From 25c0739bad7222d45b4818c7bf6987521a3509d2 Mon Sep 17 00:00:00 2001 From: folone Date: Mon, 14 Jan 2013 09:52:11 +0100 Subject: [PATCH 0001/1571] =?UTF-8?q?Moved=20to=20scala=202.10.0.=20Notabl?= =?UTF-8?q?e=20changes=20are:=20-=20akka=20=202.0.3=20=20=E2=86=92=202.1.0?= =?UTF-8?q?=20-=20spray=201.0-M1=20=E2=86=92=201.1-M7=20For=20now=20the=20?= =?UTF-8?q?repl=20subproject=20is=20commented=20out,=20as=20scala=20reflec?= =?UTF-8?q?tion=20api=20changed=20very=20much=20since=20the=20introduction?= =?UTF-8?q?=20of=20macros.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/scala/spark/CacheTracker.scala | 6 +- .../main/scala/spark/MapOutputTracker.scala | 6 +- .../spark/api/java/function/Function.java | 8 +-- .../spark/api/java/function/Function2.java | 9 +-- .../java/function/PairFlatMapFunction.java | 12 ++-- .../spark/api/java/function/PairFunction.java | 12 ++-- .../scala/spark/deploy/JsonProtocol.scala | 2 +- .../scala/spark/deploy/client/Client.scala | 4 +- .../scala/spark/deploy/master/Master.scala | 2 +- .../spark/deploy/master/MasterWebUI.scala | 45 ++++++------ .../scala/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/WorkerWebUI.scala | 43 ++++++------ .../spark/network/ConnectionManager.scala | 6 +- .../spark/network/ConnectionManagerTest.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 4 +- .../scala/spark/storage/BlockManager.scala | 6 +- .../spark/storage/BlockManagerMaster.scala | 5 +- .../storage/BlockManagerMasterActor.scala | 4 +- .../src/main/scala/spark/util/AkkaUtils.scala | 37 +++++----- .../test/scala/spark/CacheTrackerSuite.scala | 11 ++- project/SparkBuild.scala | 70 ++++++++++--------- project/build.properties | 2 +- project/plugins.sbt | 6 +- 23 files changed, 155 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index 04c26b2e40fae..a73438208aa29 100644 --- a/core/src/main/scala/spark/CacheTracker.scala +++ b/core/src/main/scala/spark/CacheTracker.scala @@ -5,12 +5,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.storage.BlockManager import spark.storage.StorageLevel diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 70eb9f702e689..08d2956782b9c 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -8,12 +8,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.scheduler.MapStatus import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java index dae8295f213ab..9f6ab9a59283f 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -15,8 +15,8 @@ public abstract class Function extends WrappedFunction1 implements Serializable { public abstract R call(T t) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java index 69bf12c8c9dc9..b32c178aa39f5 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction2; import java.io.Serializable; @@ -14,8 +14,9 @@ public abstract class Function2 extends WrappedFunction2 public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } + } diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java index b3cc4df6aa47a..296a3b5044a38 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -19,11 +19,11 @@ public abstract class PairFlatMapFunction public abstract Iterable> call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java index 9fc6df4b88844..e3f94788e26f7 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -18,11 +18,11 @@ public abstract class PairFunction public abstract Tuple2 call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 732fa080645b0..12069947fc5ad 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -2,7 +2,7 @@ package spark.deploy import master.{JobInfo, WorkerInfo} import worker.ExecutorRunner -import cc.spray.json._ +import spray.json._ /** * spray-json helper class containing implicit conversion to json for marshalling responses diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 90fe9508cdba9..0aee7597962af 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -3,7 +3,7 @@ package spark.deploy.client import spark.deploy._ import akka.actor._ import akka.pattern.ask -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.AskTimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent @@ -11,7 +11,7 @@ import akka.remote.RemoteClientShutdown import spark.deploy.RegisterJob import akka.remote.RemoteClientDisconnected import akka.actor.Terminated -import akka.dispatch.Await +import scala.concurrent.Await /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description, diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 6ecebe626a7ce..e034312c12ba1 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -49,7 +49,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } def startWebUi() { - val webUi = new MasterWebUI(context.system, self) + val webUi = new MasterWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 458ee2d66589c..a4dadb6ef9484 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,39 +1,42 @@ package spark.deploy.master -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext, ActorRefFactory} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -import akka.util.duration._ -import cc.spray.Directives -import cc.spray.directives._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ +import scala.concurrent.duration._ +import spray.routing.Directives +import spray.routing.directives._ +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy._ import spark.deploy.JsonProtocol._ private[spark] -class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { +class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => - val future = master ? RequestMasterState - respondWithMediaType(MediaTypes.`application/json`) { ctx => + val future = (master ? RequestMasterState).mapTo[MasterState] + respondWithMediaType(`application/json`) { ctx => ctx.complete(future.mapTo[MasterState]) } case _ => - completeWith { - val future = master ? RequestMasterState + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) + masterState => spark.deploy.master.html.index.render(masterState) } } } ~ @@ -50,15 +53,13 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct } } } - respondWithMediaType(MediaTypes.`application/json`) { ctx => + respondWithMediaType(`application/json`) { ctx => ctx.complete(jobInfo.mapTo[JobInfo]) } case (jobId, _) => - completeWith { - val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] + future.map { masterState => masterState.activeJobs.find(_.id == jobId) match { case Some(job) => spark.deploy.master.html.job_details.render(job) case _ => masterState.completedJobs.find(_.id == jobId) match { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 7c9e588ea2d32..ec25a19e7b9cb 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -100,7 +100,7 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self) + val webUi = new WorkerWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f9489d99fc13f..7dd1781900ea8 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,46 +1,49 @@ package spark.deploy.worker -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -import akka.util.duration._ -import cc.spray.Directives -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ +import scala.concurrent.duration._ +import spray.routing.Directives +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { - val RESOURCE_DIR = "spark/deploy/worker/webui" +class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system + val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { - (path("") & parameters('format ?)) { + (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => { - val future = worker ? RequestWorkerState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[WorkerState]) + val future = (worker ? RequestWorkerState).mapTo[WorkerState] + respondWithMediaType(`application/json`) { ctx => + ctx.complete(future) } } case _ => - completeWith{ - val future = worker ? RequestWorkerState + complete { + val future = (worker ? RequestWorkerState).mapTo[WorkerState] future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) + spark.deploy.worker.html.index(workerState) } } } ~ path("log") { parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => - respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName("work/" + jobId + "/" + executorId + "/" + logType) + respondWithMediaType(`text/plain`) { + getFromFile("work/" + jobId + "/" + executorId + "/" + logType) } } } ~ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 36c01ad629bbb..04b303afe07ce 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.SynchronizedQueue import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer -import akka.dispatch.{Await, Promise, ExecutionContext, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Promise, ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ private[spark] case class ConnectionManagerId(host: String, port: Int) { def toSocketAddress() = new InetSocketAddress(host, port) diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3a4c..1b5b0935c2fc3 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -8,8 +8,8 @@ import scala.io.Source import java.nio.ByteBuffer import java.net.InetAddress -import akka.dispatch.Await -import akka.util.duration._ +import scala.concurrent.Await +import scala.concurrent.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eeaae23dc8697..03dc5f4b9b4ab 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -3,11 +3,11 @@ package spark.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.ask import spark.{SparkException, Logging, TaskState} -import akka.dispatch.Await +import scala.concurrent.Await import java.util.concurrent.atomic.AtomicInteger import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a8ac10cdd88e..7a1344668fbca 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -8,9 +8,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} -import akka.dispatch.{Await, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index a3d8671834dbc..46e1860d09fff 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -4,10 +4,9 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.dispatch.Await +import scala.concurrent.Await import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, SparkException, Utils} diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f4d026da3329c..9bf9161c31255 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -7,8 +7,7 @@ import scala.collection.JavaConversions._ import scala.util.Random import akka.actor.{Actor, ActorRef, Cancellable} -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, Utils} @@ -42,6 +41,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { override def preStart() { if (!BlockManager.getDisableHeartBeatsForTesting) { + import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule( 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e67cb0336d066..6fd9aa70fcff8 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,16 +1,16 @@ package spark.util -import akka.actor.{Props, ActorSystemImpl, ActorSystem} +import akka.actor.{Props, ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.ask import akka.remote.RemoteActorRefProvider -import cc.spray.Route -import cc.spray.io.IoWorker -import cc.spray.{SprayCanRootService, HttpService} -import cc.spray.can.server.HttpServer -import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler -import akka.dispatch.Await +import spray.routing.Route +import spray.io.IOExtension +import spray.routing.HttpServiceActor +import spray.can.server.{HttpServer, ServerSettings} +import spray.io.SingletonHandler +import scala.concurrent.Await import spark.SparkException import java.util.concurrent.TimeoutException @@ -23,11 +23,11 @@ private[spark] object AkkaUtils { * ActorSystem itself and its port (which is hard to get from Akka). */ def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt + val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val akkaConf = ConfigFactory.parseString(""" + val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.actor.provider = "akka.remote.RemoteActorRefProvider" @@ -44,7 +44,7 @@ private[spark] object AkkaUtils { // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. - val provider = actorSystem.asInstanceOf[ActorSystemImpl].provider + val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } @@ -54,14 +54,13 @@ private[spark] object AkkaUtils { * handle requests. Throws a SparkException if this fails. */ def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { - val ioWorker = new IoWorker(actorSystem).start() - val httpService = actorSystem.actorOf(Props(new HttpService(route))) - val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) - val server = actorSystem.actorOf( - Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = "HttpServer") - actorSystem.registerOnTermination { ioWorker.stop() } + val ioWorker = IOExtension(actorSystem).ioBridge() + val httpService = actorSystem.actorOf(Props(HttpServiceActor(route))) + val server = actorSystem.actorOf( + Props(new HttpServer(ioWorker, SingletonHandler(httpService), ServerSettings())), name = "HttpServer") + actorSystem.registerOnTermination { actorSystem.stop(ioWorker) } val timeout = 3.seconds - val future = server.ask(HttpServer.Bind(ip, port))(timeout) + val future = server.ask(HttpServer.Bind(ip, port))(timeout) try { Await.result(future, timeout) match { case bound: HttpServer.Bound => diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala index 467605981b3d9..d0c2bd47fc02b 100644 --- a/core/src/test/scala/spark/CacheTrackerSuite.scala +++ b/core/src/test/scala/spark/CacheTrackerSuite.scala @@ -5,20 +5,19 @@ import org.scalatest.FunSuite import scala.collection.mutable.HashMap import akka.actor._ -import akka.dispatch._ -import akka.pattern.ask +import scala.concurrent.{Await, Future} import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ class CacheTrackerSuite extends FunSuite { // Send a message to an actor and wait for a reply, in a blocking manner private def ask(actor: ActorRef, message: Any): Any = { try { val timeout = 10.seconds - val future = actor.ask(message)(timeout) - return Await.result(future, timeout) + val future: Future[Any] = akka.pattern.ask(actor, message)(timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with actor", e) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 219674028eb7d..d0b3c350f12bd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) +// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -32,10 +32,10 @@ object SparkBuild extends Build { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") def sharedSettings = Defaults.defaultSettings ++ Seq( - organization := "org.spark-project", - version := "0.7.0-SNAPSHOT", - scalaVersion := "2.9.2", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + organization := "org.spark-project", + version := "0.7.0-SNAPSHOT", + scalaVersion := "2.10.0", + scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -87,11 +87,11 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test" - ), + "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", + "com.novocode" % "junit-interface" % "0.8" % "test" + ), parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, @@ -112,31 +112,33 @@ object SparkBuild extends Build { name := "spark-core", resolvers ++= Seq( "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Spray Repository" at "http://repo.spray.cc/", + "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", + "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "11.0.1", - "log4j" % "log4j" % "1.2.16", - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, - "asm" % "asm-all" % "3.3.1", - "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.20", - "com.typesafe.akka" % "akka-actor" % "2.0.3", - "com.typesafe.akka" % "akka-remote" % "2.0.3", - "com.typesafe.akka" % "akka-slf4j" % "2.0.3", - "it.unimi.dsi" % "fastutil" % "6.4.4", - "colt" % "colt" % "1.2.0", - "cc.spray" % "spray-can" % "1.0-M2.1", - "cc.spray" % "spray-server" % "1.0-M2.1", - "cc.spray" %% "spray-json" % "1.1.1", - "org.apache.mesos" % "mesos" % "0.9.0-incubating" - ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, + "com.google.guava" % "guava" % "11.0.1", + "log4j" % "log4j" % "1.2.16", + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "com.ning" % "compress-lzf" % "0.8.4", + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "asm" % "asm-all" % "3.3.1", + "com.google.protobuf" % "protobuf-java" % "2.4.1", + "de.javakaffee" % "kryo-serializers" % "0.20", + "com.typesafe.akka" %% "akka-remote" % "2.1.0", + "com.typesafe.akka" %% "akka-slf4j" % "2.1.0", + "it.unimi.dsi" % "fastutil" % "6.4.4", + "io.spray" % "spray-can" % "1.1-M7", + "io.spray" % "spray-io" % "1.1-M7", + "io.spray" % "spray-routing" % "1.1-M7", + "io.spray" %% "spray-json" % "1.2.3", + "colt" % "colt" % "1.2.0", + "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.scala-lang" % "scala-actors" % "2.10.0" + ) ++ (if (HADOOP_MAJOR_VERSION == "2") + Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings @@ -144,10 +146,10 @@ object SparkBuild extends Build { publish := {} ) - def replSettings = sharedSettings ++ Seq( +/* def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + )*/ def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/build.properties b/project/build.properties index d4287112c6afb..4474a03e1aa96 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.3 +sbt.version=0.12.1 diff --git a/project/plugins.sbt b/project/plugins.sbt index 4d0e696a11358..0e4eb7085c01f 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,11 +6,11 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2") +addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) From a5403acd4ead9ecabfa9f2ed6d6d58afe912a238 Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 14:42:16 +0100 Subject: [PATCH 0002/1571] Updated maven build for scala 2.10. --- core/pom.xml | 20 ++++++++++-------- pom.xml | 57 ++++++++++++++++++++++++++++------------------------ 2 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 862d3ec37ac8f..6316b28a7befd 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,16 +44,16 @@ kryo-serializers - com.typesafe.akka - akka-actor + org.scala-lang + scala-actors com.typesafe.akka - akka-remote + akka-remote_${scala.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} it.unimi.dsi @@ -64,15 +64,19 @@ colt - cc.spray + io.spray spray-can - cc.spray - spray-server + io.spray + spray-routing + + + io.spray + spray-io - cc.spray + io.spray spray-json_${scala.version} diff --git a/pom.xml b/pom.xml index 751189a9d8749..756fe8783bdbc 100644 --- a/pom.xml +++ b/pom.xml @@ -41,8 +41,8 @@ core bagel examples - repl - repl-bin + @@ -50,20 +50,20 @@ UTF-8 1.5 - 2.9.2 + 2.10 0.9.0-incubating - 2.0.3 - 1.0-M2.1 - 1.1.1 + 2.1.0 + 1.1-M7 + 1.2.3 1.6.1 4.1.2 - jboss-repo - JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases/ + typesafe-repo + Typesafe Repository + http://repo.typesafe.com/typesafe/releases/ true @@ -72,9 +72,9 @@ - cloudera-repo - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos/ + jboss-repo + JBoss Repository + http://repository.jboss.org/nexus/content/repositories/releases/ true @@ -83,9 +83,9 @@ - typesafe-repo - Typesafe Repository - http://repo.typesafe.com/typesafe/releases/ + cloudera-repo + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos/ true @@ -189,18 +189,18 @@ 0.20 - com.typesafe.akka - akka-actor - ${akka.version} + org.scala-lang + scala-actors + 2.10.0 com.typesafe.akka - akka-remote + akka-remote_${scala.version} ${akka.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} ${akka.version} @@ -214,17 +214,22 @@ 1.2.0 - cc.spray + io.spray spray-can ${spray.version} - cc.spray - spray-server + io.spray + spray-routing + ${spray.version} + + + io.spray + spray-io ${spray.version} - cc.spray + io.spray spray-json_${scala.version} ${spray.json.version} @@ -258,13 +263,13 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test From fd6e51deec83f01be3db41e84255329eedbe15da Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 17:02:58 +0100 Subject: [PATCH 0003/1571] Fixed the failing test. --- .../test/scala/spark/AccumulatorSuite.scala | 33 +++++++++---------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d8be99dde71f8..9f5335978f35a 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -13,6 +13,20 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter var sc: SparkContext = null + implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } + } + after { if (sc != null) { sc.stop() @@ -40,7 +54,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("add value to collection accumulators") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -60,22 +73,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] { - def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[Any], t2: Any) : mutable.Set[Any] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[Any]) : mutable.Set[Any] = { - new mutable.HashSet[Any]() - } - } - test ("value not readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -123,7 +121,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("localValue readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -135,7 +132,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } acc.value should be ( (0 to maxI).toSet) sc.stop() - sc = null + sc = null } } From 81c4d19c612208b932f8579427cb895385336c6e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Feb 2013 12:43:13 -0800 Subject: [PATCH 0004/1571] Maven and sbt build changes for SparkGraph. --- graph/pom.xml | 106 ++++++++++++++++++ .../src/main/scala/spark/graph/package.scala | 7 ++ pom.xml | 1 + project/SparkBuild.scala | 6 +- run | 2 + run2.cmd | 2 + 6 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 graph/pom.xml create mode 100644 graph/src/main/scala/spark/graph/package.scala diff --git a/graph/pom.xml b/graph/pom.xml new file mode 100644 index 0000000000000..1cd9cda98bc4b --- /dev/null +++ b/graph/pom.xml @@ -0,0 +1,106 @@ + + + 4.0.0 + + org.spark-project + parent + 0.7.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-graph + jar + Spark Graph + http://spark-project.org/ + + + + org.eclipse.jetty + jetty-server + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala new file mode 100644 index 0000000000000..35b9f24461330 --- /dev/null +++ b/graph/src/main/scala/spark/graph/package.scala @@ -0,0 +1,7 @@ +package spark + +package object graph { + type Vid = Int + type Pid = Int + type Status = Boolean +} diff --git a/pom.xml b/pom.xml index 3ea989a082b2a..11a1f87ed3715 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,7 @@ core bagel examples + graph streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 03b8094f7dce5..3e383b1300b37 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,7 +17,7 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, graph, streaming) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -27,6 +27,8 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn (core) + lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) // A configuration to set an alternative publishLocalConfiguration @@ -161,6 +163,8 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def graphSettings = sharedSettings ++ Seq(name := "spark-graph") + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( diff --git a/run b/run index a0946294497a8..4ecaa70511729 100755 --- a/run +++ b/run @@ -63,6 +63,7 @@ CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +GRAPH_DIR="$FWDIR/graph" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -95,6 +96,7 @@ if [ -e repl-bin/target ]; then done fi CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH+=":$GRAPH_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH+=":$jar" done diff --git a/run2.cmd b/run2.cmd index 67f1e465e47b3..eb5e283e9f4ca 100644 --- a/run2.cmd +++ b/run2.cmd @@ -34,6 +34,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set GRAPH_DIR=%FWDIR%graph set PYSPARK_DIR=%FWDIR%python rem Build up classpath @@ -45,6 +46,7 @@ for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH! for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j for /R "%PYSPARK_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%GRAPH_DIR%\target\scala-%SCALA_VERSION%\classes rem Figure out whether to run our class with java or with the scala launcher. rem In most cases, we'd prefer to execute our process with java because scala From 15530c2b23d1f3871a51fd24d14ce11b8ffaff4a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 17 Mar 2013 10:47:17 +0530 Subject: [PATCH 0005/1571] porting of repl to scala-2.10 --- project/SparkBuild.scala | 17 +- project/plugins.sbt | 4 +- repl/src/main/scala/spark/repl/Main.scala | 8 +- .../scala/spark/repl/SparkExprTyper.scala | 109 ++ .../main/scala/spark/repl/SparkILoop.scala | 908 +++++---- .../scala/spark/repl/SparkILoopInit.scala | 142 ++ .../main/scala/spark/repl/SparkIMain.scala | 1675 +++++++++-------- .../scala/spark/repl/SparkISettings.scala | 63 - .../main/scala/spark/repl/SparkImports.scala | 113 +- .../spark/repl/SparkJLineCompletion.scala | 207 +- .../scala/spark/repl/SparkJLineReader.scala | 65 +- .../spark/repl/SparkMemberHandlers.scala | 112 +- run | 2 +- 13 files changed, 1908 insertions(+), 1517 deletions(-) create mode 100644 repl/src/main/scala/spark/repl/SparkExprTyper.scala create mode 100644 repl/src/main/scala/spark/repl/SparkILoopInit.scala delete mode 100644 repl/src/main/scala/spark/repl/SparkISettings.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d0b3c350f12bd..9fcdbb5b92e19 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) -// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -35,7 +35,7 @@ object SparkBuild extends Build { organization := "org.spark-project", version := "0.7.0-SNAPSHOT", scalaVersion := "2.10.0", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + scalacOptions := Seq("-unchecked", "-optimize"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -136,7 +136,9 @@ object SparkBuild extends Build { "io.spray" %% "spray-json" % "1.2.3", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", - "org.scala-lang" % "scala-actors" % "2.10.0" + "org.scala-lang" % "scala-actors" % "2.10.0", + "org.scala-lang" % "jline" % "2.10.0", + "org.scala-lang" % "scala-reflect" % "2.10.0" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } @@ -146,10 +148,11 @@ object SparkBuild extends Build { publish := {} ) -/* def replSettings = sharedSettings ++ Seq( + def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - )*/ + // libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) + libraryDependencies ++= Seq("org.scala-lang" % "scala-compiler" % "2.10.0") + ) def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/plugins.sbt b/project/plugins.sbt index 0e4eb7085c01f..4bb4a06a505ad 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,9 +6,9 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") +// addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") +// addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646a96..b0a78e0bb811e 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -3,12 +3,12 @@ package spark.repl import scala.collection.mutable.Set object Main { - private var _interp: SparkILoop = null - + private var _interp: SparkILoop = _ + def interp = _interp - + def interp_=(i: SparkILoop) { _interp = i } - + def main(args: Array[String]) { _interp = new SparkILoop _interp.process(args) diff --git a/repl/src/main/scala/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/spark/repl/SparkExprTyper.scala new file mode 100644 index 0000000000000..c55797eb54ef0 --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkExprTyper.scala @@ -0,0 +1,109 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.BatchSourceFile +import scala.tools.nsc.ast.parser.Tokens.EOF + +import spark.Logging + +trait SparkExprTyper extends Logging { + val repl: SparkIMain + + import repl._ + import global.{ reporter => _, Import => _, _ } + import definitions._ + import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name } + import naming.freshInternalVarName + + object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] { + def applyRule[T](code: String, rule: UnitParser => T): T = { + reporter.reset() + val scanner = newUnitParser(code) + val result = rule(scanner) + + if (!reporter.hasErrors) + scanner.accept(EOF) + + result + } + + def defns(code: String) = stmts(code) collect { case x: DefTree => x } + def expr(code: String) = applyRule(code, _.expr()) + def stmts(code: String) = applyRule(code, _.templateStats()) + def stmt(code: String) = stmts(code).last // guaranteed nonempty + } + + /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ + def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") { + var isIncomplete = false + reporter.withIncompleteHandler((_, _) => isIncomplete = true) { + val trees = codeParser.stmts(line) + if (reporter.hasErrors) Some(Nil) + else if (isIncomplete) None + else Some(trees) + } + } + // def parsesAsExpr(line: String) = { + // import codeParser._ + // (opt expr line).isDefined + // } + + def symbolOfLine(code: String): Symbol = { + def asExpr(): Symbol = { + val name = freshInternalVarName() + // Typing it with a lazy val would give us the right type, but runs + // into compiler bugs with things like existentials, so we compile it + // behind a def and strip the NullaryMethodType which wraps the expr. + val line = "def " + name + " = {\n" + code + "\n}" + + interpretSynthetic(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + // drop NullaryMethodType + val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) + if (sym.info.typeSymbol eq UnitClass) NoSymbol + else sym + case _ => NoSymbol + } + } + def asDefn(): Symbol = { + val old = repl.definedSymbolList.toSet + + interpretSynthetic(code) match { + case IR.Success => + repl.definedSymbolList filterNot old match { + case Nil => NoSymbol + case sym :: Nil => sym + case syms => NoSymbol.newOverloaded(NoPrefix, syms) + } + case _ => NoSymbol + } + } + beQuietDuring(asExpr()) orElse beQuietDuring(asDefn()) + } + + private var typeOfExpressionDepth = 0 + def typeOfExpression(expr: String, silent: Boolean = true): Type = { + if (typeOfExpressionDepth > 2) { + logDebug("Terminating typeOfExpression recursion for expression: " + expr) + return NoType + } + typeOfExpressionDepth += 1 + // Don't presently have a good way to suppress undesirable success output + // while letting errors through, so it is first trying it silently: if there + // is an error, and errors are desired, then it re-evaluates non-silently + // to induce the error message. + try beSilentDuring(symbolOfLine(expr).tpe) match { + case NoType if !silent => symbolOfLine(expr).tpe // generate error + case tpe => tpe + } + finally typeOfExpressionDepth -= 1 + } +} diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 22bcb4be8a8f8..2f2b5b237262d 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -1,26 +1,38 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Alexander Spoon */ package spark.repl + import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.{ Results => IR } import Predef.{ println => _, _ } -import java.io.{ BufferedReader, FileReader, PrintWriter } +import java.io.{ BufferedReader, FileReader } +import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process -import session._ -import scala.tools.nsc.interpreter.{ Results => IR } -import scala.tools.util.{ SignalManager, Signallable, Javap } +import scala.tools.nsc.interpreter.session._ +import scala.util.Properties.{ jdkHome, javaVersion } +import scala.tools.util.{ Javap } import scala.annotation.tailrec -import scala.util.control.Exception.{ ignoring } import scala.collection.mutable.ListBuffer import scala.concurrent.ops -import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } -import interpreter._ -import io.{ File, Sources } +import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } +import scala.tools.nsc.interpreter._ +import scala.tools.nsc.io.{ File, Directory } +import scala.reflect.NameTransformer._ +import scala.tools.nsc.util.ScalaClassLoader +import scala.tools.nsc.util.ScalaClassLoader._ +import scala.tools.util._ +import scala.language.{implicitConversions, existentials} +import scala.reflect.{ClassTag, classTag} +import scala.tools.reflect.StdRuntimeTags._ +import scala.reflect.{ClassTag, classTag} +import java.lang.{Class => jClass} +import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import spark.Logging import spark.SparkContext @@ -37,45 +49,85 @@ import spark.SparkContext * @author Lex Spoon * @version 1.2 */ -class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Option[String]) +class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, + val master: Option[String]) extends AnyRef with LoopCommands + with SparkILoopInit with Logging { - def this(in0: BufferedReader, out: PrintWriter, master: String) = this(Some(in0), out, Some(master)) - def this(in0: BufferedReader, out: PrintWriter) = this(Some(in0), out, None) - def this() = this(None, new PrintWriter(Console.out, true), None) - + def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) + def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) + def this() = this(None, new JPrintWriter(Console.out, true), None) + var in: InteractiveReader = _ // the input stream from which commands come var settings: Settings = _ var intp: SparkIMain = _ - /* - lazy val power = { - val g = intp.global - Power[g.type](this, g) - } - */ - - // TODO - // object opt extends AestheticSettings - // - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter = intp - - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter_= (i: SparkIMain): Unit = intp = i - + @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp + @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i + + /** Having inherited the difficult "var-ness" of the repl instance, + * I'm trying to work around it by moving operations into a class from + * which it will appear a stable prefix. + */ + private def onIntp[T](f: SparkIMain => T): T = f(intp) + + class IMainOps[T <: SparkIMain](val intp: T) { + import intp._ + import global._ + + def printAfterTyper(msg: => String) = + intp.reporter printMessage afterTyper(msg) + + /** Strip NullaryMethodType artifacts. */ + private def replInfo(sym: Symbol) = { + sym.info match { + case NullaryMethodType(restpe) if sym.isAccessor => restpe + case info => info + } + } + def echoTypeStructure(sym: Symbol) = + printAfterTyper("" + deconstruct.show(replInfo(sym))) + + def echoTypeSignature(sym: Symbol, verbose: Boolean) = { + if (verbose) SparkILoop.this.echo("// Type signature") + printAfterTyper("" + replInfo(sym)) + + if (verbose) { + SparkILoop.this.echo("\n// Internal Type structure") + echoTypeStructure(sym) + } + } + } + implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp) + + /** TODO - + * -n normalize + * -l label with case class parameter names + * -c complete - leave nothing out + */ + private def typeCommandInternal(expr: String, verbose: Boolean): Result = { + onIntp { intp => + val sym = intp.symbolOfLine(expr) + if (sym.exists) intp.echoTypeSignature(sym, verbose) + else "" + } + } + + var sparkContext: SparkContext = _ + + override def echoCommandMessage(msg: String) { + intp.reporter printMessage msg + } + + def isAsync = !settings.Yreplsync.value + // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) def history = in.history /** The context class loader at the time this object was created */ protected val originalClassLoader = Thread.currentThread.getContextClassLoader - // Install a signal handler so we can be prodded. - private val signallable = - /*if (isReplDebug) Signallable("Dump repl state.")(dumpCommand()) - else*/ null - // classpath entries added via :cp var addedClasspath: String = "" @@ -87,74 +139,41 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: /** Record a command for replay should the user request a :replay */ def addReplay(cmd: String) = replayCommandStack ::= cmd - - /** Try to install sigint handler: ignore failure. Signal handler - * will interrupt current line execution if any is in progress. - * - * Attempting to protect the repl from accidental exit, we only honor - * a single ctrl-C if the current buffer is empty: otherwise we look - * for a second one within a short time. - */ - private def installSigIntHandler() { - def onExit() { - Console.println("") // avoiding "shell prompt in middle of line" syndrome - sys.exit(1) - } - ignoring(classOf[Exception]) { - SignalManager("INT") = { - if (intp == null) - onExit() - else if (intp.lineManager.running) - intp.lineManager.cancel() - else if (in.currentLine != "") { - // non-empty buffer, so make them hit ctrl-C a second time - SignalManager("INT") = onExit() - io.timer(5)(installSigIntHandler()) // and restore original handler if they don't - } - else onExit() - } - } + + def savingReplayStack[T](body: => T): T = { + val saved = replayCommandStack + try body + finally replayCommandStack = saved + } + def savingReader[T](body: => T): T = { + val saved = in + try body + finally in = saved } /** Close the interpreter and set the var to null. */ def closeInterpreter() { if (intp ne null) { - intp.close + intp.close() intp = null - Thread.currentThread.setContextClassLoader(originalClassLoader) } } - + class SparkILoopInterpreter extends SparkIMain(settings, out) { + outer => + override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def createLineManager() = new Line.Manager { - override def onRunaway(line: Line[_]): Unit = { - val template = """ - |// She's gone rogue, captain! Have to take her out! - |// Calling Thread.stop on runaway %s with offending code: - |// scala> %s""".stripMargin - - echo(template.format(line.thread, line.code)) - // XXX no way to suppress the deprecation warning - line.thread.stop() - in.redrawLine() - } - } - override protected def parentClassLoader = { - SparkHelper.explicitParentLoader(settings).getOrElse( classOf[SparkILoop].getClassLoader ) - } + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ def createInterpreter() { if (addedClasspath != "") settings.classpath append addedClasspath - + intp = new SparkILoopInterpreter - intp.setContextClassLoader() - installSigIntHandler() } /** print a friendly help message */ @@ -168,10 +187,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private def helpSummary() = { val usageWidth = commands map (_.usageMsg.length) max val formatStr = "%-" + usageWidth + "s %s %s" - + echo("All commands can be abbreviated, e.g. :he instead of :help.") echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - + commands foreach { cmd => val star = if (cmd.hasLongHelp) "*" else " " echo(formatStr.format(cmd.usageMsg, star, cmd.help)) @@ -182,7 +201,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case Nil => echo(cmd + ": no such command. Type :help for help.") case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") } - Result(true, None) + Result(true, None) } private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) private def uniqueCommand(cmd: String): Option[LoopCommand] = { @@ -193,31 +212,16 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case xs => xs find (_.name == cmd) } } - - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.7.0 - /_/ -""") - import Properties._ - val welcomeMsg = "Using Scala %s (%s, Java %s)".format( - versionString, javaVmName, javaVersion) - echo(welcomeMsg) - } - + /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 - + def apply(line: String): Result = { if (history eq NoHistory) return "No history available." - + val xs = words(line) val current = history.index val count = try xs.head.toInt catch { case _: Exception => defaultLines } @@ -229,32 +233,38 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - private def echo(msg: String) = { + // When you know you are most likely breaking into the middle + // of a line being typed. This softens the blow. + protected def echoAndRefresh(msg: String) = { + echo("\n" + msg) + in.redrawLine() + } + protected def echo(msg: String) = { out println msg out.flush() } - private def echoNoNL(msg: String) = { + protected def echoNoNL(msg: String) = { out print msg out.flush() } - + /** Search the history */ def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 - + for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) echo("%d %s".format(index + offset, line)) } - + private var currentPrompt = Properties.shellPromptString def setPrompt(prompt: String) = currentPrompt = prompt /** Prompt to print when awaiting input */ def prompt = currentPrompt - + import LoopCommand.{ cmd, nullary } - /** Standard commands **/ + /** Standard commands */ lazy val standardCommands = List( cmd("cp", "", "add a jar or directory to the classpath", addClasspath), cmd("help", "[command]", "print this summary or command-specific help", helpCommand), @@ -263,53 +273,30 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand), cmd("javap", "", "disassemble a file or class name", javapCommand), - nullary("keybindings", "show how ctrl-[A-Z] and other keys are bound", keybindingsCommand), cmd("load", "", "load and interpret a Scala file", loadCommand), nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand), - //nullary("power", "enable power user mode", powerCmd), - nullary("quit", "exit the interpreter", () => Result(false, None)), +// nullary("power", "enable power user mode", powerCmd), + nullary("quit", "exit the repl", () => Result(false, None)), nullary("replay", "reset execution and replay all previous commands", replay), + nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), shCommand, nullary("silent", "disable/enable automatic printing of results", verbosity), - cmd("type", "", "display the type of an expression without evaluating it", typeCommand) + cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), + nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) ) - + /** Power user commands */ lazy val powerCommands: List[LoopCommand] = List( - //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand), - //cmd("phase", "", "set the implicit phase for power commands", phaseCommand), - cmd("wrap", "", "name of method to wrap around each repl line", wrapCommand) withLongHelp (""" - |:wrap - |:wrap clear - |:wrap - | - |Installs a wrapper around each line entered into the repl. - |Currently it must be the simple name of an existing method - |with the specific signature shown in the following example. - | - |def timed[T](body: => T): T = { - | val start = System.nanoTime - | try body - | finally println((System.nanoTime - start) + " nanos elapsed.") - |} - |:wrap timed - | - |If given no argument, :wrap names the wrapper installed. - |An argument of clear will remove the wrapper if any is active. - |Note that wrappers do not compose (a new one replaces the old - |one) and also that the :phase command uses the same machinery, - |so setting :wrap will clear any :phase setting. - """.stripMargin.trim) + // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) ) - - /* - private def dumpCommand(): Result = { - echo("" + power) - history.asStrings takeRight 30 foreach echo - in.redrawLine() - } - */ - + + // private def dumpCommand(): Result = { + // echo("" + power) + // history.asStrings takeRight 30 foreach echo + // in.redrawLine() + // } + // private def valsCommand(): Result = power.valsDescription + private val typeTransforms = List( "scala.collection.immutable." -> "immutable.", "scala.collection.mutable." -> "mutable.", @@ -317,7 +304,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: "java.lang." -> "jl.", "scala.runtime." -> "runtime." ) - + private def importsCommand(line: String): Result = { val tokens = words(line) val handlers = intp.languageWildcardHandlers ++ intp.importHandlers @@ -333,7 +320,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - + intp.reporter.printMessage("%2d) %-30s %s%s".format( idx + 1, handler.importString, @@ -342,12 +329,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: )) } } - - private def implicitsCommand(line: String): Result = { - val intp = SparkILoop.this.intp + + private def implicitsCommand(line: String): Result = onIntp { intp => import intp._ - import global.Symbol - + import global._ + def p(x: Any) = intp.reporter.printMessage("" + x) // If an argument is given, only show a source with that @@ -360,17 +346,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else (args exists (source.name.toString contains _)) } } - + if (filtered.isEmpty) return "No implicits have been imported other than those in Predef." - + filtered foreach { case (source, syms) => p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - + // This groups the members by where the symbol is defined val byOwner = syms groupBy (_.owner) - val sortedOwners = byOwner.toList sortBy { case (owner, _) => intp.afterTyper(source.info.baseClasses indexOf owner) } + val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) } sortedOwners foreach { case (owner, members) => @@ -388,10 +374,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: xss map (xs => xs sortBy (_.name.toString)) } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " + + val ownerMessage = if (owner == source) " defined in " else " inherited from " p(" /* " + members.size + ownerMessage + owner.fullName + " */") - + memberGroups foreach { group => group foreach (s => p(" " + intp.symbolDefString(s))) p("") @@ -400,158 +386,182 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: p("") } } - - protected def newJavap() = new Javap(intp.classLoader, new SparkIMain.ReplStrippingWriter(intp)) { - override def tryClass(path: String): Array[Byte] = { - // Look for Foo first, then Foo$, but if Foo$ is given explicitly, - // we have to drop the $ to find object Foo, then tack it back onto - // the end of the flattened name. - def className = intp flatName path - def moduleName = (intp flatName path.stripSuffix("$")) + "$" - val bytes = super.tryClass(className) - if (bytes.nonEmpty) bytes - else super.tryClass(moduleName) + private def findToolsJar() = { + val jdkPath = Directory(jdkHome) + val jar = jdkPath / "lib" / "tools.jar" toFile; + + if (jar isFile) + Some(jar) + else if (jdkPath.isDirectory) + jdkPath.deepFiles find (_.name == "tools.jar") + else None + } + private def addToolsJarToLoader() = { + val cl = findToolsJar match { + case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) + case _ => intp.classLoader + } + if (Javap.isAvailable(cl)) { + logDebug(":javap available.") + cl + } + else { + logDebug(":javap unavailable: no tools.jar at " + jdkHome) + intp.classLoader + } + } + + protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { + override def tryClass(path: String): Array[Byte] = { + val hd :: rest = path split '.' toList; + // If there are dots in the name, the first segment is the + // key to finding it. + if (rest.nonEmpty) { + intp optFlatName hd match { + case Some(flat) => + val clazz = flat :: rest mkString NAME_JOIN_STRING + val bytes = super.tryClass(clazz) + if (bytes.nonEmpty) bytes + else super.tryClass(clazz + MODULE_SUFFIX_STRING) + case _ => super.tryClass(path) + } + } + else { + // Look for Foo first, then Foo$, but if Foo$ is given explicitly, + // we have to drop the $ to find object Foo, then tack it back onto + // the end of the flattened name. + def className = intp flatName path + def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING + + val bytes = super.tryClass(className) + if (bytes.nonEmpty) bytes + else super.tryClass(moduleName) + } } } + // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) private lazy val javap = try newJavap() catch { case _: Exception => null } - - private def typeCommand(line: String): Result = { - intp.typeOfExpression(line) match { - case Some(tp) => tp.toString - case _ => "Failed to determine type." + + // Still todo: modules. + private def typeCommand(line0: String): Result = { + line0.trim match { + case "" => ":type [-v] " + case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true) + case s => typeCommandInternal(s, false) } } - + + private def warningsCommand(): Result = { + if (intp.lastWarnings.isEmpty) + "Can't find any cached warnings." + else + intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } + } + private def javapCommand(line: String): Result = { if (javap == null) - return ":javap unavailable on this platform." - if (line == "") - return ":javap [-lcsvp] [path1 path2 ...]" - - javap(words(line)) foreach { res => - if (res.isError) return "Failed: " + res.value - else res.show() - } - } - private def keybindingsCommand(): Result = { - if (in.keyBindings.isEmpty) "Key bindings unavailable." - else { - echo("Reading jline properties for default key bindings.") - echo("Accuracy not guaranteed: treat this as a guideline only.\n") - in.keyBindings foreach (x => echo ("" + x)) - } + ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) + else if (javaVersion startsWith "1.7") + ":javap not yet working with java 1.7" + else if (line == "") + ":javap [-lcsvp] [path1 path2 ...]" + else + javap(words(line)) foreach { res => + if (res.isError) return "Failed: " + res.value + else res.show() + } } + private def wrapCommand(line: String): Result = { def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T" - val intp = SparkILoop.this.intp - val g: intp.global.type = intp.global - import g._ - - words(line) match { - case Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => "Current execution wrapper: " + s - } - case "clear" :: Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." - } - case wrapper :: Nil => - intp.typeOfExpression(wrapper) match { - case Some(PolyType(List(targ), MethodType(List(arg), restpe))) => - intp setExecutionWrapper intp.pathToTerm(wrapper) - "Set wrapper to '" + wrapper + "'" - case Some(x) => - failMsg + "\nFound: " + x - case _ => - failMsg + "\nFound: " - } - case _ => failMsg + onIntp { intp => + import intp._ + import global._ + + words(line) match { + case Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => "Current execution wrapper: " + s + } + case "clear" :: Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." + } + case wrapper :: Nil => + intp.typeOfExpression(wrapper) match { + case PolyType(List(targ), MethodType(List(arg), restpe)) => + intp setExecutionWrapper intp.pathToTerm(wrapper) + "Set wrapper to '" + wrapper + "'" + case tp => + failMsg + "\nFound: " + } + case _ => failMsg + } } } private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent" - /* - private def phaseCommand(name: String): Result = { - // This line crashes us in TreeGen: - // - // if (intp.power.phased set name) "..." - // - // Exception in thread "main" java.lang.AssertionError: assertion failed: ._7.type - // at scala.Predef$.assert(Predef.scala:99) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:69) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:44) - // at scala.tools.nsc.ast.TreeGen.mkAttributedRef(TreeGen.scala:101) - // at scala.tools.nsc.ast.TreeGen.mkAttributedStableRef(TreeGen.scala:143) - // - // But it works like so, type annotated. - val phased: Phased = power.phased - import phased.NoPhaseName - - if (name == "clear") { - phased.set(NoPhaseName) - intp.clearExecutionWrapper() - "Cleared active phase." - } - else if (name == "") phased.get match { - case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" - case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) - } - else { - val what = phased.parse(name) - if (what.isEmpty || !phased.set(what)) - "'" + name + "' does not appear to represent a valid phase." - else { - intp.setExecutionWrapper(pathToPhaseWrapper) - val activeMessage = - if (what.toString.length == name.length) "" + what - else "%s (%s)".format(what, name) - - "Active phase is now: " + activeMessage - } - } - } - */ - + // private def phaseCommand(name: String): Result = { + // val phased: Phased = power.phased + // import phased.NoPhaseName + + // if (name == "clear") { + // phased.set(NoPhaseName) + // intp.clearExecutionWrapper() + // "Cleared active phase." + // } + // else if (name == "") phased.get match { + // case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" + // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) + // } + // else { + // val what = phased.parse(name) + // if (what.isEmpty || !phased.set(what)) + // "'" + name + "' does not appear to represent a valid phase." + // else { + // intp.setExecutionWrapper(pathToPhaseWrapper) + // val activeMessage = + // if (what.toString.length == name.length) "" + what + // else "%s (%s)".format(what, name) + + // "Active phase is now: " + activeMessage + // } + // } + // } + /** Available commands */ - def commands: List[LoopCommand] = standardCommands /* ++ ( + def commands: List[LoopCommand] = standardCommands /*++ ( if (isReplPower) powerCommands else Nil )*/ - + val replayQuestionMessage = - """|The repl compiler has crashed spectacularly. Shall I replay your - |session? I can re-run all lines except the last one. + """|That entry seems to have slain the compiler. Shall I replay + |your session? I can re-run each line except the last one. |[y/n] """.trim.stripMargin - private val crashRecovery: PartialFunction[Throwable, Unit] = { + private val crashRecovery: PartialFunction[Throwable, Boolean] = { case ex: Throwable => - if (settings.YrichExes.value) { - val sources = implicitly[Sources] - echo("\n" + ex.getMessage) - echo( - if (isReplDebug) "[searching " + sources.path + " for exception contexts...]" - else "[searching for exception contexts...]" - ) - echo(Exceptional(ex).force().context()) - } - else { - echo(util.stackTraceString(ex)) - } + echo(intp.global.throwableAsString(ex)) + ex match { case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("Unrecoverable error.") + echo("\nUnrecoverable error.") throw ex case _ => - def fn(): Boolean = in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + def fn(): Boolean = + try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + if (fn()) replay() else echo("\nAbandoning crashed session.") } + true } /** The main read-eval-print loop for the repl. It calls @@ -564,66 +574,88 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: in readLine prompt } // return false if repl should exit - def processLine(line: String): Boolean = + def processLine(line: String): Boolean = { + if (isAsync) { + if (!awaitInitialized()) return false + runThunks() + } if (line eq null) false // assume null means EOF else command(line) match { case Result(false, _) => false case Result(_, Some(finalLine)) => addReplay(finalLine) ; true case _ => true } - - while (true) { - try if (!processLine(readOneLine)) return - catch crashRecovery } + def innerLoop() { + if ( try processLine(readOneLine()) catch crashRecovery ) + innerLoop() + } + innerLoop() } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { - val oldIn = in - val oldReplay = replayCommandStack - - try file applyReader { reader => - in = SimpleReader(reader, out, false) - echo("Loading " + file + "...") - loop() - } - finally { - in = oldIn - replayCommandStack = oldReplay + def interpretAllFrom(file: File) { + savingReader { + savingReplayStack { + file applyReader { reader => + in = SimpleReader(reader, out, false) + echo("Loading " + file + "...") + loop() + } + } } } - /** create a new interpreter and replay all commands so far */ + /** create a new interpreter and replay the given commands */ def replay() { - closeInterpreter() - createInterpreter() - for (cmd <- replayCommands) { + reset() + if (replayCommandStack.isEmpty) + echo("Nothing to replay.") + else for (cmd <- replayCommands) { echo("Replaying: " + cmd) // flush because maybe cmd will have its own output command(cmd) echo("") } } - + def resetCommand() { + echo("Resetting repl state.") + if (replayCommandStack.nonEmpty) { + echo("Forgetting this session history:\n") + replayCommands foreach echo + echo("") + replayCommandStack = Nil + } + if (intp.namedDefinedTerms.nonEmpty) + echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) + if (intp.definedTypes.nonEmpty) + echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) + + reset() + } + def reset() { + intp.reset() + // unleashAndSetPhase() + } + /** fork a shell and run a command */ lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { override def usage = "" def apply(line: String): Result = line match { case "" => showUsage() - case _ => + case _ => val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" intp interpret toRun () } } - + def withFile(filename: String)(action: File => Unit) { val f = File(filename) - + if (f.exists) action(f) else echo("That file does not exist") } - + def loadCommand(arg: String) = { var shouldReplay: Option[String] = None withFile(arg)(f => { @@ -643,23 +675,36 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } else echo("The path '" + f + "' doesn't seem to exist.") } - + def powerCmd(): Result = { if (isReplPower) "Already in power mode." - else enablePowerMode() + else enablePowerMode(false) + } + + def enablePowerMode(isDuringInit: Boolean) = { + // replProps.power setValue true + // unleashAndSetPhase() + // asyncEcho(isDuringInit, power.banner) } - def enablePowerMode() = { - //replProps.power setValue true - //power.unleash() - //echo(power.banner) + // private def unleashAndSetPhase() { +// if (isReplPower) { +// // power.unleash() +// // Set the phase to "typer" +// intp beSilentDuring phaseCommand("typer") +// } +// } + + def asyncEcho(async: Boolean, msg: => String) { + if (async) asyncMessage(msg) + else echo(msg) } - + def verbosity() = { - val old = intp.printResults - intp.printResults = !old - echo("Switched " + (if (old) "off" else "on") + " result printing.") + // val old = intp.printResults + // intp.printResults = !old + // echo("Switched " + (if (old) "off" else "on") + " result printing.") } - + /** Run one command submitted by the user. Two values are returned: * (1) whether to keep running, (2) the line to record for replay, * if any. */ @@ -674,11 +719,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (intp.global == null) Result(false, None) // Notice failure to create compiler else Result(true, interpretStartingWith(line)) } - + private def readWhile(cond: String => Boolean) = { Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - + def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" @@ -686,23 +731,19 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: intp interpret code () } - + private object paste extends Pasted { val ContinueString = " | " val PromptString = "scala> " - + def interpret(line: String): Unit = { echo(line.trim) intp interpret line echo("") } - + def transcript(start: String) = { - // Printing this message doesn't work very well because it's buried in the - // transcript they just pasted. Todo: a short timer goes off when - // lines stop coming which tells them to hit ctrl-D. - // - // echo("// Detected repl transcript paste: ctrl-D to finish.") + echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) } } @@ -717,7 +758,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() - + def reallyInterpret = { val reallyResult = intp.interpret(code) (reallyResult, reallyResult match { @@ -727,7 +768,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (in.interactive && code.endsWith("\n\n")) { echo("You typed two blank lines. Starting a new command.") None - } + } else in.readLine(ContinueString) match { case null => // we know compilation is going to fail since we're at EOF and the @@ -741,10 +782,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } }) } - + /** Here we place ourselves between the user and the interpreter and examine * the input they are ostensibly submitting. We intervene in several cases: - * + * * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation * on the previous result. @@ -759,28 +800,12 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { interpretStartingWith(intp.mostRecentVar + code) } - else { - def runCompletion = in.completion execute code map (intp bindValue _) - /** Due to my accidentally letting file completion execution sneak ahead - * of actual parsing this now operates in such a way that the scala - * interpretation always wins. However to avoid losing useful file - * completion I let it fail and then check the others. So if you - * type /tmp it will echo a failure and then give you a Directory object. - * It's not pretty: maybe I'll implement the silence bits I need to avoid - * echoing the failure. - */ - if (intp isParseable code) { - val (code, result) = reallyInterpret - //if (power != null && code == IR.Error) - // runCompletion - - result - } - else runCompletion match { - case Some(_) => None // completion hit: avoid the latent error - case _ => reallyInterpret._2 // trigger the latent error - } + else if (code.trim startsWith "//") { + // line comment, do nothing + None } + else + reallyInterpret._2 } // runs :load `file` on any files passed via -i @@ -794,7 +819,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } case _ => } - + /** Tries to create a JLineReader, falling back to SimpleReader: * unless settings or properties are such that it should start * with SimpleReader. @@ -802,7 +827,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def chooseReader(settings: Settings): InteractiveReader = { if (settings.Xnojline.value || Properties.isEmacsShell) SimpleReader() - else try SparkJLineReader( + else try new SparkJLineReader( if (settings.noCompletion.value) NoCompletion else new SparkJLineCompletion(intp) ) @@ -813,96 +838,97 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - def initializeSpark() { - intp.beQuietDuring { - command(""" - spark.repl.Main.interp.out.println("Creating SparkContext..."); - spark.repl.Main.interp.out.flush(); - @transient val sc = spark.repl.Main.interp.createSparkContext(); - spark.repl.Main.interp.out.println("Spark context available as sc."); - spark.repl.Main.interp.out.flush(); - """) - command("import spark.SparkContext._"); - } - echo("Type in expressions to have them evaluated.") - echo("Type :help for more information.") - } - - var sparkContext: SparkContext = null - - def createSparkContext(): SparkContext = { - val master = this.master match { - case Some(m) => m - case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local" - } - } - sparkContext = new SparkContext(master, "Spark shell") - sparkContext - } - - def process(settings: Settings): Boolean = { - // Ensure logging is initialized before any Spark threads try to use logs - // (because SLF4J initialization is not thread safe) - initLogging() - - printWelcome() - echo("Initializing interpreter...") + val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val m = u.runtimeMirror(getClass.getClassLoader) + private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = + u.TypeTag[T]( + m, + new TypeCreator { + def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type = + m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] + }) + def process(settings: Settings): Boolean = savingContextLoader { this.settings = settings createInterpreter() - + // sets in to some kind of reader depending on environmental cues in = in0 match { case Some(reader) => SimpleReader(reader, out, true) - case None => chooseReader(settings) + case None => + // some post-initialization + chooseReader(settings) match { + case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x + case x => x + } } + lazy val tagOfSparkIMain = tagOfStaticClass[spark.repl.SparkIMain] + // Bind intp somewhere out of the regular namespace where + // we can get at it in generated code. + addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain]))) + addThunk({ + import scala.tools.nsc.io._ + import Properties.userHome + import scala.compat.Platform.EOL + val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp()) + if (autorun.isDefined) intp.quietRun(autorun.get) + }) loadFiles(settings) // it is broken on startup; go ahead and exit if (intp.reporter.hasErrors) return false - - try { - // this is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try - // our best to look ready. Ideally the user will spend a - // couple seconds saying "wow, it starts so fast!" and by the time - // they type a command the compiler is ready to roll. - intp.initialize() - initializeSpark() - if (isReplPower) { - echo("Starting in power mode, one moment...\n") - enablePowerMode() - } - loop() + + // This is about the illusion of snappiness. We call initialize() + // which spins off a separate thread, then print the prompt and try + // our best to look ready. The interlocking lazy vals tend to + // inter-deadlock, so we break the cycle with a single asynchronous + // message to an actor. + if (isAsync) { + intp initialize initializedCallback() + addThunk(initializeSpark()) + createAsyncListener() // listens for signal to run postInitialization + } + else { + intp.initializeSynchronous() + postInitialization() } + printWelcome() + + try loop() + catch AbstractOrMissingHandler() finally closeInterpreter() + true } + def createSparkContext(): SparkContext = { + val master = this.master match { + case Some(m) => m + case None => { + val prop = System.getenv("MASTER") + if (prop != null) prop else "local" + } + } + sparkContext = new SparkContext(master, "Spark shell") + echo("Created spark context..") + sparkContext + } + /** process command-line arguments and do as they request */ def process(args: Array[String]): Boolean = { - val command = new CommandLine(args.toList, msg => echo("scala: " + msg)) + val command = new CommandLine(args.toList, echo) def neededHelp(): String = (if (command.settings.help.value) command.usageMsg + "\n" else "") + (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - + // if they asked for no help and command is valid, we call the real main neededHelp() match { case "" => command.ok && process(command.settings) case help => echoNoNL(help) ; true } } - - @deprecated("Use `process` instead", "2.9.0") - def main(args: Array[String]): Unit = { - if (isReplDebug) - System.out.println(new java.util.Date) - - process(args) - } + @deprecated("Use `process` instead", "2.9.0") def main(settings: Settings): Unit = process(settings) } @@ -916,10 +942,10 @@ object SparkILoop { // like if you'd just typed it into the repl. def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { - val output = new PrintWriter(new OutputStreamWriter(ostream), true) { + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { override def write(str: String) = { // completely skip continuation lines if (str forall (ch => ch.isWhitespace || ch == '|')) () @@ -945,19 +971,19 @@ object SparkILoop { } } } - + /** Creates an interpreter loop with default settings and feeds * the given code to it as input. */ def run(code: String, sets: Settings = new Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { val input = new BufferedReader(new StringReader(code)) - val output = new PrintWriter(new OutputStreamWriter(ostream), true) - val repl = new SparkILoop(input, output) - + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) + val repl = new ILoop(input, output) + if (sets.classpath.isDefault) sets.classpath.value = sys.props("java.class.path") @@ -966,32 +992,4 @@ object SparkILoop { } } def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) - - // provide the enclosing type T - // in order to set up the interpreter's classpath and parent class loader properly - def breakIf[T: Manifest](assertion: => Boolean, args: NamedParam*): Unit = - if (assertion) break[T](args.toList) - - // start a repl, binding supplied args - def break[T: Manifest](args: List[NamedParam]): Unit = { - val msg = if (args.isEmpty) "" else " Binding " + args.size + " value%s.".format( - if (args.size == 1) "" else "s" - ) - echo("Debug repl starting." + msg) - val repl = new SparkILoop { - override def prompt = "\ndebug> " - } - repl.settings = new Settings(echo) - repl.settings.embeddedDefaults[T] - repl.createInterpreter() - repl.in = SparkJLineReader(repl) - - // rebind exit so people don't accidentally call sys.exit by way of predef - repl.quietRun("""def exit = println("Type :quit to resume program execution.")""") - args foreach (p => repl.bind(p.name, p.tpe, p.value)) - repl.loop() - - echo("\nDebug repl exiting.") - repl.closeInterpreter() - } } diff --git a/repl/src/main/scala/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/spark/repl/SparkILoopInit.scala new file mode 100644 index 0000000000000..b52c477474aa6 --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkILoopInit.scala @@ -0,0 +1,142 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.Position +import scala.util.control.Exception.ignoring +import scala.tools.nsc.util.stackTraceString + +/** + * Machinery for the asynchronous initialization of the repl. + */ +trait SparkILoopInit { + self: SparkILoop => + + /** Print a welcome message */ + def printWelcome() { + echo("""Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version 0.7.1-SNAPSHOT + /_/ +""") + import Properties._ + val welcomeMsg = "Using Scala %s (%s, Java %s)".format( + versionString, javaVmName, javaVersion) + echo(welcomeMsg) + } + + protected def asyncMessage(msg: String) { + if (isReplInfo || isReplPower) + echoAndRefresh(msg) + } + + private val initLock = new java.util.concurrent.locks.ReentrantLock() + private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized + private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized + private val initStart = System.nanoTime + + private def withLock[T](body: => T): T = { + initLock.lock() + try body + finally initLock.unlock() + } + // a condition used to ensure serial access to the compiler. + @volatile private var initIsComplete = false + @volatile private var initError: String = null + private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L) + + // the method to be called when the interpreter is initialized. + // Very important this method does nothing synchronous (i.e. do + // not try to use the interpreter) because until it returns, the + // repl's lazy val `global` is still locked. + protected def initializedCallback() = withLock(initCompilerCondition.signal()) + + // Spins off a thread which awaits a single message once the interpreter + // has been initialized. + protected def createAsyncListener() = { + io.spawn { + withLock(initCompilerCondition.await()) + asyncMessage("[info] compiler init time: " + elapsed() + " s.") + postInitialization() + } + } + + // called from main repl loop + protected def awaitInitialized(): Boolean = { + if (!initIsComplete) + withLock { while (!initIsComplete) initLoopCondition.await() } + if (initError != null) { + println(""" + |Failed to initialize the REPL due to an unexpected error. + |This is a bug, please, report it along with the error diagnostics printed below. + |%s.""".stripMargin.format(initError) + ) + false + } else true + } + // private def warningsThunks = List( + // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _), + // ) + + protected def postInitThunks = List[Option[() => Unit]]( + Some(intp.setContextClassLoader _), + if (isReplPower) Some(() => enablePowerMode(true)) else None + ).flatten + // ++ ( + // warningsThunks + // ) + // called once after init condition is signalled + protected def postInitialization() { + try { + postInitThunks foreach (f => addThunk(f())) + runThunks() + } catch { + case ex: Throwable => + initError = stackTraceString(ex) + throw ex + } finally { + initIsComplete = true + + if (isAsync) { + asyncMessage("[info] total init time: " + elapsed() + " s.") + withLock(initLoopCondition.signal()) + } + } + } + + def initializeSpark() { + intp.beQuietDuring { + command(""" + @transient lazy val sc = spark.repl.Main.interp.createSparkContext(); + """) + command("import spark.SparkContext._"); + } + // echo("Type in expressions to have them evaluated.") + // echo("Type :help for more information.") + } + + // code to be executed only after the interpreter is initialized + // and the lazy val `global` can be accessed without risk of deadlock. + private var pendingThunks: List[() => Unit] = Nil + protected def addThunk(body: => Unit) = synchronized { + pendingThunks :+= (() => body) + } + protected def runThunks(): Unit = synchronized { + if (pendingThunks.nonEmpty) + println("Clearing " + pendingThunks.size + " thunks.") + + while (pendingThunks.nonEmpty) { + val thunk = pendingThunks.head + pendingThunks = pendingThunks.tail + thunk() + } + } +} diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/spark/repl/SparkIMain.scala index 43b6a6c950030..845362823339a 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/spark/repl/SparkIMain.scala @@ -1,5 +1,5 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Martin Odersky */ @@ -9,304 +9,334 @@ import scala.tools.nsc._ import scala.tools.nsc.interpreter._ import Predef.{ println => _, _ } -import java.io.{ PrintWriter } -import java.lang.reflect +import util.stringFromWriter +import scala.reflect.internal.util._ import java.net.URL -import util.{ Set => _, _ } -import io.{ AbstractFile, PlainFile, VirtualDirectory } -import reporters.{ ConsoleReporter, Reporter } -import symtab.{ Flags, Names } -import scala.tools.nsc.interpreter.{ Results => IR } +import scala.sys.BooleanProp +import io.{AbstractFile, PlainFile, VirtualDirectory} + +import reporters._ +import symtab.Flags +import scala.reflect.internal.Names import scala.tools.util.PathResolver -import scala.tools.nsc.util.{ ScalaClassLoader, Exceptional } +import scala.tools.nsc.util.ScalaClassLoader import ScalaClassLoader.URLClassLoader -import Exceptional.unwrap +import scala.tools.nsc.util.Exceptional.unwrap import scala.collection.{ mutable, immutable } -import scala.PartialFunction.{ cond, condOpt } import scala.util.control.Exception.{ ultimately } -import scala.reflect.NameTransformer import SparkIMain._ +import java.util.concurrent.Future +import typechecker.Analyzer +import scala.language.implicitConversions +import scala.reflect.runtime.{ universe => ru } +import scala.reflect.{ ClassTag, classTag } +import scala.tools.reflect.StdRuntimeTags._ +import scala.util.control.ControlThrowable +import util.stackTraceString import spark.HttpServer import spark.Utils import spark.SparkEnv +import spark.Logging + +// /** directory to save .class files to */ +// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { +// private def pp(root: AbstractFile, indentLevel: Int) { +// val spaces = " " * indentLevel +// out.println(spaces + root.name) +// if (root.isDirectory) +// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) +// } +// // print the contents hierarchically +// def show() = pp(this, 0) +// } + + /** An interpreter for Scala code. + * + * The main public entry points are compile(), interpret(), and bind(). + * The compile() method loads a complete Scala file. The interpret() method + * executes one line of Scala code at the request of the user. The bind() + * method binds an object to a variable that can then be used by later + * interpreted code. + * + * The overall approach is based on compiling the requested code and then + * using a Java classloader and Java reflection to run the code + * and access its results. + * + * In more detail, a single compiler instance is used + * to accumulate all successfully compiled or interpreted Scala code. To + * "interpret" a line of code, the compiler generates a fresh object that + * includes the line of code and which has public member(s) to export + * all variables defined by that code. To extract the result of an + * interpreted line to show the user, a second "result object" is created + * which imports the variables exported by the above object and then + * exports members called "$eval" and "$print". To accomodate user expressions + * that read from variables or methods defined in previous statements, "import" + * statements are used. + * + * This interpreter shares the strengths and weaknesses of using the + * full compiler-to-Java. The main strength is that interpreted code + * behaves exactly as does compiled code, including running at full speed. + * The main weakness is that redefining classes and methods is not handled + * properly, because rebinding at the Java level is technically difficult. + * + * @author Moez A. Abdel-Gawad + * @author Lex Spoon + */ + class SparkIMain(initialSettings: Settings, val out: JPrintWriter) extends SparkImports with Logging { + imain => -/** An interpreter for Scala code. - * - * The main public entry points are compile(), interpret(), and bind(). - * The compile() method loads a complete Scala file. The interpret() method - * executes one line of Scala code at the request of the user. The bind() - * method binds an object to a variable that can then be used by later - * interpreted code. - * - * The overall approach is based on compiling the requested code and then - * using a Java classloader and Java reflection to run the code - * and access its results. - * - * In more detail, a single compiler instance is used - * to accumulate all successfully compiled or interpreted Scala code. To - * "interpret" a line of code, the compiler generates a fresh object that - * includes the line of code and which has public member(s) to export - * all variables defined by that code. To extract the result of an - * interpreted line to show the user, a second "result object" is created - * which imports the variables exported by the above object and then - * exports a single member named "$export". To accomodate user expressions - * that read from variables or methods defined in previous statements, "import" - * statements are used. - * - * This interpreter shares the strengths and weaknesses of using the - * full compiler-to-Java. The main strength is that interpreted code - * behaves exactly as does compiled code, including running at full speed. - * The main weakness is that redefining classes and methods is not handled - * properly, because rebinding at the Java level is technically difficult. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - */ -class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends SparkImports { - imain => - - /** construct an interpreter that reports to Console */ - def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) - def this() = this(new Settings()) - - /** whether to print out result lines */ - var printResults: Boolean = true - - /** whether to print errors */ - var totalSilence: Boolean = false - - private val RESULT_OBJECT_PREFIX = "RequestResult$" - - lazy val formatting: Formatting = new Formatting { - val prompt = Properties.shellPromptString - } - import formatting._ - - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - - /** Local directory to save .class files too */ - val outputDir = { - val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) - Utils.createTempDir(rootDir) - } - if (SPARK_DEBUG_REPL) { - echo("Output directory: " + outputDir) - } - /** Scala compiler virtual directory for outputDir */ - val virtualDirectory = new PlainFile(outputDir) + val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** Jetty server that will serve our classes to worker nodes */ - val classServer = new HttpServer(outputDir) + /** Local directory to save .class files too */ + val outputDir = { + val tmp = System.getProperty("java.io.tmpdir") + val rootDir = System.getProperty("spark.repl.classdir", tmp) + Utils.createTempDir(rootDir) + } + if (SPARK_DEBUG_REPL) { + echo("Output directory: " + outputDir) + } - // Start the classServer and store its URI in a spark system property - // (which will be passed to executors so that they can connect to it) - classServer.start() - System.setProperty("spark.repl.class.uri", classServer.uri) - if (SPARK_DEBUG_REPL) { - echo("Class server started, URI = " + classServer.uri) - } + val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles + val classServer = new HttpServer(outputDir) /** Jetty server that will serve our classes to worker nodes */ + private var currentSettings: Settings = initialSettings + var printResults = true // whether to print result lines + var totalSilence = false // whether to print anything + private var _initializeComplete = false // compiler is initialized + private var _isInitialized: Future[Boolean] = null // set up initialization future + private var bindExceptions = true // whether to bind the lastException variable + private var _executionWrapper = "" // code to be wrapped around all lines + + + // Start the classServer and store its URI in a spark system property + // (which will be passed to executors so that they can connect to it) + classServer.start() + System.setProperty("spark.repl.class.uri", classServer.uri) + if (SPARK_DEBUG_REPL) { + echo("Class server started, URI = " + classServer.uri) + } + + /** We're going to go to some trouble to initialize the compiler asynchronously. + * It's critical that nothing call into it until it's been initialized or we will + * run into unrecoverable issues, but the perceived repl startup time goes + * through the roof if we wait for it. So we initialize it with a future and + * use a lazy val to ensure that any attempt to use the compiler object waits + * on the future. + */ + private var _classLoader: AbstractFileClassLoader = null // active classloader + private val _compiler: Global = newCompiler(settings, reporter) // our private compiler - /* - // directory to save .class files to - val virtualDirectory = new VirtualDirectory("(memory)", None) { - private def pp(root: io.AbstractFile, indentLevel: Int) { - val spaces = " " * indentLevel - out.println(spaces + root.name) - if (root.isDirectory) - root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) + private val nextReqId = { + var counter = 0 + () => { counter += 1 ; counter } } - // print the contents hierarchically - def show() = pp(this, 0) - } - */ - - /** reporter */ - lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) - import reporter.{ printMessage, withoutTruncating } - - // not sure if we have some motivation to print directly to console - private def echo(msg: String) { Console println msg } - - // protected def defaultImports: List[String] = List("_root_.scala.sys.exit") - - /** We're going to go to some trouble to initialize the compiler asynchronously. - * It's critical that nothing call into it until it's been initialized or we will - * run into unrecoverable issues, but the perceived repl startup time goes - * through the roof if we wait for it. So we initialize it with a future and - * use a lazy val to ensure that any attempt to use the compiler object waits - * on the future. - */ - private val _compiler: Global = newCompiler(settings, reporter) - private var _initializeComplete = false - def isInitializeComplete = _initializeComplete - - private def _initialize(): Boolean = { - val source = """ - |class $repl_$init { - | List(1) map (_ + 1) - |} - |""".stripMargin - - val result = try { - new _compiler.Run() compileSources List(new BatchSourceFile("", source)) - if (isReplDebug || settings.debug.value) { - // Can't use printMessage here, it deadlocks - Console.println("Repl compiler initialized.") - } - // addImports(defaultImports: _*) - true - } - catch { - case x: AbstractMethodError => - printMessage(""" - |Failed to initialize compiler: abstract method error. - |This is most often remedied by a full clean and recompile. - |""".stripMargin - ) - x.printStackTrace() - false - case x: MissingRequirementError => printMessage(""" - |Failed to initialize compiler: %s not found. - |** Note that as of 2.8 scala does not assume use of the java classpath. - |** For the old behavior pass -usejavacp to scala, or if using a Settings - |** object programatically, settings.usejavacp.value = true.""".stripMargin.format(x.req) + + def compilerClasspath: Seq[URL] = ( + if (isInitializeComplete) global.classPath.asURLs + else new PathResolver(settings).result.asURLs // the compiler's classpath ) - false + def settings = currentSettings + def mostRecentLine = prevRequestList match { + case Nil => "" + case req :: _ => req.originalLine + } + // Run the code body with the given boolean settings flipped to true. + def withoutWarnings[T](body: => T): T = beQuietDuring { + val saved = settings.nowarn.value + if (!saved) + settings.nowarn.value = true + + try body + finally if (!saved) settings.nowarn.value = false } - - try result - finally _initializeComplete = result - } - - // set up initialization future - private var _isInitialized: () => Boolean = null - def initialize() = synchronized { - if (_isInitialized == null) - _isInitialized = scala.concurrent.ops future _initialize() - } - /** the public, go through the future compiler */ - lazy val global: Global = { - initialize() + /** construct an interpreter that reports to Console */ + def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) + def this() = this(new Settings()) - // blocks until it is ; false means catastrophic failure - if (_isInitialized()) _compiler - else null - } - @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - lazy val compiler: global.type = global - - import global._ - - object naming extends { - val global: imain.global.type = imain.global - } with Naming { - // make sure we don't overwrite their unwisely named res3 etc. - override def freshUserVarName(): String = { - val name = super.freshUserVarName() - if (definedNameMap contains name) freshUserVarName() - else name + lazy val repllog: Logger = new Logger { + val out: JPrintWriter = imain.out + val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" + val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" + val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" } - } - import naming._ - - // object dossiers extends { - // val intp: imain.type = imain - // } with Dossiers { } - // import dossiers._ - - lazy val memberHandlers = new { - val intp: imain.type = imain - } with SparkMemberHandlers - import memberHandlers._ - - def atPickler[T](op: => T): T = atPhase(currentRun.picklerPhase)(op) - def afterTyper[T](op: => T): T = atPhase(currentRun.typerPhase.next)(op) - - /** Temporarily be quiet */ - def beQuietDuring[T](operation: => T): T = { - val wasPrinting = printResults - ultimately(printResults = wasPrinting) { - if (isReplDebug) echo(">> beQuietDuring") - else printResults = false - - operation + lazy val formatting: Formatting = new Formatting { + val prompt = Properties.shellPromptString } - } - def beSilentDuring[T](operation: => T): T = { - val saved = totalSilence - totalSilence = true - try operation - finally totalSilence = saved - } - - def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - /** whether to bind the lastException variable */ - private var bindLastException = true - - /** A string representing code to be wrapped around all lines. */ - private var _executionWrapper: String = "" - def executionWrapper = _executionWrapper - def setExecutionWrapper(code: String) = _executionWrapper = code - def clearExecutionWrapper() = _executionWrapper = "" - - /** Temporarily stop binding lastException */ - def withoutBindingLastException[T](operation: => T): T = { - val wasBinding = bindLastException - ultimately(bindLastException = wasBinding) { - bindLastException = false - operation + lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + import formatting._ + import reporter.{ printMessage, withoutTruncating } + + // This exists mostly because using the reporter too early leads to deadlock. + private def echo(msg: String) { Console println msg } + private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) + private def _initialize() = { + try { + // todo. if this crashes, REPL will hang + new _compiler.Run() compileSources _initSources + _initializeComplete = true + true + } + catch AbstractOrMissingHandler() + } + private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" + + // argument is a thunk to execute after init is done + def initialize(postInitSignal: => Unit) { + synchronized { + if (_isInitialized == null) { + _isInitialized = io.spawn { + try _initialize() + finally postInitSignal + } + } + } + } + def initializeSynchronous(): Unit = { + if (!isInitializeComplete) { + _initialize() + assert(global != null, global) + } + } + def isInitializeComplete = _initializeComplete + + /** the public, go through the future compiler */ + lazy val global: Global = { + if (isInitializeComplete) _compiler + else { + // If init hasn't been called yet you're on your own. + if (_isInitialized == null) { + logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.") + initialize(()) + } + // // blocks until it is ; false means catastrophic failure + if (_isInitialized.get()) _compiler + else null + } + } + @deprecated("Use `global` for access to the compiler instance.", "2.9.0") + lazy val compiler: global.type = global + + import global._ + import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} + import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} + + implicit class ReplTypeOps(tp: Type) { + def orElse(other: => Type): Type = if (tp ne NoType) tp else other + def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) + } + + // TODO: If we try to make naming a lazy val, we run into big time + // scalac unhappiness with what look like cycles. It has not been easy to + // reduce, but name resolution clearly takes different paths. + object naming extends { + val global: imain.global.type = imain.global + } with Naming { + // make sure we don't overwrite their unwisely named res3 etc. + def freshUserTermName(): TermName = { + val name = newTermName(freshUserVarName()) + if (definedNameMap contains name) freshUserTermName() + else name + } + def isUserTermName(name: Name) = isUserVarName("" + name) + def isInternalTermName(name: Name) = isInternalVarName("" + name) + } + import naming._ + + object deconstruct extends { + val global: imain.global.type = imain.global + } with StructuredTypeStrings + + lazy val memberHandlers = new { + val intp: imain.type = imain + } with SparkMemberHandlers + import memberHandlers._ + + /** Temporarily be quiet */ + def beQuietDuring[T](body: => T): T = { + val saved = printResults + printResults = false + try body + finally printResults = saved + } + def beSilentDuring[T](operation: => T): T = { + val saved = totalSilence + totalSilence = true + try operation + finally totalSilence = saved + } + + def quietRun[T](code: String) = beQuietDuring(interpret(code)) + + + private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { + case t: ControlThrowable => throw t + case t: Throwable => + logDebug(label + ": " + unwrap(t)) + logDebug(stackTraceString(unwrap(t))) + alt + } + /** takes AnyRef because it may be binding a Throwable or an Exceptional */ + + private def withLastExceptionLock[T](body: => T, alt: => T): T = { + assert(bindExceptions, "withLastExceptionLock called incorrectly.") + bindExceptions = false + + try beQuietDuring(body) + catch logAndDiscard("withLastExceptionLock", alt) + finally bindExceptions = true } - } - - protected def createLineManager(): Line.Manager = new Line.Manager - lazy val lineManager = createLineManager() - - /** interpreter settings */ - lazy val isettings = new SparkISettings(this) - - /** Instantiate a compiler. Subclasses can override this to - * change the compiler class used by this interpreter. */ - protected def newCompiler(settings: Settings, reporter: Reporter) = { - settings.outputDirs setSingleOutput virtualDirectory - settings.exposeEmptyPackage.value = true - new Global(settings, reporter) - } - - /** the compiler's classpath, as URL's */ - lazy val compilerClasspath: List[URL] = new PathResolver(settings) asURLs - /* A single class loader is used for all commands interpreted by this Interpreter. + def executionWrapper = _executionWrapper + def setExecutionWrapper(code: String) = _executionWrapper = code + def clearExecutionWrapper() = _executionWrapper = "" + + /** interpreter settings */ + lazy val isettings = new SparkISettings(this) + + /** Instantiate a compiler. Overridable. */ + protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { + settings.outputDirs setSingleOutput virtualDirectory + settings.exposeEmptyPackage.value = true + new Global(settings, reporter) with ReplGlobal { + override def toString: String = "" + } + } + + /** Parent classloader. Overridable. */ + protected def parentClassLoader: ClassLoader = + SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) + + /* A single class loader is used for all commands interpreted by this Interpreter. It would also be possible to create a new class loader for each command to interpret. The advantages of the current approach are: - - Expressions are only evaluated one time. This is especially - significant for I/O, e.g. "val x = Console.readLine" - - The main disadvantage is: - - - Objects, classes, and methods cannot be rebound. Instead, definitions - shadow the old ones, and old code objects refer to the old - definitions. - */ - private var _classLoader: AbstractFileClassLoader = null - def resetClassLoader() = _classLoader = makeClassLoader() - def classLoader: AbstractFileClassLoader = { - if (_classLoader == null) - resetClassLoader() - - _classLoader - } - private def makeClassLoader(): AbstractFileClassLoader = { - val parent = - if (parentClassLoader == null) ScalaClassLoader fromURLs compilerClasspath - else new URLClassLoader(compilerClasspath, parentClassLoader) + - Expressions are only evaluated one time. This is especially + significant for I/O, e.g. "val x = Console.readLine" + + The main disadvantage is: - new AbstractFileClassLoader(virtualDirectory, parent) { + - Objects, classes, and methods cannot be rebound. Instead, definitions + shadow the old ones, and old code objects refer to the old + definitions. + */ + def resetClassLoader() = { + logDebug("Setting new classloader: was " + _classLoader) + _classLoader = null + ensureClassLoader() + } + final def ensureClassLoader() { + if (_classLoader == null) + _classLoader = makeClassLoader() + } + def classLoader: AbstractFileClassLoader = { + ensureClassLoader() + _classLoader + } + private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) { /** Overridden here to try translating a simple name to the generated * class name if the original attempt fails. This method is used by * getResourceAsStream as well as findClass. @@ -314,223 +344,300 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends override protected def findAbstractFile(name: String): AbstractFile = { super.findAbstractFile(name) match { // deadlocks on startup if we try to translate names too early - case null if isInitializeComplete => generatedName(name) map (x => super.findAbstractFile(x)) orNull - case file => file + case null if isInitializeComplete => + generatedName(name) map (x => super.findAbstractFile(x)) orNull + case file => + file } } } - } - private def loadByName(s: String): JClass = - (classLoader tryToInitializeClass s) getOrElse sys.error("Failed to load expected class: '" + s + "'") - - protected def parentClassLoader: ClassLoader = - SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) - - def getInterpreterClassLoader() = classLoader - - // Set the current Java "context" class loader to this interpreter's class loader - def setContextClassLoader() = classLoader.setAsContext() - - /** Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - */ - def generatedName(simpleName: String): Option[String] = { - if (simpleName endsWith "$") optFlatName(simpleName.init) map (_ + "$") - else optFlatName(simpleName) - } - def flatName(id: String) = optFlatName(id) getOrElse id - def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) - - def allDefinedNames = definedNameMap.keys.toList sortBy (_.toString) - def pathToType(id: String): String = pathToName(newTypeName(id)) - def pathToTerm(id: String): String = pathToName(newTermName(id)) - def pathToName(name: Name): String = { - if (definedNameMap contains name) - definedNameMap(name) fullPath name - else name.toString - } + private def makeClassLoader(): AbstractFileClassLoader = + new TranslatingClassLoader(parentClassLoader match { + case null => ScalaClassLoader fromURLs compilerClasspath + case p => new URLClassLoader(compilerClasspath, p) + }) + + def getInterpreterClassLoader() = classLoader + + // Set the current Java "context" class loader to this interpreter's class loader + def setContextClassLoader() = classLoader.setAsContext() + + /** Given a simple repl-defined name, returns the real name of + * the class representing it, e.g. for "Bippy" it may return + * {{{ + * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy + * }}} + */ + def generatedName(simpleName: String): Option[String] = { + if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) + else optFlatName(simpleName) + } + def flatName(id: String) = optFlatName(id) getOrElse id + def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + + def allDefinedNames = definedNameMap.keys.toList.sorted + def pathToType(id: String): String = pathToName(newTypeName(id)) + def pathToTerm(id: String): String = pathToName(newTermName(id)) + def pathToName(name: Name): String = { + if (definedNameMap contains name) + definedNameMap(name) fullPath name + else name.toString + } - /** Most recent tree handled which wasn't wholly synthetic. */ - private def mostRecentlyHandledTree: Option[Tree] = { - prevRequests.reverse foreach { req => - req.handlers.reverse foreach { - case x: MemberDefHandler if x.definesValue && !isInternalVarName(x.name.toString) => return Some(x.member) - case _ => () + /** Most recent tree handled which wasn't wholly synthetic. */ + private def mostRecentlyHandledTree: Option[Tree] = { + prevRequests.reverse foreach { req => + req.handlers.reverse foreach { + case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) + case _ => () + } } + None } - None - } - - /** Stubs for work in progress. */ - def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { - for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { - DBG("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) + + /** Stubs for work in progress. */ + def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { + for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { + logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) + } } - } - def handleTermRedefinition(name: TermName, old: Request, req: Request) = { - for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { - // Printing the types here has a tendency to cause assertion errors, like - // assertion failed: fatal: has owner value x, but a class owner is required - // so DBG is by-name now to keep it in the family. (It also traps the assertion error, - // but we don't want to unnecessarily risk hosing the compiler's internal state.) - DBG("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + def handleTermRedefinition(name: TermName, old: Request, req: Request) = { + for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { + // Printing the types here has a tendency to cause assertion errors, like + // assertion failed: fatal: has owner value x, but a class owner is required + // so DBG is by-name now to keep it in the family. (It also traps the assertion error, + // but we don't want to unnecessarily risk hosing the compiler's internal state.) + logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + } } - } - def recordRequest(req: Request) { - if (req == null || referencedNameMap == null) - return - - prevRequests += req - req.referencedNames foreach (x => referencedNameMap(x) = req) - - // warning about serially defining companions. It'd be easy - // enough to just redefine them together but that may not always - // be what people want so I'm waiting until I can do it better. - if (!settings.nowarnings.value) { + + def recordRequest(req: Request) { + if (req == null || referencedNameMap == null) + return + + prevRequests += req + req.referencedNames foreach (x => referencedNameMap(x) = req) + + // warning about serially defining companions. It'd be easy + // enough to just redefine them together but that may not always + // be what people want so I'm waiting until I can do it better. for { name <- req.definedNames filterNot (x => req.definedNames contains x.companionName) oldReq <- definedNameMap get name.companionName newSym <- req.definedSymbols get name oldSym <- oldReq.definedSymbols get name.companionName + if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule } } { - printMessage("warning: previously defined %s is not a companion to %s.".format(oldSym, newSym)) - printMessage("Companions must be defined together; you may wish to use :paste mode for this.") + afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")) + replwarn("Companions must be defined together; you may wish to use :paste mode for this.") } - } - - // Updating the defined name map - req.definedNames foreach { name => - if (definedNameMap contains name) { - if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) - else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + + // Updating the defined name map + req.definedNames foreach { name => + if (definedNameMap contains name) { + if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) + else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + } + definedNameMap(name) = req } - definedNameMap(name) = req } - } - /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ - def parse(line: String): Option[List[Tree]] = { - var justNeedsMore = false - reporter.withIncompleteHandler((pos,msg) => {justNeedsMore = true}) { - // simple parse: just parse it, nothing else - def simpleParse(code: String): List[Tree] = { - reporter.reset() - val unit = new CompilationUnit(new BatchSourceFile("", code)) - val scanner = new syntaxAnalyzer.UnitParser(unit) - - scanner.templateStatSeq(false)._2 - } - val trees = simpleParse(line) - - if (reporter.hasErrors) Some(Nil) // the result did not parse, so stop - else if (justNeedsMore) None - else Some(trees) + def replwarn(msg: => String) { + if (!settings.nowarnings.value) + printMessage(msg) } - } - - def isParseable(line: String): Boolean = { - beSilentDuring { - parse(line) match { - case Some(xs) => xs.nonEmpty // parses as-is - case None => true // incomplete + + def isParseable(line: String): Boolean = { + beSilentDuring { + try parse(line) match { + case Some(xs) => xs.nonEmpty // parses as-is + case None => true // incomplete + } + catch { case x: Exception => // crashed the compiler + replwarn("Exception in isParseable(\"" + line + "\"): " + x) + false + } } } + + def compileSourcesKeepingRun(sources: SourceFile*) = { + val run = new Run() + reporter.reset() + run compileSources sources.toList + (!reporter.hasErrors, run) + } + + /** Compile an nsc SourceFile. Returns true if there are + * no compilation errors, or false otherwise. + */ + def compileSources(sources: SourceFile*): Boolean = + compileSourcesKeepingRun(sources: _*)._1 + + /** Compile a string. Returns true if there are no + * compilation errors, or false otherwise. + */ + def compileString(code: String): Boolean = + compileSources(new BatchSourceFile(" - - - - - - - - - - - - - - - - - - - - - -
- - - - - - - -
-
- - - - - - - -
- - -
- - - - - -
- - This repository - - -
-
- -
- - -
This repository
-
- -
- - -
All repositories
-
- -
-
-
- - - - - - - - -
-
- -
-
- - - - - -
- -
-
- - - - -

- public - - - - - / - incubator-spark - - - Octocat-spinner-32 - - - - mirrored from git://git.apache.org/incubator-spark.git - -

-
-
- -
- - -
- -
- - -
-
- -
- - - -
-
- -
- - - - -
-

HTTPS clone URL

-
- - - -
-
- - - -
-

Subversion checkout URL

-
- - - -
-
- - -

You can clone with - HTTPS, - or Subversion. - - - - - -

- - - - - - Download ZIP - -
-
- -
- - - - - - - - - -
- - -
- - - branch: - master - - -
- -
-
- Switch branches/tags - -
- -
-
- -
-
- -
-
- -
- -
- - -
- - akka-actors -
-
- - arthur -
-
- - branch-0.5 -
-
- - branch-0.6 -
-
- - branch-0.7 -
-
- - branch-0.8 -
- -
- - dev -
-
- - formatting -
-
- - hive -
-
- - java-api -
-
- - master -
-
- - mesos-0.9 -
-
- - mos-bt -
-
- - mos-bt-dev -
-
- - mos-bt-topo -
-
- - mos-shuffle -
- - - - - -
- - old-mesos -
-
- - old-rdds -
-
- - perf -
- -
- - rxin -
-
- - scala-2.8 -
-
- - scala-2.9 -
-
- - scala-2.10 -
- -
- - shuffle-fix -
-
- - sparkplug -
-
- - streaming -
-
- - td-checksum -
-
- - td-rdd-save -
- - -
- - yarn -
-
- -
Nothing to show
-
- -
-
- - - - - -
- - v0.7.2 -
-
- - v0.7.1 -
- -
- - v0.7.0 -
-
- - v0.6.2 -
-
- - v0.6.1 -
-
- - v0.6.0-yarn -
-
- - v0.6.0 -
-
- - v0.5.2 -
-
- - v0.5.1 -
-
- - v0.5.0 -
- -
- - alpha-0.2 -
-
- - alpha-0.1 -
- - -
- -
Nothing to show
-
- -
-
-
- - -
- - - -
- - - -
- Cleanup -
- -
-

15 contributors

- - - - - - - - - - - - - - - - - -
- -
- -
-
-
-
- - file - 232 lines (228 sloc) - 7.715 kb -
- - -
-
- - - - - -
- 1 -2 -3 -4 -5 -6 -7 -8 -9 -10 -11 -12 -13 -14 -15 -16 -17 -18 -19 -20 -21 -22 -23 -24 -25 -26 -27 -28 -29 -30 -31 -32 -33 -34 -35 -36 -37 -38 -39 -40 -41 -42 -43 -44 -45 -46 -47 -48 -49 -50 -51 -52 -53 -54 -55 -56 -57 -58 -59 -60 -61 -62 -63 -64 -65 -66 -67 -68 -69 -70 -71 -72 -73 -74 -75 -76 -77 -78 -79 -80 -81 -82 -83 -84 -85 -86 -87 -88 -89 -90 -91 -92 -93 -94 -95 -96 -97 -98 -99 -100 -101 -102 -103 -104 -105 -106 -107 -108 -109 -110 -111 -112 -113 -114 -115 -116 -117 -118 -119 -120 -121 -122 -123 -124 -125 -126 -127 -128 -129 -130 -131 -132 -133 -134 -135 -136 -137 -138 -139 -140 -141 -142 -143 -144 -145 -146 -147 -148 -149 -150 -151 -152 -153 -154 -155 -156 -157 -158 -159 -160 -161 -162 -163 -164 -165 -166 -167 -168 -169 -170 -171 -172 -173 -174 -175 -176 -177 -178 -179 -180 -181 -182 -183 -184 -185 -186 -187 -188 -189 -190 -191 -192 -193 -194 -195 -196 -197 -198 -199 -200 -201 -202 -203 -204 -205 -206 -207 -208 -209 -210 -211 -212 -213 -214 -215 -216 -217 -218 -219 -220 -221 -222 -223 -224 -225 -226 -227 -228 -229 -230 -231 - - -
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
  <modelVersion>4.0.0</modelVersion>
  <parent>
    <groupId>org.apache.spark</groupId>
    <artifactId>spark-parent</artifactId>
    <version>0.9.0-incubating-SNAPSHOT</version>
    <relativePath>../pom.xml</relativePath>
  </parent>

  <groupId>org.apache.spark</groupId>
  <artifactId>spark-core_2.10</artifactId>
  <packaging>jar</packaging>
  <name>Spark Project Core</name>
  <url>http://spark.incubator.apache.org/</url>

  <dependencies>
    <dependency>
      <groupId>org.apache.hadoop</groupId>
      <artifactId>hadoop-client</artifactId>
    </dependency>
    <dependency>
      <groupId>net.java.dev.jets3t</groupId>
      <artifactId>jets3t</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.avro</groupId>
      <artifactId>avro</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.avro</groupId>
      <artifactId>avro-ipc</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.zookeeper</groupId>
      <artifactId>zookeeper</artifactId>
    </dependency>
    <dependency>
      <groupId>org.eclipse.jetty</groupId>
      <artifactId>jetty-server</artifactId>
    </dependency>
    <dependency>
      <groupId>com.google.guava</groupId>
      <artifactId>guava</artifactId>
    </dependency>
    <dependency>
      <groupId>com.google.code.findbugs</groupId>
      <artifactId>jsr305</artifactId>
    </dependency>
    <dependency>
      <groupId>org.slf4j</groupId>
      <artifactId>slf4j-api</artifactId>
    </dependency>
    <dependency>
      <groupId>com.ning</groupId>
      <artifactId>compress-lzf</artifactId>
    </dependency>
    <dependency>
      <groupId>org.xerial.snappy</groupId>
      <artifactId>snappy-java</artifactId>
    </dependency>
    <dependency>
      <groupId>org.ow2.asm</groupId>
      <artifactId>asm</artifactId>
    </dependency>
    <dependency>
      <groupId>com.twitter</groupId>
      <artifactId>chill_${scala.binary.version}</artifactId>
      <version>0.3.1</version>
    </dependency>
    <dependency>
      <groupId>com.twitter</groupId>
      <artifactId>chill-java</artifactId>
      <version>0.3.1</version>
    </dependency>
    <dependency>
      <groupId>${akka.group}</groupId>
      <artifactId>akka-remote_${scala.binary.version}</artifactId>
    </dependency>
    <dependency>
      <groupId>${akka.group}</groupId>
      <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
    </dependency>
    <dependency>
      <groupId>org.scala-lang</groupId>
      <artifactId>scala-library</artifactId>
    </dependency>
    <dependency>
      <groupId>net.liftweb</groupId>
      <artifactId>lift-json_${scala.binary.version}</artifactId>
    </dependency>
    <dependency>
      <groupId>it.unimi.dsi</groupId>
      <artifactId>fastutil</artifactId>
    </dependency>
    <dependency>
      <groupId>colt</groupId>
      <artifactId>colt</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.mesos</groupId>
      <artifactId>mesos</artifactId>
    </dependency>
    <dependency>
      <groupId>io.netty</groupId>
      <artifactId>netty-all</artifactId>
    </dependency>
    <dependency>
      <groupId>log4j</groupId>
      <artifactId>log4j</artifactId>
    </dependency>
    <dependency>
      <groupId>com.codahale.metrics</groupId>
      <artifactId>metrics-core</artifactId>
    </dependency>
    <dependency>
      <groupId>com.codahale.metrics</groupId>
      <artifactId>metrics-jvm</artifactId>
    </dependency>
    <dependency>
      <groupId>com.codahale.metrics</groupId>
      <artifactId>metrics-json</artifactId>
    </dependency>
    <dependency>
      <groupId>com.codahale.metrics</groupId>
      <artifactId>metrics-ganglia</artifactId>
    </dependency>
    <dependency>
      <groupId>com.codahale.metrics</groupId>
      <artifactId>metrics-graphite</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.derby</groupId>
      <artifactId>derby</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>commons-io</groupId>
      <artifactId>commons-io</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>org.scalatest</groupId>
      <artifactId>scalatest_${scala.binary.version}</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>org.scalacheck</groupId>
      <artifactId>scalacheck_${scala.binary.version}</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>org.easymock</groupId>
      <artifactId>easymock</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>com.novocode</groupId>
      <artifactId>junit-interface</artifactId>
      <scope>test</scope>
    </dependency>
    <dependency>
      <groupId>org.slf4j</groupId>
      <artifactId>slf4j-log4j12</artifactId>
      <scope>test</scope>
    </dependency>
  </dependencies>
  <build>
    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
    <plugins>
      <plugin>
        <groupId>org.apache.maven.plugins</groupId>
        <artifactId>maven-antrun-plugin</artifactId>
        <executions>
          <execution>
            <phase>test</phase>
            <goals>
              <goal>run</goal>
            </goals>
            <configuration>
              <exportAntProperties>true</exportAntProperties>
              <tasks>
                <property name="spark.classpath" refid="maven.test.classpath" />
                <property environment="env" />
                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">
                  <condition>
                    <not>
                      <or>
                        <isset property="env.SCALA_HOME" />
                        <isset property="env.SCALA_LIBRARY_PATH" />
                      </or>
                    </not>
                  </condition>
                </fail>
              </tasks>
            </configuration>
          </execution>
        </executions>
      </plugin>
      <plugin>
        <groupId>org.scalatest</groupId>
        <artifactId>scalatest-maven-plugin</artifactId>
        <configuration>
          <environmentVariables>
            <SPARK_HOME>${basedir}/..</SPARK_HOME>
            <SPARK_TESTING>1</SPARK_TESTING>
            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
          </environmentVariables>
        </configuration>
      </plugin>
    </plugins>
  </build>
</project>
-
-
- -
-
- - - - -
- -
- -
-
- - -
- -
- -
- - -
-
-
- -
-
-
-
-
-
- -
- - - -
- - - Something went wrong with that request. Please try again. -
- - - - + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-core_2.10 + jar + Spark Project Core + http://spark.incubator.apache.org/ + + + + org.apache.hadoop + hadoop-client + + + net.java.dev.jets3t + jets3t + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.apache.zookeeper + zookeeper + + + org.eclipse.jetty + jetty-server + + + com.google.guava + guava + + + com.google.code.findbugs + jsr305 + + + org.slf4j + slf4j-api + + + com.ning + compress-lzf + + + org.xerial.snappy + snappy-java + + + org.ow2.asm + asm + + + com.twitter + chill_${scala.binary.version} + 0.3.1 + + + com.twitter + chill-java + 0.3.1 + + + ${akka.group} + akka-remote_${scala.binary.version} + + + ${akka.group} + akka-slf4j_${scala.binary.version} + + + org.scala-lang + scala-library + + + net.liftweb + lift-json_${scala.binary.version} + + + it.unimi.dsi + fastutil + + + colt + colt + + + org.apache.mesos + mesos + + + io.netty + netty-all + + + log4j + log4j + + + com.clearspring.analytics + stream + + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + + + com.codahale.metrics + metrics-json + + + com.codahale.metrics + metrics-ganglia + + + com.codahale.metrics + metrics-graphite + + + org.apache.derby + derby + test + + + commons-io + commons-io + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.easymock + easymock + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/.. + 1 + ${spark.classpath} + + + + + + diff --git a/pom.xml b/pom.xml index 6545c82b31709..3a8eb882ccd21 100644 --- a/pom.xml +++ b/pom.xml @@ -200,16 +200,16 @@ asm 4.0
- com.clearspring.analytics stream 2.4.0 + com.google.protobuf protobuf-java From 83dfa1666487a4772c95fea21fde0d47471e063d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 Dec 2013 20:02:05 -0800 Subject: [PATCH 1110/1571] Address Patrick's and Reynold's comments --- .../collection/ExternalAppendOnlyMap.scala | 120 +++++++++++------- 1 file changed, 71 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 492b4fc7c6205..311405f0cf388 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -20,6 +20,8 @@ package org.apache.spark.util.collection import java.io._ import java.util.Comparator +import it.unimi.dsi.fastutil.io.FastBufferedInputStream + import scala.collection.mutable.{ArrayBuffer, PriorityQueue} import scala.reflect.ClassTag @@ -53,7 +55,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, - Predef.identity, serializer, diskBlockManager) + identity, serializer, diskBlockManager) } else { // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) @@ -111,9 +113,10 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( import SpillableAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, G] - private val oldMaps = new ArrayBuffer[DiskKGIterator] + private val spilledMaps = new ArrayBuffer[DiskIterator] private val memoryThresholdMB = { + // TODO: Turn this into a fraction of memory per reducer val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent @@ -152,31 +155,37 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( writer.close() } currentMap = new SizeTrackingAppendOnlyMap[K, G] - oldMaps.append(new DiskKGIterator(file)) + spilledMaps.append(new DiskIterator(file)) } override def iterator: Iterator[(K, C)] = { - if (oldMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { + if (spilledMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { currentMap.iterator.asInstanceOf[Iterator[(K, C)]] } else { new ExternalIterator() } } - // An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs + /** An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs. */ private class ExternalIterator extends Iterator[(K, C)] { - val mergeHeap = new PriorityQueue[KGITuple] - val inputStreams = oldMaps ++ Seq(currentMap.destructiveSortedIterator(comparator)) - // Invariant: size of mergeHeap == number of input streams + // A fixed-size queue that maintains a buffer for each stream we are currently merging + val mergeHeap = new PriorityQueue[StreamBuffer] + + // Input streams are derived both from the in-memory map and spilled maps on disk + // The in-memory map is sorted in place, while the spilled maps are already in sorted order + val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps + inputStreams.foreach{ it => - val kgPairs = readFromIterator(it) - mergeHeap.enqueue(KGITuple(it, kgPairs)) + val kgPairs = getMorePairs(it) + mergeHeap.enqueue(StreamBuffer(it, kgPairs)) } - // Read from the given iterator until a key of different hash is retrieved. - // The resulting ArrayBuffer includes this key, and is ordered by key hash. - def readFromIterator(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { + /** + * Fetch from the given iterator until a key of different hash is retrieved. In the + * event of key hash collisions, this ensures no pairs are hidden from being merged. + */ + def getMorePairs(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { val kgPairs = new ArrayBuffer[(K, G)] if (it.hasNext) { var kg = it.next() @@ -190,20 +199,26 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( kgPairs } - // From the given KGITuple, remove the (K, G) pair with K = key and merge it into baseGroup - def mergeIntoGroup(key: K, baseGroup: G, kgi: KGITuple): G = { - kgi.pairs.zipWithIndex.foreach { case ((k, g), i) => + /** + * If the given buffer contains a value for the given key, merge that value into + * baseGroup and remove the corresponding (K, G) pair from the buffer + */ + def mergeIfKeyExists(key: K, baseGroup: G, buffer: StreamBuffer): G = { + var i = 0 + while (i < buffer.pairs.size) { + val (k, g) = buffer.pairs(i) if (k == key) { - kgi.pairs.remove(i) + buffer.pairs.remove(i) return mergeGroups(baseGroup, g) } + i += 1 } baseGroup } override def hasNext: Boolean = { - mergeHeap.foreach{ kgi => - if (!kgi.pairs.isEmpty) { + mergeHeap.foreach{ buffer => + if (!buffer.pairs.isEmpty) { return true } } @@ -211,66 +226,74 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } override def next(): (K, C) = { - val minKGI = mergeHeap.dequeue() - val (minPairs, minHash) = (minKGI.pairs, minKGI.minHash) + // Select a return key from the StreamBuffer that holds the lowest key hash + val minBuffer = mergeHeap.dequeue() + val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) if (minPairs.length == 0) { - // Should only happen when hasNext is false + // Should only happen when no other stream buffers have any pairs left throw new NoSuchElementException } - - // Select a return key with the minimum hash var (minKey, minGroup) = minPairs.remove(0) assert(minKey.hashCode() == minHash) - // Merge all other KGITuple's with the same minHash - val dequeuedKGI = ArrayBuffer[KGITuple](minKGI) - while (!mergeHeap.isEmpty && mergeHeap.head.minHash == minHash) { - val newKGI = mergeHeap.dequeue() - minGroup = mergeIntoGroup(minKey, minGroup, newKGI) - dequeuedKGI += newKGI + // For all other streams that may have this key (i.e. have the same minimum key hash), + // merge in the corresponding value (if any) from that stream + val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) + while (!mergeHeap.isEmpty && mergeHeap.head.minKeyHash == minHash) { + val newBuffer = mergeHeap.dequeue() + minGroup = mergeIfKeyExists(minKey, minGroup, newBuffer) + mergedBuffers += newBuffer } - // Repopulate and add back all dequeued KGI to mergeHeap - dequeuedKGI.foreach { kgi => - if (kgi.pairs.length == 0) { - kgi.pairs ++= readFromIterator(kgi.iterator) + // Repopulate each visited stream buffer and add it back to the merge heap + mergedBuffers.foreach { buffer => + if (buffer.pairs.length == 0) { + buffer.pairs ++= getMorePairs(buffer.iterator) } - mergeHeap.enqueue(kgi) + mergeHeap.enqueue(buffer) } (minKey, createCombiner(minGroup)) } - case class KGITuple(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) - extends Comparable[KGITuple] { - - // Invariant: pairs are ordered by key hash - def minHash: Int = { + /** + * A buffer for streaming from a map iterator (in-memory or on-disk) sorted by key hash. + * Each buffer maintains the lowest-ordered keys in the corresponding iterator. Due to + * hash collisions, it is possible for multiple keys to be "tied" for being the lowest. + * + * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. + */ + case class StreamBuffer(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) + extends Comparable[StreamBuffer] { + + def minKeyHash: Int = { if (pairs.length > 0){ + // pairs are already sorted by key hash pairs(0)._1.hashCode() } else { Int.MaxValue } } - override def compareTo(other: KGITuple): Int = { - // mutable.PriorityQueue dequeues the max, not the min - -minHash.compareTo(other.minHash) + override def compareTo(other: StreamBuffer): Int = { + // minus sign because mutable.PriorityQueue dequeues the max, not the min + -minKeyHash.compareTo(other.minKeyHash) } } } // Iterate through (K, G) pairs in sorted order from an on-disk map - private class DiskKGIterator(file: File) extends Iterator[(K, G)] { - val fstream = new FileInputStream(file) - val dstream = ser.deserializeStream(fstream) + private class DiskIterator(file: File) extends Iterator[(K, G)] { + val fileStream = new FileInputStream(file) + val bufferedStream = new FastBufferedInputStream(fileStream) + val deserializeStream = ser.deserializeStream(bufferedStream) var nextItem: Option[(K, G)] = None var eof = false def readNextItem(): Option[(K, G)] = { if (!eof) { try { - return Some(dstream.readObject().asInstanceOf[(K, G)]) + return Some(deserializeStream.readObject().asInstanceOf[(K, G)]) } catch { case e: EOFException => eof = true @@ -302,8 +325,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // TODO: Ensure this gets called even if the iterator isn't drained. def cleanup() { - fstream.close() - dstream.close() + deserializeStream.close() file.delete() } } From 2f2524fd11876a19e315f994020877564a8a0df8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 31 Dec 2013 21:37:51 -0800 Subject: [PATCH 1111/1571] Addressing issue in compute where compute is invoked instead of iterator on the parent RDD. --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 8 ++++---- .../src/main/scala/org/apache/spark/graph/VertexRDD.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index b1640bf9ceb02..9aa76c93945f9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -22,9 +22,8 @@ class EdgeRDD[@specialized ED: ClassManifest]( override val partitioner = partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) - override def compute(split: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val edgePartition = partitionsRDD.compute(split, context).next()._2 - edgePartition.iterator + override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { + firstParent[(Pid, EdgePartition[ED])].iterator(part, context).next._2.iterator } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -33,7 +32,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { - super.persist(newLevel) + partitionsRDD.persist(newLevel) this } @@ -45,6 +44,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) : EdgeRDD[ED2] = { +// iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() Iterator(Tuple2(pid, f(ep))) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 671cf496f8a80..c274e342c7566 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -86,7 +86,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): VertexRDD[VD] = { - super.persist(newLevel) + partitionsRDD.persist(newLevel) this } @@ -105,7 +105,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Provide the `RDD[(Vid, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = { - partitionsRDD.compute(part, context).next().iterator + firstParent[VertexPartition[VD]].iterator(part, context).next.iterator } /** From dd6033e6853e32e9de2c910797c7fbc0072e7491 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Thu, 2 Jan 2014 01:38:24 +0800 Subject: [PATCH 1112/1571] Aggregated all sample points to driver without any shuffle --- .../mllib/classification/NaiveBayes.scala | 76 +++++++------------ .../classification/NaiveBayesSuite.scala | 8 +- 2 files changed, 31 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 9fd1adddb0f3b..524300d6aebc5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.classification +import scala.collection.mutable + import org.jblas.DoubleMatrix import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ /** * Model for Naive Bayes Classifiers. @@ -60,62 +61,39 @@ class NaiveBayes private (val lambda: Double = 1.0) * @param data RDD of (label, array of features) pairs. */ def run(data: RDD[LabeledPoint]) = { - // Prepares input data, the shape of resulted RDD is: - // - // label: Int -> (count: Int, features: DoubleMatrix) - // - // The added count field is initialized to 1 to enable the following `foldByKey` transformation. - val mappedData = data.map { case LabeledPoint(label, features) => - label.toInt -> (1, new DoubleMatrix(features.length, 1, features: _*)) - } - - // Gets a map from labels to their corresponding sample point counts and summed feature vectors. - // Shape of resulted RDD is: - // - // label: Int -> (count: Int, summedFeatureVector: DoubleMatrix) + // Aggregates all sample points to driver side to get sample count and summed feature vector + // for each label. The shape of `zeroCombiner` & `aggregated` is: // - // Two tricky parts worth explaining: - // - // 1. Feature vectors are summed with the inplace jblas matrix addition operation, thus we - // chose `foldByKey` instead of `reduceByKey` to avoid modifying original input data. - // - // 2. The zero value passed to `foldByKey` contains a `null` rather than a zero vector because - // the dimension of the feature vector is unknown. Calling `data.first.length` to get the - // dimension is not preferable since it requires an expensive RDD action. - val countsAndSummedFeatures = mappedData.foldByKey((0, null)) { (lhs, rhs) => - if (lhs._1 == 0) { - (rhs._1, new DoubleMatrix().copy(rhs._2)) - } else { - (lhs._1 + rhs._1, lhs._2.addi(rhs._2)) + // label: Int -> (count: Int, featuresSum: DoubleMatrix) + val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)] + val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) => + point match { + case LabeledPoint(label, features) => + val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1))) + val fs = new DoubleMatrix(features.length, 1, features: _*) + combiner += label.toInt -> (count + 1, featuresSum.addi(fs)) } - } - - val collected = countsAndSummedFeatures.mapValues { case (count, summedFeatureVector) => - val p = math.log(count + lambda) - val logDenom = math.log(summedFeatureVector.sum + summedFeatureVector.length * lambda) - val t = summedFeatureVector - var i = 0 - while (i < t.length) { - t.put(i, math.log(t.get(i) + lambda) - logDenom) - i += 1 + }, { (lhs, rhs) => + for ((label, (c, fs)) <- rhs) { + val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1))) + lhs(label) = (count + c, featuresSum.addi(fs)) } - (count, p, t) - }.collectAsMap() - - // Total sample count. Calling `data.count` to get `N` is not preferable since it triggers - // an expensive RDD action - val N = collected.values.map(_._1).sum + lhs + }) - // Kinds of label. - val C = collected.size + // Kinds of label + val C = aggregated.size + // Total sample count + val N = aggregated.values.map(_._1).sum - val logDenom = math.log(N + C * lambda) val pi = new Array[Double](C) val theta = new Array[Array[Double]](C) + val piLogDenom = math.log(N + C * lambda) - for ((label, (_, p, t)) <- collected) { - pi(label) = p - logDenom - theta(label) = t.toArray + for ((label, (count, fs)) <- aggregated) { + val thetaLogDenom = math.log(fs.sum() + fs.length * lambda) + pi(label) = math.log(count + lambda) - piLogDenom + theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom) } new NaiveBayesModel(pi, theta) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 18575f410c53b..b615f76e66cf9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -27,16 +27,16 @@ import org.apache.spark.SparkContext object NaiveBayesSuite { - private def calcLabel(p: Double, weightPerLabel: Array[Double]): Int = { + private def calcLabel(p: Double, pi: Array[Double]): Int = { var sum = 0.0 - for (j <- 0 until weightPerLabel.length) { - sum += weightPerLabel(j) + for (j <- 0 until pi.length) { + sum += pi(j) if (p < sum) return j } -1 } - // Generate input of the form Y = (weightMatrix*x).argmax() + // Generate input of the form Y = (theta * x).argmax() def generateNaiveBayesInput( pi: Array[Double], // 1XC theta: Array[Array[Double]], // CXD From 92c304fd0321d77941f0b029dc7b7f61804d8bca Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 1 Jan 2014 11:42:33 -0800 Subject: [PATCH 1113/1571] Simplify ExternalAppendOnlyMap on the assumption that the mergeCombiners function is specified --- .../scala/org/apache/spark/Aggregator.scala | 3 +- .../collection/ExternalAppendOnlyMap.scala | 162 ++++++------------ .../ExternalAppendOnlyMapSuite.scala | 23 --- 3 files changed, 53 insertions(+), 135 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 784c09ec51b0f..c408d5f145667 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -71,8 +71,7 @@ case class Aggregator[K, V, C: ClassTag] ( } combiners.iterator } else { - val combiners = - new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) + val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { val kc = iter.next() combiners.insert(kc._1, kc._2) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 223fae128e147..9e147feec42d7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -23,98 +23,40 @@ import java.util.Comparator import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import scala.reflect.ClassTag import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} /** - * A wrapper for SpillableAppendOnlyMap that handles two cases: + * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * - * (1) If a mergeCombiners function is specified, merge values into combiners before disk - * spill, as it is possible to merge the resulting combiners later. + * This map takes two passes over the data: + * (1) Values are merged into combiners, which are sorted and spilled to disk in as necessary. + * (2) Combiners are read from disk and merged together * - * (2) Otherwise, group values of the same key together before disk spill, and merge them - * into combiners only after reading them back from disk. + * Two parameters control the memory threshold: `spark.shuffle.buffer.mb` specifies the maximum + * size of the in-memory map before a spill, and `spark.shuffle.buffer.fraction` specifies an + * additional margin of safety. The second parameter is important for the following reason: * - * In the latter case, values occupy much more space because they are not collapsed as soon - * as they are inserted. This in turn leads to more disk spills, degrading performance. - * For this reason, a mergeCombiners function should be specified if possible. + * If the spill threshold is set too high, the in-memory map may occupy more memory than is + * available, resulting in OOM. However, if the spill threshold is set too low, we spill + * frequently and incur unnecessary disk writes. This may lead to a performance regression + * compared to the normal case of using the non-spilling AppendOnlyMap. */ -private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( + +private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializerManager.default, diskBlockManager: DiskBlockManager = SparkEnv.get.blockManager.diskBlockManager) - extends Iterable[(K, C)] with Serializable { - - private val mergeBeforeSpill: Boolean = mergeCombiners != null - - private val map: SpillableAppendOnlyMap[K, V, _, C] = { - if (mergeBeforeSpill) { - new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, - identity, serializer, diskBlockManager) - } else { - // Use ArrayBuffer[V] as the intermediate combiner - val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) - val mergeValueIntoGroup: (ArrayBuffer[V], V) => ArrayBuffer[V] = (group, value) => { - group += value - } - val mergeGroups: (ArrayBuffer[V], ArrayBuffer[V]) => ArrayBuffer[V] = (group1, group2) => { - group1 ++= group2 - } - val combineGroup: (ArrayBuffer[V] => C) = group => { - var combiner : Option[C] = None - group.foreach { v => - combiner match { - case None => combiner = Some(createCombiner(v)) - case Some(c) => combiner = Some(mergeValue(c, v)) - } - } - combiner.getOrElse(null.asInstanceOf[C]) - } - new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, - mergeGroups, combineGroup, serializer, diskBlockManager) - } - } - - def insert(key: K, value: V): Unit = map.insert(key, value) - - override def iterator: Iterator[(K, C)] = map.iterator -} - -/** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. - * A group is an intermediate combiner, with type G equal to either C or ArrayBuffer[V]. - * - * This map takes two passes over the data: - * (1) Values are merged into groups, which are spilled to disk as necessary. - * (2) Groups are read from disk and merged into combiners, which are returned. - * - * If we never spill to disk, we avoid the second pass provided that groups G are already - * combiners C. - * - * Note that OOM is still possible with the SpillableAppendOnlyMap. This may occur if the - * collective G values do not fit into memory, or if the size estimation is not sufficiently - * accurate. To account for the latter, `spark.shuffle.buffer.fraction` specifies an additional - * margin of safety, while `spark.shuffle.buffer.mb` specifies the raw memory threshold. - */ -private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( - createGroup: V => G, - mergeValue: (G, V) => G, - mergeGroups: (G, G) => G, - createCombiner: G => C, - serializer: Serializer, - diskBlockManager: DiskBlockManager) extends Iterable[(K, C)] with Serializable with Logging { - import SpillableAppendOnlyMap._ + import ExternalAppendOnlyMap._ - private var currentMap = new SizeTrackingAppendOnlyMap[K, G] + private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskIterator] - private val memoryThresholdMB = { // TODO: Turn this into a fraction of memory per reducer val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong @@ -123,13 +65,13 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } private val fileBufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 - private val comparator = new KeyGroupComparator[K, G] + private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() private var spillCount = 0 def insert(key: K, value: V): Unit = { - val update: (Boolean, G) => G = (hadVal, oldVal) => { - if (hadVal) mergeValue(oldVal, value) else createGroup(value) + val update: (Boolean, C) => C = (hadVal, oldVal) => { + if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } currentMap.changeValue(key, update) if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { @@ -154,19 +96,19 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Partial failures cannot be tolerated; do not revert partial writes writer.close() } - currentMap = new SizeTrackingAppendOnlyMap[K, G] + currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskIterator(file)) } override def iterator: Iterator[(K, C)] = { - if (spilledMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { - currentMap.iterator.asInstanceOf[Iterator[(K, C)]] + if (spilledMaps.isEmpty) { + currentMap.iterator } else { new ExternalIterator() } } - /** An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs. */ + /** An iterator that sort-merges (K, C) pairs from the in-memory and on-disk maps */ private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging @@ -177,43 +119,43 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps inputStreams.foreach{ it => - val kgPairs = getMorePairs(it) - mergeHeap.enqueue(StreamBuffer(it, kgPairs)) + val kcPairs = getMorePairs(it) + mergeHeap.enqueue(StreamBuffer(it, kcPairs)) } /** * Fetch from the given iterator until a key of different hash is retrieved. In the * event of key hash collisions, this ensures no pairs are hidden from being merged. */ - def getMorePairs(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { - val kgPairs = new ArrayBuffer[(K, G)] + def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { + val kcPairs = new ArrayBuffer[(K, C)] if (it.hasNext) { - var kg = it.next() - kgPairs += kg - val minHash = kg._1.hashCode() - while (it.hasNext && kg._1.hashCode() == minHash) { - kg = it.next() - kgPairs += kg + var kc = it.next() + kcPairs += kc + val minHash = kc._1.hashCode() + while (it.hasNext && kc._1.hashCode() == minHash) { + kc = it.next() + kcPairs += kc } } - kgPairs + kcPairs } /** * If the given buffer contains a value for the given key, merge that value into - * baseGroup and remove the corresponding (K, G) pair from the buffer + * baseCombiner and remove the corresponding (K, C) pair from the buffer */ - def mergeIfKeyExists(key: K, baseGroup: G, buffer: StreamBuffer): G = { + def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { var i = 0 while (i < buffer.pairs.size) { - val (k, g) = buffer.pairs(i) + val (k, c) = buffer.pairs(i) if (k == key) { buffer.pairs.remove(i) - return mergeGroups(baseGroup, g) + return mergeCombiners(baseCombiner, c) } i += 1 } - baseGroup + baseCombiner } override def hasNext: Boolean = { @@ -233,7 +175,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Should only happen when no other stream buffers have any pairs left throw new NoSuchElementException } - var (minKey, minGroup) = minPairs.remove(0) + var (minKey, minCombiner) = minPairs.remove(0) assert(minKey.hashCode() == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), @@ -241,7 +183,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) while (!mergeHeap.isEmpty && mergeHeap.head.minKeyHash == minHash) { val newBuffer = mergeHeap.dequeue() - minGroup = mergeIfKeyExists(minKey, minGroup, newBuffer) + minCombiner = mergeIfKeyExists(minKey, minCombiner, newBuffer) mergedBuffers += newBuffer } @@ -253,7 +195,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( mergeHeap.enqueue(buffer) } - (minKey, createCombiner(minGroup)) + (minKey, minCombiner) } /** @@ -263,7 +205,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( * * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ - case class StreamBuffer(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) + case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def minKeyHash: Int = { @@ -282,18 +224,18 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } - // Iterate through (K, G) pairs in sorted order from an on-disk map - private class DiskIterator(file: File) extends Iterator[(K, G)] { + // Iterate through (K, C) pairs in sorted order from an on-disk map + private class DiskIterator(file: File) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) val deserializeStream = ser.deserializeStream(bufferedStream) - var nextItem: Option[(K, G)] = None + var nextItem: Option[(K, C)] = None var eof = false - def readNextItem(): Option[(K, G)] = { + def readNextItem(): Option[(K, C)] = { if (!eof) { try { - return Some(deserializeStream.readObject().asInstanceOf[(K, G)]) + return Some(deserializeStream.readObject().asInstanceOf[(K, C)]) } catch { case e: EOFException => eof = true @@ -312,7 +254,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } - override def next(): (K, G) = { + override def next(): (K, C) = { nextItem match { case Some(item) => nextItem = None @@ -331,10 +273,10 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } -private[spark] object SpillableAppendOnlyMap { - private class KeyGroupComparator[K, G] extends Comparator[(K, G)] { - def compare(kg1: (K, G), kg2: (K, G)): Int = { - kg1._1.hashCode().compareTo(kg2._1.hashCode()) +private[spark] object ExternalAppendOnlyMap { + private class KCComparator[K, C] extends Comparator[(K, C)] { + def compare(kc1: (K, C), kc2: (K, C)): Int = { + kc1._1.hashCode().compareTo(kc2._1.hashCode()) } } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index baf94b4728ec8..a18d466baaff5 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -229,27 +229,4 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } } } - - test("spilling with no mergeCombiners function") { - System.setProperty("spark.shuffle.buffer.mb", "1") - System.setProperty("spark.shuffle.buffer.fraction", "0.05") - - // combineByKey - should spill exactly 11 times - val _createCombiner: Int => ArrayBuffer[Int] = i => ArrayBuffer[Int](i) - val _mergeValue: (ArrayBuffer[Int], Int) => ArrayBuffer[Int] = (buf, i) => buf += i - val rdd = sc.parallelize(0 until 10000).map(i => (i/4, i)) - val result = rdd.combineByKey[ArrayBuffer[Int]](_createCombiner, _mergeValue, null, - new HashPartitioner(1), mapSideCombine=false).collect() - - // result should be the same as groupByKey - assert(result.length == 2500) - result.foreach { case(i, seq) => - i match { - case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) - case 1250 => assert(seq.toSet == Set[Int](5000, 5001, 5002, 5003)) - case 2499 => assert(seq.toSet == Set[Int](9996, 9997, 9998, 9999)) - case _ => - } - } - } } From e2c68642c64345434e2034082cf9b299491e9e9f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 1 Jan 2014 22:03:39 -0500 Subject: [PATCH 1114/1571] Miscellaneous fixes from code review. Also replaced SparkConf.getOrElse with just a "get" that takes a default value, and added getInt, getLong, etc to make code that uses this simpler later on. --- .../scala/org/apache/spark/SparkConf.scala | 60 +++++++++------ .../scala/org/apache/spark/SparkContext.scala | 73 ++++++++++++------- .../scala/org/apache/spark/SparkEnv.scala | 10 +-- .../apache/spark/api/python/PythonRDD.scala | 4 +- .../apache/spark/broadcast/Broadcast.scala | 4 +- .../spark/broadcast/HttpBroadcast.scala | 4 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 12 +-- .../spark/deploy/master/MasterArguments.scala | 6 +- .../deploy/master/SparkZooKeeperSession.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 2 +- .../master/ZooKeeperPersistenceEngine.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/io/CompressionCodec.scala | 4 +- .../apache/spark/metrics/MetricsSystem.scala | 2 +- .../spark/network/ConnectionManager.scala | 18 ++--- .../spark/network/netty/ShuffleCopier.scala | 2 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 10 +-- .../spark/scheduler/TaskSetManager.scala | 16 ++-- .../CoarseGrainedSchedulerBackend.scala | 9 ++- .../cluster/SimrSchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/serializer/KryoSerializer.scala | 13 ++-- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 24 +++--- .../spark/storage/BlockManagerMaster.scala | 4 +- .../storage/BlockManagerMasterActor.scala | 4 +- .../spark/storage/DiskBlockManager.scala | 2 +- .../spark/storage/ShuffleBlockManager.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/util/AkkaUtils.scala | 18 ++--- .../apache/spark/util/MetadataCleaner.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 4 +- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../spark/storage/DiskBlockManagerSuite.scala | 12 +-- python/pyspark/conf.py | 12 +-- .../org/apache/spark/repl/SparkIMain.scala | 2 +- .../dstream/NetworkInputDStream.scala | 6 +- .../streaming/scheduler/JobGenerator.scala | 4 +- .../streaming/scheduler/JobScheduler.scala | 2 +- 49 files changed, 206 insertions(+), 189 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 96239cf4be018..98343e9532300 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -42,6 +42,12 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set a configuration variable. */ def set(key: String, value: String): SparkConf = { + if (key == null) { + throw new NullPointerException("null key") + } + if (value == null) { + throw new NullPointerException("null value") + } settings(key) = value this } @@ -51,26 +57,17 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. */ def setMaster(master: String): SparkConf = { - if (master != null) { - settings("spark.master") = master - } - this + set("spark.master", master) } /** Set a name for your application. Shown in the Spark web UI. */ def setAppName(name: String): SparkConf = { - if (name != null) { - settings("spark.app.name") = name - } - this + set("spark.app.name", name) } /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - if (!jars.isEmpty) { - settings("spark.jars") = jars.mkString(",") - } - this + set("spark.jars", jars.mkString(",")) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ @@ -84,8 +81,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * (for example spark.executorEnv.PATH) but this method makes them easier to set. */ def setExecutorEnv(variable: String, value: String): SparkConf = { - settings("spark.executorEnv." + variable) = value - this + set("spark.executorEnv." + variable, value) } /** @@ -112,10 +108,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * Set the location where Spark is installed on worker nodes. */ def setSparkHome(home: String): SparkConf = { - if (home != null) { - settings("spark.home") = home - } - this + set("spark.home", home) } /** Set multiple parameters together */ @@ -132,9 +125,20 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { this } - /** Get a parameter; throws an exception if it's not set */ + /** Remove a parameter from the configuration */ + def remove(key: String): SparkConf = { + settings.remove(key) + this + } + + /** Get a parameter; throws a NoSuchElementException if it's not set */ def get(key: String): String = { - settings(key) + settings.getOrElse(key, throw new NoSuchElementException(key)) + } + + /** Get a parameter, falling back to a default if not set */ + def get(key: String, defaultValue: String): String = { + settings.getOrElse(key, defaultValue) } /** Get a parameter as an Option */ @@ -145,9 +149,19 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Get all parameters as a list of pairs */ def getAll: Array[(String, String)] = settings.clone().toArray - /** Get a parameter, falling back to a default if not set */ - def getOrElse(k: String, defaultValue: String): String = { - settings.getOrElse(k, defaultValue) + /** Get a parameter as an integer, falling back to a default if not set */ + def getInt(key: String, defaultValue: Int): Int = { + getOption(key).map(_.toInt).getOrElse(defaultValue) + } + + /** Get a parameter as a long, falling back to a default if not set */ + def getLong(key: String, defaultValue: Long): Long = { + getOption(key).map(_.toLong).getOrElse(defaultValue) + } + + /** Get a parameter as a double, falling back to a default if not set */ + def getDouble(key: String, defaultValue: Double): Double = { + getOption(key).map(_.toDouble).getOrElse(defaultValue) } /** Get all executor environment variables set on this SparkConf */ diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 46874c41a23dd..84bd0f7ffdf64 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,12 +22,11 @@ import java.net.URI import java.util.{UUID, Properties} import java.util.concurrent.atomic.AtomicInteger -import scala.collection.{Map, Set, immutable} +import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} -import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -49,7 +48,8 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util._ +import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType, +ClosureCleaner} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -77,7 +77,7 @@ class SparkContext( * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters */ def this(master: String, appName: String, conf: SparkConf) = - this(conf.clone().setMaster(master).setAppName(appName)) + this(SparkContext.updatedConf(conf, master, appName)) /** * Alternative constructor that allows setting common Spark properties directly @@ -97,13 +97,7 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this( - new SparkConf() - .setMaster(master) - .setAppName(appName) - .setJars(jars) - .setExecutorEnv(environment.toSeq) - .setSparkHome(sparkHome), + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), preferredNodeLocationData) } @@ -175,11 +169,9 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { - val value = System.getenv(key) - if (value != null) { - executorEnvs(key) = value - } + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING"); + value <- Option(System.getenv(key))) { + executorEnvs(key) = value } // Since memory can be set with a system property too, use that executorEnvs("SPARK_MEM") = executorMemory + "m" @@ -220,7 +212,7 @@ class SparkContext( hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + val bufferSize = conf.get("spark.buffer.size", "65536") hadoopConf.set("io.file.buffer.size", bufferSize) hadoopConf } @@ -733,13 +725,7 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (conf.contains("spark.home")) { - Some(conf.get("spark.home")) - } else if (System.getenv("SPARK_HOME") != null) { - Some(System.getenv("SPARK_HOME")) - } else { - None - } + conf.getOption("spark.home").orElse(Option(System.getenv("SPARK_HOME"))) } /** @@ -1026,7 +1012,7 @@ object SparkContext { /** * Find the JAR from which a given class was loaded, to make it easy for users to pass - * their JARs to SparkContext + * their JARs to SparkContext. */ def jarOfClass(cls: Class[_]): Seq[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") @@ -1043,10 +1029,41 @@ object SparkContext { } } - /** Find the JAR that contains the class of a particular object */ + /** + * Find the JAR that contains the class of a particular object, to make it easy for users + * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in + * your driver program. + */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) - // Creates a task scheduler based on a given master URL. Extracted for testing. + /** + * Creates a modified version of a SparkConf with the parameters that can be passed separately + * to SparkContext, to make it easier to write SparkContext's constructors. This ignores + * parameters that are passed as the default value of null, instead of throwing an exception + * like SparkConf would. + */ + private def updatedConf( + conf: SparkConf, + master: String, + appName: String, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map()): SparkConf = + { + val res = conf.clone() + res.setMaster(master) + res.setAppName(appName) + if (sparkHome != null) { + res.setSparkHome(sparkHome) + } + if (!jars.isEmpty) { + res.setJars(jars) + } + res.setExecutorEnv(environment.toSeq) + res + } + + /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String, appName: String) : TaskScheduler = { @@ -1156,7 +1173,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new TaskSchedulerImpl(sc) - val coarseGrained = sc.conf.getOrElse("spark.mesos.coarse", "false").toBoolean + val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d06af8e667664..634a94f0a7f06 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -144,17 +144,17 @@ object SparkEnv extends Logging { // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = conf.getOrElse(propertyName, defaultClassName) + val name = conf.get(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) + conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) val closureSerializer = serializerManager.get( - conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), + conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { @@ -162,8 +162,8 @@ object SparkEnv extends Logging { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") - val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt + val driverHost: String = conf.get("spark.driver.host", "localhost") + val driverPort: Int = conf.get("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 05fd82425419f..32cc70e8c9dda 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = conf.get("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -250,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index be99d229efb53..0fc478a41967c 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -31,7 +31,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { override def toString = "Broadcast(" + id + ")" } -private[spark] +private[spark] class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { private var initialized = false @@ -43,7 +43,7 @@ class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging private def initialize() { synchronized { if (!initialized) { - val broadcastFactoryClass = conf.getOrElse( + val broadcastFactoryClass = conf.get( "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 47528bcee8351..db596d5fcc054 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -92,8 +92,8 @@ private object HttpBroadcast extends Logging { def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt - compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.get("spark.buffer.size", "65536").toInt + compress = conf.get("spark.broadcast.compress", "true").toBoolean if (isDriver) { createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 00ec3b971b56e..95309382786a9 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -180,7 +180,7 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9c89e36b146a6..7b696cfcca66e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -43,11 +43,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val conf = new SparkConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -88,7 +88,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.get("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 7ce83f9c368a4..e7f3224091ca8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -27,8 +27,8 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_MASTER_HOST") != null) { host = System.getenv("SPARK_MASTER_HOST") } @@ -38,7 +38,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (conf.get("master.ui.port") != null) { + if (conf.contains("master.ui.port")) { webUiPort = conf.get("master.ui.port").toInt } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 60c7a7c2d6e14..999090ad7464d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging} */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, conf: SparkConf) extends Logging { - val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") + val ZK_URL = conf.get("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index a61597bbdf9a0..77c23fb9fbeea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() private val zk = new SparkZooKeeperSession(this, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 245a558a592fb..52000d4f9c11c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk = new SparkZooKeeperSession(this, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f844fcbbfcdf1..fcaf4e92b1858 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -55,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index a801d857707b4..c382034c99e00 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -37,7 +37,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( - worker.conf.getOrElse("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) var server: Option[Server] = None var boundPort: Option[Int] = None diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5b70165c35358..3c92c205ea978 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -302,7 +302,7 @@ private[spark] class Executor( * new classes defined by the REPL as the user types code */ private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { - val classUri = conf.getOrElse("spark.repl.class.uri", null) + val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 075a18b068561..a1e98845f6a84 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -39,7 +39,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.getOrElse( + createCodec(conf, conf.get( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } @@ -71,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 0e41c73ce75a4..9930537b34db0 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -65,7 +65,7 @@ import org.apache.spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String, conf: SparkConf) extends Logging { - val confFile = conf.getOrElse("spark.metrics.conf", null) + val confFile = conf.get("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 697096fa7636a..46c40d0a2a029 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.handler.threads.min", "20").toInt, - conf.getOrElse("spark.core.connection.handler.threads.max", "60").toInt, - conf.getOrElse("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.handler.threads.min", "20").toInt, + conf.get("spark.core.connection.handler.threads.max", "60").toInt, + conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.io.threads.min", "4").toInt, - conf.getOrElse("spark.core.connection.io.threads.max", "32").toInt, - conf.getOrElse("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.io.threads.min", "4").toInt, + conf.get("spark.core.connection.io.threads.max", "32").toInt, + conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.connect.threads.min", "1").toInt, - conf.getOrElse("spark.core.connection.connect.threads.max", "8").toInt, - conf.getOrElse("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.connect.threads.min", "1").toInt, + conf.get("spark.core.connection.connect.threads.max", "8").toInt, + conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index db28ddf9ac697..b729eb11c5142 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 172ba6b01c422..6d4f46125f1a6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -97,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -131,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging { ): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) - val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 29b0247f8a851..e22b1e53e8048 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends Logging { - private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt + private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index bffd990e16dbe..d94b706854f3a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -51,15 +51,15 @@ private[spark] class TaskSchedulerImpl( isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.getOrElse("spark.task.maxFailures", "4").toInt) + def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt) val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -96,7 +96,7 @@ private[spark] class TaskSchedulerImpl( var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.get("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -125,7 +125,7 @@ private[spark] class TaskSchedulerImpl( override def start() { backend.start() - if (!isLocal && conf.getOrElse("spark.speculation", "false").toBoolean) { + if (!isLocal && conf.get("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b99664ae00317..67ad99a4d72ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -57,11 +57,11 @@ private[spark] class TaskSetManager( val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -116,7 +116,7 @@ private[spark] class TaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong + conf.get("spark.logging.exceptionPrintInterval", "10000").toLong // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception @@ -678,14 +678,14 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.getOrElse("spark.locality.wait", "3000") + val defaultWait = conf.get("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - conf.getOrElse("spark.locality.wait.process", defaultWait).toLong + conf.get("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - conf.getOrElse("spark.locality.wait.node", defaultWait).toLong + conf.get("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong + conf.get("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index b4a3ecca39095..2f5bcafe40394 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.util.Try import akka.actor._ import akka.pattern.ask @@ -64,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -209,8 +208,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! KillTask(taskId, executorId) } - override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption - .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) + override def defaultParallelism(): Int = { + conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse( + math.max(totalCoreCount.get(), 2)) + } // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index f41fbbd1f3441..b44d1e43c85c7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -33,7 +33,7 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt + val maxCores = conf.get("spark.simr.executor.cores", "1").toInt override def start() { super.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 224077566d06a..9858717d134a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -38,7 +38,7 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt override def start() { super.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 9e2cd3f6994e7..d247fa4244ec5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -62,7 +62,7 @@ private[spark] class CoarseMesosSchedulerBackend( var driver: SchedulerDriver = null // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt // Cores we have acquired with each Mesos task ID val coresByTaskId = new HashMap[Int, Int] @@ -77,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index be96382983022..c20fc418e83cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 2367f3f52154b..a24a3b04b87cc 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -29,17 +29,14 @@ import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ -import scala.util.Try -import org.apache.spark.storage.PutBlock -import org.apache.spark.storage.GetBlock -import org.apache.spark.storage.GotBlock +import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = { - conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -51,7 +48,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -61,13 +58,13 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow the user to register their own classes by setting spark.kryo.registrator try { - Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls => + for (regCls <- conf.getOption("spark.kryo.registrator")) { logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } } catch { - case _: Exception => println("Failed to register spark.kryo.registrator") + case e: Exception => logError("Failed to run spark.kryo.registrator", e) } // Register Chill's classes; we do this after our ranges and the user's own classes to let diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 3b25f68ca8e5b..47478631a11f0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 16ee208617439..6d2cda97b04eb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -49,7 +49,7 @@ private[spark] class BlockManager( val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -58,8 +58,8 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } @@ -72,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) @@ -443,7 +443,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.get("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -469,7 +469,7 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } @@ -864,15 +864,15 @@ private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator def getMaxMemory(conf: SparkConf): Long = { - val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = - conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = - conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 8e4a88b20a4e4..b5afe8cd23ca8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -31,8 +31,8 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index dbbeeb39ebedc..58452d96574c9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -50,10 +50,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", + val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong - val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", + val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong var timeoutCheckingTask: Cancellable = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 7697092e1b8c8..55dcb3742c967 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 151eedb7837fb..39dc7bb19afee 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -27,8 +27,6 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup -import scala.util.Try -import org.apache.spark.SparkConf /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -66,9 +64,9 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean + conf.get("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 0ce8d9c8c449d..50dfdbdf5ae9b 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -1,4 +1,4 @@ -/* +/* * 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. @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None var server: Option[Server] = None diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 6b4602f928cd2..88f41be8d3dd2 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -66,7 +66,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(sc.conf.getOrElse("path.separator", ":")) + .split(sc.conf.get("path.separator", ":")) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 315014d27d850..b7b87250b98ee 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 58b26f7f12b0d..362cea5e3e462 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -41,19 +41,19 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt - val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.get("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt - val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt - val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt + conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -89,6 +89,6 @@ private[spark] object AkkaUtils { /** Returns the default Spark timeout to use for Akka ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") + Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 9ea7fc2dfd42b..aa7f52cafbf37 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -74,12 +74,12 @@ object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { def getDelaySeconds(conf: SparkConf) = { - conf.getOrElse("spark.cleaner.ttl", "3500").toInt + conf.get("spark.cleaner.ttl", "3500").toInt } def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) + conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) .toInt } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ca3320b22bc93..5f1253100b338 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -311,7 +311,7 @@ private[spark] object Utils extends Logging { * multiple paths. */ def getLocalDir(conf: SparkConf): String = { - conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } /** @@ -397,7 +397,7 @@ private[spark] object Utils extends Logging { } def localHostPort(conf: SparkConf): String = { - val retval = conf.getOrElse("spark.hostPort", null) + val retval = conf.get("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 77c7b829b310c..ef5936dd2f588 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -74,7 +74,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { assert(!conf.contains("k4"), "conf contained k4") assert(conf.get("k1") === "v4") intercept[Exception] { conf.get("k4") } - assert(conf.getOrElse("k4", "not found") === "not found") + assert(conf.get("k4", "not found") === "not found") assert(conf.getOption("k1") === Some("v4")) assert(conf.getOption("k4") === None) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 5d33e662535df..1eec6726f48bc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -83,7 +83,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong + val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong val MAX_TASK_FAILURES = 4 test("TaskSet with no preferences") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index f940448abd5fc..af4b31d53c26c 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -27,8 +27,8 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import scala.util.Try import akka.actor.{Props, ActorSelection, ActorSystem} -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { - private val testConf = new SparkConf +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { + private val testConf = new SparkConf(false) val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() val rootDir1 = Files.createTempDir() @@ -38,9 +38,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. - val consolidateProp = "spark.shuffle.consolidateFiles" - val oldConsolidate = Try(testConf.get(consolidateProp)).toOption - testConf.set(consolidateProp, "true") + testConf.set("spark.shuffle.consolidateFiles", "true") val shuffleBlockManager = new ShuffleBlockManager(null) { override def conf = testConf.clone @@ -50,10 +48,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before var diskBlockManager: DiskBlockManager = _ - override def afterAll() { - oldConsolidate.map(c => System.setProperty(consolidateProp, c)) - } - override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 9dcdcfaa673c1..c111e2e90f607 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -93,7 +93,7 @@ def __init__(self, loadDefaults=True, _jvm=None): def set(self, key, value): """Set a configuration property.""" - self._jconf.set(key, value) + self._jconf.set(key, unicode(value)) return self def setMaster(self, value): @@ -132,13 +132,9 @@ def setAll(self, pairs): self._jconf.set(k, v) return self - def get(self, key): - """Get the configured value for some key, if set.""" - return self._jconf.get(key) - - def getOrElse(self, key, defaultValue): - """Get the value for some key, or return a default otherwise.""" - return self._jconf.getOrElse(key, defaultValue) + def get(self, key, defaultValue=None): + """Get the configured value for some key, or return a default otherwise.""" + return self._jconf.get(key, defaultValue) def getAll(self): """Get all values as a list of key-value pairs.""" diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index a993083164977..59fdb0b37a766 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -89,7 +89,7 @@ import org.apache.spark.util.Utils /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = new SparkConf().getOrElse("spark.repl.classdir", tmp) + val rootDir = new SparkConf().get("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index a230845b92715..27d474c0a0459 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -174,8 +174,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = env.conf.getOrElse("spark.driver.host", "localhost") - val port = env.conf.getOrElse("spark.driver.port", "7077").toInt + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.get("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -212,7 +212,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 844180c81a65b..5f8be93a98518 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -46,7 +46,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } })) val clock = { - val clockClass = ssc.sc.conf.getOrElse( + val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") Class.forName(clockClass).newInstance().asInstanceOf[Clock] } @@ -104,7 +104,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 651cdaaa6d0b2..9304fc1a9338d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -31,7 +31,7 @@ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { val jobSets = new ConcurrentHashMap[Time, JobSet] - val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt + val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt val executor = Executors.newFixedThreadPool(numConcurrentJobs) val generator = new JobGenerator(this) val listenerBus = new StreamingListenerBus() From 0f6060733da83a862038fd397875cdb49d8c144d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 1 Jan 2014 22:09:42 -0500 Subject: [PATCH 1115/1571] Fixed two uses of conf.get with no default value in Mesos --- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d247fa4244ec5..08811520cfefc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val uri = conf.get("spark.executor.uri") + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c20fc418e83cd..bb278fb155c79 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -100,7 +100,7 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri") + val uri = sc.conf.get("spark.executor.uri", null) if (uri == null) { command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) } else { From 7e8d2e8a5c88d16c771923504c433491b109ab2a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 1 Jan 2014 23:21:34 -0500 Subject: [PATCH 1116/1571] Fix Python code after change of getOrElse --- python/pyspark/conf.py | 7 ++++++- python/pyspark/context.py | 14 ++++++++------ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index c111e2e90f607..d72aed6a30ec1 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -134,7 +134,12 @@ def setAll(self, pairs): def get(self, key, defaultValue=None): """Get the configured value for some key, or return a default otherwise.""" - return self._jconf.get(key, defaultValue) + if defaultValue == None: # Py4J doesn't call the right get() if we pass None + if not self._jconf.contains(key): + return None + return self._jconf.get(key) + else: + return self._jconf.get(key, defaultValue) def getAll(self): """Get all values as a list of key-value pairs.""" diff --git a/python/pyspark/context.py b/python/pyspark/context.py index d77dd7676599d..f955aad7a4f12 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -92,11 +92,13 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) - # Set parameters passed directly to us on the conf; these operations will be - # no-ops if the parameters were None - self._conf.setMaster(master) - self._conf.setAppName(appName) - self._conf.setSparkHome(sparkHome) + # Set any parameters passed directly to us on the conf + if master: + self._conf.setMaster(master) + if appName: + self._conf.setAppName(appName) + if sparkHome: + self._conf.setSparkHome(sparkHome) if environment: for key, value in environment.iteritems(): self._conf.setExecutorEnv(key, value) @@ -111,7 +113,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, # the classpath or an external config file self.master = self._conf.get("spark.master") self.appName = self._conf.get("spark.app.name") - self.sparkHome = self._conf.getOrElse("spark.home", None) + self.sparkHome = self._conf.get("spark.home", None) for (k, v) in self._conf.getAll(): if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] From 8821c3a5262d6893d2a1fd6ed86afd1213114b4d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 13:07:52 +0530 Subject: [PATCH 1117/1571] Deleted py4j jar and added to assembly dependency --- assembly/lib/PY4J_LICENSE.txt | 27 ------------------ assembly/lib/PY4J_VERSION.txt | 1 - .../lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar | Bin 103286 -> 0 bytes .../lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom | 9 ------ .../net/sf/py4j/py4j/maven-metadata-local.xml | 12 -------- assembly/pom.xml | 2 +- project/SparkBuild.scala | 1 + 7 files changed, 2 insertions(+), 50 deletions(-) delete mode 100644 assembly/lib/PY4J_LICENSE.txt delete mode 100644 assembly/lib/PY4J_VERSION.txt delete mode 100644 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar delete mode 100644 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom delete mode 100644 assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml diff --git a/assembly/lib/PY4J_LICENSE.txt b/assembly/lib/PY4J_LICENSE.txt deleted file mode 100644 index a70279ca14ae3..0000000000000 --- a/assembly/lib/PY4J_LICENSE.txt +++ /dev/null @@ -1,27 +0,0 @@ - -Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -- Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. - -- Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -- The name of the author may not be used to endorse or promote products -derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE -ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE -LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR -CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF -SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS -INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN -CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. diff --git a/assembly/lib/PY4J_VERSION.txt b/assembly/lib/PY4J_VERSION.txt deleted file mode 100644 index 04a0cd52a8d9c..0000000000000 --- a/assembly/lib/PY4J_VERSION.txt +++ /dev/null @@ -1 +0,0 @@ -b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar deleted file mode 100644 index 73b7ddb7d115be2fb1ba4cf8003fe90c662feef6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 103286 zcma&NV{oQjlrP#;Un3h%ti2|iyc!=?8&xuw+lT%T3%cT`Ln|eUli;Z5jj{&$LBKpTp#Uh}y zvYZ*S~i=KB8!Ywl%i z?__8G|H8umyRW^YnYrEnV^idRfiC72cIKvT){YMUh5qLk2ZY3Qru>al3m*goJJEN(cv8!uyCcK;aQs&6lj^MNN6`7M5_h6UT#6Fz(*dy(=Fg={4C`p$;vI`ol zf#{?dSW^V2x>QB8&W1{gXcHPybLeWJDUJpao9SuHrOk?)MaR00W^RF%F5mv5wrGjx zr*D3;t0S@hWYd5?kL9(a(9HJN&gav%-+3!jJ_IvfCbguAH#(VSYss?DK(DVkP(J*HyKV5mq!EN|-4}TgP()>N=pjAsx)( zgNsBs@^_h)K z=~4YOK(4WzlHGnG=`=fy*>q)ZT49THfHQU?4X3f`Ha<7O;?sVgh%Py1dV6-g!tjyd zvZ|!Aq^PDlTmQn!q%d`9Sa^6*tEr>2{Ivxyix&{zk8O)vZF*^p4nGX2+*x*$Qgv@B z|Fe9yw3Tar**eX}psT-?`&Ps#dq@pecm+*we`Zoyk=YI3lu;vH%GgnrK7;uOfjv8J zNO#@+bC^KR!~MQ-;G;hlnCXB0Q{yJ=)TQsiD-`T)t2ngvMdQ z+;^GxbCFzs(=qP+$#NG#Ja&<5{f4GN@NEHVc6K#qvySEx4 z@4Ok2y_fMr!kPSGl!DQYB#xqNLBbJpk_guhC=c z4G}N9h22s|$8l@D<+A1C>4|`#)zicMd9%LJbLDEHHbXzTgVld&$(6A(ZXeU`Td4jT zSe!$Zu#i}m!)ilyN#;J+wNxgo{&8Z+(4NyTkrBjSN#EIfH89MJfin1cClpSTIZApw_pTm*%!zG<% z+|o;~Wl=f;C$Q8%WxP;y+Pe^(E9hjcp39~%kJ|HC3Bj9qsp^?dBcKq6)NGo-PzB~2 za$x3%e_syTLll49Tvbs0V8}rG3SchKd`{fT)~Xe4^u&egqMd|}pk=$~qJAEQ zj=;TeH$d;ugLo4Qv$o%DGLm2rn`&-#!&Cw2QwviGW!ePJ_ZXx*O$hzq=!L1BRbme_johyl`9WkxVZR z;NM8i1}Um+#hb7R#$IvVsKb4kv68W)8e$caar+3`@RD)h zOtG%ah^U-GcoMhC&u7us2g87@#V>r()(F}kQK}gZIvdl=$-rjD&5^Vb6>c^y3(%-( z*TRwu{16m0-XCUhTd`p*st-c1yoQFv#}Fzh zYh~m+I%cKvk@&WHM5W01w9GH7S?Ol?rUnx z)HXxv>D5aZV~-MAx1Qr6l5ugD6%%S)G>%RF`!G9YOYVW%%(XS$i>PJav3rKychB`; z4R&#zqG;cxrWY%7;#JjFJ)JRne(=0d)nx6A(0;IS?i$X95jg<^MtR;?QlEI$Z_sEj zOBc8C7FT}ldglzMTlwNR?qx5oc;2`keL$)PL>k^cfBPJuu8y}=q!HVXg~L=)tvUMU zB^#|`O;gkavoV^bmM#09psW)iyJ4P!xx64#a;nZS`~<`;;koEzFKY6>7;{?~_Nbt> zUOy=`XzmG-G<261w<*WdogP4?@i51nWzEhoCAws;VA9aOGfCCIDz}R+ZNOK@lqZxj z(#q90ar40M6D{fH$(g5omtAOoOtKKwu@~Yf33zy4YH1X<=jd5?(+|{gto_Zd!18YzgCa zetZh6ZG}#JdT1fK%-eptJ3hN)ytsW#8^F+OZF=?_S_`FWlf8FZePq%-)u!ZFE;3`QUDXx=-o@wLAZ(5iJlQ{95s-3*Q;Yxb6NT2E z_*r9#WW^x0RhNg+ja^B@DM>vz2KIX>cwKDJLU6@6z{r(XE3qsP^>9qwU%v2!*+hZ4 zp0tt;!biR(0nJU+8%YjPCBEfYZJ@x;iq4;U%TmZkerB~GOFL5ed!pBMPl#3-a|;sz zQ)4bn^XXtL={-v)v!r_Xw}LRj%y~whs$aVG;Nb1PYn(hKgU&ibf%53>WbJ zopPd0Ov7_;&ZNd2lO`phh~l}}+2_&STnT)t7RMz4T;IX9`y%aywKPa6E2&>OGU#n- z9uYH`@an}K`WfcKG}c0V$wQ?|FrZ5if0*IL$zqn4fd5L{kZ|TQAQw=f@f_t|qd_xI z*Q50{3-{BiFt43VOzU$dZAL9i#h;5tdh?2h zp7V_`KsBo4{vo$x2|bf!i#Yf*`zsb&{J2gyC8J#7mAOkETh_1iI^>2<(hpfB5GHIf zA3jWt1g4yXxHvp0;q42WpVV-;^du)QDg7oZrQCde?#|tXmKU>4lhoFZ8B-2e$tt$ylW9vqjubTjW5fAZSdk*pe)1gmL`Pw&G6o5Kqcr$BqoT-UO~aF5Ka=Gese88 z*jOe{$xCvCcZ zh3M7cJ1EEU$U#p0nCs?zXNF&9Z(9a^Ns2yz*B@|Jw>fe^5~b!C71FRXU{<_&Xskj? z^u%mUro%`{u({cu!5_2ZlYzer+BaktGy;v4YVIOIvRT1b#xNPjN%heyVxxdb2#C-|)ys#<8 zqQ<8-BW-kN{hoTQAO%b(fJtH5g;U@@eXQ2_G6bWFPISkR zXFjVHgwD-i6cwPROR~j_Hk0bM%C($Er5&(Ds$BaK9grhS7JjN6`|hcI*%gpqB|5h| zHgwY!8wsk~uw@?Y0QYgfB?>zARx|bZhqByHh6#mD>)A+UeyT~F0ZE3>^<*D{xK5C} z;>OKR^8#5Yijf4*qW%QmqCzp9a7<%cmQg6Lp7j%wv?N=tPD#5OdUm7l-cf7chk#o+ z7N8Lu8gXGipgi@Ei?;mqti@PNKGWzGSL>gft}Ksg%>Y@N1CWxc@(hJyl{jG%jIvJk zJR>O73z|)YJcm~#y6g9Yj833%9Uu&o_XBLs-;b!6!-)K!vGBXz7moLt1@^F#j*zo| zP!X}+yeKn1Z9nFwX)cCZP9f|1Ve499XLG7qIg!4F%m$hAiD2P#85UaAiIaAO`xA_E zXlUTHdc$a6qn8%Is$LI`jl(C@Yjd8En%bJV>GvgI&3HyIWWh;xz&7C;0FsW(<#!Pe zIsHMaKEYVrg(NHf92|nCltZ&<`{#7=$LKs#<1Sj(=?a8HFYTXb{urG+zKJ>;u|0#$ zQC8d|c=Yb7gu2P|fjVAlEL?PD)fQl)aR=0>GmT-0pA{sGSMDvk8qDEDDqtA@J9%5YaL?_;DZf>rK%8;YS# z3Ebiet~kHf2e-uP1#vnPtVQ5%j^ebBU9bM$LR{tfanIN1+NONDJ_6_eW~EcZM$~c< z_+O&@9|@b-e-Q2t3Id`D1_FZjuY?sdwzD%aHnkPAvo?2dBWL+X%-XAL$bz#V3)EL` z)Ko6l=v;I8m0?sxy%fGQP{XK;lw6;3E-q#;_@aE!_rbsXhJH~rptLQ3(O(@qIGD`L z1PK1x5dyg|?goK({T9x0{WAsRpf9(!cy^2g zC9XJajAg*ioY=In&*;@yjv266f{AtNuPV;fRSrY5G!Rq?6<@seEId(c1`_HEY>x07 z+Sfu>AxtqcO^+ItD5unA8Ou3mjs?O`nQ`LA5DC(l62ZYTiSPC~YxGEY>%KRoGJI&_@}o zqX?*@)AfGU_gMhMD2~>>2y(^hB@6GHo+=1dPtC3l z*Im7;LCu0wH{MP!*H#-S*Z62_ar;?V#UbmuJ=+AX{NT4q(#T9%=7j#@+|^nPteL50?z7j5!sz z%xxyI(kRd3T*cvzspWK3#^oBWu6O!h4eR_uiFCv1lMm2c3-Muemo({n9bou7R1f%6EU^S9i+e) z9!z%9$`~)LZMIO^r)g5*g_fUdwwYO3NdmiJ0HVuS=?p65C^yR2b&Cg`3hV>~W}S{g z3NE?>yL+iDJr~rBhSe?m2|7SU8GfW+5e=7}tWRT#tLO=a5P9u!SIzCYl(>QJJ_|VgvKdUDV1@HTzF<&jAV8Uk`$gx5LDKO`yT$T z4^MaNiV3uODUH*-sfcUVD%~%7$p~L@`+%=K*NTUCYcj=Y)Y!`L=4gD?E!{;V_!`dU zO)Cnma1}>BWiRx{(7~HRl%!31uxKdf!j_W|-L|4Xy=3hsNv1TFWft4+A!Iv49F`}O zS@9mCDYxRbiokTN8zD9(s?kcjt&*IGW>=|gh2ir^8SdGm=RNF+#tX$+>Au#?$k|^rLuPn-Nvx? zaJB8Y4$iqOE+BDvy=*OgBdjUd<&4%acHX(fs1pI}Ty{W2zg;c3X45Q@Hf1dy{zO(t zb57lQi%R1I$4^7EgE)KLjZ_%!kSeD%UFf^qNfti4V6y^9%<>|(K1O-{rfiI(?hYUM zrF`_c4gPB(zYRZ`isn#=Q;1P+nXDU!OrhfibJ0d8=e(^LIns!Y?s}tLuEZY*_|y<~1PT?}tQs6Ccg|{?Gb6;~K%wlWBcrxvSH_VTYI<|m^>C?nYEw}; zQ7c;+D}PB7^L_+vEJ1jcSWo-6EeVFH-8PD*jsmh-R4w`;rD;@oH;rFRf~;9+ndRYT zu3p(VXGT66+)VF-1m#wXqRj`z>KihJ1w0f)AS%}3MR+C^&rZxxBE*#AVae<#j=qBM%cMkgm z7Kh7kjs=TCzM6oTK-FmMZe}E-NWE8=U^hnWo%AMkyY?xP2*AQGc_r(Wjsqc+z}Dqi zA4#;=3b{Pk|Be~co)GBTw86Y?k9~=QJ&#s%q%B}UI1JVlST@ zr*xxRgZZpnn;$faq3n}ExT==y)ZlToJQWaQBYV0S@{P62RfM1)5Zkgk%|#`}R19(6 z!cNJNXR0u5S0AVD#Rc>gWDinZU3m)0XS_IBDF|cap2Ex!3D#e>u3pJ(@%!G-sMb_M zMJv!#0Xn(O$Vl2eR8n3faXpRwnKw=CwlTl@1_YJb-3ax)E)5`^3kmD zcR8F?TXsFQ(M-j8!x6sjlPA-h06B%yM%F3%?I|u})o%M4ZKULxH2D!YM2ky_>QKpr zle;x-NqU<`!-}WtmbgINv(;r3pIrMw9BLz9x{VEH+}4S&yNtE#;tPSYzV4K+AIkIj zLrFf_Uys_7)IqL3+w%xdipdt%E~BWhS)(j&Rsgm(1!LpwHPvy4;nUM#9?TN(C2($k zDoV9uM57vj+Vv0)rnJIj&ticGfj2GuS=+0^d-Y-_XlG73>`UvmkH!5i3E`8mfhsgM zW<2sm5yTrd_?Ppvkcc3^M7~h};|#Ne(8SUV@{Ma6s2{lxB2cI9p82~iqdAz9(hNV8 z8ZU-8>kpucP&qYzt2aDHF^fIz%w)ry_smB4gk)OyV-*2Xj;2d7LhXb$uE7;$gl zz4COWaouAV08d_I(qk!7o@~6kKPajIAjmRm(?VNbZ#1v2^+A4A_q+{5%QzT6)$^X_ zJG%49yJr#}i0&FUEF7du@y?#FP5P=uxjqN@Il3<4aYyf*MBifC-+!yU4Na3jlQCoB z2#hNql!D+(*`@jmc>iUNMG z%u*aE5RhsF5D#0j z_Vw|8YzWd~+V{hrn`S=+DSp?PNI|Sx6i3oK@)or=B%B3jUr=qg7z=jW-GDN~H_1Dx z3-DTp0mVcq1oT@CV`HVqv1YO(!cDl{t3q1f9(h5?x9Ii?Q5QHJQtc9&qRmgdtwO@P z&51cS@9G`F>h>D@k$xM6lzmHU8z4V?&bn=MBTuArgNYZQJIsg5pV@?&P8>VSVy?qU zP=sp6LD<^lrXwo066T|Uh?hkWH{Y;RaGj`>$!VI5ds|EG@&Jn(ng$%^kn6XBUYv50 zvaka*HodR}GtV@YY+Uh+ff?r&^9WA!+Y;o(R8tKA7GiIm(t9&(YaLEJ_?#h&uC6$d zRJ>5wYGlez=Mk%^*f=V*|D0JRQGYVB0V$!?<&KZ>=S->G-~#0}Z7hz(xeFo088{HR(8z<(oP~tOGRVzerHc{OGRO&9(GcBV70#mI?)W3@T&0(E>>jC5r)yC_) zcz4xDAgFj*mu4lj-)}4{36b`;Le)58Uv1n}p$e|rlEo7t( z#K=%kdI)x#25(*r_J7H{8f8&V8@d{M5$TI>EBg(4vA3Xz%K7BH4oOvSjvxSRC>JI# zP-#g5<#kaLQr!isV3__z{epp2=8UD!#u*S9(`m>IM~@^Pt2PYuvpGu%rFlX8r3#Pm z;MsDObo(P@qOJd|=FVkG;#leGuf_rI#_9aI2#TRMfBdx-q}9=-*c5(%00elSGSt zd3te(QV#*GNXd~p{{n2;o^u;*=7aUIXUi{eQPFwW7>bFu==p8ZHDBi3IQgQ*nyipO zLAb<^MiHi`B}e@Sb|U9UgzUg}sC#TH6ZB3t#D%iR*29G|iG88ChC~bdB18qb8fp7( z+~`i8$3^F;g~tX?(h=I%&2;j>Gvu9)WSp|HoK3a{TAgo4k83AeG!RETo)?Gsfceo7%R2OK;)>5?Vp#o?`JvQ_+GT9~-H_+95SMzuBefE4-sE9S z&;ff|qS#LM@amDW$1Uj>I=D7cYmn26JiXf%!7HwOQ3Qjeg30~4Ci>CH9X8NE3_I_g zv3HG$L;oI+o;q=oI4370%^Js!vZLIvw^*F(HPNP`7m{W2fqD{he;

xD{vKwgz{6QLt^fzt#+`sHqZmv+6@ob#-YDUdun=MHi#yy^ z2JSdXO~k{75o1vU&ZI(IUV;RPEaZIf>wnf1qz59B^1rsA1q28P)xR}G+}y(0-Of$Q z*v;J2*jv@y#lzeMVD0K=?qKfnPj&gOGNG_9j9j&wC>-=1qQ8OPgKG~}xE~eC8D?D| zYy=cOA*4+)hlCq0&x#fFPCF-o`v%kxy-=V9A9Hfr+@}Lr+0l8CzEWxlqWh9Nz?{~o zHMVDo`3sx%J>d7b5W>J=vs`#$g&j<|^$OCU^)--j5wf-wmZoif|5C2zgE_tr#sAV1 zM`KGNu=Ef=f^OUP+_Ywh;Cdyj17rNLQwKs+U3wu|gqZ(w;Zsq>@8I)Q}wk#Qu`Qt>P-UH$`2r(B_3@+R-_kzf|FgE{&pGo;E*Oe*#CtohjO z#y9AZzpephk&!!>gEq9zeG2>v6rXiyiYk%OK$}y<|HJlwOqC5B3PMBvW%?Bw1O($> zwk1rh9RD$Z|KZkOT~~EO1Nk$K2`0=25Byv3wj+x|D7Xxs4vf81aEG&fKSRj_&#_s*I;%$8n_i%+l%4^sAQZnkjbYnQVByxKj3*D&ygi$#0%qV-@zb@al(7ZNm7e(q7lo^^@kHKYpB+Au)-eEj*0SJ%=(@Gnu$P`X$r2Tg zCvEEmS!?43eQo0f?`X@;?#b>q632A|47uhwp;py=yLV|4`RSjFj-6@P#^sqEaNku@ zxXaPS==7)1*}HS?38sG$FCdU3qYU+MEB8@18V;rnDq1WaLM3#Af!TkTy1AP$?|RWG zhIfQZYZVS3QJ&veB>1)`&!D_YO)zPMYnx2>>@gHu6s~<3N$<-~f;? zJz*@3hz)icp{xrIP2zM}*r=%(k;QjN#Jm=pbv55+?%D$sS07DKzwAuJ&xv$U%~=!QNdO=i7Cz?e0^=Z>-(yM58aA-$KvY>+BCZy`aVO!C zt3+&opHI&GCT>yn@eC`PS^pvwD6WY7sUB`FJ&a804UPhbJH+oWFMAJuOX!dYTqfCo zG@t6RI^qfoxC%y)+20a&YQpl%7XI@aW1V1cD-&nWMMflbSt9CHiM04bUmqf1?~R8= z-|vn^qaZq!m^3lFG)d_IUF^?y22RCsPJF|Pp=phL?4bI6vkK}%W zWL_R2R94em2^M+tCf-lc))gAYFNBrXFnYu02h#Yt7 zaP`hV^Ze)XRq*+E-!K5u3ES|Z7xcl#3ztegXa0s#!8YP|41{C(iK?^40FT+;$x6ad zP&hXfM;Z4oa4cqYDTM(E1i48H3^H6)Zk&-aJtjFPiHfccG{^m{@bfm3-~yAsBQzPb z!|E^?XmBRmDeC--1wScOwYXVx1_^nl-Mwx~*1cgJD%wg+ zL9dzR(h(+#^2cCDAFe){H&ZD;Pt08F5+j81jQ`haL znAcnqa^G_U$wBxL$aqwad|+MFXdL(>5)oke-E+a&l2HY1OR(|q;2d! z;Pud}XlK=qF`qHvaQrXF&&%rUEmbBv;N*>JcZ+ZB9hD7kR#hLDB!wT-&PSI>%M?Do zc+q_wVN9VMsN#Nzk738yf#EsYizXn5*y7MG+{5Wzcmvpd6g?|sZNU?D&fni^29B*k zp_&bb2vo2lJEb=L_tM(FqTa8WG0hsX*Uo^$oJ|Xo&|e_Muw*IGg~?w$q^#+--Zj0g zUXdJo7kyk_IJL;GDKF$E@pAjwd`VwE;Qwa|6v68VE&OY~xc)U?{%8m`nhLO%VVXae0I}1Z8gKP_}0*Dg<_yp ze(9I!7fRp_OOK}q77Ts7Inh?T_VhlJLX4iV6AsD$ za~~mq_J)n1y3LRSiSj1lj7Xy~pe9SojX!Ec?Y^xSjrZyI7(|{*VnAZAHbLi&8(4Qm zpd`l(+=t3R{WZgK?=2&NezHDEN{D`p3O$rV_m&2?pM5ueM4_GCF4{DzE^>Y8mmK+&yVtCL&_IFyjlo4CDu zV5JVliHGg7`b}Yma3QrW%*86+NUH`F5SpGk!~n^>nHm z#8j^>cWuGd%SmF*21J1EZ_l=$n+9rA*OhGqbSu4AQ6|&~8h_GKpgx0EmOi-D?xny- znC*Tmda3?q0FQoY3_p-Whx|18ZU3SE+Yu`5=9{e<{7=LWg6oD^H}G_)wO(JZk>2-F z7(b(*L4`JH?P3*+KZB7l3T09S30hLertsQACd^E=bmsHYlkKBE0NKG97{7^LN`vEX zSSp3?*wRx@^Fi z$KrQcf!4Q_W6LR?qeh`-Z(sF2hdh~Df%1yomb}6w8=Y*mwWSnJ=JdzIF^XC_i|%k| zM;Y~%kDcHN%b!UaCvDnu&bAo5vdR6u9`k9(ItADFvvSY8X2Bo?@aEL*#R7&75<0GV z`_%I*`KBc*{9c!me94yj_W?KkMvnLx6m!G)8Z&zxC)>tChheWcN9cW-seK{G`Me5lUiLZP>~Xj)-@LNMt)T6y z?3#zS${DV5%d}id!n|8!-hdKJu; zi8O}P!%S02(K|GWYT_)nWBZ)Ri1DB5etxhl zp9X`FJ^gCG2HhBc$CerHS^q-s0}^NM;eB(0O^}*cunLLCJb*m`d%Go$+atjr8EAu! zzj=J2W*{73^TMCmg?$rFo7hn3i=LRlW3KK1OJ!9Juki1HDX)Jc6U z{6R~#4Eq58q4e7!7&KVqH`^k-nb6QB`Zq*ad=BnVO)6c9rhek#*W|%p&P=yVMMBbP zzfw!;_#L_u#og3Ltxn&#p4=*SwJgQwuzxw)05E1hvkJL-ydDj66A$I1isJ6e78>-g zYe@5Ub#Vrnf{zmq7K$^k)#YRY90Wwb|OY>gZov z{Htb-!d_uf>Q)%*>{h6x>$K9%);dMj!FDkw)wxqHh;jprS4F13l6coWWHS?Sgo5%* z|E%0@%F7za{g}LHf9?u6ZqE+_JmxEFh$d4Q8OUaSP;JU}nAH8*NPLEnUEB8^bXq2e`8ERvFFDcv4DAzI zYL#7B$2a91<5uT5WOi6?Y`VE}>7;$mGH2`d+obkaN-iww%}}P>S~_XlZZxqsW~%CE z4_2N-ur*j@gm+FOD;SouiAODe1C7%}NZxsT!rQE|j>VLrq)V%5KFhL)=^=}0>x6PQ zDRjBOW1Dm^VjpPC;18B1RZyEO1RXjEacD|(hK;bKE~R;%aE@Meb1!52{OSdcnLIG!-XLj^s=iOH{e$+#T{qx z4;8D7J7X4&r~ztDL_&||jdQE}9eA4#xZCY+Gm8~@)=}RdIFyOnkmUR?+WhicX78jE zZG3Sx9FoH?=SSOH6!r7rklCc6Lu}_rOXfv z^I~Lo*Y}@1G)Ize&wgv~(zCfoQV*yyHZTi_858&#(O~qT|nO!1g zRd=2n;xS3GNV2Gk!EWiKlf|uze#4y(YTCgPxse{6pm$gu@$?7gfW*G^R|-tep&cV7 zUUJtayGv_XPOF-1v6~ro$O_sD_5(I5LWvz*15~lUBguv#Qo0B*C;B;&+A$}RzHahU zBzg-aCaf-eTgKFeuUup)zBC$O=M$ru7MDqj}3#7`)u?4 zHXiQx=jr`Qz;b#r@M!Yy1Zn57LEsS^NR9eF$>e~hSj4V6xFLK2dTm)3&cF`b;q}4o^W6)y_eW81AlzPl=u0}X&26pdTPyLW z{_0l|{>rWp;)fspQ9Ma*fQ2%`qaR7cl;fJ`|hoScedszPNvH9X2 z7VDc(K{sO&7q?2n4^}l99vyX>v2iN&Uj<-cjnv^vAtdTRQC5cZa_chh!w4+|4AJtG zIX`c;v7&V)4w5EiZS@JXGTybHdBaNSqo%ME8&v?c3 zYmxB*#o?&syH%oxM{_3yR_Ef3iFTKmrktjJ^A0_Iw>-0`Er@=alO9+4@rO;T4gTi(QR z47pVuTI|f4MJG~;ZS(Arcs-%@tVkkh7n($cW(U`BOr*8Y`o1_bQbs3t%s?sjO#GPP zZ#rfTY6wF};#5iPdBNMk;vdYgm_-6qQpA~rglVA+^mUQ5UJ=VvKJxqtqff_$nR*BPDx+Af8GeVqAnY*meg1AG7;l}0eU z;+ND=<`=|MmLtlDnp;>ByW=0hOn$kv&CHDe+S;=vgFbr^3|5upNp#p{FILDZ7`8=3 zd@7Y6^U(XKu&&+)KZcLQJ!%BR_U8a@F^Rdx2}>7`tj0kTWY)MgW8P3FOLSX0@R4gt z5NooM<#S-sp67|@zCJK(?wa(1jHLbS(~s+GkNq4N>Fn&4;=4Mh!F=8X{ttxexz~bu zqW;b*+^l(u2$~&UD?a*gX%6~kvRd_3yd_TOLsMswhxTVzJagL}8>cqx*st*V@Ru zIlDkBkFanS7Kr{TuZ6>bK^Rm@$wVbBpZY3u6MWAobNF#!fkyV3JL_XON~^voHc43VqHRp6A!dHENOIKQsdg!2kMkM7#}+O z7WtDV;wPol&I5I7*8xj4yk?8m6^xsX6^Nu7F6<^1u{Oz(J0m~j{Ro+uX|gKth*;E- z+|^nurYJx0nKCNU*}FzjdLq)b!J@>r)%?MJN!WATq8b=UP*+p{JLEO;us-a8XPO)G z&0^GsPPeQ8c@AQ_{A7903Vq-@L%e_uTQ&1UnPV)*tGkN*5If81Ju4cELCmO%>Ltec zg=%Z|*|+AM!J@NEO2T2OjKi+`it<96Sv3}l*n`({j$eOS$Dl&Y^a3F|f~?0#oZ3GN zDi0r`B%>u+P*x>Sld13H0XoF8V@}03wNQm$zE9~}&T#n-R9bl2E0$6qWOQs@F_LQg z=lNps)kMe5vGmVlEL4k(ZF`{oqDo`|T17Q&GNjV=PZ@qz+_Pr(n`2ek0!$&ZDT}lv zdr0IF8X4fX^bl*8>JCQejvchRvJWeF)b5t`T}a)Dr_8b<<4cf+nDWpp;eFlw`15d+ zmLK}J)aI(2vU6I{FVXh%I=szXt~9h3SN%!qjTm!1y-w?j9}*PO*K#8_NAVdr*gnA; zcqIx}9S=7;TCP`@TLX>cJWH2a5sT5phWOoMw}&bXxSDxAq9Og(29-PEQ|NTMR!<7d zTA!hoAdH@<)0m{)J=&VrDb%3l_&NOYXzmD${cRtqAlITr%u542H@smZUwyrT!xwK` zR!}gta-DMADUVpYGfEvvPaAT-K+a){wnj^Gr;WZ_s%n!M5;xK)8i)SSAWS;u+Pxxl zTgl1yb%pc3+AnbX|JZuRAW?!STeof7wr$(CZQFg?wr$%yZJf4k+xG2=xp&^o#EZ!K zQBe^UkyV*hJ9n=2Z3TZeS(Ah1m3f-c=LDJedX_AcP8q%+Cd`J;vLiGWFQ{?ZRE*P> zEZZ65&mRqn_~wIZ8g9n)7JHMNl`YpS=V_it>a&6=%cF%%B4?{Ymrt8@ovZb?)b{wn zPsY@27`joOA%I{7+8h)VI0v;r!|62&M;+R~NITGu2$HpNXHUb$O&fC;mkEzqF>!uP zO>_3oqM2zmh&xe~-YC5zCw0zl&{7>h7<*)_u&z?fcngsb9q53Mdv>wsv}~VX*MN@_ zt*Tg;xNNHrx5#j-=(%E&uP{>X@`LP<{9%A_F_IwWVWv2t95i{sn6>@glq!??PDl@f zB}wH_?6QSin>-O%jZ4?HwNZ}D|w_nH#7T6G15`eK>N zb14AIMx&P>xG#FbZ%fm))$h<|-y^gF-L*-2<8R*-2*zNO98~n*?7H&?qqm2^dqVjG z;6EbNx}k;!0m_e=pwp6 z?_7AI`9X1{%|u2!aWmCY1Yqy`f(j~LgNSw@qrh_*0JCYRDnF$tx1V!-o{u>!prk^Z zRD@Q~&iw$6T!KS7`UAS80PBW;vpzb~mW*@yK=|)EYAw{92j!L>p$CxaAd$7jHLGXp z4NVlTEsZK}vjmN7BYV`IQC54v6krB(dP}KgRWF%M+gIX=G=0Swc;%=aCP(NumQg29 z*Z9Hcx|6&dl{9;hXXO8ZlzlDhJ}tv&lAp`ZBRe56S57~^(L8rWD|Lk6swuG;N7BER37(zX#Rq;oh;azx~;dkakGHu@`>YYw&YqX ziyTk3nzytNtV@H?4mFr)TVC3Q2#9!j&aoBu%)VL||)mD{Ck$0$y!(J7Hr? zf?x+BW2n3X89YYXjMvGsZ+|O5~H{{)8>_wqxSzA0x3- zINT#!d{gh|pO*6Gy?vsspKrDo7>mT@hRlqnD2}lHT%VGL8RAofto9JWo zk>n9r#I32tHyoZw?LK+e$2g*{q(5go$y#Qy4`mcVFNNv&c0p~O*ZQ{!#_s_ZO(`#+ z3|0m!eIAa9;qPFdOj*Ffkz0g^Sl3nbQEQ^twT{^PSgK~*6XklBCzN)xFwToDt&@J# zQO}=94A^gUMe17XTlR>kSf1KMLf zSig{vfsa1}~2l%wPb?_~U!N`P(oF5mD6a2146RZKiJLY@JV zkM_!0Ie+qvKCjT>CYQ*(aNYGU!2MS17i^oe!1XV|Wh3D({yr|{nys6%L7JD3Go^_y zKgWQB?<5(_)DQV{9{oabBnYSbFjs<3S50PZ+D+WCgS)FE1?!(>lPldziJn+bl_@zgIVTwEhoN;Q!HiyWCttC??OU+q2ek)*rQ&uqNUY-%Sv)$R_i zvXxZ;ZS@!an>}Tv+ky^1iK4>;y5?^<>{(CvJ^s<>6opG&HAa7UxB99xX^r{g4^k2iuGs@@IlwUi)v+*F9k6*L|30Y z&3|b>D9f2FN%tsmcdXGZUuwY`66~5S^lDW?2*_}#R@55mmi_romz|d-rR3a~?>v6y zvEaymKR3PJJ6$ii-q*hN9`9RW_`nS6I*c%$9s+|wdWQyeY4{)A8gYNIy#C%A_oBn^ z{aNIs!1u*^XT`bSXTW>j4M&gLRit;}ITq!Mm5MRO!dtn;t+>;r7d!CQL)n9*pS+_v0a@_0^BfUA~P*SH7+G;W_DlnBRwf;|Rb%JP`d<;M1FTV8Um8kNZ`c{PhtY%z7*D)4sn;^7%yJf7m_2^4lBcxqRA9yZ@qodu>7Q?YHNB&C&Vx zKHK=cR~+7_WazCq{5MhBkE-1xyXE-}-ly>S_d<}r=be4`)v&*Vl-G|wyxhbKH`aE_ zb?zJp3n5EFQG||cJImg(NG=(RtOpCD46>f=A@pfhOu8oa*(J+B4P!t8takL3m?;q+ zj4B(5@!c@cpjP0OA#rYaCJ;J2g1Yt&A@Qc3CN`5%!+B$9ZSI>k74UOx%JXtm+HIVG z9u3l})d;AVCiI|LccZARq@gHHFJ^Kg%@c8m*RrORP91F$>JSI zwu4yl+lGaUqDBg55}trG4;D-tJsuN)7QE>CM#&STW!FE>+A13bCPuceUFJS=IHQJYt^ogwS-6vLKo2%3hQk-4#V7mj2T6bPb~=!t1v@B z2@Ek7SIfnHw=%>)+nu<8P15JXECtYupZx4#h0M3wot!-J;lso^lq#Y+@3}%|vSB7@ zQ*l!;{EDuU9R5H_Udi|p8mWKb-OI>^KzcHH9lGqmI8%EfGJxAiaW1YI zH9#G|1|*EBbQXA=KGQprF=~c08#fTAz!T_bxrdruQmY~2x#3KVF}Shge_8}N+LxL1 zQcBE4=dNR|Dte`+qD;&g9RYG9Om+}M620|vmS&|ZwYGrn1SUG(H4A@~F!QSkI@D`Q zl-=MojV{dlGBHSJU7iu9XTz*zW%=h!DOh<3g-`4Q?PwiU1$q z=U3-81a%68uv8iYvn66thNp@aiNy^`ii5CEjKbAUZ%eo+l89LwMbnK$I&!Vg)*evl z$bMah1qnDRDd1mA4UIu9oHm=#nCeEO4;TMiXPuZpw=@c*Ba4)L6p%@cqvV3InQnaX z`dH13N2!(?%9)}qpeu#S+ce4aky&ws-Ij+dE!49d;#$>m0hz5lfZMDn5(Z*+G6+k& zm{}NpYtI0YC){h2)TDY$CTPR6=hW z#*s>UOPuOQ51%(|Y7s(r9_Gmv2BxTqdYj)*ukKzR25L}1M;~9LogWr0luETBLNSts zuU^d*VJJpo?_f)WXn7wj64&Z9$%veMO|Y11i;D%nTM<+Cp2-Rwt? zd!@PlkkRB-Rna3eJ1-RDvm%Z4!$#J}%0;@sv$pMnv19kmF{3eS&Vsp8V$57+*M6vx z#6RqDHR_;K3s31RPx}+2gT0Lv>k_uGnN3Y+++t&#H#>Y(D2N=>X@OMbvPptVsr+;m z1p*X75pi^JZE*G48Ixblq>n8s=2QoTt3Q;MDCtU|wsf%Dt;G_551eQaEF_113oE0* zffh#wWV0VT4gfq6WLCa}C^wB#%Ob2>1(`^UZ=gLtl^{Er%$k{#U=c^0lSygIwjrcu zh7&U{6O$S%(w$;pe!HQVn_S_u);Z!T*tCHtKZe46=#Daf?yz)B`xp!(A36F2OC(GR)J9E-vI73>`QBTyp&xPGMY0JhAoO>_w_Mp<=%rt^B#2k zW@%1>2Pqt3h7nGF>1?mLT9b4Qacmi0$M#(6M6I{)%?_H>2)CI^UhtwdZG(>5*(`Oo z39M2xO_;8F0_jYlcPp%OY5dUQB8`=OiM>XGHRNRos)J=_GnGN80T49FMSr^yrC8Ri zn;U7XGj*fUw2Irkyo?PcqJl5;X~n*FZavcS-=G&nH#`>EzYm&Gb)bL!WGLew6r&;c zBW2s&!jAZ(glaKFt3!5kDV+qWK`JF!Y&Q&X&8b00G2mb^TPq9CJDJXFKAc)@? z8}W_gxx&Z98BBp? z*Lu{mO5-4PSfJFa?pF8s!*A~AqaIgT?sKVN#9{ za63+_rJ5VFhd4=0eu7SSIyhi7?_-Ehm%K?_?JlW7zK^tsq!tzGd`9O$Z+YrJuF_`+6=;_wt{Xvq90N%=^+H4o)M!5U| zGu5Fjl$T@^URX0PoTc}z=J_#4bF7_H3;f+CZ z=!|vyiLFdZMw7%l==V_^B6rZs0(^&a#}5J;VIw=_}ToHWA^CfCp=Cy;D&=wtP1?K>AX0QU98JBPL8 znz8grh;@k%b#o&Z93pGP$+$ML6Kin2aPmH&wjctx%J+s5cqF3i z-49y)>-S={lfh;$iBbP;WifRU*z9F7+VdmUzinaUqS~>}*+(0EDTOD3y@9LFh`1-S z<`GLJp>@Jj*Wx&2&k9df+rw~mx@sV1a{j8D2Eq+FV)E<1?*;yc5+LCMh4kZB0&v9$ z0Pss4`5$YX|1Jh-(Sr0*Sz6*hF->I8-~p19ARrMy2xcGzAqs4?{0EdkGDPx65HV~F z8U}{h5RI{wuEnm#wSra)wc0H}?TUyMG8yg4?ncNsP(`Jq~ao>d! z?V@`~T=@;=EW6@C319V=>5L|A8H&Dc@}TL}hEpadP5rGs7u&^dv13f!Pmi|Qy4Yg+ zz}ce)h1PR(+?G#k=%RRoZp+#AIzB;8aQW!yDbO`Y#(whPp}BFYNX zn=4V+g-QRDl8-e`sfY9q0OV?G&9xJ}iW%#cum?L_*> zk~s052tnd10tC4%8L{YcPi;|DL{?LHYtOjKVYIt@jpj;esp&ausbodPyQ{5j%#*J6 zuW1c?r6$dis5GMDEOw+bbP-3n|5w>Q663n1&4ta)#^PpUX=gAG%*)HjjAu^OFpb5C z%sNGJUhSB@vVa~#n(pKK6>&8u(RCi(#N|q9%M8B@sw0<7M!;cXo zGHFmq-8>OZCYXiJ49qJ>PmZ(IIu*?*+wd$;CbWlz8B}kjwTP2`xF#cnfV%*)GM}oz z69C$gQ(&7QK!h12U|uRqk1w{2346_gJfRjifW4`O50``Q>NjRkSt9J(}@ zbSL7X?(4ErD^LuMUH5NN^Q?^zCww$bU9&onO)^F{3fFXIlOcRE922Z$}&h1u4cD9|!mgnTO+o!_dw8(E>R zuNl1Zn#A|hG}LgHf%ZECjTsG4PA>#MVlc3Lx?Lc%xLt!y@?Z`(n)qvY`_&SZnBra| zcPnU^!XkELN2_i0aA5<>10I`be`!WqxbqB4%-Af=9pfOBom_f@01+1OICugRp|=p^ zdqUShk1=Eu_!~M;p^PS1J-jj#&wOka<7HHoF0uv30qEcX+etH{c0`YDtUSU$=TWF- zP!0C^eFQqHlUT&v#q;J{n7kbrQsa?E>UJmU;|q*NEv7X-CP0i67)|N>xf_SkH%H$F z3j}FpMjN9e3*vzBd*kEU)8XUWtACNCgk<)c2^wxXhrtwz9g-a-%@ngOy7SPI+FnJB zB0W4UJS|nWFa;K}51od_0zKhRT|folAaM`k# z2peQ>nE}AjCFCZxQPq*6?wJYC3i#cYZcKQqW|5bhZJ10DWgxwxO7uDKXFPbE6hrkq z7_G#Wb(q-Xiyk-PTz8J4(7O|3A!)na#(;l zr}5%5o;JDQE7`HDWV~hy;+J{Dd~^8&-{WVHUwq02^$ttP>*Yv^6t=znsrVn!;y3g?&#G^e^)#P{I1-{qE`K-@Uo`6iz@y zc_IF8=;u$^%Zn_PV&Ycbdus2Ozct)^+uMVZPD33R#r$OUo8QfQpx-@$`6m4t-z|H! zEZ#95MGeX;*ePx1pN}(Yt|davp$cpe@abCnWw#w}Z{J&k@lW)!y|O<-2jv}vk*X@B zH4FV;8V67?)EK|N7dAM{ogCtlJ{`Ki1P0^ZOu^duMX z&)J!MYw!O-o9i8`6Y$TVfccKoV6D%dm8e}ITu9&fGgityr8=h2E{;QlnOSNo zxyRUS(~y%m$%nx{)sJ{4P1ZLGbJ=(dLo2?}P;s0>v@+|nzY-hnVMeiG&)^L_=0J3S z`bQK#ua+1yLgVc#_%R!C+Sp`-V}D(MdyW^)bnmp=7=&9ID4flhH*znWL}cUYEd?`h zc?@qciD@zeJRt}-%^qOz$+Vi- z9NMa>6dMD(@?^FW=ZLXn`Ui9IrE%Hjs+B55AOsy^XZ}1bPPf|O5t9*s`At&uQM61} zwBNXySE$^ii<7i%)jRF#cf%4JR||H+DOafpF9HGqvV{J=ad$I2Alx>NhjsUqIlzTS zh0U+8#GH%MGd88LAR$^qA6@OQ?mPcEY3iPD#==^-L6o@>+GTi!pUngoZBqj%vJ;*9 z7(@>D7+589{TN9h7qRU8-1)1)JMa*4<$%8+<3)5x%R4?}j(SLdwo zo$8zk2(2(@iUEO5$(@)^b-(03Z_0U7K&{g`37w=I0YyJhtu*GJoUQ7xV8JeKOeW;L zZ?A|Or?k;mtCGYEgWq)JVa0x;==e+}s%eU`FGs@Kouj86$w8^})MigJzMK#GqVcg{9&9Etq;iL$y*>dNd)5;Bu*~tF zFxAwjC1!yBsP3xKXMiH&JtoKYDe|mmvcigg1!Km;PurNWVRZV5Y=nh0Shmm123QvArJsWu9%eCj;S9kKOtlQL;BL^U#a8!PIg7{~>SiL)}-vIA<&QJVt zKk*wCY}>3>yt!26=71jnutrA+UMcBm{6+EbNp9hdU}F#Y?*RT7CGx-7VxD!lN@8Or zQ}g_D;3Yc+Ur(9JY#5caz;j*p{JKI_ zBia(<$NF`uIJ|_j_=C*=u5D%sE~o$0j<;eiIz1<>E;T`ksSfp z5g$H`6QgUh9|X@Y(PVB3J^x{A9CE*(J)&Ou2tp=ruvE$rMxoSSH|EF&t+D^hiU z;;bUp%&Sr{d}6I4#x5#J#jZ;1BQsVvufy#AwrcuILw;_wl6eadXWti$1=iHb(mvlJ zOv$OAT(HE(Rx#5zs&Si7yH97|WOM50P)WBOs^GO8xZ9xK0E5#X!+ow$JyW2XQ=^^~ zSF}4!8&;ywZp7RNvG`T^C4X*}yd$Xc+MMCO)B2ZXnxEFE39+Dah!EkjAW}$)X#>a? zYURpEwFnBWT;jFQRJmD1xz!{gXeAx~2 zx63mxg%ux)DzGSRzOyHx?YCf zbe>?_eMWO!8fRqZQc3w&Y~eF^Ds#Fy8GEKke@eo zmq}if*6Uh1x#L0gd7=1fYBE;{O*n106tw5HnPEvXNF6Y9XZkMPy0PVdW?VsQh0MXJ z#|)Od(DZbFZ39Oe`xJRW_H9G=L|%pKth>4H;b`b|0W{O`HLB+CK2Ywgm)6a-ig+S156zg z)H={#)_||97B|h+o?OB0YZ3UVGnlA6i3C%z8w zx`4(8tGI)E=%x zg`Em=i4FQ&GCsmrMirtv)S&h$ESMG4tp(^m&lKXRl4J&zHvS_6qTC*Re3zm!m^O%0 z`-i<3>DVC!=yDVgjldR>Ky#N-5^&i7Y6+@d=a37#zAu~62Ea7AM4%HPwh7_e6egJV z2Eg#5OV|SzK@U231$Ylu;1!VIlK^2GAejC82I1QkCfN2Gn-^@r^#*Q_R^Zi;;1dF2 z8!Q+fbl|Um2R4G=rGMD1y)l7}KRCpUuRFvI(15Ri2Rp(xbTB^nfcKwUJ%Z1}A1J~J zcnP)vtS39-@sWs$`N2&@?UYMW#Lkn+o$-`ymD{e=koKoZ)EaslkU+-$J|NXf68)3hkDF>FG9 zWvg18ZUhbDX^|k-=Tk39r0JbqQ?jjZvKf@Cw!Ru#e=<2%r-=Kxlv0H?haZ%y)bvYk z;^eb&*0K8Q-}x2PJQla_BfT%0dZ$0TUx!sV1up-2MsQTfD}in_g{J8-7bF$q{D;Yv z7;Uf);U~!Dn|%77%H$c#Zm8FSt5>e+6}EWMp2(L+PW99N^qF6Ra3}DBdv7!KEIytz zhDuwlx|!iJ#%YYXK;&AAt3_Rg65*@49cDKXVMmsTPM);kGGP=Sl?aD&$-hvOS>`1Y|Ore4;7R~zY zUpKizlA1t`7NM)9s7tPz3}8*Lm%@u>5{@Sgm_7Af9^_5J{dAY^eLBImCz;N@Bh^() zwV0o8i)*(8r@0Ove`BXY=3k5WR<*s(c+QFqn%>)W9J)cuA$qwG1h-_uEgmWEu3zbE z+kWU3mju@c@gP0VQ|vq%fD5f>%R;$;w#e}M8H=X2%+Wd~Ts_czeOQfrure-l&N!#b zzaA*Od9p$De>aoQ*5_)6;O5)WJge19 zw$D>*M#OPi0?4dlSahumIklMIT9#BwY6?WxIj4Cm1E+Y$^+f$UrSG{cxm~8GT(?d9 z+9B~z*3dl5zCexjmm>#6vVJh|D#9otPJJ6r7_I{1?(rvj!p1)}TSC%cl=yQeYb|G> zdVkW(%AOf?J3lFo+?W-5%qzs59$usz*Y+=kUEUNs3%$c@X?%7FbXBu?C2sJGtNQal z;WE9GYtQ(kR^0FlFG5e>3E%tXao#rX`eQV$9yUulZHR?B@B%FAh?Aiawi z8WMcgJ!cKsAKM~IEl(D7u7I@iZA1UG_QeCcw6>-!TM1gN$gGz?&D+^>leXlz{6#Cs zYNpt#9JZ)i7H_leI-{fA^OPT(2)wYNTnpp7K3FP2i%bko_`kPOJrKzWhxwZcW$H%wv!PF zHWsuD@CW;k@!~V{vx>R7R-->qx+t|q8c6uzU+rt0UP~pCt=>(`;-s^1RYvrTtmCKI zRii5n_IwhLxyo8*+(_+r(xfTm(^A5buhYCUk-aP57YOMr`nUI~8{jUWAX*M(D-FmTdo3K_poXyy0#ukvA%5~emb z_W$X!;eN7T;R6E$BL=f`1Cw(DqZ0#Lj$X{p_7MX^gq6*!9%>Z%${y-=`Pxk^oa}b7 z76W6eZ$=~z>HsTkCKoAhCSexeA#Z9Q!m2{l>3=H`9jma*;H-#-@c|TlLro#97$)^1 zD2ayI1Wh5+e9y&0P=_FR{2(_iAxo=gWME)oVEhAs3{4Eo3=9nn9B*a{AO&E$D!31W z^WWb1fAr&%ityC-mmkmjivsyCD!HVci>Z^Dp|Po`or{y_fA+^VNn0LS5F>QQvTX~m zH6%4?+4$~{sbZzZFo>~{jH1LG7KF%6scCS{gdMXkjI7L~kUh^AtUYT20?QrvYf!Kz)(hP93s6d zQI1WEt`?QMIGK*s@WE5sME6;oK$VF^bvt(>9U2X*QYY2c)^nwWCv&AJux2y{hM3rB zbwCycrUap^wJOp%cZ1y;HVpAM4|lF9`i=D125Qw2W_!p6SZFejcQ+;%i-9f6%$`;5 zB39w`94kI%JYnNfLs3Y8b+jF*ItDTWNJG@i2ZYbu+d zS`j{N=)U6pHGv3TJi|->SWzNg^fmcenz zeA{vU;-lO6{U=_pI`IB*C&1m}nG9<(Wk*kS;q+OVy#TDGaG}U< zX_HcuRkJj8|Ifi0qpGWnZHnT@MhHnVAwC#Zqm(b94UueNYnuWu+Khn3M-mUEde%r| zNJx@l667`l%}@6zWd1k;?H`2hCRQqH{t5U^HhSl|vlVQr89Zk4(rrG&IrEVF(B1Xx zY#r4Huq(n4ML*7*jR@OMj5Xkk1lwpLpo>DJ5uLN}=Cc`cg2Aj7X^6CeiMvgxUkJ!RikUY>m|XcNDYeSE12L^`Ir$FW{xlJ)5XHF-iVpBv_X^A=WP? zw<0SRLuoOd-Vtb~IuRP!0bLxu!RT2NFKt%MWH>74O{U4A*3jH2EFoA`B|L@2SAJqi2(#?Q(JxU0L|#L42D=s#6dY8C!d!#Wt=`z-UtL#@Pxf=hzY?u> z>%Nu0J$Uyk+5&BIot9G8JOt1Xaa3x&jy=CXT|ADP|yg8Vl= zveNo6Q_bKGIV;$YpV0*VeIEOq$wRkS^{rtMLY#R}9x3r3Zr7f)&BZG{ZP| zwT50`dIhQkAmeZfX#f01-!xWPuu?-DZICtu!Q{=Xt$OYS&x?xQx0xca4XAa}iyMz@ zLT}*e#2!KN*SocYhN@0LUc+Q@4iPtSLimJs*k#;yNvmdU&du%+W>QF%x+8=IyRw_k z1c}{I=7eZ4-)i(IsngxmoGS7?acvk!vaudX(tKmBR6C{ilWBr|+C%rI46%^-V2bwl zC+1A)#W%P+w!wczdtP^JyQ3)0vAeRHEd10RgrJ-S89GG`TfrIUzN*Yc^qrni%f4B; zZev6wR@Yo}T=sY?k^qQH@Az(FRG!KglLmxU2X)RX%{##;WtN396nW|W=3Ouh@MT{RY+s}A0uMvhFsu0VNMfBny~OPl}v9J(fJD=!HA{;NZT z*5ic&oSp$Fv?7(@DIg$(070c3OV52FxDmo3Ls1UVUzNMT%x@j`O3=S4M%@?Y-STC~ zv$k&PcbY!_p6&Se`T=_ccE^H-7$nZniP5zA28;gdPFu*kh$;-IGZUOoE5qI_erlzxH1kbz~+4Y5-S7+DDPQoqDyy&yOyG|$}WoXTso)N3(h^N zzt#P+wi8^mR-1k~YnXtgS=HJE_aRURWKK%KMPkTA#=)&D*)KMZC z0jbw030yKY@$$ZgG3qoIa{5(b9loE##MKTG834xGqscS)htIVupc?kZ90 zsP44Rcdf{;PE7Yzi*B93_4kUQ8oDyn!^cv9wn@dRK=!Q|JCz7(5sRvZ0IgB=$iubn zDv|5F?yL~%#O}0icFD-uAC~r8`unkSx+B=5Qmk>hMfYl;?3G`&LfWV|3vb~<fCQ(<<>`P_Lq(g+JZJvpE=rvF^tpFw`v^9l3s3O9Y=AHHm9m!gnsCUr7I5_SiiJ$`_||=*W%jz4ak*))(WgBA6?ixyL0JS$U4}co!Bb=r#X{@o0HjT?=$7 z%|k4BS+Zotdxvu6%GxXNpj!=w z6e^dn;w`lm5rnjO5OAU_geEj7(U0H- z`K=knp-hYIzthw ziCn~kp(r+8Xi)MXh+a<_?T-}^22Rx-v^-X~LY+Ph%xt8;vLE@b*lkQ%t0FBPXl3$G zcrtq$uFMc)_+s!ddF10~c~bJSL_1yShW?y6r)kJB-1LlKhw)hz4kj9s@}B56S^a@j z*Ou$*@fq+L+zwE>@&Fm#hV6=34RMVq7k{uVmWq_E5v zAPdJZS&zHxs4`aKfy^0%7-AZxnzO`h#Asp4yyC*^CQLX{QfDqG$(3*AM68MZvxfQ( zN}+6aNTo=Gt3EX=OXnY6&4(&HR@r%L2zSYHN0_a$)!XATuC@lqKaa%1Hx6b+1C;b38#^y+(ycChE>9tX;{RAP-Meo zy4=Yrk}BYf!S_KKDsAG;rxr?^+=e7}J|q}OPeqrz0}mp;nTw_s+4`UPO^N{2bQ>*2 ze&$h^BrG9*`Ll7eM16`1wc^-w`xH^DDfsenQHO~Tu%4d82nUHwf3*&7qiS9{6P zq`~I#bn@07z|R{!S}7bkc-Am;Le&Q1>j|Z-^(`bJ;f zN>id1^;3F8^wrL2rYatpy#!12=Rs3?gvwOCqV)QyVPWLP>C0x3s_qbI$hukYa z9WzDS92P5Xb*Xk}pL@maA&_N?PN+037*znI-C{ux`s1TJ3LmTS;8Jrc;;H21T}6xp zhukf_Mxtfx6yK0P$2>0^VF>CI^!p_+&i}nSjR9CIz2Wt!AIab4gtq{6lB-M9ccr&N zrxZ05Q)9Shgp841B0`A`f5snYN?8on8VsOj!;c?oRjV=4%8n(0#etDGU(u!TPrX$5 z@cV;_s*k{pD}$t9Ze-GqHDEW2G;>g8jls(F6cW~C5~@3QI)Mf{E|MrqqtK_| zAB`f&m?61XjuAJ!8cJh)XZ!VwQ2FQ56(>3grbvNKe^!#P{8kh{@BuQ8Z4^&de zjRNN%ake3=c~&in#|xq zZOzTi`~uUCjhtCBGd*dBzbtg0CNL7)qW_P|L21os47O>Niv(Okf-Fqz3|3Cgs!)V} zW^uiALv@pza(LyZ7pw45%^RX)JJQ*97S1-DiIz_OC!Aq|2N=uD0@12f8{OqEgT7wZ zC>OnoMZ$8+G!~U{!DFMEJlDw|znQL@UfoF2DmpKHrf7^t>j(vfW@XQaF$oUJt|^#a zOD4B0;sy`&#*DK{QJWMiTc}5CMKqwVuw=!yackxV^b(BjhWu~o11C0LNew{;aN-0) zm&I!d4P-(EP``@1Pf!Q@iS-A11KnHUeE)6Sj7r!MUOl9x*nfs)pEKdJj^%7oeQEA~ z+wfJKVz`r^(nDZx(N(A zy8JbTUn~M5)a99Qvw}ukAXH&?K)y{OcpzVh(6}JHCI!OzVYJO)UPD50fp|@`hO-A~ z?+mBmA-Mp!EwhZXNoenAvyRX?06ERG%K2rqkBGD4w2v^S+O&_1v-mBu_`tqdAw9T{ zxM=Tkv-oYZ?g2d;LjFLHB>g7|mFRi`itKZ9lnUn)T!#XQ4XOG6S-5 zwtG3U{ZhINZMv{Gj3@Q8c_g^Z(J%aZ>el5A3KE{xBi&|JI+L6ZwT(I}{gR!`N7*Gj z6&QfHjDV>Mwu8t)z%p}y>qZfYy0_S69=Vj!r5)Em;_N|kY*~S{bJ8?v_#|S0RpRRt zyAe-3VH8WDg0mh)799DTfew7cU1z}jAtif-|8%4wf$=?r{wkTK1$q$@k262z0u=>y zhwmz%Qrhj&qP}BKoS0C)?E;wI>Fp(EYf($Bmm(LDzA|y0P^iiZ6heFbNuJ*x52AKu zgb#(n8Ylze4Yy;}bC~vujLpjQo@n(+ppIfT5$(=NoGofOzc;SOPco0fYclC$QbJbdZ}yy86wpAhypc@7~BoP z!3{~a4bk$}G;9x0+XK1ej1}qXx$Kb)3wEcy4q7!Xj8gGj0Xl$BwQ1NS0K9fohio4_ zh{=d@E>3)m{qCV5*}evHOR!pohUY|@CCwn8 z0~WXl4miCIrpC_^YmF%A2eXSHK#y_9u54dE5?1b>?zJh=sFJGx4_3uu!;Ch+=JWIk z55#`?GjvNQ*)HBgwPL;GsK&|JexgDAietLApbcU$hA2u{GMj2cIxr9~93QvpeUR zykq@W=U15UMX~Lq%RWyn!^B9SaTS=q3qF+h$y$c`!p~eij$0idKGCK+W|r2=uPMzd z)HY4KDHiLG9nd;k3F51*>OV_uBRl)_jlAQB%KuVdH0?g1Jw`h>bZQi{PbrT>X;=F;L~^4 zPcXOThif0jx+3)tybH7tdUc263Xe~`85bc`N3Z7%#;gxIw0Ng*2K&RucdHpwg5n=& zKiI0Kbaj~w3JHuu;V_F&>`x?G-t-6FV|U4UJ}Q+y9vZU^rI}|Q$p+EY^3=$}%1Y}x~H!Z3N(Fqx$=*0CKm zqHiHGJK_c=+K{V=HyAk^Qe(#|s#6AeG|vrUaY-$Wu0rbAfd?~a_Y91AVsTF3t{vL% z+k=P)T=_~CrB90)Ib_k*aTt9)-2B;aP?|yE&_5~p^@d9Yr?QJWhFGF8W{$Hg2jq?w zV6+f6!?OW#(~3j_GN$ji@qc0U2Dft_-sP;0>Knq3Pq6)Zi9I5fJO)nmUNxwm8!b4x zY$2g*ib4~{bPkY)0lme*@$A~6(M^@~CLKrs&g zBn|tILf(Xdg&| zAL%TPF?T1n2kAc0qwe^$r?yi-7S0U;NEamyU*Sl8$0}SO7#|#nWrj;%xnF~|i}y73 z$4{7-Ug0(l(g68*V;;Gt4VTP?@xSZI5M=|3DwHV_Tw)wy#9#iagv0ehUG10$)%8M6 z=0Wd^#&Or?LAX;OX(l?bcX?2)!xjQ53*F>F1`JSXMtJBeWayeahv83r-}|F7$v*=0s`4iwb^ z**gu_2Ll5&^DTY@ePA0yA|QdHqzTG2HD+@{E5j_K5_+3aYjC6GLeng~E2bI}z-N@{ z5}qA5xV|;y_S>+@*~{)#R&O_r+-U(8Z|xvR5Qj-744}7yb^@L}GMuB*%|5HinTG%Vhc42k*E zm%RVrDQX^YIs!A>HNfk0Zx7_uE66a&&;39M#Bj(O=<n+#Ny|!;p~K5IIW13z*2M-$eL& zk!_?Hs1h7!rM(=Ku!L!`NEz)S@A zGGHFL%p+@jn&@ygcWmZMi#NmlgDDp+^;Xa+^TESs&KjKo!gnN_|K2eJ;KbX-JDNym zu3eS-sb!IB+at!()#?uaL8*oQ8f-@Z0)j5@QseNkw>J9_5}3x$VVg{ueNJdU>C;wj zYo*I13`WH+;jO->>q3Soq#;p^ii?WoXN-Q@BUU#ihLUK;CsI!hG}=6>`3Y+XDe4%M z0=G;Ow3s+k6}KeMDC4AO9^T^dC^`dMXfdzd`*+Od$9d_xTO#e~StK2iM7ea1EJG5>EOhKuNua z&?co^&J8O+MX;_h|S&T_dt z{YLSN1^0QkzbO%qs$Ka}bwKrkg;u9;!K0m8R)*#NZI29;h6xM?V7IEA8N_^2}#IMh{EVf((TnD@B(K?vzQ8xJ?ezwHU*7-Y0qZH3bt7Et33kwrj zW$jb-D!phnMTL{rKJ$r|IHZ^xlYO5>=c8GCx+2nAjvh6wsXyEXj__Abq8in?GVwun zE2k}+9|yfLf&Qx&P(Qlrgo%&jnzSogMVqT-}1a3G*6=znWA zQ1o=Mu($j7TBs!TRTUKt%r8C24Tnx>s7PU*6#Dc5iveow9Mv8;m|o;W2oAeGaQfgs z4$dSqV(r)8i8k%>20mv&DpDC{MZOm1-1Aclp|8~qzU3cX2_cyKA`}VEc3p?=U4P$v zk2l^QuMq@5R-)}upOL<+$#bA7JUFRLWQE|eD*D}mUo3TpsIZFBYzDEwp{J-z`nj-B zm|5AyQ5Q(c4DrY_XSi&ohZ+LSf;pW4*{~F5o>|VsMTYp~p@Uo$z5O5v=n^_n7Ne5X z9zacjs8i#bF2+*5%vr|A^BJ#EQRb$)agIvDBC;}T!6QL zb$losZzcdc=DPR>HX03#Tpg+ebeW@b$D{?7ih;RCsm=HrqGQ&*fUbpwM7z@jlD>YX z$Ydjp7%Y@5dV1+>1Y>WW+Z_X4(jdTsMyxmjuO> zhW&7#OKD{eig49l%lcz^tyE6WuAVmJh~kP8zZFA(ZehX2fh?MLO$wNS7*^(PX8ngJ z8beKeYo0dlZJspTS4~994KtKM8U<7(`0%eZ$yL?Ds>x#?YW?1F*n4b*YKVYX-l zJ0fV8kR{KM{nY$J`e@mVFwtlODz{XRW6!dKratn%+Dqc3x^Gv$Jnpl_dP5l0T>~1r z`^(gBjsaCX*bnSv0FrvIMA<8~dC4(W=gBR_F0M6uOBPgOF`k96qhhmr*qdo>x-py_ zKFSJY~>RO1+gCtb1i3M1ei@rC~l>}u(iAXk?f>XpfoGr;jR`E#?a{fYKi(k^SaAhV6H!qSms-?F}Z75z2mEgd~X zda-fo0qo?gChsVQRv?g#$R3iD0xe+;rdE{2zG5>a zH1c0(thnWv?|$HPT>y0bL8VB}O>o>(9^q=DbvXlcHY1ILYgj7{!nZ0R`F^fcF+ENg zvJSpzHmnk?3cN96ytK2`g}4A3Q1_W*J6$0a^vPXV?RU!x1(TrmBmnCgH=bv^{X2I0 z$kN#e7iIq9ZWl$gH-pL7*L0`Tc?zEDp|7RQ#_u8WSG%GMI@QGwjS`1+f(~l851!8) z4xh#yvM_OMo)du{{su%A{k{A@{JtKj`zHa>!w$edPW;XXpH3vuf83(L@0#NdRdWh! z4ntm2+8F}*?>Y9`B2X;_q6wemK6E@MGr}W&mnDc&`ZnBgqC?UM!Z`(X{eX%+Z%fnI zXY_6Eb&N!>{_Q9_q=W^^95Wgmo26qBBx;5aftSdsY7-FS^TnxEaqdx~`$_l{Hf7|R zc8q}3TmJK`;~hoI2CltXFLW5~g$DFmQOLtz>DloA{g#&~4+`5Fsk6EJ4R)@vvrq8n z)H70mVlb|oau8EVDJ~Dx+7dPwm|}!oyr~8;7oj-pU|w|>>n@8dR(#3%XJ!tQazRG` zr_oqBrFU#jh9d{e7#?aSNxEn1$n~G>m`Ii3#O2&Gg3nSU*2Z1r6e{NGL}4IKv@4Xn@QWHKH7;j(MnMApf{WU^K1 zKZ3{^MbVk48=T=dJn&MKE_C)6uBM}#dhN9dAaV?X41#~Qa!}{i83m167Ayf%jJ`)` zW*$2MKkzXDKk&bQ?B0Et$1csCRos#~$GANDY=3=xy?oZ+*?;{#-$tC}yZ4LA;Iuqo zH`3fQN(&t}ap96za`M#NHA)JldB3oV%G!Aucbq$=c|R?8^op+G{yVSM&}xC5-JTYwHBc#I zQm&;*Edy_^#C5g_opxE9T|r^`k7u*jpJssVWU}*6$Imr0CIs@!KykQa=Q+&Xyat(jV75U&2(xl7URF5lbPtoCY8} zqh2P~8BGM0we=8=P<4;>O#Ho0rHrg$A4;dns}Zi8@#m#FyXk`!g8J}yQ?=j1L35Be z-RJRWl33FLUNb?PwiWZW$HsO)(zVH5vEw#6ta4mI8a1Hu>tCn=<+8Sx-Fs zkW9+%vO?D}rE2-=zg4UcLnm;VT_W4jvwygKW49t^pIl#oH)kw=H5YoE?B`~O>NV&T zH=p7~pV7mg-_v&%&*S?sj)(xr$OFYh!lMSo>chB6RY)4ywe3HE57m&nx@YM;Wlj*9 zS@jt!j9ddZm&5(Bi4m{XFRwFpIUl zH>EKO>JyxDOi!a+WXFD7#fKTcozI5UrWOuXZ;4gP`mWelZ1L?ntq*O0y^W<|O zj&n}_uDEvz?+a;FiZ4VYS*~>BJ>a2ZODO6o>{IzHGd@X1Q_RsaWVZN3BG1I>FVBc0 zMIi*;M$X)jEu11DDPU0iMpTk?AV}F*HvJ5Je@`mq(LZZGY0_ChSN@d=Da0ByD|#cd zAXT4Oz=gRLC`AwwCoFasdIJ1fA^B;S=0gErk@0@~4oYq0j&~$G3EXHtrk+;mWJ`XM zAO_|M8*j5P96+E;ls}&TFm0HUI16TATW8PhQ;*Jm^=Ev?ARthUvL&#q6`F&$_O#2> zq27zzmv3muD+}z4?Zgw>D+`cio^!k^cg|H?%hEm>r8CN3ayUNx1*f6dYM#R7$LDrlCE8oOP`VYt zqQ?EBjN&Wvf`J5kwIV`i1g!q>3+ z&qf_cR=vLwKz0OZdbKb+&^HjlzQmxWsCwyyZnhJ58jv@z!M^mM@D&5sgLeEWd#T7; zNFYCpkvEFL+#!0ulmp+AcB%-UyQOdN$vClqtRJLra*#XdAY-U{vB*5U5jQHp1cU)# z1if}wJr>x_@*+75jp|l2$Y{%2s?0YOp8*pA#Z(6UNxim62Ub5cBp@-q!py3$8mpEF zH8@(lgj?p6Q%aAl5S(nmHJF2~1)cPion66NufUdL5UUao($$<@AzIx6tlc3svJcW# zmb>^z7JpQ!LN4poJtk{wXvEQgR@p#mbT6>QZf8U5Z396OM*s6~BlUMA(0^qCskk^<+L`~GaZp+B+vJbRHy^usIVVk} zE_wrlC=-$g37#8Rm=gzt9r?T%3~L%P=&JL5bI093L}3U6-)9y>_8*IXWE9icDjghk zs@%~Qm-983^Yq1Xc6JvKW?x5)neHrO!ERZMQIy4q73Lyi(QbN#BOa3OEj)P#?J?c5 znrvArvPC)_`klL(28{DdJ4hZOB8}<=p2pf$M1z%h95fj)nfY1~4;7;9LnUM|bdsw` zj%RN}bm3Y(TfbJaHTv%X;X>L4FoWxa4gqDvjKj;0BGFxW^JTi7haw%f?QBM98Nrf)=OB)a3i!~=?7 zdAHHDDzVvZuVm5S-SJjRTa$(cV~S&PhCOljv*B;l5G7zt0^5PKk$%c+v}O@9k5FRv zpT;5*yvMUwMIaP8#9KG$3+AarA-Uu@cXM4Q0Nb955+ZGgP$J1w>MNpP_rLj{Kz!y{ z_giM%o=&l!3~-jK8T0gRn2X~31$U)xiOkRSq}WiV!cp{(&gjr0koNa$74EaGB}jMN zLugYrN7jClhU7oF|0c5P7OD`aS`L5*i9r}R`_*NDGtmhhxoL3y`h@tm3j4mA`*fXs zpj@QXJee`XyCpzPb|31u+|pX=Ux|!;%+2@XsacN@pK_lV z`n%>mrd#}A&MQz|L-fZXv1)rUpmkO0Sy(*s*En*ga1V|K0~4_ zf&DS}F#7@|AF0^AJNy5cv;QH+a(&%WZNJ^M_i#W!B>&$!F|oI`HM0BZ%phd!?BZl( z;v({25B_WB&eX8hK+!Ghv(FhLyn|P2|yGFaR39kJy3>M2voSU9kJys>X0(|L-|eB1Yl zkLBI*V|g3MrY9CS*g>*)7YM(9AD9=|7M>SV5fP>*Rs@(Ib}A$$w>8W}Qy&(bg!xDk z2r5e2K$$-sIBHkQvi^Xqum?T|J9O-K@{^-yS{!%`1Y>wHDlgxU>77+4UW(}HA2%TS zst&TfG9c2r^$$?C$~krBHg%SyYlr;JkQ32q`T5#TH5xdJ;ALLo4UTl_Gx4_;-L_bx zoFeZ~EcnZG0c2Giwm9J>4YPQcjy}v*k-u3l)C1}qHu-_=L%BwR|9V9gZ2iHO9Un3(d#j(KrfksxiB-GWXGGLrHJj>Tcw7an6)ELGtjlyH{yc;hk?q% z9gpwu8oHqK1iQk=oHcQiu^zANXXYKRiSW~j#9J+l=2}I|P;`j5c#HN>uAfIgZ~V01 zG80lB^*tF$F?VgLN)#-|Toy=xun!#DpKa5_^^|##Hie;u#K6Zd#b1<+`6F?!*T!c9 z5A8BdJ!U*UsaN6r_wn?AE={ZZ*G{1I9r>|~w1MFQ)k&SQr3!J|rThgs^y1h!X-$o& zmm4K`b&Vw?ZWrDJnR~@OH^_(CTEr<2Y-#TqmV{TPY(8-mcfesY?>n@d z(6Z`ANTcqCobF|Ep`5e>HGU5{wesM(M^K&WE%}!6E&J9kQhF;hTDe1Upu1W}=#!ey zF9fxUA#8ASwG5Gzh*$^x!C_8@8jGWiOG}d7jO}!OTjGN?VJ$fJP*tBm1vMY?dY>2u zRG-p4nv1DG`VBR3gYstx46voAQ>IPKsygt?0W1Hssr|Zs@NElZI;yrmLH@rvBQPl?pcj>ioh8<8}lBL5u`o%t|$r;!ZL*i zFjCEWxAa+;Nqzc#ITs#!P`{Tp&swcBi{eUr)C3EEwnV0kpp zUBcbISkSh#GZ87C3>%EQuO6^b=YNUUGy(Mz;RK}aJkL!x-T;k;4V99ml$YzNakozk z7LY_4q`{8h_v~ppzSAkqPPT>ix;pO>ePfAZ4tW}_3kahPHitHyztIo8rqJdqIac$- z1bJedM`v{?Xk(cPbCqR49NcWED4v`ifm<0FA-5rh(As6#`sipj2>I$UrEDjDP@%lG z?Fi%oncUL8Mgs$zGETysM0>`ws;|yn!gv8kn=k1I^u)jDcG#Z=L{1r#<`DXHTnTe| zDTVWaJs_^tECVl(8N;P2v{+-J9SC#m>rasYVbrj+AxsAAPI<#%92zA@uZ$^S9gn|f zMOD)suZtV-_`>Y8#^eNEPF3#{IJIs8lViGd_iIrM^UNd}agMbg$k1DS4&s*j4SNvA zDjGm!YjhDaALu!=BcF`LOI)Gfv&-k7;ih}&c)&+boVQ0)9dzMLyM z@=;d1j2ZmPK#g1EW=FqwDiq`05>2mA*&vzuLh_KJ$fV*+HD|(85@@FipR_&8r9d-S z^M^m}+@4=*(iUw}zcl|9BEGI*)f@O>(uu%Fz(gy#?y#VItFU5;BZSAwcV3go)$_O1 zIAg4^fwtfplXmR9UwRhGyTUOkhD(HK#^`Bvx`ZtP8qN zR;cLvUL4{7eOoy>8F~K4t14O}(jQd=?UQ_UQ;Qu1G5}l{MKVOVj#Lz>X4B9H5(aKE z-$s}?gx!Wc!PKZ8$ITpNPFqK3fxn_sT8OGik)Y&ZmYO!V($?47SDp%>s6C_Wxz-ijyhsllUJW0~5khGXHEl!!Y8>vg|V#U{= z;JzD1-J72-FwZQnV-zheB?n7Z&40EAq$lB}xlDx>Mt{2ntLG~koiz&lOYE;7xzH11 zoQ(fCG`#9Ih>{fNEG?a!U4*rq72wk7{l?dT^HMP%Mv88DYX1{XQxf)Fd?@Y>fbimR zN6fJub(ynJQQVs0%;}l`jg^}#rZ?KKl|X>ARR2-Cn@Df*bFX$mkoHevBi%h_X#Aef z^7Fz`Ki%!biY6^zytU=DnTZPCQbelSK)u(*j9h785d_Tg&mvQAKNr>MGZG~}_C8+L zHsC7Kmt`q|`l9$iG>!EsMDf@nG%)da6EE(RpLwXNXoa0LX)Gn*MZ4E|*GctOnU*3# ztA1H~3g-z;*@LIoc>EQ^rq#)d(LU_9m1wlJAhGCBr41FRWu>=RElX35X6xS^M8VK& zl;*pnyUoVNP_pVR+T)*y>asBAu94ane!H-J>aHTe#n{u^4xzW-fEP%8xJu8vUYof% zQdr5pij9YXA&W&0%PvQetu2vkrxzoh-TQ5jQT=j{m=WPteDn479H%>XsP|ER1?2%Njk2L{hlIIoSw+rS6Oa7DI`Lk_a?I?aq;z zU_Zw#vvct-dq>``nos1`1a8~1y+1lEN4qK^(bd797Fn?H?t-SlDW{XDZF4EBfNcthW8;O0PNS|ChO*eAlHmf;IVhIB#(}kkaqepEBdb}Pk46_%))t_;?afiKBAP+g zt}df4{d}564Jf(8=-YRjNvFR)a?8kfg9w9lm%P&=&)h9B(Xr>oc`s*TEZv2cB~7{e zmHKbdQDnw>xc(%oHst4=omXGnO5gd~9b7Dj^gIx{+HNylbR9>Hy~h^R**q?nBzRGg zEfeydk#g5|o=(%XtMQMENG{{_GFq8&(wBIqu!V;A!@@1X9mzo%VK$rN;S;pd# z@tH#0fSF+S(^z#B`QaS+1Ohnj#wTK}S`9eMdwHIPs&+CnQhJ9U%xoN&bXxs1W?#jt z?5VXwTkYty?e||d#m?EQwpD)mgQ`w>VmNKcOCgv-XVVC$Don#dUYs2pkME>+6T1wIei5fO2P zxU8-4*Ho2y0T*;9)G*VhqRtzyq! z5w<<>qER-jwGTKgR5{xjYcl_E0&h_<5g4c2-B<7?AcXru-hDA}C1Si3f;?)#APcR~BG z0^U=1B$9UvX5iO9k28Ey(-z&QCREb6Z~~_4K{>@}bRdn%9!epFe}tL|+Bb z{MAzh-&+16eo!Vb(8E~qT0aucP@9rJ==8K*#d9Z**`SOXs%)|@h4PsRc@+?Lvgx-I zRxW8-rjOOZI>GW!2d4e}ji2wWA{Nld9niRDN|ButswHo_p(Y{d#!`a_ET=?nYmt=N zT%tX8Kr;)!i^SjQ!#)LPRAY$^XG>hTAH7#oJfyOB!|c4z=d#0(?=G;6Fmt+JRR4!A zKc+r9rrx!we!K?Ea(%uAzTWkY)~G4mxLd~37ElSj!{D8l z!P;ESdvc0)mtNTJTBzErhW2U=9Y3?g!k_TG#RxM_<9WCEtebiTX5ta}?h}uhu?Odv z&d)E!&n+*kOsl{ni(x7unMgfOd*HAY95nv@F~Ui)dQ@FH=tycXbli%j)i&a0 zu+`yuh2~?%rcl)9=WbG_Ct&+0Nx{DS>-D#gw zvX-@<`gG|@?eYDOdyEGah3yMFSnQL~%(TwGTbUeBogCkx0!Iu$9J5NbzjUCxlyBMW z$|-lIV2f(-hdG?K6df_1RJG9#YfAHi>A+3Vu5ku=p4Ho~N<4mrI?bS|Vj4Hi zpE{)M=J{Fha@=LrTyzXn=u=d`dAnmC$%yG~Gb~ak8M|6{YgZ&Bdz6gK?Qu?}=$tQeslso1O6u|m!-V*OY(TXKgi5@+C#!VvD z+I?FG#YvYlN!mi0E-B-VgWyh^d)G%A~s;qosI8|;N0u76Ld+!1!k2! zOo6VO(bGUPQ3+fPoGs7|Z<_6;iOljy9J!Vni&b}k3~)c-Lgi7g%d>!XVdW7;fjwpN zJ0K0ainuT**9gHCZhOBJKyZ^EXpZgIHw}xiHGNTJzUrqP9{>$T7b1b4%zCkp0S7p+ z>QkMW>k%q$F!f2z8g0}Ay$dgAWcTsBn*K%Iuet7~BJ&YzhZW0onA}>{gu9>ow`EX*+qW9JN z7>(a7^M$NU)BFgOoSV-sL|}W2gY0loU%c;Lc$a&8!g(4~mdQgj)(3wIJxu}{ub#vK zCWbr^lEj|_Y{mRcg5gf#T3zn+m;w`#5^%E0j-fPR-lrdV{=(j` z(B>$|fK<6*OH5Ixx^(;8j>9MVSZmRU=X^gc(a5gWM|M@#+5^!xaukTpO)KW? za9{l9a1 zrm?;9eS+Uc#(*_7Rj>tjN{m04c3Z3#tfEn*Ui!@<^>DUQtJJ=1-_BG7Q(Vfu0IrLl zU1=X*Y1i4)3_vL-R1IkIkP#_1(!;?? z2!(>>n+WaNzN|i3ZYmeoXjk_a{1Wbd2yNJOCl3HhXwy93;yKP*-a6h8`FuZa769Un zB_QSumIBuxX#tc%7X7yDR~Z2~>RoXVj~J2n%~$B!i8F@kKz}sf@3cB|F*Vr^Ykq zt4hkXx?_N~&96>O*j$-;vHxjLn37kB{W)~?Q@m&xJTFU6DgQIdb~ftu(z0H)wlSWs z7!5B|2@oA`WPoZRcRv9IHg^P4h^HbhHO)!$w~eiQ62tX$V#0cm>znd)m&2?{iHhj? zmdU-ooT%-qDl7^^q{rZlxIEVz=CYy!&_24d+-e+lDp#O8^J*~ zh`LR&*bkYP^%{R+*@}VG>cTM3O<=e#=~@e9rW@SVcvO?l6lsB*i!rPQHx~zdQRN$} zNHw~w*GjLxnDW`fxOjpcU68O&#xo*pK!=o`|YEyq32%5j)9N{W;`P(ZO#dSp2yRV%m7iyfXJ6X#%Lv@Q+~dbJ^MM;}evFa}PUZM`Vx9-xfuio9 zw}-*tkGbVgkDMJ6G=lwr2n7e(!QyAx-ec;n9>z)EF!h_li0Yfhp0tdt?kZRX!W((TR38Ix`K9k{H2+)neR+b%m#I>Yh1p_VpnuPy4dI z{uI1@3yXhqD-2}O;4rGUe#KrHKlPx7DRQ`#pt=QLkXnac!G41 z1~)iqrI7nh8eLszVFPUI^>b#3nMV6A{{Ce;?jNaKdxmXuPJA<-Eu|ezw`}Svs;oc3 z|24kVoRJv49d!y<@mjy84dtsw!4MEW3j5_f_-;kUb`Vu0NSJ9+HfICW8ox?5uA87T z0p6NoirGv%zqXb-8r&VC5O4$gl}oUv06-T#)ZC>46eET1L?;4f##ks&m-iEYQlf0M zC&UuQ>sgfMJurbFAjyi0AmqMZX!DJ;MF1@mRhiF@wz>fIa|(r@t?rW|Z-9Hn6@=V> zaF~R&#%7ig^P);RS>r&Hn*?aqM2sj>qI*T@iAdjRfcb}dy&g)IjT}U9>kj|OyHbY zJTEzG4~cP$YY%$Q`HSVf7p-8MzvGwR7G7?5eTSMu6<~QDJ7iW{T@XNzDH5g|FhTr z%ZU9SJFTypwey-dDxWoIUxe|np-tFd2LtvQ!?naBI;nX>VdP>=$|BCwaWr8=E)XWK zf!pabS&>w?qWS3$k`Ii`t)!k!!`>X8U-Ju9rIPBKG>?m|tyMy-Er1Mm4IN zG%ONA;kb*}GCP!LF#|?)m+81~JJorNG}o1-?_}3B;~!ZzED*7Lxvh}8B@G(+>cVum z*1~e$)Z7s-6?V2Rd>+up05hhFq#whr{T0UdUQXS%rZUy2u7=j)-6x)ul{sC?Xv5^P zR9tf7e=Wq_#!p>TRcx2lTIB}iUY=wp$py^4EHkQzpFFUNA`Qa~P$Q>sTP?a-_FG6} zOV?t!h^82e<*kguGKsc08KX()L5xJ_R6J)HO+&&U%^F=xsM$ppR0^*k?lazR!a0+F zwf)j_y2OqtEs15Nx4f{uW%pEuT6PTD9cp()pNYHSO9y!*{f^9mawNrE5$xku2Y)Br zm0%@%56%XECpIGJMwZLYBOS4TQY38b*hXmJzOTm6-;m# zneY5XpF@$0pcmMC-s0C4BRpOw#rG!q5Y}$MGsQ=jIYyAFyXN)~<-VJ!u9QM|CZ1cq zrmsRvrsAhS=r?PUcTWC2u$<19l)CeaeodJ`{yjQih)77VeURigOMP0~uWdf6JVvv{ zZpy`M8#bMj;p&!fhWBrpOEA;PL!?O2uHhvVp*~$7?J}m5XJdkrG_o#Qnzy+u^zySN z;hH2ld_%0dxx@9|$ggTicz4u{T#-wYGWF!5tb=YMMD z`1AwYM1YFM+pLAZ!S_AaI4L%QK>FOVNbXUq2;5CShnp4HM@YSpyHnj@LsaMnmXTm$ z8MuH9ScWVj$z4YDb5^qcNdg(CN6Zr|Hb6PfYzj(>5c_ls91TmbMsKZxe{AQ5pmP#RI5 z2(9TeZ!(tgOQJe9@v54%E1Z%v4uvvD_yjl7Xq;~#95r(|w=(@^n}nro%Jh<+=d(@F z)MQIN>R@JJH#UFKCXkrlFCA0{$y(|~U_R2A^cS{1;=i5fo2UP`S)=mv zQ(^pDgtmtN|D*NAElq8H{>OHZr1A2tKx1}-N|D2Y)`g`usW;F|p`IEyTc`=u6O=8~+_o8v+_*fyFA%-M7gQxwvussn7TSv`+Po{gYr4Cg z4WCv7&HWKL^r8!dMpH9$LCDVADpi-*l^<(#@=+2`sFkG3|!w9Z9=9pd46#jM`t8U1`u+~#&Dcx3pd?i|rU!d8* z)1%3iC^Id{LpS!`THdMXGhAi$FsX3bxd9Wz-=a^S+meWUU3nphl>BD@OQ_|kcU^#E zkTW3GC91pEbZe`6n-OuW)*%=p=1XY-+s6L}yUuGlA%(p_u=M|Z9L zLR*kCNK1jj;f<0PgDpd278xcp{P1na=yowG^-vm?;Vj-W#jD;?!^5dj3SchXcMDGa z6|?LRgQG$!GkoqLB7xxLx)G!gUKvSs;)E=vB|1r8wqZ5?7Ts{_fntB^fphHA5%iaO zYw;F;>fD2X;{I4eA)0RP16bhfS*oC+3^!@&wjz^h&#hJ+tt$a866L3oRHLu6 z&J#LJ$|-t;%e3L>Rm51Kj#*RB&vRtg0s1k*McK_h zADIwwh-+o4A&^qAWJCy&*TtzL@UEtl{K0$Y5@?=}{1qpmbf(*X(x~R}2_ASCbgHtnZg^_~2Q_zO*?GJZFL^v|i zq>9JMT0?b}ElH{ljXN6=T(>65PBXRhQ^J)?XxwI>$Q{DChw?P!?NR75Jo$mqNnuqh z!Z=`Uhro8Q>TwmC?UQr2#)iw_B43C2TiT~qwAU?3@s;Dr{mtiJJsQAh>PL6P5dG?T zGq$yWZN>AK(Ic@0i-rtvL{UiSz6_0T3cD!Y6>j;D%+lq%xqTu!>g)#B*2p5z6B@q; zfI$d~BcD3-%<~_*+V1X~uI^mu3O7OF#pU7JiV}CpMcZ5%!`oauzM}J?Zb55S5?T2K zU5tWRDHrAa<9s900c@`#2fy~G1jfEAcrEN0gq3NYBX6CZpvz>d)UB7)rWFi2)RR_XhNlKY6zrzz+EAGY2x%F-a& z8m_J`+qQAawr$(CtuEVkmuXa{*n4wK%*4_U zGQiw&d86#53F1(uA(bkC@k!>tK~#<*CnH=2>yapf*r^H*xdxILz%w{RCn!vnpB&-c zD=i7xDI_q6m(ikS1N|j8(2g$eX{#zeU`I*GulG5J_9q|~x)C2efeP-7ZR9RrhwicW*yH)rXamQ>6k?8=*&H>@UyL3 z4b2^Uhey1*^WGtKe^TrtJAn-;pecc^XWyo4`+~x9%)_aSx_(K0Xp?Ys>iFMimQ58+ zHgdlc;mrRzjrC2pY%gzMX7Vq&Fgo=gxe#Tz=cj}&BMdFTvQky63Iu9Sq@&Oh8l|P4 z5`{`f9|;TzgVBKzaj?j>X5-0c>%2Rg)p6Cy^VR7q;FC$G+m0+e31;=@5zSMV*H%}q z+m_7j@K(<|*f!XEsUh?wB743e%2RnO{F9h`FnAwE;zuhKZa94c)uN$b(Lmf};h&@k z&2?l^$w!M7X{ifLR5DYO#Q4{g2tW+fa{cru6Ar2ePU1WV>ac_0Ah+z;38nvhJs&YC zd^vk$C)Rz`|RH7uZnytCtto>m|ywj_O=|KehvyLH;$&x@eqhD&mPc)emAUox%_=2T?guSCfs z6zkK`oMncMoJLc#hY8_opB@8K@O#RFapV~*101#lk5HnjDt`Ge!HZH}|;_PK(X(KLx&80Ln(KqD_5R@5=yzbYPCZ` zoa1ZsYv8iy?8g~7Y=q-%p1X)zOWTqW_3^pXVpbt5p25dNT;wHgfSm?M~ zk2zNHh1|VN)b;H`1elUeMg7l1Ab4g8$ifT*a|R2lhOa#*n!F6wZ3RCQ4t^)MBo1vh zPGp6Ua@6abTAGVw21KV$kol7{bB8+t%UQT%O`9)ASYc>N_RBKQioj0q;((@v99?&Y z^dJA7@s|a)N(H`}D3M{aIpvW@v7ybu-m2UI?JV6P&6>Yv)J1ex>9=wh=|3}{zXsbx zahK{}a~JC^FA8Dlq+ZzUe*?xhe@4K!c*e+u@GjqB%0hqE?ni%C>{olvD0y2CYrKwT z{;K-phSfa`kE+j!#4)^>D9a&`Ok&r&LCjQaUG5B|bj|-obb|FYuz%KbEXNQDYPT*G z9g=3ZS*=yw%n)3vg=P2T@iCcN9%=AWYr?Sm#u9Vrk~qaYM8#B<4o_HU))2Bk)BH}E zoJt4p0FV>LPUNOO0&o->8!dycir1BcY~1$w&P^OCBBftqIK4uGQg^nrPG^o&?QO){ zaH!5P@(32SH1^lrEK=6TdWVA7vlHB< zlGQXsjVPELR{zBH1H#e5P~ zv?dZyWcMPWivq}cVD73)E5_ExM=}k0xqLp%b-gF#$C?lK!6UymZBpC1Bk^dG8G25P zZw^n5lHFn|o@LCSdt^c7$s;Mn6s(1iQ3lwn1;g$RMeZPuXd(}n6qVweGA~>uUn!K9 zYpmLUii+@5K$YnTu9*U8T=H@WeeIi8R~8kJTeB}@D~E936Bhl?C}z2Rw5My_x1UAN ze}^=Fke8pp+slW8YeGW|2hQs26aqcS{B8c8!Ij~*$1BINxZxj{gZPYCB}T8!w>9}u zk^h4gw37CBwM@*}8kBk5rgF!owUO~3-Kj?hXP^i6u;^pLMBdlsSX&r*&yffF*dviy zA&%7(r1*kNd#0}B0xf&$DhV%#YB2MrpDhk31GH>onpI=A~5%0DYnL zg@_R7WsF#1V(3;zJZ?GBJah9OnWTePCm*V!Y(X+2cn+2NT*7u&qCZ7z@x)qL$8?_N z8yG~nW$loom?XL+F0=m7!&Db*i)#F}-$HZVkri!5#um4y193SOFN8F(ggPgmhj;)n zb8Ax`9#%lFZQ2Z_hq_=0V^< zb^JEzlbi@0Pg7#n=ub}n3e=!wc&kqMX0KT#rRCn;DK)o(k=jK}br(ibD>$tmxFQ}! zy5UhDT{I?Xi27?7CDvx@q?RL#XT&-=d$C8tGR?&yvBC70cMzYWWc0mK$|uL8Dcdg) z37CcZGr$wiPhw*aKMyvx=RMsy{!2RjA0Nq+=eyY0JACrXmDZTwx!;YN`$auU)I0RQ z_SuuA*}$B>`#yc&7d@i?=>7aR$N9geH?>GTobL%PFjqHyMs8pden|_gHDkg!8Pqbt zXyA=vVb{Dk0BQ4XE!Us8jcgZJG^904tBZb1ALXh=vnEULMbS)9?C6K82fiD>+Sd)P zhJK`!<*nGQbg${Ir>raQqxFW=mQCM1&R=eOdRSn!#k&C^07tCbT5N?-!JoXe-7z6V z+OVXR70WZfck<=+=B^>R^LG>+g!_%Kc*Er&Z`^<8%v@V=Q~u3G+@;L()jZCbb-&dh z?ISh6-XHZmu54^n1=RRNSIndzR9*gIi;n7?B|%D3ot?JY>0S@Em*#{U($B6;o?~WQ96+oTEG1;cFswb5 zVX;(ySGS^ne6jO5!lSzBC`b9N$y$UGM0O4n+br$CqA^C3KBZwqX$7U{x;jz}GmPY< zhv}$B(|7iOQ*@y{HDVUQ;J?VoM%KxZ&0K^#Z!e=)qfc;3*mAz?ZsD=wJTXyeVVoA) ztw|_?(^mT1s>UcNfz464@Ng1#TC4$l?1?fwU02iS9%{9#NMoLc!$WS=g~pe`1`UKm z1a(VcCa#X$33P48|9aKama%}prLga(XztmZ(tMUxAstCnfU(G+n+NsrRHu2d6fMJG zK}X;!b)3$t$FL(P-NYns?sj1mb3LJ&W&Ccgf%xEDLM~{ZvvKFA_TUr7uaT~R4~5vc z`H9&O2MJCkwN43sW0QCQyXllJ8})waW)fb~-8m+eE?H~71^bB#QCfqP>}uyU(!pGJ z1HCN7IwvoXs^0V@mH6P3%R5j_>2qET{?0Hbm_xwjqBD0GBIjQRjml5qVYwS7;L52^ z`)em${J^|jbWY!JHI;YCVXM~s-6c-nz$F)Lk@UmYbWYsN<+&Rz?b(|Ugr)0cxwipHQ6B>E-Ne##CblK&R{jv_4dTJx}y9u3=1}KFRvm_ zvXVexwHZ%h`%QZb^TWBB^l~3?t0hXVfoA{kN(Def1cOcuGgm%Kjl+`ehMEBJwCWJK zrFLp}rpqPREMBqkxK%uZd+3w{6S|Z~LmDfTHg|i=PBwq+g@=I~j*JS`h4VpqBYo6q zwPPS<&%*tJHQ$>X<%(NGG4<<|zvjqE_o!65QHSfQX>HX#pN=DR(ZxM}5z}PGUr${h9g7PV{%*p1j@BB5P#vhiV&uu z2YKiESiT{|0(ttuzHKha3O@!=@ulh{Z{u_oC1nS}2 zP&eVX%~3m>iY3a5e!DvrE({2f>Bw`}*`+9>gKuaqTg6@I+RHxX_B_8xrg<=q3vo!I zygmA`i+7*P9*@c$;nbEE3;yBzW%{mmlnE->>>`pV7~yJouDZW+be0xa3)FQ7Jcb=&Bk}UU)<{`d~1UnE~02 za7@MrpjQTRA9#;PpDjaYe8hSJ1kyf4%@Y#%MD{?>9IM_?3uBvy$S#LFQ=A{gAQhnwyemKauV)5qjOUMGJ6nmiq*Lbm{{02}1nghUFFCh9?Jw~4{egLYG%eYBWpXiSb~!z)Ipb#%Nb}u} zZNf`vsTAj+&p_Q^oTg?`5)s54FZF~0J+m*QtUOtDt4qG_aIV+%JFc4DW7iWlq0UwO zyh$;OFNW~T-)=qT`R2~@%PcSii8uf2D8s6Ozlg7W;FqK!0Z^ ziU!{$x%!HeM2F1o5Kcd(3J_7fk?FvjO2Ud2eafEWZqsp7qT<9%-W0`*;q*!k>uC8^ zjtUgjAq`DG;Quvc|9Wyr7yecvA^Rq{`X_Ua(fQB?C=uePXOHOMt1BV9er(gP7R#jDMRz-^uHeFSXP2id` z0gZSn6-^6xx(j%gSM$1$^-v!@=_>{a(#Kb7{%l)bcbiXLPkzH}XNlmKen5Yqn&<;m zmaBF`Ab5l2#I7;d)kBchl_N3_JfuKcc3Cwx@Op8dD?_-5jUORivf`$pUj#wib|SI5 z3wOp5m3OJ<>0!MFw{1O~V&iH=?ddKfe}fVuMj1+Xv@nSv>Rl z66|BbPDij!hd+A|(uJMEvvigs$eMtGO;mmZ1#O003)*%;*xobkj!{C zndf!K7=zzeX_sd!7&Q0@nX&{xxo`*T;D^d9YOza;-TGhLtBaF}>^|Y_Bx&-y3RGa1 z2Og+i^Wxa&6gQH2(@y%x{`JW`opQE#iJ}Q%-cjyHxQv&obl%`%POha<>AO`yzn*lKw|1F=*doNL zr>cO;bUNetsYPjKozh2&W_a=mLXLp;oyw-fzV)&x0;Dpb7%KC=2eLM(G0D6>rFNtQ z_gX?blBp}^5C9N!=YXr}nbk9*mm1-&XWaa?5nkL#A+l)q(q@grn7vN{Z1@7hlvD5> zx2iyk!trW-v^$mZ%C?(V5OWL(W3cnh_6ep`Cj&C-&NG@5jUeP(Q3N8C)3P+mZBR>> zZM~I}2!lj7N9Kf-GY=XP+Ju8qTt|qTy4`!i-ZFiFwn1bfHY^`uE)JkD0+BMs(_#|` z=d@PC)!^he!+EEUHXDOeH&7@OTu@QqXrqzlAx99u9a{s(ya92WGNXdAWb`?|!3@eD z$%IH$QxDlF1|Z_xc!IGA%kzXdlNx$2Z}JNqD%bg4NZnj!4w(uyi)YG~Nt1dJI$~_b zVCz2Z)Fc`z?y45;N*8P3otpG|0hZ3nJ=D(f9Wzef7`Gh(5$r}I^8b<;Pox(kt&Z50v1DFJ+{o4b79dLfl;ZhfMQWnOD>PFm|Fm-R8_24!qK%0bq~yWB(~&MACBd)Smsn8 zR%q(b$RBjp4IP!)NYUo9AcdML4-TZM%YJUC3#ZW;>fo(YS0T%8wcRq;*gM|b!m&H6 zGMntH_pbg*Z^3p~&9)Pp%s#C)8=Y%z)xBg=K81hEmZ$31t|qzZ9m(X1$_5B}U01bu zSf1!^^477Fq)PkjK2x%9gpg;Bnzz2pgY^JGpMZwUW&7nAR3mX4&)2U2O89aTgGf zE{kwuqBL-ZoT|;ZcA!-qsYlXqX1K|e4HplYy=-Z`7A6yir6AyHJ|gYajZ-{!)Ebfs zJZ2X0fplqxJGx!@xK=P+lE=!Gsxs2v^n5Od>Zo<1OzV+n;lnP$+*y4-Zmy~WQ004> zC+sWah?jA2m8RWEvbgs7m6|Ttul%r=`WuLjp?zmv!}@D9l^}5bL}V?Gt8B;5C&8VV zLZ-dJz*VQ~E4ajl#nHiu-LAKf5x$tl^u5#GhgPOt@BRlTS?j=ippldef0ep~W0-d6 zyN^r5?NkEkZdfZFUdc}s_$^uA>`*+E%bkvq+{>M5f~|`kwz#e1o0{+(>b7bLHYKDI=^Ks~1C;VSC{kjGC_;C9LD=M- zx8EB!8rc$>;RF^@ji5{?dpNJIDZxP|u?-F}Mc`M_1 z8GDfUQ2HtsWv%!Q(0Oc2!Gf7RFMDqrk>(LO@}9(ePKvRPv$2mAa^N0;2y@*gxHVc@%kd2JwSJ9e$DjNi>OOSC=z3eIHAkiD2}O3 z2v}<-)b}$UBhmV0EIt#TbLiL@U@4t^Nmy372O&~?!aFXhs24WbWd0pbxo=c75 zz5<*{Ww?BomPh-1#}W0rjKvjAj!K=lv0D;a@srJ1V)Dyb)RC=MLp9d((w0saua z8lKu+8fMDT2lywkU3urw-t(A1-B(gq?I5EF}o*-REBX^qXmcyBl*0Al*o)O|!n0W^d zp^ku+b|kSorLJ_nX_{vWv+0G2li}s?r5RIn|F#s=8wpJvx#*3g&Fvo!snEN$FCO|X zleExNNLDV02Jq@fIuo)`w|}LoWa~Rj!TRhB@|3+nzo?sj!Oq|QRCUqbRUJN?!}HEz z%Hyoug=4-H>gwZV2Kgh{GG+`ju6a(Virb0E$%2LYsP9`#syvyr9+Dm{%cPB^=mpoU zy3)7=TIPQV%vO(O+h@2wLDY>(ZO;`mj08R2N_55kdtsWLuPyKd>R zsO8dWp$$Iq>%Awt>;A~L`l{~Tx^`0gs=V@MP!7gYdrFgThsjvQ{nr__H@r3PyMMvD ze$<)I*Ra2uXT8mIK-%zc?$p*#F4i=202-;T2M5cB&Q?ZBAjNBb^OdMZd6K{X= zncngSf@@oWY@ea{NWqtqd&=t*k~&0*1M!S|G`eGUU}L}9A%QD&F<}nKt$|!W?r|`W zK_9j_A#i;v{XW6!1zdXKQHAz-PGL?0tCw%6S`qs?>|51~Oin9is(C^Ci5dJH=|$mO zx|H}N1Lk>Pz$@?!)d5|>b*K&sc#XGG+4`x##05vhz)&067lMYE-4|b$!XRI6fAM>> zw?kcLoJzQbvT{gcrsMr`++Luq^UytNx{W!uq7}zXonB#(Ye;eW)trEXD|e~ElI(9b zkGFLuakPzC<^_E#F6ys&mj>VVrk-M|TxgoW0Cs8qq&YAsU%rC%wbt5b)pwJEXZwIo z=|nZSOwhp}RIE#1=>J-PPu%i2J$%ngnSJ-o2>+vJCSzh_=lHLnM<10pr8QB6&nfs( z2t*{V9SzNd?%JQNHsXmHK|n(i5+A=Yr1Rj3#zPU~Y1DXGcH~a{W+#}*5~uE;4*euc zyb|(2q)pGeI=VKx?0VLCx?Y|hwt%_=3JfsZRJ;oa13h4$?0et|0Rz|kF_Bm)wE!~> zSn3fA_b@RA?+OFdSQbUU5~3Od1UDm!kva+yi6M>oYo5g-ii%}@dFvA0BierT&p*$T z;$D6A^NMCHXD6LKY}0WR`%py>7aP`P^6FI8=V+-=93k%8hSR&B3!M zsQr)Z{q9RHq^VmpcC@*WdFu@sDeC$Va-`HS@s&!%xxJkW`yw?82bUoVO>Nv`46q%B zttr}W$%u7!O_*f2RsC&&jVH`lq793(=Me zx_B;cS2@`%lg#SXcq7j27^AfJhY+0Jmn&GnXr9*hX$%k{saP?UKeIA(2pZeU?sn*) zJY;(gGzE!BVqQDK(W1dbRB~Rpy0%?>jwQ8exm#9Dg6NyOPDc%Y17Pkfdd*)7%S%g_ zi4yDg+Q|IOU-o7`^IL^F#mcR~e%hrrMYE!o zY90}_R#hnRSXuAqFB%;aZ|%KU$3jVV*Vu%5rlO-(0xVaIk+KM%nfBs#6lcU@1DdR} z=B+}kx?)NKa3Iv}ad*;UWF(OG6Ua4lO4(H}f>eUPPBC5Iq_cigA0;{;Ma)_{Sa3Kw ztyh#$~_YhtZ9J#zxy%!2uW!r*FV`|qpL;6ilXx}<&BVeu35<-d57;{IUOb;hhw zAQNM||B}JK5wyRA+!c9IetrkME(?7EqUk6U)1KcFT!~R`;Cz^dDQmPEx8NVqjHnKe zkH|~o)@=9&YkbBo|9!@C@hi&u&o@pQ{u?Jv`X6V8qKT>XH-p|cJ@vm`-6OTM%y50- zNjR9fDaV*=`VI37AI>$jRl)=i4$DJ!r{B^A+w(j3Aryvi`h#9_$c9c4~9w{)}zI4FzcSuuwx^sV|N=+ z-37Vj!Y&dG9}AbI_)l`U2=6O!bl+R0F81V`D5sf~-1*7bKcU zZH9Jf4CzvEY)+b|G)XJhanJEN5jL?eF=5~zaE~#d?t=s`c65Yx9_8f&P~zkpvN& zHNCIscaRyGBewMBXE8L)UUN<}s!Ix5X#O3RBVIZhuN?JHZrsk%r;<_47>6ZKY;@P$5sW&iJ;NK#%kOo$gLmZfT3AN_ROhOrdWCau52wkwG zIBK-<@p3Pga&{a9EnXIKB-JhZH>^%n*;A2%5t<;JZ0#T|4%2A42}DQ$lJL#Za*`CJZ9bFQAqE@gf9FTz4KK3x z62ocRrx`Hki%FDMo-Z&VVC`jY+sdo)T*{=n;2^+={5`~aOc5ET{e6){G&7KL_TLjkczCRDseb$E>mIJyZ4 zU^rKr3G@LVIti5#<3?(&)c%b@vSrx^dBaBWM%7cgCNf3w#!WT$7V2elkmzmUr|+)X z=4!3nu5!5RG>fA$Z+-xY)Mjmo8<{}cCB7LdDh#ORy1o%);{pmuHb zD&K2&tp!>2G1Q1)a|&xol3%0^ibux$XAc z6~MP0Ysf03BpH+wz_&Avlw=qR_3=f7@>~%VQXvw${lNx`+^W>ObpsE%H0`SRWj;~x_^1Pt0CYn z1wc=Yc52|dGB(YYS~?qDv-CwJ{BqH}8bi%kjCC@n0fzF-V?B3R+M}Zy|&zU`>Jk7 zh~(;d#Fh!FVh&hHOWKVX5ikC=yGUYS1c?|;>ZM+kEfHdSipXQ@=AYBZ5#OEQhuuX= zAr;cT5pvPF{PHEJcbwxAT+Az$r)l@Q@Od@hQlNh(y6;>3n*v(%Vq|n7;KEMz!}}K$ z%&qOR=4qFeYX$YeCJl7L0k%11FLswdo4KhG!`CN+P%cspRM8b}lcAMuMAaRu?uQI| z#Y^W%Z721fJPlg7AUL1w6!giff2VbXf|%*zGhVgoIb$G-NljiZ`ma~Xbo2HvRh8}-9QJ6k)@SoWBPT& zqlAH{Fe%KD9x_f~Qo)=DDV*4~BZ^y-Zefz=!;JTasIPT+fy(=6&bzATy*4QkHqBTZ zsDl7~0b0q`Bnum!CQKV;2p!*`TaouXMy4G)XBPC^3 zF~_A5*D9Bdt4342VGkn3KqmG`mquNE9211_*>HtbzL3GNn;`*Ct|=}^Fztk}(*fIn zaEQ51j?o2_%U!i=pH^(K=Rt3nr|Y|W-K@T!NjC&VIK3Rc*nH!3>N^GfFol_; zK71UA#-FdplTf!>(LXV^Uom{R#iz%T{E{}if<3GHKkje1PNJQ(H)6Qhk-h>-m{R;! z_#A#*<Rl@=~GYK6HIrm0E>EkU_Hd>3{zyW#F5d` zVm%d{yACEhs4MjW6_7iEQ>ci-8qe>tv3J~Uiw#iLVg(Q91kA4a&kBP#6fz!RLwpmR zJ|NT5{f)Sr`PNTToPmC5UY^w+Y;jpElh0Tk_O&vtGzGs zfa1GUx@FMoM#auEOy#|b%vA*5v3Hq4c_zJijWCtXlu(qbk#d3e$r(IBbzc9Miuykg z&#e5kSRUU5fXUwkfb{=S0x6lhI2+r!*@_uBo46TxD496An*1xcFIvgwn;Q_p7Y%AD zB~UW5l!7Wk6Rt4g8TeUsJRHj$wu;o(vU#Zp$Kq^5mEeI|NT6cacVPr84Am3 z^zyucX?lXY;pt;IwFc<07GoG_PCj3sc=DHnw1L%d>!`(hH^M5HqPhm4%-Mr`&esDcpPkJG)!0BWBaFyyN8i zCSjRxDa-M!ki`bN{X&-N;e+&9msnAHfoM}P2BmKaVG*`#4FgtM7OXdPE~Y9BoFVbX z`ZTyk;>R-#rF)18oVMF!vu^M@{YJQQwfw}KAL7?k!Hus|k25`=dT_J0HF8 z9mp0Kiaj9STF3kaRrJV#n;Pgsm9)lr?KA+S+eb`z{qmns?@9>#%q?&EQcb@~vH z^x4t#yNna98;c=4L?H@%{;-2FCGdnEf9!L`%y0@%5sVV46Re74l_cV|8lj+@mzW7a zdU}^5WfL`uMNfy}#HG3ufFm~56hIm~mdCQ0Hklihb0(|BEIgVf>wYCFu_$Ex(r2D) zlidwuj&`Q7X{DQqGtvBW@%Yxo`$29~ugyz>SZhk>4rcXes!hxkjp5jrkOwY@6?o z4V?W&JKS?;hdsV&Z8$Co(3XAmWmh0n)4c>!OB{5yO_540=wB=E=T(8=0#u>$z77Fj=s!S`HKr+ zEVqv%1*Cco3u}450Vfw=miK~Y6>MOxw|H$SbYxYFu+HpZas6E*^}DLEwdsC3#y)TB zlSA0s<;9Dmsnsm4{P^*&roXT~VuSuRd)asMhxk8CFX&=wYT_vNO-}7-@C{70AU-PUm#B?sT~Bbah#?#pesW+*=((S!t;*wO1B{k#iz$pwk}`QV$1PvD1~1XU#NG zzEf9ZkV$v|?LkN)w8`1yxxxVNPC)yjqNDNz;djs~d!0e3Tb)Huvguh8KxoBL&3^in zA?Uk3NvqrzQ$$@u7!F3aGgrvfUMkBvUM z(jPsI<#;}EW}@}Np@CTQN58kn?f$%gUVDn!nlig>%N2rH4JyM4xORg4>lLefeveti zC+n)1`LBAA17@~tj`nmE%)$Lbp!19bjKgE;GDAoY&S=CRcN7$Pq3U6HJwjQ}0~kb$ zBZJ`L>I>6sSnYj_l(rPvePP%qLC3^^1Vu0r?iiFUYA^m9t^}b%EFtsgA9FvAy9K|1 zKd|wlWDdh%lt%~9SeD7^R%!Z000!xXVAO2$)s!GJNl6Y>hnQX)d2McGx57;n|L#lM zP6MMs-#(P~z5dxm{O2(u=;3VQBxnD>J&EhVZiyER42&2|))mav6%0-k>@xpv!dyOg z$ymZ)Ls2l+Xmj#Ob8=Cz>h1Tr{C(w?i7aJNFv8kOKLJYxLkZt_T~$L3BH}=2fg*_% zUrP}~TLD?iI^GB*QLrCzL?2lLoz6aAg_VEzkyu5+M8U|w5cG`=jP*_Rf7D^9`;#F5 z#0L7;aG=*C$pHQihwSfg!2FMJ5V3W!Vf>#UxU!qmNAT6vG^fP1{@uFs&X8SgB5C4~ zOE7*|Pf8n2;+g>P_3VuxEdm77zUpzLuoujpH+)CIW(-D{KHvJmX2((6QT5Z=Q%f#T zv%HMZP%RKTRiyJQy zeHTE)RzqO{aAY3+&zgcTGfrD07i+Y*g?OP9LavP*!j;FjA1F2@w~g z22Vb`7=KMK+%uY*wThr;Pbv#HP-k%na*=)ONcNRPMX@|k__HZJ`E|WlA2?=eeX+xzS(JWhe@5)18vhWN{*f|zx@4j*KshzfIRX_yr6Rtce0z9G zs(AF}Pu)t~xM!>F1W$FD*rn)8k^b@3)>sB6a>k{Z`hkE2tx)aRv-DGoLx(FEsd)&U*V5RJfh<~ z{F&GY)0RKFQLXluS#3;ahg06(-CaM;^D~67u(43A%fEZzrZY8mwpF$zOMF-NnJ}bY zefX^>jvTCBc~>JhkV3t;#t4p>aIM(k04kXD>rF_@VHi}e$G;hAyYX^n)esZSQNVw^jeLQS7y>%G4O>N>f4ybq@2xy21^3<_lsLO0FA!&oZ~XII z`bSXzk8NZ;{_*d>?V@O4H}_rL#G*kJ*^aCy*_;|nYZX_d$?)-`P4p#@s__0M8Ip_D ztSd|^YK(kSvz5x4HGf>eH|eZF`02o@oA~Cq#dEZNr+YQ6r{@Rc9v}(ExTum;n@3Qx z?GJ_84k3!AksM*-zpQ*6C$MVAim6B!gYEm*YzEGI3-Xbh7l%_QB;UoobAu^6w$tyJ zLrCctRzV?izA|Q#;#=E8`>dh59~U&Q%RrZSson9AG>C`Y^^`$ci}scW<^z}Pb%Yx? zr-EV8p;t)BHwls0`hDvbiqa?4r0XWu^19XCAJhaRos}aWvA>^lBu^|$Zv8qT zMN*o07&eBX_}c9{#RslFc%nZjCE6(pt(w`$?Bwi&6IT~>-OAclv4s6E&O7={Y=Ij| z_n9Xmzq04AA+(^RLz*g-Spd6rAtsekLTFAfPV=W}@19s4$zMJ`>`EjNE`S9{%nH5Lk z+3%s%`YP_j7Z^_97VAaqj<->^M2ll=BX+kD&N26xjJIxMg-RJYOtp!8Y6{`BSlYxM z7J!xzuN&P$CI8=}{XXUoegDR1%>ypm3;P}HHQ(!><7)qxL;aU0xyZ{%_46b6tQG4- z&n`ZT+~m&$deX~xQ4%FV3H}Y7dn2Q5CtOMLMtN)ChQxdS@k!ypig-@iaX#V2WEYbD z@%i!&Oy8Fi&8i(k6JryD3-+dJl^$&re*`nd7tb+FEA(W`Fl!>Xt!Sa5&CnPdQv$Qd z?vnz;qi5lvN9I3~BaEOl@GvhwpcGHp{}rHmQ(pSf&8L+mmg9{)3P@=)>6!>PV zGeH}}HlcczYMU3I2o6+Ph6x;kQ#Nwe1>8qIAqX=9FhZhlcWyVqC+j`^aPmd)&EN{8 z3gUE#T~!@^JKg?xl>H4FS}B94tRL^39j6po>#tjMqUMUMb4yDYsT<0i^*;8n{coWu zSKyiu|GkSL-{JXxgyw%8ng4n5P}Gsa{-)GVtXXiuW*HKGJ75zI282BuQic`!Xk-XV zM%-H@#GD09TBk`jzG(O`y4CLJC+1xXP+9OY8e6^DgsgkKE71eM6vOEXe8rCKI*}0-p>}Cw zSGA?E`2sndNJC-PdiA~6?*{2M@r4bK+2;CJ^btmB*%(zzl-G_F8I&#g0`PhceFs#I zTcT2^xMj$kaFLeApeX7_69l!uV4y_V}hGQSchYMD{z&hLen=usxcD+u6(5*9i;jn%6_ zzf?(E;_uUZI0mSDHGfGZR zZk>=CkEmV6dGJ5({%DJYyhTO{=+|QZ9(zLt4!zetb$(Rnl@QdCSgQ+gR7@VGGH@-h zoa=b}H>2K$6|zEoKO~1D{x78c{~sRzWz_$j0aZQxkX2B4n9_~sEX<|M1FaPi2xXv2 zX}s$ABh4+P77Q7-SuIK~k}QxK2vp>imDknQ9njTKlr`&4!LNm$1>QfjFTI#NaLFL& z_zrEJa-6ohjyCRN{d`|=`N4hvA`FIOG4CoodQ8B-)udF(mb+^QTITZQ?MlF6$(FJg z5e6X(a1RcZtQ=uJa_B^w-Do3BSGWrh!`FK=grx?!bA+i`-cad6@4B#d=k3638t1ks zcUJAE7_zg2nqW1q_aMvmne!} z{>rI0CtuISoUJn8F{^KCC_^Ee=nOjeSUHhNyclBep|Zk!dCa_!s3|(S6D}4;hSAJs zghIwd<5eIXXcrPxY6Q&D9wLsMa`;e9jf_u?`mIykXTAgki3sa!p+FZeOtG%Kh!>~P zVsJ=CN(iX0H|DL=Ivz|GHx#W^-7H@MV)HZK(P2>U0yl~b;;gUrc-nV-Wv=iYH&BxfER!$_S}00^K|TpEDvCqxFHzKJ z!AePvN4OJB+y>V{Wje~O%_d6rAu$JT6AJ2BK(ot_5Vp`8^P0-Ke$hCTJc!7ooaObk zh0M2kG|gRJrZs57ZszMh+h0NObhQJ`b`!M9*Rc!ka;yiZzQb$ z;T8-lAPzzs5HvT*WvwJ{9P45Tf;BIR`R+cneV+3+w5|^~ifV-B9ezRbn&eIkKI0sf zLI+)gc9WnwNwDBg>;ohp=1!+WEl+|=WLIVdEbj+37QM5xv#GkFcC509vtgv+rEBKe zmv6CAw?gdkwt}yDXvNJdmGQ2c6ZsAu4esd^Dvb#lHSQKXo)i?QY~nu*4@uHx>)*XLG8Jul(;&Jf z)6(Q%vJgdnzi3fmy+1{Dtru9cun+mhzEW{76ic>T7{)f%*Pe(WR~+|*QGBrNhE9M; zGlg5E#gZ8DCg{Jk#>YxohRwqu9f*3gbO=gDJ z8I%!eOlpKI1~J112YMziRQWJ~L0GL4)QTy6ii4N3{iY4z#th1EqtO=Vnl{*Oq=Y zzbO*CM~mJ@GD5l@WGmYY-Ek5nkX5oq4RCUrInHSM*?oyR&f5!X;{-A5DbQ*Q_CReq zLmwENikRYXoh0|VyT!Lg8<*HeNv!|n`$!O`JJze)Za|0Xf)7t9%8F$>fS2*&?l0Lp zFdAq=G;;Bz$j-*)*5vj$J4NFy_KBSlp;gpV3kK;s%UjI$1$3AtV*a-s@N4UvR6F#E_ zfGS#4J^_v=4>6N5G2>Ga?GS7bKz(D>kj)uP1KL;%SJ&%gmIy-vE+gaHDw^(8D|0_08vQmpJ zqN=JM_O=CRbXW0UQ0yX773K1vBzyPBXkCOdl8h@J^-juu|D!Mp@326)`T0bD)#bs| zFy7t8*8_ZPFfM?3hRTeK(Qd0;D^V*e4A`g4`+FG$VanIG6Vy&_$eShOoukXZrzkxy z1&kK0PIa#xSKNt;nRg&_B$hXhzifh!7ciNS8*p;oQC%iY(i>`MNz2 zr8e&ebS*c~Grh|o%Ee5;Q68V^2|V!1E( zZuvBF*^4Cee0UZU^ZUxY#BsP7Z#E^vQ+0|y!MubT6lxa3UrignefV`HZY-}Z=J?q( z;}Uebg!p604b`tqt~p3_P!t{YT1?3$!Ln9{Fl1Ev^AEn{yj!LlTZ{;W2(oMyWFU-KS6&(dl}x{PxY7LUOn%km&o<2sxTe`abF1mBh)(4T=vmEUy6 zXr((}|9^bFV|-`rk~Q36$F{AG?R0G0w(X>2+wR!x*tTukHlF{PnRCxM_nG(ou)o*d zzq)o^wQAK`KP)Qplg1k!VE-LSgLR4?jsO55Lx2bCKL8PNfXsxI?SFsV|3v|50T7@I zxQqyR^lraI%zS31mk2l_taMIAf1S{Ofq5U>@pHf3UO!M#cHX`Go<2rmA6J zredmLV01J8kPD3SgC&{lhWi^Bh_D&SXaoq|5%A#s=PTGa89SKh8yfTf{WSncriu$V z0CD^`)Y50FJ$^o~zJFgywBsKyBmy4hFOfj~F(2N)?w>ht;2p?K9Pmvn$VbLNr?t~p zLB+xWa+>)cpe2N!ksf%w6b$v>;+u43Kl25|mHYo6pTOVa`wMJMzMn@D6#>TteByr( z<$ps3eCmIHi{`|H{{dY=;VNY!VSouxUpd0uPhLO8)k{Z6ts-F}VWeOPx`%s*yT`kM z4H#;G#mT@~{}wvU9BTp>pyv|>Kr;V%2*v;TX@&p0SNNYpsAy@61nBw{@u1gelynwX zRWx)ZOSSPI#2XCq2SF{7r4ovAz;WzH>S8wvN2g}t5v+Fse>cplbEH~`^AG@3cE#a( z)$*sy=j$1An|73n*=l@`s%pR&12nrVrHG_g`S;0DPP@Tcre%g8=@Gg6aT0kMmB}-Fuo#DV8BDh1gXe_qc$%9K;&}*kM znpA5q>H_kJxH=*QNFhG0_W?uY!Mv1CqK>(rK0zWO$Dwc6 zrLV*z+MMkjO8=7iIAIQ1Cjqm{P9~@?7E+@9F#CDjQyIVT$)aJy`E)mlGGi*bsf7lW zg_lZ|u zW5n=h;?#O&!)JWa$?Hxd4ncP1lr9~b#p2{D9_S6Os^m7FRA%Pf8<)Wt$xbO&g ze`!!YGVtk``e8sSNWVHe0KxxID@-k5U9=mx;<&n7{JT#1nW~t^7$Et?!TvA!<{#UJ zasUfI!0G<)ymMC4RQQL1A1;}cSTKn|C;_h=0vUps5(=0Q5ZKHwrGZ|7E>3^~uYP>` z2?6y0>T@|?ZY&l0Ht%|%GOvy6PXfq0d)p`Pthe;fo1->BT>Q_maB=%O z5PQvZBOs+2vEd$yEL(>_QS24E56+E4av>G~8hMoos4Y>L(iYVoJ zkd$c;b^d{1!9YxZCS2Xih|cpEvnA$1g)dPQSbjJdkwSWwxn-?2-uk2Pdj2AGAsa*@ zw3m-J(}>$;5IRE~9vfqDV*56s%*v*meRGF;nIcl|xF2iqPWJM%iCP(K*Q7&yfiN2vF;G%z7_$1>g2++=V8*|2m~5)j?mKQ8nq z2r6`T#3s9m-I|dug~l|1(3}->W{z(@*r>eoWf@jEVxsq-I@SMA7R?{NKjz`v_QlqsN5rFgerG zAu7S*G%NYaxvGe6Z!cF=jEMqMKfMX8aZ z;NL7G`k#qH$ny*WtuhfS2g<%7sbEIR^HR1JD;M3Q;~&rqa0|GTmqM*9G90A5{!Jv+ z7_!VG2v9q+0S^w~v2$m(p!@44^#3)g`>RM_CVGAO&*$~WkH2$^fUCgD%1X-uMs@_S zCU#&)DQKj{t;UB1jEvU;N{8qt#-z!ljEteEO6f)?#VRC+X!`|dXMIMPn4mvz|Ihl- z|6lc^xBFX>x*yTcv!A;#=YOjqWqA9vJAgSI9DsxTpQG!4hW~FRBum*^6=kZ`L^0!aV2-dk+1CKP!5BJEAA(2sdu;=O^$d#h;)t zQ{%w}kd&eyjIKwUEmPc2X`*0V}FaVJ(?VI4gSHXL!E!?g3Q zu>e+eTo}ySNPHPzE>E7&u&A`)4A!(q;}O-+3vD@&>SM7iX7IDTsj825|KB z?iqkQf*T`mw3kJxGlXnq=2o_cZ8f^1iFc@8XVvDIp>g{z6XI8^7;ll#P^s;h(cyNI znT#6BNftAsq7#m`HC$q|WObJ&nU2DdEqy&`@pfBOAUvL1K&bFkTh=+wmuH-9Hz zbcCf>v4Ph!=%5`UalNg>VIKuPHzD?~JyH*adVf}Sd(fK*ja$~CSclG-F&t-GpsMXM zU>x%1l=h%+IWl_O@1D)a@F?Madbt=dsNsZ_a`U}GNh76(&tA_hLD;!qWa7l7HA5Zm zWR340->;CoQS{qG&a`NTcR@5-wjApqrfssjb-4xf4nX-=r!5>ThQD?ZR;wZK^QS^Y zO>q;%zZi&k-@}dx?~S&TMi^+HXLKl6UQyQ=PQR^_q-ZAIv7UAIRkyi?dxn1m4iydf zB%JRV;T%4Q(VrSFQk7>B>C}}L9JTPvN|fFbdb4SwcJ4GF5!>p5+VxA6_GW#aqL|X<*^hGuvrmB~F>iUC#cFX>Q8Ue)ttyCuu;A zL6NN&`v8x7V@vNIy5kb^SU#z~FY5-`r>3GM($1YdtGx>3GREUofc0*e^fdA}r8@|a zRlzwxp$`WX`v08fD>~^LTK-RoPt^Q}NAJ+roK4%b%~%JOPA9kby9`|qv;^wkFZ;}Q1MJJ}eQLlk=I10bmIn9OsJ&6e0IGZmgN z9|ItJ!^14xbEA+`P=Z&{>b0=dI}X8N-H#TI)|f7&O?xdx+Ug^U5>5;C(sWY@lSZzX zi4PiS@o(Wp6HV9)JeQr!kJJFVttDySDvOq4g?I7pMTT>5T%Kc{TkPJD$U?1K*0sn< z>~i>A$<>@oQCodUp+kgf%LSyH>Nl4Cd7R)KsmHvDPN*j+TnCu!?~TxmD+li}FlA=W z6LlEDjQo+uEJiYNbk6#q*KD&VWN3 zEDftlFF~Ybmy7#C_Xy`EUb0yUAy{{(<&-4SBDHG-H9)|hO?P^0aSiu)~c3^Jn>NvrT-1FPFxu)WF$c} zmHM|XAJ|+M(fujPj;tK-@G&CRfGbR-AU_smUA+l6?M~f(LME?efEtm?9G?KBYz`rk zNGwwrdl3owLgWKSZ)>=vkU?`aHm`k}flxyMNj4!?uWQMb=6K2a!rH4u<7;b}CJHG9 zhd3CKKay0orDXK;h;-@TFq=$>jlkw~#dKzo;{(0bL}qTv4volgD8tNRBkvpLjjOM3 zI6y*AC>{loZ$O?J!9qNF*4#ExB5W==cb404O(DWbGJ^Sp-}5jBBG_r-rhm+*n9yIf zeFrJN^hNp-@1|S+1$2JDbk0yMZLs5LM9=P+E4zSlBsum$Y}jAyL;wxg;6 zK!DM{Y$(BC*D1%c*yE2VGA>TUyhlsf#l@%JCwfJf*Fo<|gE;=4fB6(p@$kwVX6%ba2zHgl9$p>yNt(M`u=OD@ z7eOASgfwD}~prURW`naKq;+F?%una*2cc6wA;Chkb0#f)z zA6;G=LdE+`W-4>B9j;`;AsaJ=$k8mu>kWoDQZiOgiE?oBJ*0_=OXwK-zzT^~#ATy2 zHy5EV?`}7v&2-uzwTc{sab+;TdjGfCPEZb=I!x^8mI_VBt2c{ZWfV)h{R3wz>}^z3zAn`BY3JX*8Eg%5Y4S;wks(`$A{57QH0a1UsI zjiA=(d}3;>OnicW=#bq(Gd6ez=SH5aj+$)HB{l$B?8_7bU;i7BKIcrOY0fT83{c#? z(XiYd11)*>Ag}mGqRRD?!zXhmfH%)8(F3ysa1hC@Vcyt=mehhbnxpfV)bJxmHr+@P?q8F+Qk2+Asy7t=7wQBYK8nUYdl?^-lU7) zr&Ny`Eplks#@pCztMlFTBrZMED9r>fHcY$#aoo`YdVOLK#+W)y)qRiti+c)|UzQ^? ze|?g^X{7B>JL<^9^*wjo*5L=mJ-Qb)8m-yX!p{f_Sb1-cNHan*5Ak$N#ShQWQ?SeM zdG|CSH1P&0q%lh63|LRx1xD`|KCw{-<%k0~xd!C2V$_ANeup23WB%py@wsX+GvV4P zYzjvprDJCp3X>8!4rmGoT2e{c;+ui$pNNBc*BZJtY2b8})d#+RDLkVROR#-f>J;HTG# zCzH?=R#qlVkfb>=wq<2mXRv~y#^Qq^qW0kb6q4`JK$(M3lfQO@OYt}=Own=QVdqZF3>;XnYEMnroG>F;Q=fRowSqu#ybpO>|Jh`G%SS$ zU{)#${Ls>KnZ4Nu9Clt{zdF1e%#@9Jt2KtXMOu^hXFHL*?*A0gvcC-+?sCi}XZZ2a6&{iq#`)+G?h+RdBhgJ-&v}?fwyGYTuOd^@|^9$ zU=oAviUoI$rQLswzy^b@14?_l>d@kI{XQBbt5KfnLIdkFE?I-poI-ssdM8Qyv@D0Q zBdoWidv$>G?frY5DD2V(QO)J?El%S}tl&M=yn1{8AoX0)WYGW;Pq>UE#du9<6AMa&^vhVmT;9LMLZZaj$w?IC>F z>Kjk%umRWWs^BN~$PWGQMeh_7q>)`isEsp0?4Q5bQZopC&6~)a94hHk#~NH;l`n<* zWTTzkLvycc?mg#A)U!|0hV99Q^Vi=UsCnYxZwne!zITb<+#*bzhJ~2PqvK&$H;^@F zp+T|JGnyargin@KFio{plbesLSE|3!MevU)wVIi`)>%dx%s7cuuPm^HNbUUu?Q1J) zIqa{v>u{W4$l%umBWL2)W`>^y^V76kF5_|GtlalsL~TC!a)?u~mw=_Im>WDiND9fK zt?6tre$if4yYmxIWo&;BOdq^h=yaCBxQul#v*L9BGiY%P9q6)9BB?8^Thg-s6==`J zI=!SP)cj`Atf9UT;h9IS_DVHBspnuZrutZa50z9TufBg(Z)!K0DEiW?Gy~;Bplulr zx|v+ z71am*rmg&%OvwEx{GcTz5hj-)vOy=gV65n5P(0XUoMow_Bgvz?VCwGElG2p|nO>2orDq(iY z5USvIipx&sq=m$M>SEY>)U6cpB?Z~?}2={igigXp#T{IYeq z8dLe{ZXXHop)53YjV^Hw&P(;l?J|~U=;~`V%Nnf)*8G1eMpyhFyAanfMf?F58Nu^@ zumUf+PfrQAS8%NtfhiS6z8z{B#ut<423h=$=Y`KUdx+-Y#)WmvpN@z3Yrpuz$0OaC zIwqkyCVO#@a*(wnUiiupO_ds7-fjV<%yv+b=Z;K zo0lPY4VZV~F47}4yL1iY^H=GbKKC$WcvwnWME>}bgou4)OXx1Ux~qVXz*9jmGFC!f z&oq4=11U=99hCuozI0y+!kw#PXgQ8Ni=*LAw@$$#ggz@yQ>T+b0+mC0heLv|cmdV( z1W+f(w{nnMa2cVW&yn!Gn-GVc$`eeZ<8$U!@l0wVf0`lZ%mz7L?XwZb=~dB}Y2aU+ z#lbqC7k>FFVs4We>y|d39dq#=2yEpT;+;ZxYJT|#_a}3BJt|-b@1ng5#MnwRlWUmF zq%Ws0=l$fIDf;P2l;3R~Hia0Qz-!t>b@NgrBge7~C(qrBfS3P5?)^Q(tR`Zh(I06> zUYYv&AoS-kiOlOv&imh{%n&+<(7FI%G8!O`@YneBzw4@;JF~>Uvfos#U6oW(eQX>| zCygBZ+F|#Do8(HQg2feeEI~vBD40{Hm*psW7G)Sy?B+(KIGMs=!y6uO2zJ&W_bz-wvOKc9P=1{)p9tTn{nwLbB_oqwDkt+%{xW~hF% z>R#h#-u?rX1DyvvN+Hdy{UV-@NwlE% z5bGa=jFJ*YE+f&)m=gVrm}81Xl8qF|I5=6b2jYSjWMQsy?jsji|YB zOUIdo4G}3Wo`{`*eIJEVjnQzZ)Uvu)kK7H0u@CSZfC%r8Ivbm|1TSdIXthMMs#C6R*(*r9Pe-492^RisJYw z&t7gPqhH*M7o-z}Pg!qn_7EuY;`G+EBF?pzyCc+GPr)}HE=rT~V~degvq(boktt(^ z*hR#mP>X}iFu&qEDQ&-^pwvN4(#7af7p4v6*f(ejws|m*WRd zSGHCM?&YEiBail}Y*cYbg?1ytbgw|B4KgW+=Ct^1XV})H6DKPr0V_gZ45SbUUynNo z-@p`#kIgo)4ly(AJ8(6&8;By*95}7PU^i)dsohF9-A%IJk7v|~`yyR{JZVil>^l}Q zxKF?xR2R!N*SpRRJouwQviPk8+1@TDABb;%j)Ze&WY}K7?EA`3luB?L!3@8(+Rg2V z_L*)ntHV2aG9&P+@DSA-bdaTM)!_CO<9I11#YHW*3nL9gYu0PFNH-2j=CR^}5-rTH z7bylhkt$*=sy+og=l&s?h#0BWaCEsXdVCl6Zs=qWMf(iUj7fIz;WYpkIx}Ggp@zwO zF90&bxGZvI38hh;GhSXs?BgYyhg-x;72JEiE z(Q9<+hS4_Vwy?cmT;;QzkWKySAjUGkKdarGFw4i?FL+&QX^J?@tH>g-&tODp_VByE z=5XFJ&rzI(6Wnl?MRB~f0LK#|!#I|(aF~K+U*XT|qJnAZVdp|5j{N9=0o#&p8Dm8_ zuxsTER{=a|Jj>nj5O-;uyB2J>GdIQGe5|a8?FW^3RE1dXw}#Dil&+BF&!uhAIs3fx zkmPnsFVcIqE6$_SZyIo{k@zT3c13MY&CHQd>zhpQAKX z1p=Q;5E-4ZtJ8S&AamCf-M&yP@=P^}+1mbP-F|x#T%LRQ^DrouRayD!2WGSuEFeq} zlk)H6sO4K%VtqWb^TLk&GjY&s-WqLN_Y--O_X~zI*#=dDaFJUA{*krbrP<(+PhRLQe@Z8G~d+;BJQ z8?gT*whk7bSg!~8=E8sF4*J%@v)TPD@Bc{e<*&p7(SXIXoBeYA^jRCnx8`ya`St@c zpZ1(&dlA9JRUciyCV6^oB5t=u(oKQw01k;;ds(pMJEWK@tw>3$!;#q)@^S7uC zfehMt7VR?eFNd0h866L)Ys;+p6XI!$dM?OYUV+7V8}jitj9g3hU|EH}p*P%t4r)j~ z+b4vp=7dL^X=2KlUm$yJExVf)0Y5I)fz6%*y@QZ1e`MBl*$I4?wY$Lko%B|^5CX=A z5&o-RR-fR@Rm`w*{gv+-&>RKHny;q6KLbf8Jh;|IQbED%h!PJp{qTC_JY7D7IwRan zgXhzvT)w*x>NJox(!M837S<@P;Wju`R6}I&R#d}e_)s*9PPKq4*KxcDWBNgojA2NP z$ImibA{?AHYQQmAi-jzcedT;%0AU}-CW@9{%#q)Rqs&fBm?mLLVn3f1hY&S|3Ayl` z6&D+o_edYT&uwc+x9AneR&y%Q!;)f7c0?aVcjOPAp+~2zqf_BEuK1!)!kw5Ak%a%G zgeN>J=q_A)i<~`5DblLsZd`ktkUbhh&l6pTX5dCzft+Fw&Ee>5o7i6)Y~^;w%j2Bvgdo!W4wYS1yT_fG0b{IcukEysXA?}UL{9GGvot9uxm`#{SM z*=Ak@opLZ~rJ@6mU(*=Sj1Z6&GH9OSdgL1@Zy&q3_<~?pbV+ail|gM$N5G+LrkymW zANSt{3l=SWP?i8V@H=24@_$@Q{)O_ErDQD&Snl?r^*m9bs=493F%BJ70Y}Y*f|19B z@rz3Rh4HM?$S5Jv=9CrZTNnmI9LCp;Z_M4K4IUYf!FRstIK^}1p6xh!;q&$K0^Ln5 z0bC>`CAc%jgvgY7u;_>gqmIbX9vc4;Pge~(9)`d$b%kO>BNU5V8@lvfL$VDkIL%RM z4s)|x`>L)Cvrs{`7G0UKfFoFMl@?u=kv>6y$Ly4^FwK(ROP|n+WMdf;tm1<5M4Ljp zDD!7%G)#F0=Ah5qrOy#rK3fZ>L%4U%A`{2#G1^0~d0s`G#YI{3=qQ=V>6n4khSUIp z?zHHVn!&2brtFJP)+5DrU#yx5WLUHYe;$>Lie$XLXzMb*EM#xaiL}mV2_}EZhn@1*itZ2 zO$KC33XmfO#T)~^MN)azM;$}D+~L)QLfSn}q5tI*yrb!K-OU1dRXd3wMoR)_Ww$O3 zDIBRdfV)l_#piEnw?uZXeBhk-r6GG4tWmAF(1A-E%*zY4P%o90otT0}ycDj&s#H~v z%6cOUAIum0GrU^AL1j)#f%zc8%`q~?5!2q%CaF-fs5j((W(L+5_EVNJf1+Par5bgV0l-lZJ3Awcb6jm8?x(#0-)dk(g#^qhnxKD?M?&NA4Wx zH!&To%UO7%xo6E-7SPZ@h=Y-F4p8?~7(@<4Zozv^Y|3bYO2S?#Z6TQikU)^ZXuCvl#J{CJ4}sGGHi$fI1T~;8romx<^Ow22k0Ie{KW!N zkhYxzEODJRyI5->&4Q1bYIg3j7s4qI07(cbLzXI2h?P?HSnp}{$Q!5*^565>1HkVv zklpbZfpMF41e~Ysm8_U2mrXaodALFDCc`2-w4eGF{MO4&Kf=u{A5@u7h9fGfJ;WKX zxrhH^zt51UxqJHPzU;gvjuW_H3S%++oj0HnH;{kUsATnz@}Ia>L_*hmLjg2R{$qTY z@3?%t`&pfIn_=9We7p7p@607+R{`)aOB&?Gnmn#Qr#s&coh-kS=wjzXcZ36w@!moZ z`(WSv>sIkV@`3-Hk-r_aCZB6$GiF)96|hIxT>QqqHNr^Htlc^Qrkb18b@k{>8UCIm z8}${Ea#(UixtO)$UpiccLM~G4?N?8BlXhW-;rTiCQ>x;bu7xm^ntD493_686wO4DX zE*chE(ysJHOC_#?KC!NHX8P8{kYagvDY*5!TR41|T;3Afj?Zj01EacjSAi z&HezW4RPbwap+rAP8w6t2AM4g#~gB7Go&Y>TOHgbX*Zg1YK81%lrhSTAKrSCFTuSj zFcXU6hEdQ8SY)3PaB3s|8GrPW(D2#(X5mg+lH$2#A24J-(CZQJD^gDrF${?YyRFxA z+x=11yTV!AH$VR7I|ESD75WZHqya$N_kSjnlbNm2f6tqvl(iKB0}mgax)(il;6Q>v z0ay)E@wuW#wFadE*x6*HaB0m^Rp$iikws%R+6bogH{$l&UTj-u5rLt}`|r=h?}Qy@ zJV(-jvD7l;367K5PhL5uTOV%|y4BybglQrdQYF~3PxnEDBXaK^4e4_uhAuj*Zn1Fpc{|CmJ%R?a zrR1FPopMMvmAB*_rMoycZde6jq?K0fT(q%rLH56Ed5lG0oD^9%1A7;G&y}ITR39pV zRu!(EYa0$Rj_OpovuEg3;g54qU2vlIjaK4~Q#afLOJq9%gN&g#82dH}p`MXSnz)X( zJ_xx=``AzlwpAMKnH~|9XB1g`I0Ew(v}ztNK1U9UKG3E!N9o*HkV!k-pGRA#H~dWP z_#vbIvm8t3)2Qr5Y1Q^x(5PXxwkl3()wR0VN)zIke50@N!eKE_y}?-l6UU;Dt&;x4 zH|DWZ)lthoD#p!XQuGZ|< z;Jb{7Q(B)qf9D;vp>&_35zBrJ-e7RBw3T_Y{{?-v`E#;tXJ2D*%ryhDb{_^3)?iv@ zUm(EFb=qZ;B28NYYqW%(fqfzRr!1?ur6jp635!zPLwS_n4|*3g3UeLi%ftKK;*{j3 z`wkwgh1AFu2PRyuAxn?*wB(-!2a=kZGR|+WhI-E^k)LCX)450(M)xO;h8g#*EI*8% zsWFTtkK}0noVa=j4j5tP3awH?Z_O8x3VFRq_#L977`> zU2vqMr=~HmH{qk)ZEE_G`t)yRCRk&QKpbiAG}H#UvabYEKXblE_LlgjRUGpQe&Lb8 zS}&}*F^KLaLb2RSwBaN^+_TLoko>vM^0td3VJoGWf!-MI37wZFb)5y~AbXv;HiLV0@3Z1+Z}_ znoQR{V7d2!^&w4Re@E3%aT_3!t&po$kdO%*vEgEIU9!VK3hFyt*I7lgm6CT3W7KwQ zNlU`P41PA(ySoPEb|$l^t&gKjVv-D8jCWU8nMeEnKpe+p*=z5ZY{}-lu76FLM3~w{ zt#2xQ$?kuODUJ5{j3~>d)hETb7cxgh&Gbk30xHR0zu6X|Pl(S~mOnRdto+Ss?vhB3 z*@gO&)oPaf?bf9`-q+CHFc;Ox!IJzVuTHeXbky5RlBea^_B>*h)76F2QkDT^z@x7! zoJCLti4$|H0u&rkSOCv8Ad~>1fTc|~fx~hihS+raB-Ie(bPgSsDNT1xL36J}I}bJf zhdULTs#%QlswCZ@9UPF{f|0u&9waE@>2komd@fbI54(XAT--z8dt^a&_sVA$s#36^vaBLJp5nCJ_ZP>H37u;G*drB@(1Nv!Uyv!LFZYWcYzzlJF(j zAq8|4{%+8-WU(4T4|9c3O{L6{dyK-8R0#$B!iu^(lHS3Oc&(2>=}-AkiAFwPKJ$V| z7qLPN;>1>QPFI7Ru8K~sNly6OJI2B0EMt~R-jZ$6*b-#+;!%-1M6;>a-#`C_aPp62 z*3fA@76T;m9H1Efwb=bH$t-RBk7_7n>}cpj!Q6cpxd#Fls#$zGFumR!*{dy}Xm7qk)Rx?)T`m zQq>xKQH5EUeo{P3=rOz% zF)~B4K5?$BC*O#!i+FTc*w{nvpkm%G^9)C#Xo7zn@|l{9@F5k6Svjjc-3|Ic=>sus zO;s!ZdBI;^{&fW%YmtR}0l^eHZeqP0leBJNvP_-od^YK3Ur|S|rRL2fff*0#c&#YI zcY^>OSv&?j)N#%o?vaFhVj7}Ub7s=&MJfVud%=CI%MQcCRiP!^7!*H$J)vVaXok>ZCao&rkA5O^VBTy{{m8>Lv(g zYJW}fN0QS%110^nuJ_>BG($K8F213}@A4Hh7ng)xHr9{wgx(k0z(|RKdNwddyeq}> zmFl$ux;VZshN|0yxd}E_h#aSYG+E9he)<&huL7^09JK@@{(eb$e>ZE5=dW_$Rh5Ujye`4kvubnU#z~+1D9eDN5FRd?h*4u%{-Cd zonUeKkJy(}$@ZBOpa%o0MFc1{bh<#q>)*mISLVmF`@A?N29Si5{W9|@I@(E zK3Q1L5JElOE1?|uIwzK-Vh04a56i|JW5zB+9^n(cYF<@V4B+P=SRZ?_c^nEX8+UYH zXs-Fn=HwIe`-|WKx&FTTc;7CtDnK!Yli-UDFCW&@Y^Zg1Lvtvn?AXw8owTij>6=aTTtI&7c;Q$ z89i;;dOp%v+dJUwnF>BS>}I_wQ`>#T&+lIe7an-Ll`b1!f8)6`Iga-R0onl)4VnJ@lZ9L68Q zhIMK`Xu@VapwFUmadB0nx(v0!!knd2nkFw06y=&i1z;0dxnR&RuX47aVyP)i^I@`S z@@Iq>A{BppJmY2R>-lr(eC}(l1-n~ofE9deP}To>x!JQDicZjF3g_sQ-kAYKi|D_MAvVkF4 z9(HAf@$L@)Hx+oiZdU%+-xzpEeJ%|5VfrEAKJdy|2q`Y%`YGF8iZJlyZv9ak%Lzdp z{v-KRcz3^1Vm#&i$d%ryq2H9aPkOYt32i>tql&NV5#FIuyOii_8*>Ybs$6a~CdVdq zW}6qgDoWM;6w37~_>iIpKz7_D2@G$ePXAV~d6S3BN@I z*TE%We?K`%0*lbA{UR>*emcdVIE=N(w0qO^d5z2OQUwm8RQcVMEJfv|iv>HgBkZL; z;>4HAQF%e%+^rP35wukCkCVBT-X(h4^uf)|}Y@T1Dr`OUbQ%V2Ooh?s6K9O4J+REb)? z-b5|7`N))mV(M>k_IQTk-?c`Gptau&=GiAPCC<~$m}|uSEK)Sbc&z8>ghoh8h|&x&iRLpvU#lg!QsoqflruXU&meji?6|5G?SjE}p|k1q zOJaTg`k>NTycPxUc$CkVuE}wxuT5(`q$@uCKl&X?3`QM@M~i}Ov}_3-QtSMWiIPcR zeFx64RCc&wO}C|B8`vfLu%9ys2+G-RlEa%=+5=o!sCyJ(S=0)4?n)Nrg21q3`cAB3 zi1`<2?@lul97IMx%S$sCcKZZQnx}vRCg!{Ck9$%bMJyYEq*0x!nW2 z5XKj;b-jjJsik!~%{SFe<>gM(gJbOEmhWx|^!fsXdrm&Zh~^JrMR#?WwmS|z$FRdZ z@;&6(78Okh&KDHZtpHTL)PAqGaTDqSqvDYd(44a89B|%uX5k)5s3IKNm0`lWgB}CM zcGB5O(F&wUHDYJiSv{K=uZa>|Ked2PdFI2KOtNG(a+^NJPE%}+WMM?Y zg2q{4xdh=jF)?^Mo@l}@uWNbI@knt~HJ-|OH|PaA)u^FD@<#m9TKG;EjA% z?&C}K8n z*mXkR;YbXa^=|3;VB%&_Nkcm5R(TrG|+0SlPIg-EFZX8N>L(9}-K z;0m|pPP;&v~FNtN++xPQ@#+T^Zjk~PJmnv@6Y)9y9 znkmN#ZzQHs_J^JZOdEy!ZMZkDkQfozpAGuL@KuOM2k~T|`!?dH0T5>cj6Lrc@-QI2vfiY90_Kxp}Jr<~{oXt++`B7G@|rgU&Kxz`uC)fc(d#|K<{F>r@aC@CW<%?$cN zef`SIRA+*2Z;1{tG7$-oc|@OuV8TjorpwD!ETvo1mbP;xG(CXZSK%CcUq+Kceu^!WP~F zY~i$>jxz2;Cik6bm2`vJ0n?#DrPpdNjh_?y{jn*+Z9+KKguK51-G>AACU@?Nc-@r#-GtEY2?tWI&drBConR!4 z4ZC@5b}izTR{d#Y+)1mR7U9ihmuU+;_>fzJFTBREpxTH#=XJ2Q^KkR$83<`*uFRgV z+PV-8WZsfIx{nsTx3AQK`j(0c+fOa6QlLTzxppKetxa&DSq*$zPcTfJbPp#eJ_S#H zV+c7D^h~j2ZJ=ovLZ<-R!^MiCcTf2|KOe{&|Fh-SIfaM$H;nh zPpXdfVx_wyN=lhDAk38{p_nF>qB)f#+r6dPwg14Bnod^$l!;CjpYnxcZ0vo(Oo)pBgBh=L1hN>w^6qExgO_5l^bd zQI(mAUsOqV59|z>gXI{=deC*XU>vnME?tpYpVQfqs>d&?3^9(RzUE8UAucq&rq$cF z`z&r*g(tAL?m0MHMq2FvZYOI?XW#8L?c5j7dI&s8bh-K<6o#z{)0lC8TII+H__MLr z_bZv6q>4o%GEYT@m9bUtqUVox6RcDLF0L#n%Esa(NjK9b&q853^FRqJ@UU=`+CfMS8eIHw@xj)9{ zU$HV@p@UwTXm|+;5>>JF;TghboOyb;bcIp7vMMIVbTZ0T)D`eVS3sC@fv?1&K-a#*vL89y19$#+Zb8>^C};uqUj1y7yB%d zW}4Ig9jib}ZEGcA>+ACX=-0=yNJtnzsvu`DL*5>Y zPPH)VWcW@d#$?2ff4Dqu1IE0a{CILO1A}S5zDfWM%#iawJ>feC6&Z^bl7MPoA{;z1 zmKUr$Y>G3A+&rS5MjtA!s=f3;WcV3WS!RX5n#XLb0jc)b80lMp|K33~#WbTK`$jJnA zcah3g`=LKtN3B`6#YNOHxmHpQg>(5G-o+x5kwr4ju1yd1F^~1mt`urnw;cw0?me*w zGHXzcu8j}4wrA(RH8?louu9BOcuw79?4){j1`jw?#!qjA(t}Zi6nv7O77B5vF4MBH z{y>8)rdmQTzegUxse_50OZJM<@1UTb!+fBe+b>OmMqiP7n)*X-ozXlmnaD6Is{;R~ z0sj5FRbH!T8Te8(*F|e=*6C7X_wjcvxL*J7%EOTlXTN85b~f0U+06`d z?D!z?^m>&d6)Q;(N4S?bdpL5Qb-5Rhy-gyXa5hd)c#b@q0!s@z#)vxc3pLpZKRlkLq<}(Mp!LX2+#D0Rzit4%`X#}^RA$d28&+v0A-ja)PP))5&VK1ImP_D9vw3Sm|Z(q-fYz@{5$OFzazBXKBP+GAwVlzP|2HCoMO8Fw(xjo)@^rZWIVPo>B|V zU2THbiFcz@$lk9zfU{+gy}iz6p_}#$`en5au9OxyM>!-}f&fO?63fjikhW-yC6EVT zHdQa7$Q22zC~HxA0*621k|U?z^tiLE9|fk$R6HI~g=%qU4$;TkelK%Tm?Uf#&pz=< zknyF(@V;-0{|nJQ^rHn$LwzgB2CK%m(CoYXZ{9*P5U-mgn1FF~X_$2wZy_SDYMY!M z6s2DBt0RgB+dhskwhVu0-T}`>9mTB)_F3>oIi?{<@$@T^%n}4t`a^KYu6>*MMnX7+ zz9_##Jyyox#-oqKy~v&@fswMN#CU<`Id*J#6v?(|vnYck+a#h#mTWU~s;H&TR^+Pg zBIYl!GkBjMmm0q~t1%h%MxRo8LCj=fM8_DbI#6iA^+F<)}YICt5)EsJ3SAYRxddv7KP8_JM;tB*$#j!G9P?c z_vuKCZEz^YwIZKZM)}BW)r9-tMn=Mm+o?Svnbm_t7wM-Jt$}P1n>D#?$3t3xH=3N2A>2WM)<`K;xUrl!Jfd0T?mTk2YWw*7w`HAgOU&^d{%^YhCr-t{ zJ+nUz2(mx7P6RCsjV<(v8UEQhDF*!4V*&m=z`J|jb?zw^YgSc);@6qWrP6b~W+mX0 zgb=+PN9cojwOy3(Qsh{v$~{k1@)Gn?D6pB5-HQ!ZZdGZ#diIs@!}e~nXI8g zdojju6(P1vhsaw2+qg(Yxdw1c-$)Y^B8Jf8mfjLh2~E>NuM#7Mb3B;%e9Cz&w_Ac` z|NW=Ws)@JtT9jMe;h+dof)wd}JV^X}5$Y#bOSV%JOYImM#NexV*S2T!N!_;i!O{9A z;{4lZuPxX$ANLgE;l|ZYA^Q%F>!}^&<-%Tm;T`@G5#^3-?=x8f!%Z`k`3wV}PhOsv z4XsW&!^BoTz3=PXNv%nJzyK`)#oPysBFDCk`4zHd>7jUPQ0iIBS9T&f@V|J%kZ$TUF=|eI7LvvHZAh7 z7@}cixMdGFLt}}-uZfQm_Atw{I@78o(ubI9cwavu^QQi?$)@bXU(W=h1tzB_w}$S^ z*rO(3F!*jt#O*q81x@x-0J+1ijNTr(fo}iYIIYaUn~ih7a_z_fdjunm&-!KjpJ$L)V6zeksVor~86Y7}&x3hzuH%IT6l@@p% zBVLy*B0(BqH>Qv9LK@Lt(?8?Q@T4WcXv9qOOs@eu*1ZfAvWy$%JTAq$W_||h(k6XN zdgd9|=h^L(YJ8junkaUwAa*Oc7dv`8>}qpSEA#AT%qCp}7$IWjL+Q1?o9HLz{;d{;7tS?2kXgl^h>Y zZ?qDSw<_rLsWF;YtFIYxJ$**CfjI4;;pM}nx#GLJjs~jH3=iJ$8l!JREHth=(&R!0 zs-m`YsH!Y_TkmufWB?^x9VeoXk$0-^nv!enLT6#ZrdB9KnHD(oiq<_z}-6Mk#RVta+o=qrD_Xl%ZS!_Exkbaiox4nq_Ll0_NFewaX(j69@K2Nc($!?3#2AmhRJP$&lX$15zwlO2(beU^nQ5YQOgi(eF3_$8q z8GwDr&`QcV!036WHL`mMr6Rp7r1v7{m0wCixR!AKGG|%;lK%w`!?Yn{0S+q*nf64b z_QrtWGy6@5%4&#a^}6;-OG5t{aZ=+ne5is=3B(CO54x}gENF{pm_8bU6lD%XDHoT} zthq=nG^2S#ktSI==DL9Vd?Tt62eA5t@x&B47d1TdIQc!AK^Yb)^)-kwgn*vmrU z=tBw_Gl8%y6N!W-tI$@a3>i^9gmn>8xua^K5~L2#(%IET-!i(%RC&jJr8bT<$rgh3 z7B19`_w2hhwdS|YWGLYnvkw!9Cysl{sXz;T7K;C<_cp0Un1v&dRXaVJF-m4NYG4SDwf2&=$fSjB`LmIC_GX9h-L_@E1tq<|!t#Xxy zM6WA`r2T_-*YTt)sOI`lb5zU)k3*A&H5tv{3b2yP8@bz;XVOXga8~VUdE=N)toBg! zwalSyaAs%PVUo3Ex3`IHlKnZ zNPRK$ZWa?H@`cMAu*;7*92v_GRa6O;(EAvPG9B7h1&Z4&*vg#W#AjSd9%yV6V{Awu zW-WKF`Qj-1_z9Qm6AI<3ZL$)7JQy_RC|x!Po!V~je!5kebZRu0aYSk{@m_L$z>Ovm zCm=JSQ_b#eekuP;Dm$6JXRTb1l_;M2%Yd}aQxKs_<-F5geyl$Ec=Ez!ci5Tg={C+P z_HZzTU=6OOb#f!5#AoxoeGbcYT%%e0-IZjHJsmGb$9~!DBa4O1qfj^0v7(WW1W~73 zZcLvtj-Vl4v2c9qZ7yo?3gF;wWq>A!=R_ZIOqFuoGzeYha12hV5Hj^z;ePwnQ37m> zS%ja1yW?rk@ij+s+N?cK^J^yVp*i|@CRLrVEvQw7CX%3YUQ3CL0;62EA>Swge~XDnkRj1``A2rH+L3Ug%nRaxk;#W^BC9v z+R+QrjtDf0^Wv2R{82X1)jO$HV~iGTIe2gc_dO@jcN<&C+Y&*qN;?Ad?RJTn#$JDX zjJ!pUz|LGC#llo*8dRYkR`Vb=uYV>Po6c6`=mPWI#*rd2(duj$t%RS{S2<(3+>T6f zTx$rMvZ2c>xkTo$>R0LIdUN6Y3J0*A7iurEUu|_F^HRd&e`$;9e($-~galldjjft=+6+P|k#D3PO6Pa7XW~;& zH`2<3RT~TcWGOPs%}`EM_v=P~QKeLCrzexa7kmLWRAmEs4mx{V~QG5|&^N{?!H5S^#qn zY;q&8ztd^1TMuQcudKA9xX@f!%^N)vjQy=DSlcxtWfK$FX@WuMtNp;!^d^BEsu&Qd z-H>6cp7!%GL8b8EYUGFjv8)$5%r80k=J;ietdT_2_$6wzc7VVJ4|SR2-*@iofv1!g z72pz85ORI+~ zIl>#Iw%L>YGqH15mWIX$O$#8FmOdJ@Wf`ao+MpLxcq?`a2#PDwBAd0ZO9D=OV?*#O6fI-XMN>x=r|M>PZSPt(BWpWzX4-M?2eiH zvMd+ZDrfDS;J^W#g?1{}qO{UTc|4Sq1(~SZiA0d>(|b$Xn8sFVy~+!Eb5+ zX2)5C>;rfRi--!89!t=e>?f`pQv~iy~7wp;-N-vdA4<+XT$juMA zJ{-a#wgJoz^y|4X3yf443-CRzuxiGX4Q~oyx=yV9=PvdlYanHItHZC{uNK)12t_r$ zIa-fa?J=eXoKx0><`GlU367J*T5vtoAhvgl_5a_SlxNAkgz z=-ma{tIE%Us^sPVTB^r{H0Y#_5~+E`YA)8VOQN;7#Km7Eq^9+TPr1hyyo~%Rzre3D zY>HgOFHcmck)c9>L^ygvK^ee?Ikn3pNK@-0V~CFM)MAXUhq3#0w^M?O02WJOGmPQ3 z0?Ua12Bm_e4Sh{>8^o=o_gM~TdxDo4&y#$=j*~KHmTOp_OQyVZX(Ce|@C>ftGLHvd z4@TBE5R$Xu9m0A2!mpS&NTcht>KA<^JK+@{OM4gb`l~;6xI(y@hka?FldUI1=ivGF zCYlG{24Asdxm+*H91~?_?aKAbg|u$d`+c}Wb)R>v#`sekwv>o(Ct=X$@J*`UUWV9g zRVb!=v}No-LEK{2aZs*KGYL7LuJl$2_u86gq0@rDiAx$OC0Mc@;gxPVAWZQ<&Mf^r z5ZBy5_sWru$hAqwR=EavGYbLxKI(P0B$xe8(JH04m6U@dzoc|aC9baxb0?+_?d zJt0!7DQU2{?es2Jv6np+$vVrUwFW}i;UV5zBaQoD9BFwT8TVFw;2}9)7waiw?^7gyMwhOw$HIt)l24~5 z`@vFBWsEqpf4q&ol{O;y0H*}8EXnS)*_KDOha?PnQqOcwPgpq$QCnBWmn95Y8@vs< z6K20uwbNsQ(Sb!ct~KrCV$z~prC#RU>65BQYNjkG9`s#rLe2SWQT!DnQr9zO2XYuw z_{E*^Sgby>R#FcQPgJKDnokueB3S3MuA*`EF%^z8Za;p^IM>VM9(vO%hO>xlKAKZL zlQr3A8RdjrR3Eu^fKsq(5@dz=((e;uL&)U5tZE;!YS#(TOx6bh_yp6arW?;;(ndJv zi;)T*Iu{&OLgrY{)o~Pe{`AvCzZt3*!eOTCS;oq$7<^r6myj06h&2cxk|Jp?PjLeqE8v-n9pq_+lpS>ooGxjpFgp!pmq&>ZsQW2qg?Z>=|uL8 z?G^Wdn4U?{G^NgLhEI{!<&QpBukq>)d14)PGzRytwWO{uwBDqf6t8}=q`bYULfffN z_uEmhSnHd4Z312Wh*)x0)<4N4W_B%htfr`&x@LezX?3BP^#a>8DpebmBCL3=#+HtpL_)QqMq*EUaT&W6j8;FaoIM%Am7Z7GtpmM_KWEb zRo3^$nKm2yju1zpSkgOtFqiv|zO5yz_LtRYLbld%^g*gG;jO}PASN}<_G?$Qu#c0o z3ZE~eIY?$|vYD->iDA}gzWoXjLqK??h?Ge{)R_1Qv5?dsspW=8wO@24$-|`W++}bk z`ejH4S(hTO+jy2u7{>etWsI$xd?12b9LK_QaqQ$g$&I{FDfudea#KEI^}5L@`C%VKv0b2%&};PT#+!tIPs0c=TGNloe?vBa4y3fBB5=?N z1}zE?Ohrt|Un*j!ql;(yU*K?o7ja>%yiTbkS3{U55fdwnup}v#DDc%p)pr1)q zDTT@lbWnOgM`htA6U zKOU208Wmky#jS6N6MdFCg@PX9dF_=0VJZ~~ybDJv>O(veowHr z!ixbX(%P5(mbr`;s^!JH6$^PY=n<)8N)qT0IgA0t*{_V|=ZX{10=P0dCestbLBmwu z$V4)-SoW7m^amqYt8#-Fj)PnAXgqnZUR5yo64cYFn)so}M@~{1!-CiZ( zWX8mDDxqS%@j|nDt>)=cRwLM4cE!!{b(nr%Cg{#`knXnjT2kwxNe;m=$QTqJck=LAkEk$LjEWjO7!GUdfABP2!?qix; zXZ=E;duBgEj3D4t)SjVl-uJ(xO!+TzKm zT-)qIem%?VhPvZ)_9Mxf?U*G=ta*mL_55D(#b`%ao79|i*3&O_0fKwlBhFE}_LfBg z&XKFJx(@uycJTF5)-$N@ABVXTVP-XJ`Ek@N+0iQdG`P5$$mbD(#kIj&?p!=TKIW|6 z9WR{3Tc3fpX&JJLcO`n2X`ghoJls zatJjHqn&;`S3_5BeJD*?gwH<6^33Hz8RqA+Q{I5a9y~lt(|zTU{EmDP`I?%$1--y2 z1S@3;Vo0xcEcy9hpE3A$isSsIKDk z&4wC2*j=Zbv30yrXsP=cF(SjNleLaU z8`Mv-Uboq?mig026Z_Rz9$Egkigq4F2D~zDw|R@XCVQR}`<^D}w;*R8ie8TUwtlM^ ze9T(1E!U&XwND1&5s**pDQqksFFnu+T(S#vx>{1uOeOHwyy7bg2eA|jXUlqqn1clu zs1)R+W{{^}PCx=$6eG)T097XC-UntR_vsL1W}=4B>%mDfS~KZMF+!TSSWlijCL^bp zMU@J!0G-BxIHGf$>WBV=KJ^d%M}7JpwR3bw*YBFc{rtyagM?5=z(?fmxEQUz8WB^( zk2Laf;c0Nh28B_7c>{Rf0lOY)EVnYDMn>)*;xR9eC0hKrkbRvmz+CMpNoDznQ)E`S zwMUP3CS{-8TTy>+g)gQ&^PtjKmE{r>yZ#b-#$}e%V$;kNe`A(3oR&}9l`B@lPdC41 zzgxLoQWzWkTs5QIs6gR!i6%4oeq~nxQBZ;r^(d!=@5j2?WHQ7{sxf(6+a3pA)vn{2^CZQ;Hr2PVTo)OLDStgR!v|jJJ?!Aha9YRoB z$Ik3s;3s~+Pxz?V;xZta3K3%{I+U>pg3unEaAmxX6c zD{%Z<2{MrRnq6sovG}xME;*B!1rY*v3L!%yn)_p~I=jyo>vw&8PziYH1yiAt2P!+jh|w!z*Zx`ZF7+(*2elpjd@(b=3% zu4K~TxOqdQee|IBENTeXPz#bp4|zNP5^%ea^LQdbe1LjfX(hs?E5l9kP)N5e;?=mO zgeC&+>0qCa z)ecjT(WRhtxJyx&I@Rnj#k{nMd{giWGzMj+*moJ03`K%HB|^l^+gsP?B?iptSRoYdiyg~xq4!NpMA`mOcY&C55;iMX(!G+e#s? z&9<`d#c(v0b!W8xyN}VU@5bBr7NzP&qC;5&qP`lD$tiX#_!lu}iz+6ZBVLidbcQRS ziGcZ35@OLi@h*|2CE7qmS1hT?#wcJ@H~R1c(vY+JAin-8bt z^s}6OmMS+D#33r7S+q>-iQ#l9%NE)k2ssY@sK7VN9&)VWae=U0z(NqGJ3~bLJHZm| zrQQ=fxw`CtoPqgI^qpeJqVc0t?jgq{Ic8{y+8SXm8V}?R-Kbf2!ZYCegFEDoPV?qT zm_yY~msGe6ds#Q1>OU)C(tlC%bQQ{&MW54tXf>>Imqlk5QL+}fzfcepdZxp`AZgs8 z3ZZ3`HclI4_Jr{@8*RL|W^G0wwzb^k%UKNDxBTmFL$B)yp7snBg>y7Dh{DfQs2y!# zS34iH^JKjcB(I%W%f6DjkG}XCNn&l7LI?Z*M^!MmifQdLo-a%7JFiH$ycR59Jm!795+GNtE3oYs}mmHWHl<=(^}LUVW4-()QoTcNKq!$Ib}o%>uam1 zg7PtwaX>+ei;Ut2BaXm1b23iDjZ}(`NEt^%e#Kda3Lec&x$*g_dI~vVrpu>i=GfhH z6fICGDUrciEb#5>%=!^`9-(u-(?-Qsn9Bc+SJ{Ps+wOSxpa~Dk^mk25dg?)Wr5v{p6h3$fJ_#P_I z=6G{p_2bY;CCEN#$Ub&6Xgf?cZR+Y&+g?n`q(7>aWZL-rj1$6c)c$*`Yh?KW- zH`Mq=8EAj|7hlQuoS2jQ7T4LHalLAHWr=*x!ePjU&srKCa5fVZ0gQdri$2*FH z0z)VwhfTp{6mVOej~&-r%-h~@?k_*MGWX@_t_7VUrp%t3ocN-*_+ZW${tQf%l9;BX ziU4jg)oZG;y7L4R;`t-iwA_m4h`A5vZiwMvv{Rlf!ah3qJhnge2x(7EjfuqQwF4;4 zB4IG!Oe=9HN=knQhiTr6j!7}NQ3=*FPOwTgu$qrx^)S{L*6%Ch(XLcqEI)`wIZ9)V zHblvRpaQRU!Y9_Qri^v6c^yJA|LKd#;oDY(-WPK_FWpc`d<-;W_K3y&lH0^sk|@qi zqgk-iDfzbvJ{uq6wCo5~QS%1EC06$49kQl07ZI%w#k%Ijzi7i8mm!Optd`Z%*NutW zGCqv?nwR#hcBrDTM{YMkR))&Jsv+3N^I<_gT*MUGdu++~(1iyCcAtUML;;%=`00*r z(PCGRJFeM^op~*}(k2{GJ058BScZ<)>C1O>pVPP;6}O)g6b$4NJ4CzGEDJiKo=K`E zI4;*1L)EN3r^@jn%t4WzW>fa(@wvi*y3R=2EGrdhSv)!qHzVt4;83 zbnwd9c%6Rhc%zj+qbU)^u?6@%V7uwsY ze@5fDD170GMBenjnqEI7!w%~TqTtUnATP!Xk1 zv9~B1iW~8T&Z3~VatDI>%W0tqF0C|rxga8YqwJ=YpgmhtC{a>@syNvW>y)2VhA(y>8modn zSH_lwX7?Cn@iRy zULF5ShOL9K5FM0v?1~k+IrtIo!eFPJL|Rcd-iL8R78@6{0Iuu~{#j zu)Y1`IC?!Og28kz0y#n-_K?YoC85moVImI!5-(&}LQ+x-eTX-r6uX9>!fDwVk|lXW zbzYMV3d80cL0K$Vl8H0@A}{(BY5i5{q?yNfpCR2t`eV5pf(SjTV+16 zmtH1QFt4zr4!mJg(Fwp6;Zto0OqpgbV)S+`aE~qsv4Z6|z^z<%rQSB1W5*UY_{`I9 zW@cY-wv~ZEtv7MxALbt|T%@XYBpN+6yrtyo!0qevBGhoox`+phW2-Dfm)~SDf_=Y- z>k;FoV&5wB9vB%b942rCD<^0p;UkSMwI1ks25!gBBloDh#Xhfz&kSndsb!q)wdrHh zndwRM+8f9G6V$w6Yd5YhZC0*kF)_1!`~!Tug^9+-bo!TS6ie*R_n8#%sxIRD zWNjr1-vxE7iDO;>N7dgA>YH?(i0e-;AKIRB#xA8~5lWZgsF*cnf`cfda=e<5kD0DN zw~OkUU!ggRb;Wx=zhtv_XrQSZBkgEws(!3j!Pk*anRwJ4+miGoph zp*7n2z-UItM+w=gqI~I*P976e%)&*6;-!vrs(C3irt7)31f}w7)ft0SYN|G_}J zYBflJOgG#_K@#YzQjj;UK{8ZZ&`fxt73QR!?;J+6+^v9fgt?j5)SEo1al0G=yO;38 z^VYC-K)|Y+f%N)CFgr&Luz8H5QFF%)iw2PS`M76i*$sA_ICaSwY4z(U8y+-t^O6td z2Y{b$fuFeP=4k_VW8sHGUxLklu-|Z)wu67OnD~NYWDUJv04l%LI1A2PR&GOckV>(J zNO1_V9qYEL<8XOz^aD>M-3Azpia8KG4p!Y(;@ZsGgDEX8sUnVhy6f4H1@M1L)*ZB&(wW+Fz;TQQ6 z_ABltEFzP+ia4sdicjgYm8ld|1e5g1wsFc5dzunRJQ0)5&`-lnM|v_>MD-6iD zo%mZyyd^ocNx;eRu=#|Nlk% z`db#jBm8e!KK;m|YiVu{+^?{u`~S}i51i7y8(i#fd4c;KzkTl0A9=sK<@$?)fd#4G zGHU(E`1>7^pSl0hv4>AceKb%`Wgz?+5Ul=Z%JL5Tk^8x}tv)OBKMPsEr2Z#+MC*#b zJ@7^)z-!Alk54~RmbVNb%l}HuuWe?g1H>8PH#63^u={ZY`98=_z>Z=8C^ysNe*zW# z0^I^){Q>l6dAOVsaxtNR6VV7DAhTrLSNU;*wW(9D5PV2f_$K8 z27t|y-9Z@N z>|~(bfhqzEiT{&NP}j)vA6@-hu5XC&2A;z~0I>!%FpN8hFv{N$|48#Eb$^5KVb{IN z0Q6)F=;>S*2Jw{8#T@W^TPtR zb%3MGKU0>s0mpa5ckTW;|0{bvkOH6#+<@@`7W^{-<|}$PKd?XapVj?MBN5w=NWKE4 z_`(1J0@&csl;v%sc$f5#QvNJYAVdT04v;q-Fq|hp8Q<(L)9g=_zgx@KnY>)P@EGa~<4ru+zV)UQCnascCf@`DDvW_io!{R?PgqwV}x zng|?`<$IvgG=U!cUd5H)r+pjx{uB01=d(H=Ob-B2^Z-k^JAC?7{R{TDGSZyKTap77 zkOx=*&_sTwEbpS~e}gOlt=a1WKBxXuxNpWTTnRri0+cihC@Jwz;y1lL24+wH8zXdo z%!Ga#=wFG&H#gFKK-HQ8J^qtBM5^Zd#Q!!(|$zq`71C+ z;wh9ZP}o*LJivti`!K+3tEJ7~&3N)Q#H<-;6i7g$Ai0BcTfL9_b#dsg60$y{#c~GPehkp| z>F*#tckd(RjO^|7EFCTGOr8EpUwZf2xEg3C!@wty-JuHsE$DyN_%m=XpPruaw@#eT z%F4`GSNp&F^Dh?bPvHSzKA>2lJ5au(`_Sk11_t^z_d-GaKFoY;GoNkSdwc;2-5rKu8UriI1vnxFBy>U?g;+k^gd3|!ruHpCTj8e@44jx-O&Ol zV2=1RWqAjl{|WOwG##-hFlDm&El|h((o$5dbs+*wIH1Pw_Bte&f8$Cze-rl4MdpT< zYC!~KEd`#tjmF{XFU&u~q%$2NBLI*;Amm*Eo3H-`bJ8}qGSmM*ZRYTG7AqXk2m^rW z$djL{{MIeT-2MmiFDl&UihPU;NZ1KHciZ7S6zDyJ`Ccve!&Kg{m9ZlfBoYAk9(eBd z+e|S3fmAZKGy2W*`F$dyU08Q8P}DBq0X&HPOj+KEu=k07n~*+S2w~I%a@zvWw?h*@ zQkM5ixPNhe%uM3q?sxkZ z(I0Z%iQjl%W7|q1s(k=a6d055S~1Im`^0Zu{|jJF^z-c6ec1gd<-gR%jeH;W_x|L5 z!0=zBtN$eZwVe0k3;zP*q2Cwnw`jxnOL#vx>n{vB<{vTtR~xt=ee)ME9{WD0Ad|DE;MaDn%E_X8vS;w6yY;k~#UCh0!sek6lm9D2HcasJY}@4JBi#o=ZA z7w30}@b?9}@BaH2D1`YBpkL0we+PY=7T@;+{0r32_8s)wlGa_X!1r1A-LU>*C2;|C;$PDjYD!_+MxFTjkuR{<#Cbf57(_bH(7_%s(9W{awg!HFW=Y-Y=k| e>7RkW|3L}~{8a%0A_e?wMF#;%aRUZYkpBmm4H6#! diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom deleted file mode 100644 index 1c730e19b4b2f..0000000000000 --- a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - net.sf.py4j - py4j - 0.7 - POM was created from install:install-file - diff --git a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml deleted file mode 100644 index 6942ff45e75f1..0000000000000 --- a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - net.sf.py4j - py4j - - 0.7 - - 0.7 - - 20130828020333 - - diff --git a/assembly/pom.xml b/assembly/pom.xml index fc2adc1fbb5d2..c19e4bc986349 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -67,7 +67,7 @@ net.sf.py4j py4j - 0.7 + 0.8.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b3b5fc788f791..2d2d63508403b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -338,6 +338,7 @@ object SparkBuild extends Build { ) def assemblyProjSettings = sharedSettings ++ Seq( + libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1", name := "spark-assembly", assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn, jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, From 6be4c1119493dea2af9734ad8b59fcded31f2676 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 14:09:37 +0530 Subject: [PATCH 1118/1571] Removed sbt folder and changed docs accordingly --- README.md | 30 ++++++++++++---- docs/README.md | 4 +-- docs/_plugins/copy_api_dirs.rb | 4 +-- docs/api.md | 2 +- docs/hadoop-third-party-distributions.md | 2 +- docs/index.md | 6 ++-- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 2 +- docs/running-on-yarn.md | 6 ++-- docs/scala-programming-guide.md | 2 +- make-distribution.sh | 12 +++++-- pyspark | 2 +- run-example | 2 +- sbt/sbt | 43 ----------------------- sbt/sbt-launch-0.11.3-2.jar | Bin 1096763 -> 0 bytes sbt/sbt.cmd | 25 ------------- spark-class | 2 +- 17 files changed, 51 insertions(+), 95 deletions(-) delete mode 100755 sbt/sbt delete mode 100644 sbt/sbt-launch-0.11.3-2.jar delete mode 100644 sbt/sbt.cmd diff --git a/README.md b/README.md index 1550a8b5512d9..22e7ab824577a 100644 --- a/README.md +++ b/README.md @@ -13,9 +13,9 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which is packaged with it. To build Spark and its example programs, run: +which can be obtained from [here](http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html). To build Spark and its example programs, run: - sbt/sbt assembly + sbt assembly Once you've built Spark, the easiest way to start using it is the shell: @@ -36,6 +36,22 @@ All of the Spark samples take a `` parameter that is the cluster URL to connect to. This can be a mesos:// or spark:// URL, or "local" to run locally with one thread, or "local[N]" to run locally with N threads. +## Running tests + +### With sbt. (you need sbt installed) +Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows on *nix based systems using sbt. + +`SPARK_HOME=$(pwd) SPARK_TESTING=1 sbt test` + +TODO: figure out instructions for windows. + +### With maven. + +1. Build assembly by +`mvn package -DskipTests` + +2. Run tests +`mvn test` ## A Note About Hadoop Versions @@ -49,22 +65,22 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 - $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + $ SPARK_HADOOP_VERSION=1.2.1 sbt assembly # Cloudera CDH 4.2.0 with MapReduce v1 - $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt assembly For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly # Cloudera CDH 4.2.0 with MapReduce v2 - $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt assembly # Apache Hadoop 2.2.X and newer - $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt assembly When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're diff --git a/docs/README.md b/docs/README.md index dfcf7535538f0..e3d6c9a5bc211 100644 --- a/docs/README.md +++ b/docs/README.md @@ -27,10 +27,10 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri ## API Docs (Scaladoc and Epydoc) -You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `sbt doc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. -When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 431de909cbf4b..ef9912c808259 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -26,8 +26,8 @@ curr_dir = pwd cd("..") - puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc` + puts "Running sbt doc from " + pwd + "; this may take a few minutes..." + puts `sbt doc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/api.md b/docs/api.md index e86d07770a80b..11e2c15324ef0 100644 --- a/docs/api.md +++ b/docs/api.md @@ -3,7 +3,7 @@ layout: global title: Spark API documentation (Scaladoc) --- -Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory. +Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt doc` from the Spark project home directory. - [Spark](api/core/index.html) - [Spark Examples](api/examples/index.html) diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index de6a2b0a43bd5..141d475ba6610 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -12,7 +12,7 @@ with these distributions: When compiling Spark, you'll need to [set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + SPARK_HADOOP_VERSION=1.0.4 sbt assembly The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark diff --git a/docs/index.md b/docs/index.md index d3ac696d1e818..5278e33e1c054 100644 --- a/docs/index.md +++ b/docs/index.md @@ -17,7 +17,7 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you n Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run - sbt/sbt assembly + sbt assembly For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you will need to use this same version of Scala in your own program -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). @@ -56,12 +56,12 @@ Hadoop, you must build Spark against the same version that your cluster uses. By default, Spark links to Hadoop 1.0.4. You can change this by setting the `SPARK_HADOOP_VERSION` variable when compiling: - SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly + SPARK_HADOOP_VERSION=2.2.0 sbt assembly In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set `SPARK_YARN` to `true`: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 55e39b1de17a0..a33977ed82859 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -69,7 +69,7 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`. The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: {% highlight bash %} -$ sbt/sbt assembly +$ sbt assembly $ ./pyspark {% endhighlight %} diff --git a/docs/quick-start.md b/docs/quick-start.md index 8f782db5b822b..5c55def3985c6 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -12,7 +12,7 @@ See the [programming guide](scala-programming-guide.html) for a more complete re To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: {% highlight bash %} -$ sbt/sbt assembly +$ sbt assembly {% endhighlight %} # Interactive Analysis with the Spark Shell diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index aa75ca43241fb..13d5fd3685bfb 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -12,7 +12,7 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly The assembled JAR will be something like this: `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. @@ -25,7 +25,7 @@ The build process now also supports new YARN versions (2.2.x). See below. - The assembled jar can be installed into HDFS or used locally. - Your application code must be packaged into a separate JAR file. -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. # Configuration @@ -72,7 +72,7 @@ The command to launch the YARN Client is as follows: For example: # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly # Configure logging $ cp conf/log4j.properties.template conf/log4j.properties diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 56d2a3a4a0202..3e7075c38203a 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -31,7 +31,7 @@ In addition, if you wish to access an HDFS cluster, you need to add a dependency artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). +For other build systems, you can run `sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: diff --git a/make-distribution.sh b/make-distribution.sh index 32bbdb90a5bda..a2c8e64597143 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -43,7 +43,13 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') + +if ! test `which sbt` ;then + echo -e "You need sbt installed and available on path, please follow the instructions here: http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html" + exit -1; +fi + +VERSION=$(sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 @@ -83,7 +89,9 @@ fi # Build fat JAR export SPARK_HADOOP_VERSION export SPARK_YARN -"$FWDIR/sbt/sbt" "assembly/assembly" +cd $FWDIR + +"sbt" "assembly/assembly" # Make directories rm -rf "$DISTDIR" diff --git a/pyspark b/pyspark index 12cc926ddafa5..1d003e2a008fe 100755 --- a/pyspark +++ b/pyspark @@ -31,7 +31,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "You need to build Spark with sbt assembly before running this program" >&2 exit 1 fi fi diff --git a/run-example b/run-example index a78192d31d384..fbd81fe6f331f 100755 --- a/run-example +++ b/run-example @@ -55,7 +55,7 @@ if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "You need to build Spark with sbt assembly before running this program" >&2 exit 1 fi diff --git a/sbt/sbt b/sbt/sbt deleted file mode 100755 index 5942280585ba6..0000000000000 --- a/sbt/sbt +++ /dev/null @@ -1,43 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac - -EXTRA_ARGS="-Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m" -if [ "$MESOS_HOME" != "" ]; then - EXTRA_ARGS="$EXTRA_ARGS -Djava.library.path=$MESOS_HOME/lib/java" -fi - -export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) -export SPARK_TESTING=1 # To put test classes on classpath - -SBT_JAR="$SPARK_HOME"/sbt/sbt-launch-*.jar -if $cygwin; then - SBT_JAR=`cygpath -w $SBT_JAR` - export SPARK_HOME=`cygpath -w $SPARK_HOME` - EXTRA_ARGS="$EXTRA_ARGS -Djline.terminal=jline.UnixTerminal -Dsbt.cygwin=true" - stty -icanon min 1 -echo > /dev/null 2>&1 - java $EXTRA_ARGS $SBT_OPTS -jar $SBT_JAR "$@" - stty icanon echo > /dev/null 2>&1 -else - java $EXTRA_ARGS $SBT_OPTS -jar $SBT_JAR "$@" -fi \ No newline at end of file diff --git a/sbt/sbt-launch-0.11.3-2.jar b/sbt/sbt-launch-0.11.3-2.jar deleted file mode 100644 index 23e5c3f31149bbf2bddbf1ae8d1fd02aba7910ad..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1096763 zcmbrlWo#r35+&GXcDsyiW@cu)&CJZq%nW5_W@cYpiCT)5 zYI<^}fpwmFnR%90lI`G7Qi4XJiIs_gg^7WYg_(hYfr;(#5cHe86cqIOaqR@v-=CrQ zd&>XE2VwoaYv*W2Z(wg=WNt!l;p#yz;p#!cNM~ei;N+C7ysn5MfW!+%f}M&60HN0w z24DSV+7=lUi5w8HOC&7it<_&k6S8T-EP?rf;@1ZinflWrIhOBPA%#~$9^P=MDkJA4 zXJLWY$Mo$jc18~bepdtCO<*IXKq|r)jc_BmKyHY1LkVW9vH*`)@tBHkKL*tRA@Ro?L!biXFzckD-~i4r z3r|4)E+ZuCByj;@hyeoJk9njKAB36WbuuD&Rvj*(Dm-1Pagz{J-mDdNr;>O!+=&&` z?hAE=5L>2>Zj-c}^tJtv>kcv_Kn<+Cv`b5 z*Js0J~CDWCLJS%IwcevU!Nv>!iF zNH-Nel91m+Y-r!J+}aP(iYM_E%syO?r)JpFU$qOvunMM{39eTdK;XcyDN0H4)=~%? z)WdyQi<032`CJ0saV-do&3i3L$=l)biq& zkDnkdd?{G+)!unHN-PJyZ!ns0t=&zx2r@EFfe;%MJEkCN`~5GM-DqS-PYf-U z_(A@OesiiJ*=Eph-}E5`5hM`^01dPVIBFRYsLiU3+djMN!Ro+Un$r4Fki{nDI^3igyA?62a^4y;XD$kk_F<(LkSe| zgWeMEq;)@$;YjPmk&6@5X~Ibp)TzRmB$A{I(@9dK3Jmeki6;e(4C;Rv)rAgmL`G_O zhYTIkboUz4M^>t9Pwi={3B9|$CzrgUmsv{BY~d#q!06*X2^qv&gg~{U9Gk?VTctkiLqz{9e|F1tB>iH z+ECU@_g$6XK!DV((%OM95>8!vbZ-Gq*=R8^`MCh9@<%-u{zD#OA1KXWDQqn?M%}YD zh~*6c^ZO1q4Pq_F>%9R7Mf}WiRXf z@nx$5lQL`KR}HPf7lr9v&i#$5#*gv#oEsG zm$ZkPz190KGO}8~YH7Gq7?`LZK43TayDZHL8ddbH+iDFA^wml}htRkSvC61iJgQsb@iSdk>EJL&6oH2|RYZW3@u1QT`m;4Ra7op!+2J4RP?h!d4ivnvOti+(#V zYzq$B z@S2@HUdG6DEn2P5`Ax?xU*kx|bWjf$Uc4r7a3>g*+44s} zYz|t6KesG&EIXSED^H-}?_K-_D67XGZNpk`!*Z++$*z>(5OW13oRVLaH8it~RN&T2K(JH!3V;BphGKy95TvRx9@Cz)|GeWEiYTVQ5vH{9Gs}lS$P<9KN8mh$^y? zu5=z4c1le&dP=Ze@6zQR{ZoEateabMRZwT^LSz0$5=|jm+H_M7SctN|e@m`iH5Tg% zU+rr|N*f6_k3>EUOI5UBPL>~aEiL$A7+1a3z?cxTP-Vg#(o3{HJMDa z2lrOAZK7yIacv=kl6UQM6HRc%u!HN6+wJ=zsq>Dk15cl|uONZRPOX^7WD~C`h8QSc zrbEz>3B@A%n_mMhW!kGt_WItQil$VFiP9bLFDQrRYzz~$A~htkKdbY2__*^{s~MSqVpa~Na0w-<#R-rE)^Y05eXKsq-Ln6Cs7ph%n5YZ7U{HXOm;Ee z1MQUN4o0RQuGr;bv5drtwn`0f=r&X?vzr|kYrb5Z46=T>xPgJ|&CxMfkQKE*hQ58I z+>XF`Qte-1EdIGZg0+3=@vHn|!fSd(a38i*+o!Co8bVR+fR)p1O}!3Vs_8dU{X*ce zda$}~U6OQ<(fsZQp1=H;a9M5rdjY4kg(+X!P`e#)@5Q@>2`?#+Y=Ye>f+^&2ZMte? z*SeD^wi{XB*y7=S(Ps4TeZp8@@H5A_%`oT&X(x#7lzh7$!S2g-Lw3rvh&oDnCbO@w zgLN_1X9)QNcG3Wmtl1fE7$5lZUOtjT3x7)<%24yR$1&i>F3S-?#H1)9x`i?tvbQpY{Z_&Vi{iq(}IcyM{BPQ#oZ46?Bt}i zkfL}Ft$YsVe#9P!mvt>=d{og5KDtQ5$|@z^R1DrqV!ws8Qk)qnFqh@bddhMBeyBEv zR@MOKfg;81MVoO^uAt%SC-=nbTi>pGuwyd(ysRObFrjwA4~PQr;{M1(Ly*%!qGu6G zSrn;Rs=op`sZfqaNL`tLN_7W$Z}zYN>n)6%9WY}{K8=g)(i+S0f^>6jF&Bj~#MVhP z-CMaQ-6?wE{n?l-zP&7bN=-J0Cl-2zjw)AB8S9x8|Adq0ioE_-lBf~ER3*U8V>w1ii8sr_0m9}+a<6wr@p>d2-x$@47V+L~Bl^`R4l6An zDMO-@KgNmX0mi}mn&BAQjSX)&uD7YIAk@Ir=t5KtOvJm~s{D*W6nyV^^J^InVL9u2 z(M-9-bot5lF;_tH?*jMoGmu{gFAfxM+n&Cv1$2ETE zl?NR#o0i^-8S5IDEICm*+geISw0A4yg=MY^3IS;y1FH@ zoYme?BPh7Ye~&nff26=*-27swLev#uz4%ndvn3^SEavutg?sI~BddKuT9(dF`en^| zM}QSmu#(9iZu1d3TZ{ivbn|DhuCC|9fTlo&&E+|)sp_8FfnVE)ujXJXvhWbeo6oK~ z^?6RP4NW~(#+1cHvN4FhfrpTC-Hkd-68%#-R7@aQS17pgk$i(xCwZ(ESQ z-B2A3^#%2hPR8#6{jmr8?Hd*5f78h%|8JdaWant|Ppc?oXX|X@?)-1fT&$+$gtG+r z98O@pHE&EHo#z+fRGSKl;A(oAXeG5+%Ylv%Q~=)ghbns z*v+;r9vh}D34Ts8s3Ivbv`n!0Hm)cF3NIn_z)s$qmw8PmS~NzfuCm*yMDEG0{q3If z)aNO1`vb36_g!>^-Pv1dq{Z1=YsAgDvuvLfm;UUzI_B!+xjP2m+8_7<3I%?N%0a?F z!$4&}-^+E*JnJ6_iDj}JHUOH-P>a_BbKM|;o^j0xHpOH$+yy!Uaf#Bp-3z%vXCKu3%w)D83qHzM9Cc)AnW#rlHm!I%Wy|eB+D6~M9~{8mG+k4 z>w%vbgmWJq{H>q)%w;OmNd#BZ&wL}4?mp zl~re2<-A2@yg)9StI+I5K*h&-K|w9#a8%!Vet3K0FjH?vu_C>=J>Byw=087$R6|M6=Z>LP9wthVSHlH7j=A8=$-qOMh>$E|$ukh*@>kjBY{wA@ zKK^2DrQ3tUi0V(ZEFufr*mynC!-y_S)&}&NR&G(ot<0FMc2Arp$?n_#nr1G}r(Y9-ea6e+i zPrs)ZzqYAi_qy|y)10iqby+Yk(=Z8#EXOe=OaL03Qk(Uv0+{7E)e&#Tion{B0LV=` z?x@(kWqb4k;emGezEN=1sK^%D%Cg@=+Hd^}!pniUX-qHB?jNbi>+($UVr#l@h}9f& zqZn1PGC$S6JnLyFZaj?3Kck`3)Ohf)kWrTo<%HlMCSq|DDHBk#soSa2eF(GKx$Qxm?EmNJR9m>9mEw&e&{;)WxK`W#va}t91-QXS5 zY$$QV@MFH79=EbmdEk*+&mEyl?M*E}KxcRNPc$v12)zqtfFKnbW~jg~+hujCj)f@U zkO~Guy^*E1io8N9ilujJisW!w*^f51R>g|VOU>ZLgAkKMdo^WnTcER{{)CY@_rNOG ziP3xfMHqkm?;9~T($CT90sPf_`%cJ=d&xnuXBP}Fj_Cd715h^7?Y>tD8xXBOb@6v4 zV-VKaVrQh)R`%hzi;#~fPX)LwhSZ30sa2|@kDzLU9XHgqq)I&^Z#-)z8?Bv#9%rD< z!4tT5$8mK|c-c(NQ-xK3gcI_?+?F{Vpt&7>P(n7~yX`}NZAbb>8TjjvVz`MTC1 zpSnbf%OI-NewZ{mI0gDXsM-$**9V-@4Wva1dnLI?;T|EiGfq~m;)Dc=VOGjLlOdxy z>xteSTqARB?Y8>o7)VNZ$kdCrrrR*t2ar4w0^eSEL-4YSQu!-A+ZN;qW|2Lc!c+FMF?6mbMlYFdLy$bm7j=O#^uD!Rm&me9S z)t2wPqNT$=x=Z0!6Qizj7SXzFfLb}nU4PiirL(Ohz57u&Uxs28Ma zePm)2j{fYuiOu5GV#R_JT9Yk&By}dmsMD&5(nCA<#*Ic{$}W@z)%hfaey%RQ>&T#g8C_|%X`wu5Og$3Z8$3-mjD6|0yG!@DSFpSk|!YSX?h8dka$^# zFGNICKv$&QK95knp%JVyP2yjGT`Wn|VTVUnkr54x+1^{R#(Yhx9<#=j)gd=HHa3u zS#pKUpd$zu*^`8aG+(XX7wsXe+`N|$#)k@vbUlm?W)t~Ug=NP8(-nP)D5bftLLgNJyxEF_hY!a#gLqO+_oE-oo9A#S)961b1Wgk0~g zIY5cCsJ|B%vcOTW3qQwE z0VI=G0r4$53k})jd}H6SQsU7N?as2yGO2lDt2*XK%^A#YUd=}oT-h^@#gSA+P{K11 z-9*i*5Xt2tlUj~!`#yZhP@e5u5+DZEmXXbB=zgnzeW`iROb$UsziX>{x}WfQxode$ zVacvo@Q-+N@u}xThXtHuG$)HYp?4I(;iT%0Ff)seW4liNvPkyPOL}MQzgfEl%317| z;4sfX-7ar=1e96$$7#s>RK;B>=FYtQnuzA^Z*qdz9efkfRktuGE)fkNWxJm)K$xnK z>Ddi`Q#POzSkPTy>cZ>x^KsR@7TM|WSh?oeRSy-WqtCK!3$1nTZ5)i_k4K$Oi=V7B zwyM%}g(;>r8Sa^ls0C_kY`Ri4vTZ5oWx3AD~k2jMnx*M<@dYFnGp6{*)W z@*PRTW`!m4F^fX0JKYyEXWJxR#+=}r(Cc^FZsAU$qBI?9+GbGAV%DbbGqZ8z7!XiK zR3a6lP2tF_)sAHz*zlaNtt6|OqU%^nsq>+pI=;p)%z+bKslDC?&ho|f;+BTaI?)?m zB%l9i+tO&_wLI{4W%sN(3avG%Ms1@yac%66l>#udb&JP(@9JS5uMR|w|laLQ~r1h*1{wlKNWx9n(SpR9gRCwz;Srsr-JcsLo-`U_Vf#M4laI_cB!OcQG>zHV6Z?2(j*~n5y4;S;=9*HQpFFU!GL(qCW{Oly-6D)leTR&PhYvj> zrOw@5b176-ckPgK6&c)-em}U(nx(DR-%j0PxDm@B*Apzb?NHr4V*3gL^9nG!N0FVK z;gIXu0sV-Oy2sus-oC;60DeD~(=O?`27E-vp5QM#7ib^eeErMRn$v0G`}TLF-w5~L z822CB>;GYD{jYZAe`DN#^Xim3jJMWd{8uc^39r;lDl#brOn4$kIL(+VophWNCmCVL zAWV!Au^|EdZ9^e}R5eW~cuJx6Azdww3z!DNnvSK%+m!Y(743%J%guJ%!37w zC>P6Sxq>rLx%BcMU~MRdnKp>*E5TG+`+yM7*ih(uoIoE?BM}RajZXxL31&l19~wa6hVbR#cf2hr>a7E<-u` zTNcvuCjw~?QrfcRF%BRy6{JM37T*C^2} zyVN7USsZs;MU}33qf7Q6qBlSnuF;}P_Rp}gNo(+cn9miW%qGrkzD0A^m{WjmNsnym z8mAjKtSJJB$n@HIj--4Gl&QscPz zuX;H-vS{I73GxS3wq()|s&;FIS4(FFp1(?MC#rcxIacH3wo;^p4SA(=Qb@m(5YM%n zE*FvS!{n6xV6M}+!WLUM4cjV~jwJUyz31cYTrZ0}-_*m1A&+X`a{v7^X}jZ;EwfP7 z$gYSVIhNIpH7%t%)19*Kv}$WkIPJs+%WkI~cxg*)kg=+)-!4#;X|iDJi<*(65tg%H zdB>JHzi^--?tMBM)OIKn@4Mf;tYnX4crQjsf@i54?4+bVmW6ct^oWtN5<#T>Y z!mU`ev2_DPb;3X|z=-hJVTtlkMc0lcZY=oblYkzjkp7_biB5K}_lYihb(|5tD%R$1 z^*SnTtk+>Y=-!@AMGY%0?MRIwE9smte))w@#jTVSAe#LShs{bUnQw^3riYC(CfzC zDpat2$RhfaHKCRF$6hJN`ygFU2Qx<0F#fo}VDZslWD!CiwkuSr;iD|Bki%*`FFfoHYwoowF3`}a~uyF0h1UM^r?K*V#_ z@?PYjcFt-r!CE9^xkRks9E^7#1AQ%d)@XMVoD}Ekg4w1qWWUaqjUa2f^D{E)BdBfn z&=$cDs9@spVlL+f$?P}wAsCOWwAM7_*L(1Q8|D+G zzPp2r@C;I(Y^5{G&K@}n7J@7hP;!sc6IEwd(z@5(62XjZCh@#iFVe$+dv2djiiEm1 zFXA?IuB#8)N;?=_XN_Zd_2}TqTpGmZe_uxLnGb6k-tV-ucx25kKXF-me7fPHid}m+ z_K#PM=WX`>x{?@#dQEd_+1fU|l--@ed~2??t0b=LvKCh(4J~5Y(!sn!L3t=^W!ESt zr+(Y}DA``oak_I)G!(;&<)2u{MTIDxG$8mNE!gsrGYyuVh^9j!~0oS zbs}TqI*_pgbM8HJtdE83c&vuJd;G=hm91s${gUS=TchM;rX{NS)}3;tEfw-(lQq6U zONj=Rk{D`ZfzAr?cd49!rmLE&T+O`RE@16}&yzxTNw;14pDjMROxOt<)&-5o*m9cm z-<4vz3wYt_$z^5q)0Czc*vH$n^k9lG$R}%)E}YDkQIa~c2e5HX{En=0)E{6S^OR>h z3Vuup2bM!-W(OvfsK(WLCd4ESiqQs4l#?tbK}nm$N|7>_uh3kH4lgJH9^)%{#ynnZ zRT2#wXcuSEhA=4xgx}yyNA+giS%f~rhG!_Hy--GeK7aSkoRHfqonjACnfduyaStU5 z#SFry4*1DvGtgta5;!}V?!o5=t9EPm{fp@R%q4s1>j>qBLsBdcfBK|86OJX*(>Hx? zKNvyeo#yF9C^OW775}?oxz3B?x=gvANfEkfk))5W(y$y5l!;L3kDe2@^aSeLdvNNk z%%h&!3i}PD&AT?AZ6MGLn4Zv3 z$GNcbs9Kl(C_eZ?Knh4wUKXaW3u^Z)5*?x?-Uy@it^<4^v(b+?=ssk*F^0V$u-2cV zS(ADKHWcbyQC&>hsIV!?P)h1f&`LZeIrl5=%)ur4>0(B(%(|rQkP)+t&s!M0vKqZF zey#q0GA{TLmqY?><9^j(>t3yjK?d}OQWO2dwM6a?JCSW zgjM$&IkV%H#_k*4qvqtHxF`5VlybOnUq?z`Lbf7}*5#gZN1zdYAsLF*r8zmQ1Dub@ zuNF!UdQ3<#j7g-tKf^_GcD=89ttVByaDoDdKU^Vs9#Pi>A1V$RS#U&LD0WoAD4mP! z$B>#s0*3lOK;XDy3aFA44lBfWiR2t$F-mGppb=&=CD*ulB61^T+~f@));WWQtR>9k z%Xgot=#~zp4YDUr4clBOgE)qSY|lWF?wny$U7>9`&#_nCKpTF9u5gO2K#6ho&J=*o z7*OSmNb|`hyMx@_5OIH6X^-|y3-E`C;*5wQTEKOOXS;`k`2{TOuH$@O?GSon#O|2j z^iw~BcaLE8%&htP*X)H(z7Xepz=Rg_V$T*}tc;jwYlRe-*z;%!W$HA9#@AkhfOT(^3o+cM~ zHkEa^mTf60UpaI0E+6IOsqw0|IK7k-d>z9;R<1IW0DzXCI%7S!@7#So6KNx-s))VZ zTu}=gQMNIX4HNyuy0-<=JWiBG$;Bp{0&B#36Y!T|{-S^=89h3Waw5K^t}4oJ#! z60ntl#q~oTIrBz4hFSeF`-+RzU{aQgI5`cM zU-txP5^YrMeG<*bOraDgCzNZ)0g4hbmxre02sv4_iVb2{7NR+B$(=76Mn_f$42LoE z>*_<#RKLe^9fsaH(eFr&!1ORRU*?eyxsnF$>{)#yt!}tC{C~G)boY(?d1}qJ6L8z5 zt&Z{A5qd!E4t7hwHG794fBEv-Naw8jfi?_VufWS$2Fz-VXIYabEmGoixsm=ZQ%uoz z!go6>kRcE|FYTFSx{$~cs$1}K&!jRJYXQZPlff z=Z@WbqkwcOy#34#hpcHrjkt(;@*ztpL z>>8wNf$(ZhUD1irre7UkHpQ-sahNVn8y=|CFZQWjV5KOwV$7}{ux`)!@CCCx2~-zl zClY5jpxum8ZHLHqrrtiX9e8l&wk_ngXWtC>I-%nce%i5U{~j7{hj{;u+gp`R_ackk zFf=|{(G1G3i*<)jQJ8VMGof~v}*M|+LZL}JK)W9!Y6fWhw9$T{GM$6 ziEZN^k>{T7&n$laE#;(d=wccsh;nNZaqIgX$BYScZr{6I!kvaSn9>$2zwOqm-9IBm zizy+CaKvxlWQqP;gZrQSIB7dGIeTXdJ6orJx3xlGuN4{r zr8(6)T~Rzs2u@V1vV83UEG$r)E-RUfVjIuqtF!?MXP+Z=f*Q9PaK!uEu4=TP7>k+n zn`oBJ!)#7OK?_YFbDD_l&J2v({TEA}$Ialw=CSfmPB=2fJcG9GPW6o#V|R;K9@WNe zHY;Pck5*cFOvp%knwiZs1@hZ9U3b6-E~WzY?4&kiVL47i>eJrv;3#DLkGcL^zs+1R}vvV#lUKdV*yFQKJOm9ra7ZhccXu z?-7VE@@y&qFIn(rKiy*XH)oi(wRQZyVDiE|uu<5^Y@}Dyni==~^GANx0QsO`S`lgP zxyAR=BLO_@3P*3?-kViAEJc~M)AYowds!V)4)XvhaEMq=SqLSnDE3PJA!e#%XXs@NDdz*Z% zdNH^LbNEuO@dQEzEjx@j;>GbYQi)dOnR@Rs1oMxI^8Qx&5k4GUCdP(Z{7!6EJYl)Y z2Eb`7qbQtAzncz=6(QxC^dlOXdC&M0S*}~5FOE=p$Z*)u^j7%+<@Cspi6wN~Q6O&gA?cftJV(yAfEtoQx`N{zF>@_sWYYfBTr>mJRLViC%I z-@Zi@il(?7@07#?)7k#7HQ%PreBHXganSo;vytvU#DdYkcHaM&-SThy<==^ztfHfY zt@6V+1-w2eIPP#jaSfSzEiPzyxkhzOQ&Or}1`M_lBC=j0rIvKc&~;(Y+^n;%kU1tM zhO1!uei)s;brut3WSYY>kol8h7->ich>v>SuvPl^t+ zz;5ro(^7mxzgAQFN+fSgOsVaug;&|~QrljCfNtJEwWl)|zamtPOc|Lz=C)BUQ#J_H z08B$%wAF;Br$Q`t%XWC_Q~;hF_o;m!8+v0SuP#Hq=+^Fb*-Vd#X#^*4&RmsqpL+9D ze++Hnn$tI!!msrY>yO9#5VNmBY~9O*MI5*j>JSsHQqsmDK^%B5V+A?-6HHKA@X1C~ zY||Xg1xjMO6*H21TVU541 zb2aT=-xE#i;+Dk_3=T47b!B7pq$K0qab(tR>vLwr<5<}n!AWT?#WP&%(m2OqW3IC&-)>8qlRqCPG~Lnx5Lw8h|0J*?QxdZlgApG$Ds7 za*hzn6|#bEA3}y?Z6kLHPTUf~Ph4`1_0H>K87CSl$>Yr$(*ksVQZJJ#%=PLNU3pA0 zT)jOx3s-+>w?`UvroVS$c9|nR!K)$cr8sPLI%Yiu1-2AGQNj!KLs)*8AHd4v<-3sW zDPXtQAB&yhIFFN=)#Rf8zOJ_d6p{m7MVkTg*u?xon}aBo=J(ZJ%at?(F^3Yr9ebPL z=N0o0doqS7JQ~_fQCRM7aU-nrsrNR25>&8#1au25iD7cq0v|_rAQy98J8=SH9g|ea zJ%;pz{iBcje>^sT_;eNpTF*IEKLvV6rWM3Jo(h+UXB*pDm0P+b{t@@d<}cB-tj%bb zKIO@AD-YGs4P1#7W=Gnf?Z3)nn<*9lblwaCTY{;HdqtmKY`rx3U7^z>_S5g>bYu=t^D4g|Bju_Uc9jMC~&h6V5zkeATRW>x-Cg9(`!NUBvV#e}c2~6l8 z8WXm2v$eJ}F#g~4U$&x-RKGr|uWspdZ1ST)i75fvBpTgu{qgX%t~ovSVvs0ZD!M>5&1C94FW5w}+1#n4KRYfOVoOJCXZnQ0H zk`ancTo5~$JmF^lhR>Zar;O*$bPO>L^-AP|oU^wg=DB!MlbJd#i&jlN!xNg!T2@JU z)7op_z@{eO)DTNL{u8Au7{=G`j5N18CI(%q_LEH}ovT`k@+yqNI;@rmp0E>nuvXL+ z4RjU0e?<^Squ-mK2xIK+AnZ2edLjuKp_gZfJ8rnxtB=IL4Cj^CbSV5A-Mo57l3BcV z9xcz;Je`7e?|7CgP&9X8i~IsikS&0;$oF5+GCDi?c>JwR#9;r2(8`!N8yFin8~pFk zCfUd>$fJ(T1lw$i#xvrf0_5XX(bNUx<>4WR-J1x<$ih2kwVSyr3Q2AHp0mC4hld6+ ze82H0-S|!56;%n0COCP`@P2LcK4l%gzkckK18C>lg{6>rlgv2!negl}g@Nh!?;zQX~LmQkdzE&T^0_P;!(5=E7s>+c(VBtgRZqn`tWZ zot(1gt|9zZn-Cfh-4Op;7FO>;YO&5!9B*aWEV5@IC+1}Q%ebvBHZ30z0yPPnPv;4<6Tk_=5&`nuF=9^Z*v<`4PO#(gU0N zEa>QQDD-BiV_K})b3f-89yS;Dg*L0N2OxQb@;9YJ9~?g|v!#>ulBt8|GdHIc^Z$xu zdIrgqDy5E!s~>x&A&B2qV(lTWNiPE|;dtw>5B7*t+s z0%PPu8WK^F8M3@XS!UO8v{?76@_%>k`)o1D^9_{ULuAJY%!E{ zrw$2ip{im%8+s551p(s@-kd-;Q~i+jv9wD>K2?)sqpYqcljs)?3a^MK8|o&Ao6}j5 zNYEFf1eFi4Rfrfi53=27M?A;HD)Pw6(PIRvXaiJw6yBYn0C!nR&^m#~JkF4-_xj0; zobiI}94Bq1lo4?P^Y%t@+0O??W=OB#W6!Jmd&$18=+iqTL z0(ijT@)06lup7a+&puem|83$bn1r*m{1t2rQ2#A4`2Q>C37eQ2xL7+Yn%LVpSvcD{ zdi;w7WejW$%>HiD{Lj}HvlGT7!32;(zG4!iG*}`hrldv*!}ZT3Sct?2m6XHwx|mO* zaLJg3%*f)7khX;c06fp%_>$CdQNB9}@Ho@T(Bi8+ygt5z>SLd=MX*V-sm_b{PWuxk z3!z7H{vw>Im9NraEVHDZrNhX#uO?M1WOM_ieIYm=&NnF{cR}OYqc^ubg38^zC&zHgr@KVqU z>5~WSwpxtm!0NRs+C2=3GaP7*r<&d55kC|kyC*fj*}YB<4RR@zNcSxspf%MCCB6VGQ*JoFFz1rwnOBsYL+-`hlR*TY%)ekXIc(B($7r%_i3I53 zoS-@D#}6`drG4%Isvj_8UbjC|Buc~w+64fW;&S`!;+fCJA0W${+Q+nG{Y69b-2b#; zUYFUH8h_~x_+O>B>i+?Xf(AxbF82R)$S#gX|5t(gU%hg+%Ie>DQ&D-NEy#iZ;1Ev; zEl8lmwq*`R4zZGL=-n=4GB`LTBk6b$w3UtTu>1p5AjwqtKG%gbcI42KiYb2%>3Dr+ zIDIKY-M&IaHPpEH-t zq=8`;)$)SrGIvCACgo3YGmq-aaj(^GlNAsT!~bA5-e+-So8AeE27R^38lc2Jif?iN z31p+oI7UWXZv`f@MQ}{klCJAz0cL{tBlGq)7UD6KVd1gsKfYH!G7dyr+k}>8QRmRZ zoiU-Q-~eEs!ks&h?qmyiw=var-;Il?;-mHpFsv{~%dS`a4;vnwA7Fh_6NWA*_!3^^y1A&hpJT>$wsNf=&a{7 z9FGr>khOX|A^)o80+WgA2UnynfgO1RDdr{yY3?}J=iw804*xyM zhVob4)Va=(y59zmd$_8|;qYdA4AY)RNQ1wNbqb{uu2@r z*Lcm&Ibq#BT2a|y9UxiR*>o9N|$X5Z}bXSQ9 z^l97uWujgxp!M{|5~K6Gwn{l>OzqF zxY`~0wqwru^P9JGyT3zj6-IMmivPHr>I+Ts?}|>@!v+;TvBT0b@9OcR@=o2uJC*LA zhj1#s730{--QtIlD&4||QYyYN<9aGDdMck8<9cc@*i?Q>hj!)fHpxAOw{9w*Ipf=E zFHlrp@`w24??%af1-Ei4pXuXYCATvwpTXlj$}ehEUmfFL`L})*1wYaLu2rgv(5obs z6|_*jmKml|#VCpdQ&FinG*XFH7*O{)x{RkSLisg>_ps+=ni z(W$%(qIIauqSdJZVk!0%M0iTVd?MVn0df?3 z5+d)4fF8xc8nik+02JyCw#d6GK#p?HLgZZ*(4#iU9O14Cn4#SB5b4Yd!$7-%5_wku z_$dxbMLahUeHMhJqTNIg>8cFsq1|K<>8cH4N4P5i>?rqaM7*m;KO>yiraz{i_4pcV zsbMz6caFf>P+G%#ZNM*o_FjN{0w_plQ+w4xN08GJz8}?9LTd}{ta-Pl>DYuqSc3t% zVR(M-$bfhH|IEilukTseRc?j>$rab<0;ao~>({Iq03PT}kJb%ffW_W?9r0H9Qal zU@t)s>o1v?^^+91e~v`-NchMDYD+9HkT;HKfKEOuy5j-rM|#Z8+H3Ih)(dP%2!o0`yUcvB4?R*FS6yKQv*CoD22R^}g3-7>z zeu?w~e=>q#01d!nNbFI8bKqp_2524qhyYg*AT9JW68E6_JmVt)um^cJ1ZWS*H7JlC zaJ>sY4jo0XLnbeyABD??6o?R150qy^8iy2I{YioSJO?fc_<3-*dI*k3evJo=y{L`n zmDZ;Rib0wDIBnI5r0Yu0xiAGNr(r?zSbn$9^vb60n{ zT)BZ4(9hxEmgn}oZA%a5HsJ5}ph7TFjROeY3Anb#?CZNn=bxZq^{Y*E z+!BF~VDyS~%7JM2e=2`dyC#O+k;K}U%u)NMR%ZiYkHGsG1sCl%FhgX|2ZV#MCbNeF zdO`Du=SBAtgZoPDV1mvdy90aiSwHE4zdng?Sw8jRYhbvyV50p4YshPXy=dUo3&Q0) z8OIYn3&Q-LN-`WL4&Q)4??^I4A>bf^Zy*_@Qx6&9QK?10NS9cixnY=o#t%tjvD;Es z#;hwsq~F4rU%_Oe*O@H4f7FoObx{t5{|9UD6kX}tZ4Xvlv7J<$?ATVtwr$(CU9oN3 zwq3Dp+yDH|cTSJK=`l9Hveocl_8C*y*oGJQxrzprHRbb#gFXWC5}99nsYFq3+4p+tdPbQP)$@5)`Og; zI=W;|D5{zEVvzRp=8YMVR7^v(w%k<`8Symg)+jA>7NhcAQmW$c$Xp-xD~!}zyu2+d zNuS$hX~4~~+0TR7j%TYVEb87B$PfCZn#m4vLrZcS2{;+4C334WZ2aUBiy_#<^x7lr zuG7&4ulqOh6Y_;~uK6yIxyK)!ys(haT>^#(Tz+$X_}QV^5+Up_W2vGf=WZFZk}~-UEN; z+D;$@thSa%^2L@ptC_n=22l--3=lBD9HVOqApc5+%|JD^iJ-QgbIN?h;MNaU`Rd_3Q8 z!QWf6#7c<}^&;^36A5RYgwJYu3F}_wNhF=n4^7dtvKzCJH|q6|vm)H8ac0?SGQPw} zm|&JQvJqVLfI%B6+%po<gmXk_hND1`$;EUK_Rq}Eh~n?a=^VAwfI z=Pbs}YcpjDX(lhb+{HxaBuST&#-agh!;BsdXsCZ8%K(o|Mr)cEE@PImrrCyCX@R+pV%S` zM7fL=oH1U91qn%j*n=j_R3)O8*j^hU9kAzvqfZ2HU~ZAwhGsn^q{tzuASgrGDcDfs zVyGMP!_UUwArbdE4Qv{sshZ9m#g<{!o`E(4H&uI;A*iMBlcn&p4Zl1I7n#-a7kVr& z6f}~l=YuGzImscEk{&@52R6Y_M&|;o7g;hMW+I!WK#&g9QdL_)l-pnzf(07-kkN6? z2Xbexct?)4H0&O8>qVB>W3Nwr;UAv43HOid{&HIep_8aBgg1r1O0Q@Teja=_E{I8- zYbCV~&rZaMTV-yz2IGM|hfYaBh)80{2sI-pSkghi=0!S>%bMFa7$DJcX?QH+=q~fo zO@4fp1mqZH+e!}%1d1FA$GoPi6ZtK3tc&K!UDVeQI{ zg=hDdOoCDmDi+~FwX(a%jfZjX%sK5(wjD#;H{{<;j4XDw9EDA!P)o;ro8 zZn;fkD<&}>FUt1tZ(VeeqTM6-#CBot4=v5RQc`Db@IR<47Kb|uF>iEFtYJk?N)^a} zW83^RkS9JmOd(mTL(=W8o%}2q=VOFUt_nxf!jI@nLk~DvfjBOGp=AeU;RbAD`QSfl z1y}Vv;T0%YE=sM#jCUnOa)dxSGoz{ZV{#Xr+KGw<@6W{Wshz>3-4 zE*%b+#7agHUlpHsZs}c9Q1W)(E*YNW^~KkwQbY6n31Y;$U!_o45s8k2w2&8I^{NIU z$_m1t9CFL(9ja$H_w@P0vu!KRHQPbdhSom803wkYlM}8Nd(rEP4_->Ifh|N=gL{Lg z-ss@z{kZ>A*F#^)S+2f2?R0k=|BN1dhxRIPtMZAPn+vBnR`{1?0TokM2GdI1fPjZn zVfXCYuU5d&$WmL*bSK|C0@H@HF5E{DNfk&u?-Xb~9?AP}A>-t%g@9ycXd=H9JIBGi zFZhL-y^Jr9Z&U?PaYF0~g!^R$Sa7$4llyDBlnn4*_D&}9)~T{nq}(r6mmWB7)d{WW zBqSEO5VOqA{@vWGS5qrOXVM4tVswJxe9}Z2Z zqN2IJey4W1khzj2Jz)g~R8$H)=bTBsIbk&YIa*=?i*KVqHM~A~>;fIMELTKTq_2_C z8c?J~xAJ7nU?91_&)LA;;wP}#sn7eJ%F@uRVPE7&QKGH4Jh}SbR zq^nQvj-*SgQ?*=HGsqtv7!MbB!X01Y3c}aHA+p#DjTa+N0FqK09k1k0^Ts*r^}qx^ zS{4rZyz(+Ow-_i!%IBT+Z3fj-S0%VB>+>sv5j!{B%O7*Rpp;kIyOCc)4*Ey06=NG8 z1I*C&Dmt63lXrB)YRqL>lin|Y06Z% za-D2VRfFA3>YdAVeT9h;)K}{2%B`v0 zRu9Mhw$cXbFl0z38yjdNd>C=6uyQI0L7m1`F4MckPO~tKZtTwEA)tYIELdorZCH62 z>V+u!d9+!w3Ixk44BR~;gL3BH7FqZS1uVJ9MGSp_)PScq^hQv+#=Q`grFoPf9VoVr zP2M->AAEiFHyuV#E9(4!*(7p?6_1a>nSBFfKQ|RddukjQ? zDNz@KNVhJ!po{^P9%E7EO5MqAIz?k+3LZ>v;WxLTn^+4TLr7F2#WuiC)WZFQH3lFz2lUfpv#kS?C(1W6BPV|?8S z@%sm<$HHJTaF96M)I^NS!&XjerGgwoO96#%=Z3#>an^!ZHIfz_b+DjPxJ3{~24WRW zirx4R(=*hkhP=`QGCqQK9y^nev9=ZxQUK$&I@omcnF{D*tneYal4sa3^SpY1-aPxv zc1b0Z2Y_i635%-lNG)YWs-B@e?jAf-u6D(k#Av8z@dz|{|FtGK79}M{*j+W@Heo-h zN&VX(uX6PJjE6s|y;n~S^W#jb=h3C*m(2AhKxRaV0guNYBg1j=Mk$I zBba3_Mi!>Y;l46tc`EEV8Eo+snQ{j20l#aiCWO5*s@Ip^iDQ9LmF~9k9tO?2$-(bIn=*-C}MhQ*j42OU;>`1xtmOH?e7COAj_A1|-f2 z@5i_fgSS06j&2ZZTc0Q=JUml=Q67*uB3;NMOQrcBMS!Qelw z=(j8oX|fxB$1o1v3c$0edK)~~)=6^sc_>MgCysXllmZz0_t9WXN*lmnQ@{f5J+BX8 z(}{{XxkvKn<#TUk`n9@=({VO{Mh)!31T=?^%da|`ZVQx}1b7JA8 zZRV}Q2!;Ji0G)=tR)lwxgx~>&^XrQUsGa06?nY>Arim-Ol~ThCootn(;*a3YH>K18 z`Ewp;eRt~V%96G6A1J-AXQA|xeNppg((%$ABziNn!w~*z?s3M5l51nP{+~1U;fIC$ z7s_1NEOWf>1=B~I;Pn#bR~=HOy$#${yrJc~*3;tkBtdQ>kh1k%#f1f1hmS=pWa~vL z4)|qnZLIX-0!L*V6_Z4tsusk8iH($hv0$8Ot|$S%OqHT2a=p5)QCliTa|XDyY5A%L zBY(Ptl7inyO?fO6mRP;l*?8N$_(r-(Q}t11=TDu)s1xR#O#&N#&q-s#j4Qj4BF7+q z^rFqlsJXyDV<Npa_&qcAkSn@>ve6^NuGLLF)M*FQ{D4n(-Y02}{nYWs4-g<0a;v=Z z%10a!rWNnC8M=H!J_R6*tLLP~PIKh)sut z;%vWgbQYcS3WERxvJ>6me5`6Q$cY_t8rEE1^`R@yo3H=`%uFF zXwxcrb&A=3eZaaqWvDJZbnA_@6A{^6xtR_{b#eQ2vqM?Gbpb z;aLU-6zCM#4x8o3IdtWxu@P>B@r6h4&lPCF=MR65 z5U7pd=Z64-UNNt>rB??>V9|%;LbrH%3m&r$;?hdFTM^S<*K4o_q7k=;3-OH zCKqNZSey-FsIOI)-$!N$$Kz*Nazfk@%@&9YncK?4B6Ke}danNWt(eri-AHBvW6#Er zAiIT$!0JwQqlX?J2+K0%>b4&)CKpW*e+!P=U;FG)1A?Y_it&y1oktKlP+FI(5JFb1 zJ3sK$D@gC9AY0{4TBZ zPv)OOgt?Y%>sq5bxA#N5gr~eOhn5x1#&zDvYTZH4Ut`fe5~QCYp3f&3tQZDa6Z;08}N_e zZ12X5%L9+$&aQ?YZ5S0$x2nu(!D%(mM3Y8iF5}LoYamaCqElGEjUr@kXWt3p_o@D( zxBj&p$6r4%Mf&0|H!X?i7f9FRg8R5MX2TM4GeM5*8Q1X0EzO1XR{0fTf*Z}og&uOW zwjwh#6RV57&wda+7P(-;mtn;SW1^Qbrraw=1>9Jngl?EjK4KOs(&hP{BPFXB=~h60 zxV>(0c}(n7RJOF&bHqo&i3fq*?~)Rr739UF+R~9*6##~5=|49H=L~+S95Sr;MJl_` z@<$&T$2wY~D)TG*w8X+5`2-X+A$WBX9Nlc;-+e$kT-z!2Du z6RoS#Q+rOq)a7&CS!CO+k0&&#FwTh)>=Wb2X@pI=2?HXfhZ8RcMP~k{rUiZH(l)-f zy@jEH^MGNTMCG`OgG>>X&ZbeJkbHT+kI6dK;blrhq)R#^B9TS`=vjl9W&7oi)C4lN zERE6YEWjLG14#v^po%?Vh)Wm29eK9sSN0x$#Z=h2tZ3AUJvgXU0Q)&FtFk$TL9NZn zC1tO;TOXwy`|JAk(EC`*D(-Ucx+M%`Yv*q5qDI6=csBl!6V!G~S4IHn4Hw3+TLdzj z?<6Z|Z{?0Iz4OQ2as*(3?7cU^m*n0zF`+D}Ahccxno^YOO!$=5EC7@A4GHeiX z9I!lYi{b&cey*fhg3!H%5f~*rJ&WLzO)p&8&aYF{TRN^7t4flw1@WD~fx9_tideVX zz{*&_%&xbNPBWRi4UGa^aP7mT(r!P*L!-#DgDIR2^FLTw6O#31use1}T@Eio_{p?a zcf!~@t`r{yXgw8S!0lY8Q?F{V3m$Q^fcL6&@lMC0u%~}zjHO=tdNZTYqrYxnoMBXM z`szQD;*4(clXO5Etay~XJmm&^`KEm#B-O>}vA?1CpW7hlefcr2B-_%zK~?GcgkY~M z*)qO?UmAV+L)1)5`&3V9l4N5V%Q(8wNmA@u9urHr4Ch?dpw<)Gw{H&3W17iy?IVLv zY~?30vM^-9J8SzAEs_c2SVSK?HrKzajpp2D7}V^nq*A^5MLmh2k+}p6>aD232%+`= z89cYH!AQsyVy9Jw55&g^HQfS{Vvw z=|x1#4nsM#M7ar3NAv*_Krj2<(O2OrAb*q5Bz_xF9S?jg7dQ>vYmw`_)N9LwEqtw@ z%ajdW|H8!I%-^%z5RVge)W(S{b{5 z;*u!xU&Gp_R<>xI5wa~TEX(t2C$>q)GE>@i&mr|L8L$Of1l7F?k#8aS+VHHSN09dw zbAL0$GcuRAM|poFtHvG76Qk5M=P6>IIo)$@WbIafn?3e>Mhpov8RMPy3dOwOp5ye# zj8jMb&HriKUYtxI5l}oJ7b8EPrOfYv6_t0&zXI>euQwS+^f84+K0W8e_lof`&ocEo z_8}{uddyP;iFOeL3roC9;RwM})vHhgeHl@2kO4+G743wEPn}+(qpOmEdqicMK4Moi zxboa->kTrb!8xWgGZg(~`s&W!h?}H2tY)vY3;S_|kxH6Fu*mY$N!dX!Ev>nC^jCUN zfx4zv5s_m-4VJkcIhwLi8m_E+cxi zAGlDk?@Avl*A*tZOik3liL}AJo{%liX0}W={x>O=GP~ZYebYZ^i^eHkf2zA;xCw3R zcVV?=LM|D6wgqK_G~YSwpmzP@IXvp^egQY$J7VPA8d-nmcZTT>#T}+KCh*2=PtzG? zxhr3$em8f9=uX<`_ZiaOU8;wFi8|Nl!VFlI;)>atfPO*SjIlZBf2jmG;E;3$RPR3R zcsk#nJ%hZ1eax*gDSb>^QnnGier6UpKtN)Nl=A=nMPHign%j$1i7dkJ^PW85d6T z-SYx)83GQ~PnPBLly5Z3B<_Z{Tm?_Q^iMC@L5*awdmblc8&wOWhZ)lAshw?1t_Q?a zVc4UzabyTlXn7=~aSnvfX~T8x-p?UH2pj_t@wN+}PS7m_Fe$1+G{>F1f_S&Yt_j9oeal?x~ z!0U1RM9diA4Q_o$R_);p0(`+t?d$Y_f0Aepbo+C@@NV;bGIQaxo%5A@1b)SH_k0ch zI{4C~z5A+Tt?lxJcoRRKo=?*_Y6M^=T00eK!HrlabHc`#J4^^@HT2s&?61-{_leg% zxLpoV2xM0moNn8)!5x`Tjz?jF{sdFZ?bD5g4}pe_MhT)5M+p$Q_umTyDGJ-9!>uN{ zSFviB%Z})u8BDsFm--tUW?7`SXua%@>gd+mRXDW{qH%34LD~vEdSn1oJ7$vw=)Pk6 z!($10sY22`3kQiZk_;lK$~q(U?~AcSFJhY)J%SOX==qJ@zI050H#O z;jm(zJJ<9|}#G>M~+Kfe#DPayPP5_io zpvSrQegPo_E$Qw-OQNo~&gBH{q(y+wE_kuZf>PkyjWJvJ^Jej#5MGg%Db?n|@XD+^ z2zGsx)R5CJ9a+EkseRwicD$~?8k7(7v~L94IfluEq5^W`1bdW?=Dw7WK&(sr5w=XX4sYqvTD|I zE;aQe#3D>adA}d|w}5>zqDV_GSqsGD51tDyaqAtW$V8iZ5oGl0()A@oID*M#*zL}A zi{n96$!E*-OB?zX3^&o_785-=pm}dy3{gv$)GN{e<^4Y+uQ0{cgOG4TQJ(mg4ey6MclnDi2rbX*#)Q9W< zB=w>9jR%?Q`!N&Ww)e2+dcwlpg(`!qRW2Z$HnsX{Vbj+3zVmy^32D9$Bn~lmhPmsH}84MN$ zamuA<@vnQDjxx){BHLsL6^$zCVi~!V;Z}@-tC|6%ooFk}Y|mECcji*jO#*CZ?7w~T zbu%7F$<;9}hd3|_)A`ZpM_c+G?eb`3PNuwmwUf*+F^qz!l48#__baUZF71J~LIM$0 zXX@4&%FWEuI&>iHRySoJh8|^&FoT@eMR)X7>E?+%^iv5YK;cN4sSZx1^6&a#4*=L> zpo|ge`QpF{7-A-jBho3g>7`W!7S=&2kIB{WMF*elt1eRv_j20{h;J%3q`g}qdv?f5)8@ACw>5OuPD;QHv=$VO#zmSi95A964aa*}m4Y#~xPe>v!BLOv5tnXj$LPY_HdI?nS$ar>g40euhS`9?U- zyV~o9aAh$^0BJ@3B&uI=g>11oa~A>W$eb+(it10E3U@6tAEn2XmIj>+X|npkPWXo$ zZ)&hb5SDe;MCreMQq;ZCC7WCo)Qk&%_ERNdD1qjZLP47FA6s2FrZdSmV45SZuD~eN z**O-Iy8+c^5I*z)O9T~6sL~y`El8?qU_{c2k9A_2znXhRRISB%{i$0ID4-Zxu@GT7 zMLhXSex_w9{(GwxpjcH#%v2Onj#aaKaVD4yMbHjEio+ArUp(><&*r%&rx3zNEBP}R zPR-Bn*_v&(FD3s9-cma+i4WyNvfr9Vcl16opQit9!pYjiiGw;lN;#CTPeJh$RI?MZ z;dXfEl#O$+W|T``;ehHE<`(A=^B8kx+fKfcOSL)?f(=40*R#c;=8mp7-h9GO5rFs0 zBqSFT&o`_zU9B=To#%{gwWGdIP{|(akTh9pz90EesS?`};#+E6t9K84XJW*w$cB0v zCO0jSt|tpO9lPhA__zv+8;CvhK#sVFQ>c2u>&&@+N#&fBz#;6UdjXoYsjrzm$1}CL zY}ZxqVwNpeI;C;yoD_HWI9Og^z0>jGc*M;<$^1s5RiAyChS~r+#q@QaJkBdI%gg$x zx7NjJkq@#HXJxAx9t$g)u3Mcjdg=m}0Hmw6#I6+A-uI^z8BX+-NGu$Q2RgcK)19nqll_e*-;{nM#*oU~I>%z&kd3kN+ z+|7em(8dq%b56*>46w9{AbB+iRC^Z1(bnzQJH0k8)WZ>QRebaS9hxBH7i3i;nE;7G zbnE!2mTIQny*UV4G2VA2Z=kD@gX>r>5T4;T8^cg~Ok=mYAY4+FTJV)IetYJ2;VQlK z1N9~)@2SfKotleV$tH=c!C&#bMzmw~D@(VUkC~d&LGKwypjCGQJ!@jDbsN( zH*U9BUCP$GDx>Mo#%_#0e4Bk=k*?L_oX^9zG@n5o={}Urkfc|r!=rBlOm5w55|>Eu zBQ47WYXjB${ky2GDjlSFs5axn17!Bu&%!)3R=1EV5$JloTO(q(&GuQI z;xtLqBl8FK_EDbFS=8Nsun!_`EU#c#CzI0Xcdz+>Mc$=kSb(YQdMVjIdhMSab71ppkc_7wk1vl(o zT_h*`jS!1u@7?>fHsQW1g#yMMRAf^hc_qC#md)YT(3|=r?4J9zuUXiPn|rHEsOt6n zP6D%2UIcU~KKGk-$K>t^}F| zFu#2Gu{jCjo8aeY4%Vbr0duDz{nq8~G94gvIs*$V6M6i@A@tLSu})0Eu^!5&S8#Of zwL4lrkU3dkY?P02DV;D1WOv69$fL6T1^@RTYF!bInsg@Z7%9JCN zb8d?U2~H#8Tvq4MZyLSO_Xs{WkbS&+VNrN&Z85ypw??_l(a8lk_R2f8x92KX8$l?~ zU`~rU4k?6eBo0Xs?x*!F(S^tJ)lT~BYv7i=pAC#3a_mb(b1u`j;eNItQ0~S%S>Wzj z2k2Zm+*^bs`v9?C{I$E4jTl$BKs&9qB50a|;H2`EF9JpmTAq68{(}F9W-EdA!}$H% zj1JB2{|B1ww@%#YA8G&pUFh&Hnr)+!+qZHV1H56V0_N`hUbUSDUc{o zZwi9`L=*05THV^jCCe-GJ8@blhSd8HZ@^yqhI}rS@F(5mr}5#WN4JZVw$JD5BS;UM zvQlJuOWAI2xEt!UB2|g199=$s^>(a38B`WUiW1!t7%n<7j2K1|T^`?{Z!k^9e9oY) zb{{INh^?9cD~)&yv7YQObEG)Nyd63w?vt&=F!3Bm5wWC%YvdXS6cQQ<6;bgX`y9;h zd_J_zMVpn9RkMC2c%L05GR(!vY%+}3$@A1r1(|fBjOdDNgk^W*-?0MZ{Wn3zogr}a z{Yzp0S3gkBEKHvF_?54C;v{R1&h4;pqowmXY-{$1C*@J(_0{_^y~HV_a)b+I*Nq$8 z=}gM2=?EiC4*M&Xa+p%FVJH7L$0-k^DZ1xSn`J#_)txRCSP;Sy30|G5b?jH+w=t5$4j5|pNU zTp_t)P4lduhr6;PzJJj!q*fnhti=>9mu`r>g6E0NR1=R*uL{eHHi9}`#{0_c5ZTsk z%yGh)cKc?D3U?Wf+ZhdxKjN3VX(U_M5#qRmf*<>1I3B*1kcfY2|gsuv3$FNhoE z9p5b!FoxipxJHOwT37|YnSKgQC@N=3rYMZ&5_g4naVu8HN$<&(O&Lv-#% z!OIKHEtW7xg7S>%OKH;qshP!vFM6}^8Tc>&TjuLsVY!&FUn*t*XTKc)#|-H(2J8w8 z`)FR`p;iM=^c2*|>~G=@>0$_ClZGs&X1jGJd5vSns3~fUqS@M@O&P~Q+xQy1%s@7*E&`vtBq^OYz@D;Ar6Nn5MQpVv$D2?Nnem8J({_-FP9FPQAyri^_02D*)N^%Fw zm`*e?GV(g|xo#QBTJNm51UpT7-h7*L=&2c(w61z#&X0s@Hl=4crzmR=hBdaw+92B~ zU=6Z9h_Jw+9Y`JIT&X~0xlZC8ra>ymc~0Z4+)ArGP`gL$NPHqzD^ML8ad=RZo4=>9Vy{=XHNl4cH$M%Mo$ zQ%Q_zm+q%S365EtvrSZ8Mo@sz?I95a`U?q3FI1*T>^#f^szMqqok6%6P^`BN&y68Wc?=bFI`nX)1(>uss}gE8?87+Y1$abEZ(~R_eoNRH zx})}pZw~rxD-ex(hhI<%?<^z+21oGGPC2CH7}&n>tT6+7#V$ZwQTOhJ55;ic-(Wa; za=bRO$bxBnm%Nst+F-eL^blUUK|XZ;345$#6(Z)hhKTn+YKVmYW7wtsYfYY@k%NJ~ zneDd{Ov=dWf8=1q|J5sntR3y${?#c971qp=3=jWEred2I1NW9~>+Ivb=IEk$UW2>??6S%4b0HTNlN+a3OsX5Ftv>#d zP5P2vnza<}GzD{Fv5b%O%i5~(2|)x@<@-v2SLtu|aFl_w*lzrWU4mI3q!8%ZbRD-< z-GVjR+C^-J=(7(Y8iy;aIk#Me%>0WyEo3RdD$c)x=nW9DYE&M-7Wy+HS5B|cH$9T% z(3cfwLH}I8_66ed73tM|$beQy88-m?YDWg6IVd~4k#>siX0`>ZPdUa`Tv9p2)9>t6 zwzMTyh@0HTV+3dr5TTp6YV&hVeB~zCy3P_wt#rWi-5DLUao>v^_H^tC$>@?~l`N#z zZZr3Ygv)hUL$lMgU`c1fyl&y%=F?$=NyBzQt~y0SS-J(-Ts*)9X436@mbrT7+6-Qw z`*afpI88+usfHULjSF11!pQ^*M32Xp|5jC!&XZGm0Yn&qwrU&9{IhX>`q&xUD}ukDdd=tVI_znq~cHHjAN9w`)(??4UWSik?5H-xe){qD(k za%p_atN*>oh4cSya{YHkDO))pE1`JP5U((40bpfS%^erQ&D2-q1Np;I&7h;0sF4ba zx~${Y9M>kTmy?(M@jhTDo_pcF`!I(wQ)5xiBlpNMZ)BcjaHcn;Y72G>uMbu{u3C?{ zjmI`wm844|vtLjt4%=1IHodd%t}izB$9J&|J3a zX%>{+dGmPSe}ti&m=t48{g$O?9^Yg(=iJ{U`u8aXaAYU)WNE_n?S+pZb~JG0%&~5` zyh_r)zgOR+uyEg|t$mm_tGYDS)Zb`N*IC9%;{vsQ7rHjNmPBw)?fkc71$Ap*J*eGi zxMZG(8m~fSG)B07Z-@eiP_r_r_a*2S++DR3MbAO3#o|l{troGiBO;=#$ z-btp~WySM|!&Yr!8_P&O;K zsn>Dh(#6zIcE@bzOpYxtyY+a^=PYltpea*#h$Y4ZG)(;?SpCeLWKJ^`w*D}D#j>3; zi6$E*#eVrtqpBu6%FwbjCCo$TeA!#>M2NQyqcv7a^RRNbBiN05rx+FBfXFeW=?c9Y z_HO0fDW^`uIGwSkGJ!rN?bO+ggAVQD{qvvhqNJL)QVuB?=FS1-O@jjV!MK-J?et`z zh(@|o)~&DawW27NTjUNmk!E?xXHzmG26`NL znPB#N)rkez9@-0TZD`QWJ#?o+Ff{F3*9q8pm;L<3%E6oAvCaI?zPMt|GbIiY9d0IE z2Dl7%b`}ojQssr`hxsL6_KIY&LhiQ4eCHXW|+Jg1`d3_)DU1!e^PW6J&uK0JOwttc$YXfRDih8hVM>LR zg#+oXdz>)V{1n#Pvi35_Ui$Mpq|sbG8Zim4=p#doR`ZLpC+54$eMu}9^Ox?vOILY5 z5;$&}6X7ezI?FOsMJPy}X;BojI;(J;!FYCPyn8smbTi;I+#Q$6IVuEooRGN(cVJ=u zxElf%2GUnljYY$X5>%0^$!nb>;!*O*aRx{VY*Vp#&BxWzx=B0;jp|eJ&0{_8rC4sl zQ$BtD_O)->HO09H?7;262sVvXWslR15$bZKpsJ9>Vg<@VC%#5Gr9VhyN#H>2D)M5g zgm@o9%~TVl)1}>!Rkmj?rvohRPJI5sUFT*R_PX!3$n+m=wAubwtx?wA#@Wp9f5x6d z1r3Gy-zc9fm9W&YNqMM3@vzrC)hr50Nx(+&sxg!(?s9~RVC4W z%hKQngaOy3V1-1-`C%%;toJg5Fl(~R^f-Y)x9D#FqCrY1YrITirz;&M_G+Cqi}|eu zW>nUw0-9+a?!cFApWLXk)JAAqa+1ENA^f5nqE4vWKzlx7^w&5S)mA6mw~A!slpu$Y zV_{EoD+=c_=r3!)pD6n*GTS#$<yx$a?qbmM8IhQ$?9p&2f%JH<(WXJq(Qor&asRfWqnU?IwEBG@o9B$5MaW8ZR;h=a{J+lD z%pgG{zOd>NONcJO^m{H+?yspF1vxg)x?VNhz*9f4!)8_E@8J*^z_l(;=zd-q2jEv8a7^ySn$&2ekdkkiy_u` z{}&SfX!ccw{2uXG|05E!{IAkY#op|Db;iFy*h+@)ADw>%q?mBN|F+OIhj&MS-%#{W)sUcp%+7`5 zuL5EP?6-maHkC3jjkYp^9n$CMpw(lp4Qtg{W6C6xj}+Lxdz|n^M$~mxyKe?`M1|%% zWc|ySW3Xlp;V@hDyfT`hdm|EM{sp=vca4+*!VE& zT8V8&;mDaNt1#B*QI@qwzj50KLeQ)Put5+3xM*Q8)Efo~=*K(+h7ZU;70Y~klDzd# zfW8Nse?QEy{>Ko?>e)LO*((@1I+|IVI7r!8|6`o+e|aS&sW~8vAPj#I1)vtx_U_uU z_32rPGzE&l9WoUMu_KMr|sb^6^F_Q;X3q#7n-ft zCR!^m6i+bsDE4?|^KJ^_a1;jpF7Yr7DI1M`ngJSWv;^Cr9i~t;Wy)>~ zRIeSZ0nQTr6n)4%j?(+Y4J9eNw?J5aFoOv05r+6+SlSV5kfqpJ7?QJxl4@4!M#dXv{L>ooVUN5C}_m*=Eu}t)lmcq#mHtmzJ zB5TO3R>OjrGA^r@`YBhZDRqS74tShRg+k2Mj*q#k<7|}j@uTQ2g^Ob@XJ-^q_La$U zZC6*CH`pOQDzARG$q9>0_Oef!8XliRL$dO0c{CbKq3>CI@&?{* zsHWuG2~OoKK#L2Elx+!ftfW z?JfebGmM&#@Qbf2z4}qAWsMLVAScpn;ZM*#i|%$wU^O1~yXYdDs=p`cKCjezR^&+Q zqM9M}7i;`uw}`%DTJE|;(wx--lo`+qo5F4YWZ%K>HIBap_h)1*nTQMVnTo8U#l^bT zR%P^V?#9>9YE!Znfly+>o2{dByd%=yJ@7u?bhM;`Twf6k1|z~pyqFINvM=FBBIm@# z9!A&hsM^q6|F|g_(o|A;Fkalimg&T@Ho9s;=pj%$iFipygAE|^}GS@ zCq6LAV^^}OtViMp%=>TGHS3s_zrO^0RWJrDVD@08o~a)+@ioAYy8p}f$9WFsL-_kH zVdm$5DpltHeX0Jd*<~uJDSYo_{{&}QNvW{^3D`!Y*Z=7U3zc^Ru9(nItK;1K1bmNW|sG9rHaGPz4h4oeL9# z^6GypQDkRv`G3fXx$`dvZ6Yw;o3#Y;iA`CwL@UH)%-Vv8!6IYK(}l%j7134bVL`&O zlC(wcb`YYmb$ko<^5Too-`bf7+Ljz@2)rgic;V4p`UxsP=-tF7? zVbw?e9D8ioJ9gSJ5=D}mK2F&tR}*OMdQNS<(wC=+>%XQD)In}>Z9ka6k~2<}xG4o_^x3TzaEl49BV4dS#q2D+ zFPOwhSZ|h0fPWj^$j`Mi2%UT+dN>Vb!e22%V0Gv^m1v#R!rdlFO)`=rKGEPR#sTE8 z*R5w-aem7%>rOPMqtb3e-cp$s<=_!^Iy9sC;lprFLrK%W$-rYTbf^$~8>euq-SNu8 z8KG{Nj)jONcyqDNB47gM$&KD7l2te+at^EOnyOKBUPG72YgwTq>{_cE6tCBYzb3U7 zebNGV64->{HY}~zxybZ%@B_lj$$n(6N^MF;9hDAi5It0z)EegD*X8eXMyw@-JR_Zo zC*;S0#qIR3$R-)n=if(6N})}&=Mcf8L&)wwr1KF?X2OWmY@yLvsk#U}C@QD*@n&WD z$pj!nFp73b8W=nJV5~SWx`^m_wfcZs%Vj9MNN@~AU)|Rdk>HjHPHdhU-&AJrv38gN zC}Iwb8bT7;#dYDG<)$D0A7kyT-J6TuZ^*ilbKsrWZ?XL7#xo&mFnZDc?Gn{f+=g^9 zLhkaCTsf?#SW{M@UVoH@%VYdE?>=MLyUB3ykJ+XceOVL*Fss~+x^J=nf!gMJIAX2u z`t|#dcY*(TTKRwWCHhwZPbyB@&(k4n2FZrps^C)_wb)wHmON!($QyIaBa!ND_rVZV zRLTt}a>XH~sqFQDT;p&-1XL_br?K@;LgIDQb?|OInW=PtzCT0$5V_{^FGh4EToyXU z!O7CTrZ*v1fyCw@QkoNb_OvaN>T@01vl-ZT?bEjk)iS3S4}+|$jZK{o?qre9x?6UF zw*c9_R^xbxk}NnS6lRs~KF%&A${u47NNt*Hm!{YPQsZWzKVZR)H$&1Gy1#cxG^a7o zJACdK-72);b&#iQ=lAUhwnJ@VFhywUoas)dn9Yw&z(DX^r$0}C^FI6LGZP8wyf1gV@WU|lNMGC}{#Y2ofL8r^4D{*n#Gj_C+RZ)kFFR%$9OT@*KTvlS2$IlZ8Ab5(_d0oiT zfo&DB`*0RsPMs9`Is+So)8n;dZ?iw=W=)E&)`yZrj8>*pkOAK*giCaN~GLg$lgZwO(7M4qWs7!+Vib3{CyR<|!BV9JvMg98R#8$G=h+udh@0Ntbik#CFN# zdi3KBZT$=SpA=gO+!p%!p5(~>Kc(2e|3}CwhF%(k9zm!({!h?vz6gLRfImQCrjRc@ zutA5?bF!Bfd?(a(!mNL`{s!(>DfbUOiPzdWzkFf$?hk4lF08 zEubtQ4cbp1mPH&`d3X(S*#P%!Jn+J-j?8Z{k}xwabvQS=)J3rr$&@O!x{54&Liq)8 zcwNe#8Hn1EG)+xhm;&x#l~>*t@frzP+)26DuRz*{K@?{6!=~|_(+>#hT}jzLUB|#t zwb21Kp$%U3$x~C$ksS_3ZXgsl*$wD`_Mf;Ide_plfB)*^@8{p29H{e8)Olb&!J62ToVU7Q}yBt&^ZN+*Mu zNkWq)tZERqtL|-FZ4Z(e(FU7D3Z+xpJZ{GJN?Lkg?Wq-RheOV-7vrPzY+4^Lpl#&! z@EvoQB}dIgxkWpn;kIa4+$E>=8M#@xOnw+VF$c{wM;tj%oMBL&1X9%D0?#>+_=H2l zjN7CFcEn+nXepxT12f6J1V5SB*dZocICc<{eb|iQBXAng`e0NLov~#hLs(7z-X0Y; zgB((H>puNZ<2n}%G{%+o#J9&Pt46D4qmA{h=`-&zJrYI)>gEgUW)x8OuKA`#m)0|t zjH+oZ8gV2t?i8Oj^yoDTBX~YK6!1HiE66Lq&W%HX(hop?spKFs_4wuTgU7nP^HW6| zS<9f2*V*c27Gd5`MV2Lf;$9v2u?10?{m(anL9go?)>=Q6KtHqgb;(u zt>(7x%Zon@m1#vajo9VN7DUP(Kg*G*= zKLE_P(kB`Y!lBMO(wg&#F%;!r(x`HAK)c#nyHYEZGASI|i{%}_zpVSS?aRu~9usZ} z%x{VKu*gUej<{CQsY}cnaK8tXc)Wn-o$!xPkJ`5RvPP{(5T#1Uk~n@2tU0q!w4Mt z7Sb7VjsL0k^NsczBuGQTouO-&W$bbDu!32%j!_&VZt6OMlr#h=yZSnRF|4PcTREdo zuAScmX~KGW)#5|H^Y)9~$@U;q%Co6x2g{$ zFa_#~I^UEghr#q%2-0K4N^JKm4A$?Y08~5Kq`ejx8y&c_K4+;?Tlnl%FTFeVnxye*&#d^oF>+i%{Qb`SxEK9f^;A2x3D zY`!IFKhl#FzIT$>eI1pJ;J^qk+#bpsmlc66QtR2>l8)Abj}RE#EaJc5^Jf6EwOB4T zS_kvr(~}`T`SJ;8KhC{s@^mH%o{DN5O7R(VlsP6%AwHSDTeh*TW(buCuLJxNT$TtM zi!n9~M0O-FtsmmO~LTF z_TuyCC0i}xX^hzy)_vvkF%?O?xtNfy05x|P{#drlKfm$*=AL@sYv|kfZ7Y@fw{HIR z+y3e4|B+~_S*ZW9105tpDFagK&b4VM3ucy*!Z{(;7@iK6{xUf&eKbu1?AA|_qKcQC zYd!l)d}Ox~m_?Se;Kcth=eJ3Gaknz`h6GyfhQR&wvE%mBk59Iz&qvwozHs&gf}nFp zc)K9|kQk)RZM2TJn|wf2<2I^oA0YAIokD*R>-+eN7BG{sH_djL_;I%;L^3p(8h$hA zvlAo;iyg64Q6YRm!Cvbd5JDX*D3~4`kC-+|_eDM@Ss5}5=R6=_bsxw6!v`c?%=5U{ z_7^NPGEL})AucT=QZqoD4OVrf&jq4Mb<{vBn5LLsSGDFdKBR3)wSIAP2o2WTIND2G z_06rcaL9fYOqQH2hMF4cjgFZ1NGYr{_@AG^rFZaEC3nykSnC^w;dWx+GnTR~IC2U} zI11?8RWiU*?rPqZr=q;E3tfEkDQ{uoLE>aBC1bNGXY8y~0%75(D=UIPQPB@@kY-!9 z89d`CWNUJN(Ofbs7!XU7XTsy_aDyHw8s94?g8 zsv^4DY=DZxY_y-^Zl(H5gGB^vdrMJzLz^!CW1jt;=)(O7C@umU|>&6&V+FZdI?s($HZwvq&3? zNLw*fF@_s*39rMAI~@~iWn}SQ@GT;VUz-9~bCewAs;=&Z7zVWfPR>j)badjRYIm%A zzP}0;4FjnY+o>XilHv!4-E}9!lOL_<7EA%XB*#6>QCp-=?LlnxDuR{Fm;AbV6SZrP zg_I0R#1XRska>h9FB?{)jsd3L8rDNfm}XdEyejys8@yr&>u?>4oy28FESRq5#5T*) z?wq9P^@>xCxa}1i`Un#V)xY*!LOxXLJ4?0cdz-l(NY@uJ{iJez4g!5sIz~N~o;LH! z_(j2ggBBth)Qn>?Arb}E9&dH?6Ynq?^5YRYsqQ__JEso3*zG5tmCc@kXxF)kraG|) z?W+|*AGcOJw;3ItkoF^)vW1eLU*Rt@mU`xA75O*h`qOt(|B8VF7{>*}ul>xV)e>ezF5&SOc(A;<&Z*RRr~CI$aNcwG3Aj zZSAtK=MZyy%6D4c!?^CYc~P1&oeVn|nt^KE{O<=!$bKDr$RnMFo$d9Flufhj2tDxh zDww3z-0ozj=y8*;Nav3XAk+_ICHM7@t-d~gp8@<`LF2FRZ~PB(6VpHU%>P@UNL2nq zIR*0tlw_+3>u;zEt$5zuI4K81tO7<*l#!&SG?XE~FjyiyOgXWw%ly*YPRR@G%Ge_MkZy#mC(&$jTMvm0nN%hj+R5GNjL96C&BgV6ond=M!;YMR*IGrTUs zf=P6skUoHXPZe8LygRe+gDX0{E~RRI$QfwhDIa1fc+jZwq|^wBXa$xI=@JNyYp%$$ zNS`P;v%|;BO~egQHZ#cXpx!hspI8jPtj7*^OW-+5BjX=T!#ng<1|EDA!8C$PhPRc)GMa^`wR8qx3=`J3ngZ2{7(gz z|K@IGs%pQA-lM&|wPuYUk{pctqL^1KWJUq8asY9Mf^3Ou{q9?T>wuloIZ+|zqws=(&3 z)>}lWCgW|o)fi?c>ur{6+0BKH+ibFd1m%Wo4ZNHFrViV|c|P_A#GE%&9{eGrLElI> z-U(Z`JD+kU%^f>xvtG>AjGka1h%$db ztuWDCTe!5bine|VsYAvr|Jue&)Vc9h<=#Np`4a&nrL?V+UIkSxB^fJ|-~c~*B)8(4 z(vuNpKYuZGWvw$wr&ol*tmc8oeNdfVtDaJ8U-imJt@^tg?m^*A7}6VzMFodT@{kqz zMIQG?-b=mPMXqje0*y&VzkxFOTAX<(HCn8nTOJv&v+*zW=}N_>di<7VWlEseyfO9V ztaxTcme^@0HCYKG8)jv{zdo*BRDf9~BbQiy;f(<(M->>>P_BzLL|FxLHhh8EGUC1^ zO*E>ebx67sF6z_2dZ(e1AjP2!9VtZ3*ptwOTpL6Olt}pQYRjvFVOb+>J9r7J&WGHi zb*b_Z7*2j9-a6~T*(R~z+oCciP%Ep6GjD<`9fum{&GD1Z4L|6+`=|Pr@U|kJRLj5^ z1EE5$JYv#I;nd?3oSq9a{Ju$?MtL4MpD)O4?J~iq1vt)zw^^1fJ>3vd{Ad2mqeKrR z%n5J#nQ(u+{bjuGKalB|VclCo>o!AuB`0JY>ukspPgR6yu~zX?@6=GTWlII7L}jb# zh}2A59WYlW$cYA7jZftVXO)6~+=Pf$p*_ICT+jT%X)>32j+$Q?8o+fJiFW8_NISK| zy@xtuu0Ll-5oz0v@%m(I zE}Quj`*Gkw%blX>Xy$li+h!S5^2B>sEf2(h1P93_OU^&4-3q_W4SD}( zwk&62@-Jilw*Zl;G$M!p+Mj3H;4nql;3f~+Loot?L|Z7rg@oh=;e=YE-O@}ZJ^S${ z@b2qKFBBXH;db(JueE*z@IW)>9Uhh(hCEEHzc#Cm4ch~nu*$_WqN)J~E_m@N6^4*= zw0WsO8ImZ)9&a#LN<>W%4$Jp*=`U*wyd5e&6l^BS9(Zjv2WI|b{oH)x);>7gp2tO3 zB`Tj1bz<0`{be$!79DseBrOjq)|>;c9M|ETD&v^1rSMTzSLx*=Po(4pl5d7aL=NNT zjN`k8LOsCR5qT*u zgS3)+X;fT#rESEekA?2lr!m0F$$gd90>J+DXLiW2Hp=Gv>aj0>n;rh$!tehA0Tdif z>;RGU&VN6A;Q)z|QYp`&=Iuc>B% zucNPGsV}W$Lx-6rg$6!ZAdjsWjRMkjjppFsJB2ss--Anxn2q&eA26VC{bK_CQ6wE9 zJ%)u8#ovupQuH=w`kRnxK;XpeK++|{&?Z2IpTAj@L0?sXQYP1!6ow}CVWik3y9X%xO6~`c(3L#Ht;-JzS%0@es<^nIUO14 zL&J?4GEB7C^sbGS%Cyg@0(>bPvQ^i?f<^vAxPh5cA?GK?2U2gc7CfR01^gi|UTLEt zG9LKILv5S8q?$v8AkMR^_uQIu(KD6Ouq%W9Q6bakHjBwp@T=G3=cv(oIU(A58RTat@!#K1@H7kwX z3oI2j&x~OWp{^j-nsguyte)sp5`|jktkQ=PI-8Fn-L#p^+&d@XM4M4~#`dGkevUxU z!JQj5_mK%MUc8b?e}n{C(i2lFviyX?<(UgGZBLNunNomM=Kx=mc%md?3LbX;X~+dc zD!02vV$nmv0)ecYs&R=!Cb0VMtIV1erb7N%k5)_d1rW#TD0^dWxNk|qU5{nL9^9hk zQR<5<=*Qyu=`H}kLwsr4G{hKZ{$Y%8B{&+3K2}Cusz4S%Lx#-my^HgQht66L?Y(=w zWN&}Vp87r7a{NnXlIay}{X=n5)WqJz*4V_>=-*2|W2z&6X=)&_AqRdN6cJJ+>PCal z3k`)<3^M=VpQPX7$d;eZ{hc{JO+woDRc9VRjFimy1liet>58jMV%{}8R^cVT&c<`= zIzKeAzTN@V9N895gXN~muU1pM6&6kbgNnI>IZEeVtpA}akFvNq=R=olWNjJiwYj-) z6Ug*>$X*uYi^A)fp0n8+g)Hv!@Dduu$NXCMlN=C0K1_+-d=GLNH53{kpmd?%#uf?T zY5Y0PxSV9=C1%BjNKlnxWG?u5d(ff6<*Vodfq}0={c+qUcRKo_HY=Pd#>_V3mZsg#OZwA5FNJ@5n~!R@9CW%HVOc{}}L z14tD@K^@w!%6s(E-P_%C$F}5?G~>WPVwPwWzqOVE+n#dGPH4Z>Uth0VpToH66no`s z9m^c+`#lDna$!KTaT72ierwxOzhbMsHQlH3SCB-G(ze6)F2zEL_To z+qOZmalby~RSH3Ht(1BaacR+`jF7G@>VvW(+6-@E?MeSuAE#!CbR!ZoyX1DcDY)Df zyv40b97CB5+Xi;9@Fsl(o}^~Y5^1r*9!dLk?ILQiWlvs0O|XhW83- zbC^ek{h7Ib$&d{JzlJREZ*?l(|JKL;^XmWZ-WRAGDgD7#P0|)l*D85O8ool~4}cbf z7Hz?V@mI8eI?5kY`YsipMoIVx`@lU8|Muk^;%N_wC+j3wje7iZd}`g^LjWtc?an(YZu2j!cl*%nDIkND#%yPCQoz zTh;V28o`-yjRQ!eOgd&Wq(zl?_uB*OtYD4}d~G6*T7C?P@wcqqL-<4!UtRsZ`gtcD z2Dao{o7Hp1?`H>WFT9s9cwU#$>IhQ&I#{N*1G>@s76!y+MC$DpH&SfE`=#cXj5TnJ zIBRoza}5?Y-}cKa_R>GgyCS1jDk*^0!gZHru$#B#huVO`kG2~H_n;UE%R-ZiyV-V? z4Gi#X5C1AxH(%CWgjFp)=qK{6y11%fqrk{VCnkgcVKc%}a>Uc$xwTHf_`9(qC zg&~g3Q9hX4C$i%?*H4#yVd4Hz_F*3O4U?)Uri zOgGzuI0&d8iJxO^XIuB-uJg1`*gorbX)&nlv}N4#Jhs&>)^Q8^`TT;=M?W*>Pp)Ak zqcGC$#|L_`PP6{>S-9LxMCE&b{BXV$Ehicz8a#SG6>K1g35Ew^u~HhZMBB>1r(&{F z7;6oJV-^|ja%eTW1W0bN`1|4l%rw-hL2!tGNla__MNye9Bod~YFYjp7>{zNc-b4tW z!ne?M<%!2t+Dh#!la&~?ms@$P4qI59rl>J8lBo1h*#2lp(A)oHi=!rkt4GQ+TRe=P`=F3wY~&!@9&B*v#_>Q(7Pyjs<`Ut%;@FY_WA z;I_({5+pQSzaV!&-jm@Y5Kpxwzf3_j!YOO;Fr(w<&f%9gO5gF$kb=6^pxj}2EP;FO zChk*HMV=uP656tyP5`MY@TUZw-I4exH4iVB%}3!|3c9rHgM>IE@3GAvv01y$H>yy;d2cv=1X z6-A>Lba>u~yf^=OxHecaV+F7@2-UT}!sD5DWd<){WVZh(Ez?bFF}O-B$_LsfBKYto zuf8roVlIfOo{9F_yx7O~g&@aVvM|UW33|ew4DT|oy|E<~l9=HF~Z2pApzY8pLaU;@DEV#k)mdhnZ z4O`s>t#O_uQ<(}bMp!|IxmdRcJ7!>+coHZ7_tEkj!q7PHKciBZN>>CFRLZ`)cXE91 zD17;H@c_GlR>r~3Pw%C1p(tDsg;$C^F?Jd_2ooLj)%bRn;{(*~k&l_~Jpk%dWJ#@erRlV_EWh&)}!`)WLxy(jI!L_dQ zYyZ!m+Dz1fkpjLskzpu%%g)1bWgm`giXZ+y2K?ZgBKR^JxWYLw!1YrF$AiCp18=NjNU`;b~#AA?ERL^9O!x3;-1B zHxUxsB~z#{2<_I^j`!;qu}}9mFA!fsPT(uh(n%T(%MIGY;9dL`seBZI<62mid9LNs zE#kDe5-kgBz2aKkV^dwo3CGH>F1tN^idfg1CMcUM9qVwZK4RGHCXUna;38b}{W4S* zoDqL?^52-2ll&sUO#dT8MZtuXdhjkDfMs5=GVxb_fL|X z&HdmvHrQtu`9nS^7k`dFA2PU_BE6pc`s?%ex$N&d1db;EIC-Uiy!`udtJ!ETnPPV2 z!IKo)sjQe=WmW~h>2EAsuD3F)v1Oi;)0#hAgY@W~jj7)8AAA0Ma(j%_>a&2l5WO^a z1#O=&ZMEClACm5Lnvwj-Lq-1dJdy1Op#htG(E#3Kl>y51=OAl~%Yw*y5*3p#@{pMz z$pRdq=isAAt4ZB~T{Qtn8i4BD4bu03^P11{m^$4IFv+o)F@P@!l1PK_gS4R^aO}0a zR9CR?7(>xPZJ>J>K>Yhw?7n!Y?nm z!$wv04w+E#X4-&*z$!02uVLJX{mg)nH>C2Rd)L_Vs9=iSd1Q`@A-zH>WnZarQ{@J5 zOJ5{?oyfSlkyL>b$?M=2>eOCpI~zHSK2z}{OxHo6a9*d*fNj;-x3BJS<>qx5P&-6C z_?KIp$zk8OB@@|^CLJofN*K#qir)Bp|D~;=ZtXj+#o4SNZK%7h*v_}%1WM0cd`aiH zpNwS-&s@QoyhVt-2L{=C(Y>e(an#!9nf>kV=i*@BPenRIG&%feV=yr zboM6C!q9bI=S3po$3SMG7csRW6BV(wwA)7}>kTv;VO`PWV5CAae_n*Dq5$bvSk7|X zcv-gdvr4X8Y=jAKpxjxhj$tpQ7Lyd0OD1qevpY$T*MU~0%x&_z{D_z$cxki*QPJ^) z|C8%5Pm?XnVIKFc+4UW1*0dR%OZjsb?f$!V>UaRE}mGnE>os+wv>$>1~0b*eL{}MjPZd&3e5|$Hpwm!sYLmj?uy2ZWpXj93+I-qx z0!QOUU*QV`54oaf_=~!^e=!qZF1F+>wU0zYO2&A`mgg^?!gn)I6y9P#;a&CqtT0aX zq)@EC){%{WD`d_8H>sM%?7#J}5Y4 zxEV)h7HC{gTu)#C@-tDheRNLx=6jKBIKY53ZgCLCCu9NScH|GB_aI*&DhuN_3#AEv zS16bnWX#_O1w=1o!%bv~U;#R{u|yjHk=kkouYs^8*80NV(DyYli;=GB&q_i{$f$?-B=kVjAG62GfUF8f|bB@OMbC%GqQL#z*5F|Z$bNL zNlCfTT$CwDc9>`)70&_ZVo{wEh0_k#(jZxGBU~I^4uBI6>8lk1NB%ChHTQJ6tE|Oc zTE3Xl=JFn)KBiebaeuirsVy!okSqG_;6!_wsoPbiM+BU^)G(^_mxs$_QfA+tDar=x z?n%^9a1N$?;^~YaEH)(UuiNFX_kz;iZtSRRPHQF3#f6Sq)?eEKZEL<{$7dmORX9^1 z6otBE9DJ_3C0ijLL~|D%oPe5?F<{7_+-+E0k3IczBYV!5VYlg=0o~aJJZVr^z6H8L z)3X!j`Ma;uw2?O;WOQr#ZgCqhFliLO?}u~{yiog*IFLd<4$)*o^iEg4t>;H%uMc*b zq@`KO7I$`|f+GVKQvGzqEC}(i3@@`@1U2;2gB5LNy4F=9ibFfLR%2mvG4Xk$yU$l^ zt>VSB+OYM2wnVOW9Mk9Ep*f9n>rV-jQavyaBqILVFU^EK`#tcK%+gG2xN1WHje(d? zN?!%Mxy&AeN^gT@D%k{}i91sZ+89=7e@4Y{wU-44jx}j_t$jd3={vpJA!=dtFsiu) zcrfYgt*RUol4TS&Am^9J z?>^ZO#CSZds`sqOc$ke>$%@!mW3>G0w4I@{T)>hybr`bObhwS|ENgLSLwU6EUu zTiLC8A=*U7`eI&_z;Cr3#)uEaoCiay+Srn@jqdS=% zURw+IZ>@GLc{!nNAi&u$GOP&93|^_m)P{j-5B3iZe$!P>Z1!V)aiM&e+=D3(fB)S!Z4 zrt@{BjlNgT4Su!I=2xLg6?*O^OVp?lC^BAQ=Lvh6YzZHB)ZD<_EVoe|E!~qGk340- zN9oCBQ-zg+@BP?oF`arb7|*dJ8fl|+tA=u$!l5k5H4x&kG#Z}{UGc#bQh**>4+kPR zUrcjvddhaFM0&&;*5Z8eP%dzM7G-h8T$S6#7ju#HsqlWXZf|t*WVE)K0jQRst33mV}vjMl?`^AiT z&^yf*)5Q0uJh`7Re`~r=lrf12OaBwMnVa6-u?UzM4n}#Oa)jJ;KZt1T+q{Ps%sl0` zc88Pk;_i_dO}y6~zkU&oV~|?p0@NZDSk(TO`d5%&qmUmeqVWx3C7haBAg)8yg_HJY zlU(0Gd8kA37X1@6xQz)2M#QJ6UWjRQehIg)_q3@WSKz}EEQ{92RxYYKkCXGU4!YT{ z71r0ySAUJYuUSVNe}q4|b!c6kg6dv=(P8+jcm+ux9Ypim(6IR}T==`<)qevQ9Gxvp zU*W=EFdI(`_ysNoX(Jfm_GrxRqlq?m)ck|N5_S; zT9sAFFt(gF)hLZNLzYH=73JN|IQFZ8Yom{jfFtsqx+JdesW`h|-;a%r_9{x$om}c^ zGnsVCEQ()gerZFnt{@^)!0QdWIT#NjSAZXEz>#}5k)H=+3I3`GNmO=g*$twt8!*Pi z)8pyTak=q%uFt5v0&(upntmrMs&~|>y=1EnnM(1loPWr}mt|W|D~UL9#f9AC134Gh z+hbaA7%F>xpSziC=fS3$JXjNCT8?h~ky;u6n`? zw&2fM5ze>^UiQ}<5aYKQxaj{lWQf?=np&8-I2!!1xQ<-S!pXwW!rH?5U)z}e;*Aw1 z*lH$e=cp=RwwN;w1}KP;h`vii<@YHH@J zJdiW(u{w>rb?`Wip9u>3LZXE$nqu-RFc*DU@PBKZ-%QyT6V5$N zyTtN^)c*rkcs-_0@)k&%!~(?#H)$&hL{N)&sU)-22zqQTFu=ZLAKj+x5#m%|<2VG_ z;zt$OL3h}f!+;vCZ6ioB1)OekYe&ZPNp8~&SiN}mSVyIs>w0`3m#u-P4@tJ}p!=kw z+>WaB%88paZ4{eI_4}*?BqnQoIzQ&_IFp+YMU>GiZv!0Wl8+)yhbL^CKgs6PLZJ%omw=> zYQR|(u4~yBnZZ_so~veMioi}(6tC!u{=vsVTjhbL7V!URm{6~?!%&RUJ z@x!qM^wG(|rWx#p*KMf0nt_goQUg2r@lndXnBw1<<<{^ofd_`+yQouF2*WtVa3vZu z2mIW^-jDFZu2AhIjUv}ms_P_5D9+EhqM)zjY>HF#AND52g4w9`Lx0jy-kvtAWVPX1 zaI6ctb(|9}?V0*=p~IqsjiQ*t_v7(BGPq>4l&mNG0$nF%q(nomBn{aSV{`i8L2KSY zQ70ZKJobw}+ov=N8h3`T_c`#l?Ngrr-JSl+-HfRp zIbqCW{i>kwGV$nY1i+2xO9Y|qBx0hAQ0s5$TT9L^fb`)Y^JTwJFX`5nLp8W~MlZkl~y0-DmePdC1A6$sE77lYIP22NWo`2s;Szoow-% z;JICKnz+x*^YXhtJ;Hfo-CG7B1FH|C4|lHcMh|gggq0}vA^@EPq+CVs;7Nu-n}A~gImO^33=q?pI`pVI4}z{nIc5!(J*N|5 z%y0?(6$WT4gmelp7V5%0#V}Thsefk_0H))P$_T&+@PAX6?Zd(^-FD8B?!nKfG2mgeCyHwUnO1Z8-zNgYiV|x{lhO@{s&m14Pgz52hurBi= zpR=HV>N=lAAYNuUDl4O&jYnDD^xArfrBG*C>gVG36qYO}_W9}xd{kUbqgiGYH;{Pi zOC35{Ji&Fx&!ZU~Q|%I&roBUmXriP^zzM(7*mgxgC-QA;@DL!-OCtKvqDk92F`RML zT$7}D#b^o}^+pHlWlr3qq>mPzPiHk7uJo?1tvLy@c!E<_oP#*rzP?p7$lInK()D${ z_ZHtTvAthZU(T=&wwy7hOvJo+v|aQL)b8h=lYwi8H&bM)!-eEHP~zWBTJuzIO3%sp zfuD$GU%2jXGKju^I&+SrB`Fp!A~4c&x{*V;$=DH>B^x_yB?Or?3jm&cT*Z^oXzRS| zsws0EzExl6tn3t@g3a>^70SLhx$64JuimZkmA2XW9SWTd+Gd%oze+(W`S71qXF5$k z5Q#Lm`WOpGgYjotu}Wlo$&HRQtpf3DqP5Z8x(CIl3o0v2jx`47Rcxl;GsJHu<+m5o zkbH}7KMgvC*HzmH=GSuK=oy~Q1Ha$hx9HD~z{57`5-6@D9m{GgcPmkf#jKT2}uRp!-t7k>S&~hb>X6hS^;!aEC&@mP5M(GZ+B_1iY-_kmqV1JYdl+>h$=)saD-8~GtNZM^MK}e^=_Usx}53SBRo*$^s z&=j(A=%kd$W^JhVon&US7(?g&X%-m*eOYUQoA_yYglpD2I@oa~qhvTgD6gH5wxT!~^NG3j~0tRRWCnV{~Sx^ELRTm6j`C?g7P;pdJ#8K3RbK<37Qej;TJ^VcmcD3~OSJ`K z?J4?EtMElP*abaD%r!hvS7o!ie#4 zuex%7MjG`+ki^_?xch~t#k&E!Toe%>_Oi^joPUVPJ+7*U+Pw4Q(|5qKUEqR3;+DUx+;zX(0!&|(54DI&(_*DXlO^jpmdDH6x@r||dE zFp-uSbD21OA{-w@2|W~@a^GowGo8|S>b;AwB(3?ZO;JsXU0ncc7m$LAKm!{1UtXowD7s92H0sda4!IR0mm6FSnKkHFmUm9`A0Uetmy*q5}DPhBOyQd<|$f7jXgI|j-BooD5EmqR0U^;@ojkEBOIW0YHS^!%Sg0P)hj7O+IR zDIV>_@^9F#p{^l=f~LrMtr=?*NMlQ|`OaiYaT7Jnb?l6`0RpzNe>%x7v+vHXh9bB)Xaqtx;iJV}8T$}>UezYsdVsT*S!jql&> z1a^MY%d?%P6@7qaN*D0D?-P&-jZmSkqgA03IWXZlxBJCWywg4!(RFt^}r+zq=JKWit{uo;a4>lL)5|c6xq8xn=RX#uhRSMZ3sgxWQk|4H(+6X>E zCEnFdb;*LJ@O~=06wT&hV8dI_eXB4qIv#3D42)L*R?dp`bzJ zx7lS|Sx{iZYhtjq1ocI{f@iO^1+Xo>qJ`Cj;gD!XS5u|7coGv*dVQWLS^HB%fh7g# zg@8+Wg|T*rRI1EF?v&CodlG#)N=y2Ri&7-8SPn|$=$DUUDs2Y4PlG8q=lR$hy`oI9 zl5Gqx@mKg6UwA^*tlX`l{czfQ2@xPRqbmw>-QF8}P%t7>Sd+7ST!{pV2nGFFR;gy2 zXw}*p+MAm}-^F|n$?O2(OYF?$^!)RA`u|8ikePn38620-|B{rSZM`aUQxTH~|WHEB3{FLNZfO&i7N zl&Mj(L0puZ2i92);~c9B^bNPoV_g$=|_rkmhl_h!0Q^Kf%1fJizex%wRm!)91eQKWeF16&hxR<5VXwKjq|t8n zQMVu~c@3H21z`zTc0xDbk?X{>__F}rSSYr$$PAllwjoGaw?c0< zMzTP3U<7V*Y#$1&9(9K4K5{Ty`e9V5d8w{Sl4Cl*T_3LJORCLBbLrtp_F5znD)pv; z#Kz_lEs#_cXS7i;AGcI3&{R>eY3>PDy?4Aaj!UM}8VIAkE$yFtS-4Q>U7FJg5KI47 zJbIo+9hNZ})PV1l+F`tp7(6ep@To>72gfMt`5KD$qA0J&3b}{ln97~0blFyex?UHm z2^85SNak$N2Ru|3_h-BISd05!B32;%*q>nyN&-)Md|w-Oa8ez!SgCObGLL3<+D%{9 zU27S-WZ^Oj%9~EpfWCzrqTJN(3g(Irz!!B<8+`P`5hz5&N*mL{7@YL=dp5EwE+*RQS_x}1;OGOSh@>HaMhkS6(hn)NLZwGY`Rh;4hGKs8P~EGQ zME%xE{%*$6w(4VK!SKzmwMNk<$&?1M z7o+$2@_XJ@UIj;;zwWo?-zX1CoXbI%p<9ce`-2}@lT6yEjAZ*;!I5mC?`Sr{KdZy! z=fZFHQFn6_2h+a`K!RjMbf*btYbM_?fiQw(L~^GHr|!qp8OZ-k12F?>3NuR69I1_n z&0MLEj)9oeJ0}34*;eo_1PFtA4hpz+Q0|tr^uAYaFSRPPJuyF8VJXk_KoAMBjc~AW zvPsps5J?_EQP!U2A={(pu`!dF8S&i2CsZ|peCLZTnP5UT*J+a@;(cQ2rw^Tu1 z9>$j9ypquz|9Xf9qqL36zY@osc37#!zbZ-CN#LiR8EF?*Ehs=|t=_V_OSrn77&kL= zMU__J)UCA)bvIEjrbfdpp1LeYCqPm0e3nmzpQ4e$Bu8@_Ao;SN5a(vA^s6JMyF*~E zn^ziPot-{#Jr1qS2nXonguN)T%5lH>l8OmH2pj?RsjWuB3Pjxp8cYHp{bpNQr3!iu+T4}ole(&bB= zY-$4(Mtv0pP)0X-SQMLl74<4U)&qa0`$4=wmzq*&q*~a+8~WY>a#}T;n_l2!NRHV7 zn4C{??-;RCxErPQV{B>k@gxMP2M1yDK2EvO9nPgx3KeoZ3}fu!YLc`~=@v@j11?i9 z)P)f0DpvPV^Y>|vQUZmU)G$$I^tgOpabFvYT-{mzhnn33)AZJ|e5827)5(%9A8MP6 zH;_TZad%|B7d1E=4W^%+*)H_ygh*G&oMHo*<+TCQ2%@lps{u4m|HIQZRd9~b18M2~<4i0*q{kfWt3@w25TF^h{~ zHYNMg;wrge5QWx;tQKNKYrVtU>}`Gmu>Xv3Yl;vS+d={4PkXf|5zC}layo3X+!xS- z5&xcEK2w;iUx8=c#QmZk`Xi%keOJe%aY*RuS&^^otd~5}Q(N{kN)1j{KhaHHC)rOz zW6uQiE9FkA!{z$>))FY+LsZc*;iC_#zCRWH$h~L0YIJ}7S+9Ff?P+xI+P0+q?QUZE z|8d;?he!F3d53?}!Tv)s;2#lKZCM%L1oM{-V1a_Zi2+BMQK=6A(2rP~*r3H!&{*~U0rk27m3Jn;FG|j53b_6hLNJjWq^%>7@qWkcwKjc~ z`8YmA?Du?)^x4Us+(5WHKL{W2KB1a^lMLYqz)3+)v65I#x9K0663T$yBxW74fbJmC zYexDW=OQ@Ni?jptq8iy{f7d=%L@L19*?(VzO(iU zcVdLF+&Y(e5CXn!Af#RH;3ghqr)HffPvk+3u15lF6XwwojUIxKGFE5rcScZYU|CUP z@;e(q_8>DeLr+%%`0LXh7PhhXSoHcvEBLXQ)F@ex|637jM_>6Hc(ee>wyr1(h=#7H zR1}+!${1-ig^UYHw&f^@#cMeDqiV^OV==|1dOHiA!kKeQzf| z0kJJQeGND48*{Mo#PYt*qQGQPRu({sy%5iCT3RB#xlA0*OvDKR>hv1 z;$FygL4yUQQ1(tL0dqz#zW7;4adKe3QKVBpe4(7xDTbew*c3KB!k19>!|Z!vGD~~L zfrXgKQ4{%uOEfRd&mO&o(VnK7;+4DV3dXBT7G(k5ipIAUkw^l2nAEd6g2JE0my;*f ztXk*GOsq*08S65N&svqTP}+<8 zJc4qB5vBG&i)f28x{Pee8%&_Vh16xvJv_1O*))}Ha_z5LhAfX?&)hs!hZq?4>R!0m z9eD}rmm;iHG*wupEKc(MmO+qhtX)#kiVwx$R|l}<3Q{#*Aa%C7top>8@{x`27X;%#laSDmj3LsPLyh?c&DTGg3gJ& zGX)Pf=GF@SsZIN>8nZ}A4#>*f_bE|r;R?v#@hy&LwyKqQHBQx|_>K(Ov3yfzD;ESY zxLc(w05z>;9GI07QtX!%>v~nVR7<)w6|qAVcttiU=6@>qQ^3=pJj32w!WlsY3vH0M z2s_2ZZ(V|OH#}E{RAuWc#4DbVgJiJys=I~QV~_DpS6kgqlce#8!lU_Th z!o<;>m=3uPF9b=Xd|`Qa+SwG=00&IyI&thNKTe`g#D9cB03B=ZdtXk-{=cokko@1D zuK)d1?c%XCQngHCn@P%}7tHJpq!w1g!<1t;%7o|etS9J&B9%c~^ZyOmU`DX@H=p$WhNB5sFcs zQiG*pzS4tf;=ba8n&Q6lgDc{`qJx~`zOsWY;=YoDWTf46Kp#?nN?^RTQSiuoFJwHM zmH7=uuq8I%#I6^VAnS8RUk}@JXrCeWJ92Ojs*n+JdLJBehB5X9`+%co97qW>bGk9a zzQoNL3|H?iF{m8-fRq6?EOOQ!hG#!Qp%;$^WPG|Yetg5O57ei5Sfpb_U*42aI2N|A zH59b34HT8H9i{#w%j(RoG86#<4^HRct~!(hvuAQ2FG3vib3&gX4n9SowMF{yx?=bM zf`Qo$WAGEU@AR%M+hpJ;hbU6#%q}n6q{tjXC6;y!bO#sAfHd2OM_E-{I23hhcYX(E|kY&TO93T&| zmc=utPZwd^!ZD;z5<$T91~b?J=Y4P&8R0J(0p$QRf`D{*`3-BZCbnj`kl9w~&clj(=|@|ZqFM2 zNU9ZHQr43#F69P5xHZ1ZfzP;iNq>kmnQ)^(xHW|Dad$!p_0&-I<`@*xF-N)gQS}VY z=M8dB?wcU#qb9V_#pf*r`OObITtmTnH|C=)0Z_7dAunU9JZeg zRi%OQ=d8){;QT|bh2rm-{n`nEW}7ZOg52bY8W`xVzdRzyvA?qV4Gi6ldymPSlO$z> z_kYUtvtDAr%=ZVjhO#P}qSc5TG3126L!3A8$L5}Sb8l>PQgra3q%9uRv`yN$;3(tjnMl(OI>EuK~BC(r)JS%iqlG*a*<3t6QYCt<1i zkP-Yje;=HeDf%hHOk`~BuditKbR3Vhh=%j%xy6q=O&E24xJ(;c%r+eFgUU7Qv1bFD z1TQ;{DHME8r|_uV2fL#SOyj3~J^thoY)x5+0;ajINwkEi+2rO1==U=STk)HAuY={% zclmkK+p*C&#q_UPfIJayo&xWv>MNP4;and(iJXLp4y=3Mg6pNVuENEK<3&QRLQCm( zr~9DY78p>0nA~FpVh9BlH9MHPGB!7{oK?U1!k%jFSGrEWc-xeY-&KYWX%vwmW{lC| z!&rs;5gX4|Hq&Qr*o)qozI#^R2F3brIq-ec-@JRAnB4g!j_1yksya|bMtWrCRsrQt zJWm&xAsgP6?PvMPHz&9AwFX_8!M#sRs!D5DXQtn=cHBP$xESl}pFT4M#xh)->-8Qs zzL$Nbr~7YaQBP+$z1u-fdPt+~)bi;o@IiT(95e_B*vdPnq3W*7p1R3Cj?EV@^pPBx zpEiMvlwz)<%DGhf8T4@S5s|pnaP;!oFy7r8E{o)X8%yPnTqB3r=Vs#S%6o9#x~wZ< zmnEL!@=Dxh4C;^-VWu)vyXGk?t3}S%E}o}NWZ_mE7!{X}Q?XIr)^9SMl78I|e&)Ir z41pW}xeaf=^Sr`uuoqXhzQq}ODK3V536-7J$~4s0ZtBGE(?hj_)0h}C_$?%H2uy+_ zASE9`T)g}n?a2tk=jU58S0Ef>36`gpMeKMyW(SSmP zw}V^P6ne^wX1!?OrhLA(4}e^CIvYfyVj6%3#KCE0^@g!c}srl`--$nS|`@3j>puGFx#cR(7)(DlbEc zFq)If%>E+hyj2>8_#&^Saz-Qw^vx|p?RZf{DY3-#zK{iVSIv@!s^AlAgqx{Joy0oCaa%Tze&o$y|ifR|4#mg_LNYE%BaulLzYtrRHq1mzCnxar& zzH)+nfFl&twbDZSzB!p$uHI-llPeK!|M;BQYW1*#Ujb*S2?@4$ff>u5TZd5C2RQ&j zg7@0T{FjXv#LqUhHuUpe*6_!tV~uai?&{^1NrHL{_)Xj^&+(Cy>fCJ1dj@>1o`NXFiOM8`s+p6 zRGhZSdIQ3aZFA=AD1R8?Ruo!Qe|V5JiN^A-oY;kmf9nD7%gI`Y{HT@)XII~|4@J{y zjl~)i4deZ_mSlmcTJl@i8osJU5WEK9xRD2?SuQU zW?h|tHS|K(`c6HK*Eh9;7*7aMxiMc$;Qyp16Gt;(hKald*`gCmzD;Y9UP2g>_`QSm zECc|`V`R=-brMpkYLvPprAvvrl_YndNZQVxF`6ajs>(OVfqo@$R*zfgUj<3mCvyT$ zH(E+Z0Ul__Ui26pLN3eB6jJi{CnND?DXmGjczS3C|L19pX<9CdS61*~Rj>jbA|oCZ$US z8q~mRQY3UOHI?{mr`@So1Vxs7;&)A0(@LWVF*n1FlgQde14eTb3zUd8z`w zpx`$;_%-Erg`{!!>}yNZF%}vXL7$mrwF9WQ4^pw~ql=M6JLsnm?M3it(gBq7x=u|J zJmHgLtM%>_8np#MI5@8Tu~IB7o#_5dN_=MIaIa*r?NFuGK@y1{kV_M;@KuWBf0ISB zE79;Y(0MI5(!@pWN?^oy*F3m!zx0Tayn}k`( z7o&N#Y-MJWD7gI|fT|hGapX7TR-3CXQE1f&RPWfMubfL-hFyVeVwNM=A$F{fPO56i z#O_iL)UO+~nUs(IE_IT#=$l!6q|lmf&g>8@Ll|0O=wL3@F5#QY$ z3p_f(rTFCCgX7rpY*%mpg^U_?HoXOxH)2*L5;DCAd9K|vVSW8gLuAbdGk&patL=!< zQD+KJjySGk6!U}cKw-`XSKGgBN$xjoDX4mVl%GZcq1>XyA#JjnjpR`GmgF^sTa2`T zJ3n|FKuh?@gFJl)-S)5Y?NPjz8u#mQd(i%+vOhNc_?#+P*&k2Yo1=ZJyQMY>O)M#m z!b7arMJnj5aQ2*n@e8?iAs zi*==`UqV9s%3gbEEB-3mxg}au+TTWoB^$vC)7n}60!+ea$b~9XivTi-@fL`&rj%m* zjA$N1-ByV@xdN{g;q8Wgr1XgzK_i6%n)(g!p$ODGaz|8m-S8nD<_LU3o(20MZ@`cG zh?t4~28BknqP&@+S{Sid?2%@NMc0GK&IZfYp~{wL(2uw{;w;s{b;8o!a?tj6I8g}1 zRMw#@4q!PVb_?%&I1&66(+e1)edk?h zO5`I2_X7}MkBzlj0a>AVPRVufEig~J+Wp`8bUjgKP}gtt_Hg(Nxq@^%q{`+$Vi~38s64uI7N4&Jv_vpx#BUo);3ceJry^VQ`pRMDAtVUEOc8!8)gc!~ z$p?PJy$VRnf=OZAOny8; zjOIEy1$lH+QzfNRrG}Uc@yuVN1TyGOZ(7?rj7x8W1ORT%_X#JyA!1UdrRywr_3mw}{XH?8Nm zz!EY`P^nc=I3?VbY|vZcE-3k`ND$MI6Hm~cT3_;0c}*lLpEf!lx$=z>8ea&6uJjGF z^W2@*FIWLpA5n!BIYXJawy-Re@d?(_pQ)lbIx2Cr9<8Ka&(&mm)->5*l*Sk_7}#) zq0|bEWR|EKaP5PI50O@)HdAF|i!Cc1NbN_kv2fnCFs9s;-Qn-ITcw)}2JiRfBBu{z zBthN=!0n>vqjaN2jaK<*S8_WA{j9bKPhNL)fL^5dcGYhpg)S3eNjAN5czb@6f~JUb z`whsaPnd@^qXN{H00j(Y;LgBZ;;_!G<@~W55Jtu0xkLr$S@a1A5K&VXX|Dr&g{3o& zsDrZ$!fOawDq4PN)!P7~*e47hhP^9qLVjew+XpxD>}Ioeka4oFV3*SsysjQ=oMG4? z#w^NbeEr7}eR?S9{M}PWdnxBDlUjO!&5FPX#6KQqp5^^8fN*m}4r&Dd-4r-i=aKc} zSx{jOFF+KQaJ+{Wn|icPp)8aqKW@`NQD2@RbP|e37Ja>`(ahE%f+TxWpE*!aW^Z!{ zXbM61(S=g7#+jb@ToG{~{+Zq>42*#E>;K(kEB6$irRGo6@@I zEDP+l4Ce_1EUi>La>I^d8``ynMrzXnFHxXKww*cgo|UussU_e{F?xGu?Mws zB!QSeWV2QKOrIm^gF-=fzT}>Kk3AGV8V8Y&~z^Bov{ zAmH?;1a(&|>832SJ8h9Y4s!W+hiT0S?^)-W|55D#N#{)g1^@vu|6+>%>ws#umj?)Zf%@Ev0`l zz2~FKFmDg#C^OjDsIpgDO0hsD(q&9%`^th@p!bm|S*3n$iG+tvBj_YF&t`}dwKLpX zumt7OQ_c;|=4CPHMdJ1Op{Bf)&(TsJps5A)Klil211kAIhQv9Ul8_2OAgf@J54U@XDFJTjwe~h__PL}j>YP)=C>QCS915)oS z8yXgSk@p%|b{j4$24Q0Nr(Hrtnmjtv$_P^DKf8&h4){qzzCwjfU(t~NANBsv(bNB$ zIsI39=PQY;K@cr$HA%E|$Yx@G(D~K>+J$~r1{Pro(*Y+kaGFSL^Le8|?{!XQOc4CEe~+fKlCR1r{3~kdJiRR!hh-Az%)VnC{N<}v6@`^&1I+o8!Z$u zEVlsez)6om+K*920F~9!xJuuv%q|H{nH)Uc_}Rk?VI#om&TMLlmNox~w>Wc7!_*}< zxTDq{d--#vKpkda{j+R`%Nm`7!U$2i0YsEnn^HJYPKZlt4Zn+=rUVO+_VfJ2vxx13 zBEK1#8TJpxYn8SDPUjpbwwExlAx=%uC#X?LGOQnc9@!94CALAke{Og+ANe1)(kJX4 zRQRwWr~OM0E?0fx0b5Z#}4Gx$>c6S!2VyNn=CD=OxSQ`tsqCf_!hm z-#@^tdu7u1_zOie?Q!%5FnFF(eFHJ-eEzEVvm3Cr_sl`8ZP`QQ=$s4VT)VNryQC?b)h`4}R*l2)lAKa#kJ zBZHGXCD3MMfTc;#8RSUPz&i5W3|6i%Vc1|;1j=<|#&Lua z<}#DIz&>w|M3-_{q9o|XF76OF<6<)PQnTqChHK+ybMP{wZT?8lt-o+ag+u6R} zv0XVR%Qa(A1rx^t0)TNL0ibXSsdOI!iteHg;dp%yp>r6?a!(zh=C>$=LI17_RNCPH z*TqEdj1gV%YCR@bu06Ae!$Om?n@vdcED(GHL1S8ZrcJ{>A8+3R1*<<4wRbM~1 zU{Ja^DB6(I)8QGOtijd@HrQ2Wb+kkld(tn=c$^0V`1GdC(S7f}T$ub{pNHC#IYXEt z_!Pl{HGN5i;C-@K{#aAoH7Dq1^S@yQTmkXV9{F%tk^u`6s zf#sRv%jSEJ>$}GBo!VtW=uQ^@a^HuTeT4x7w*=ZTy)EV}Xo4x?ECMsT=Grls+zpPf zS@fIFirO&)Pv*)|9up>f-L%oLd`gsN+H!-uW=;il;|1A$($AdQa)2ayg`+vRtE_Q! z`TcGg%B*Q4sJ|irwp-%hV>1>kqQoISUr|8mE?ol`reTserT}sDi)R|W!oZ4|a;zeY zrw#z#sfAu)#K){2;%VH3Mu0Nfe3LJ@FC)eS5Ug|5klj{gRF~98yGa(^h;xmf`v>1w z4N&RSL*Kbd8qGsld5rmT6G)FSY}-p^Rv!la{&l~x>|_{dD{S{u9Q!&=oXlmr*AEp+ z`6}ADk{%e-i7J>jMER})$ZA^~G=h(IVqEbLd>RLaKiAPu4_KNx3H+V_f_3Txuo}+% zL&DALW7S)7pnpe;XzL9&N8{*4@tQkv3Q$M;gd_}g#dr`EZpMe3)h8>v16vwAK?IYA zf=Lwt@eu+Io(MgOL;Ia+iol!(Pf?6fp#1l43TN|Mj_VPo8*%N3k8eDwi_^+f^00b@Yh_YC^(J@XGp!Jk8hx0?U3PE+asPJ@5;zM0)B zvTEEiL46J@qJs3DWow*@je2;Ku>=IS2mo@zcAb}L9=1KzoQ9WQ3#VTl%ldjK32%qr z_YxMcGz#)rSV|k#h++w%XPI2GPvNr#ij`qsQk1$3_mCq_#LL zza7O6;h64XTZ)q7lXs@0MbbW;sloigwruUgL3jg#4wah>^+S0L6Jx~U99KngORJf? z#2i0-;R|54RQP_&gx;Y-VZ~P)9B-X4JcP?uip1WoCTWS?LX{FDNX0;O=Cc~=JhFK?OC?a`olaImATMwU!bk@qPUpwJ<@ zE%$JpCLO;>dtP-^tuD$x4N|gZxVx}O1gnK+Xl47e zRXs@jq{ag_sgv7ezQt18rc4!=59p|6SHpKlNMt=^l~1`qh{?w{0ud2&W?Hf@IXil? z(99w#9I{RoyxT+5F_xCNy&u`NogvKRuiIqG94Aa;8?p>dngnxrdm^&M z&zEgxKj$xnQ3#{Qmydb2X3 zmN%BFmdRE9wyBvez0%BHwD5F;%Lj+0UikdJ9kS5+9zd~+nmRSV=%$$2^b};k$Lvag zk**+8Fm=wN{`*dAI)=2%ECnF(z-QN5bev51U?UIaY!DzDFMjHUmZ#-a3rvG@;6q_^ z%4TbT;3NcX2TwZ6%K^@dYN%KOCQmE$H~)&m!$Z|cbM-v z1dlW@13dKwPJDu3(~>8;m&i2mz&;*|hDBX0;sTLI2zMn*ZS9_mk#qmGv(u~xEB@}| zY1833K?M))%C;=`rxVQ8g~qhCKil2a?mY1?9adKb=8YAV7G$ewW%~GZ3x8eQ?OqL7 z^06Gy54-$`a@K+nyIOD2o)Bx9MA?oL2 zjq=&#@WdT0hyxR5F!kG9i^@{-Ie1FEHFYk6X2W?@bwsc;=BDGiP~c$5f6lOD9? zI}s$J8(N5g-VRb&uq5I4x;?p8ODY4C<;WqT-zb;DgJUZwu+043$wQYcE5h1eS4PNC zDKK=;kZIZf3iT@bB`LZj6ghO2Oev)=lLjnFl?ELYC^-Mj%sjZS&4d~7)3@TTXjt)H z>Y6ydE*YaZqQwJlgOH5L*pQW-DzW}1$=rm-olsTjU=Ar7Z$g?fQO{?v#n90=3^LPT zQ+*XQI6ocdf~iy*$jl-=j4+wcVVp}E-a07 zE4)?Qc-1>2*0(Y_$rUGC#FNd!%}@`hrn3KRJ~Vqo1=d|myNV{F4`f#3i%xkZIyy?L zek*}kNHS-Y!R2AcR|6zwp2R3eV=<>sT#yki-BgOqrE?DE_Lcv>zkZM@;Xdo=i>`W< z;Kse>e8$Wh7V$vm88~z7>Yl42;(uQRXcsyrpt0}s2@%A3`E93j9)4Rg7zkC6%=ONY zQ z#7QFWvYyM`Lp82Qx)(w7I}x5P@Q;#`%;WfJX)f9ocH0cwKh z@SL=X5r&wPwVyXZneV}8tiST0-wCzc8vXDyZna^xwwm@NEcGx#o_{fKfBI6h- zWBLeau;~7?8N+#$SDfU?7)A#7LVMQF?VpjA-$}i|o9QIR>DBG|>>EJK1N|(>f%fw? zNB&_3^Or)6gMs*l7CJ(33U1jlx{Bi^=*b^=Sbw-q*F>$lfU*TyarAp{)x>Q3#Nd;o zoUH4#!iEFkSqMisJC5=a_Xp`0eqHlL4awj*{!;)cp5|`hLUrjr2-B`%Cn=>a7waZN z=81Sjq}M`alf(~Qsw{xRADmJ6Vm@tSskNdFpK)OzdY(HHrc(Cy zF_J5P9J32YBhZvOebetqWtcmam5J=3BzdAo-pD0Jhu1po6N|J4iN7c`YEay$_MUQ0 z+juug+>~Gy^G1BbGmXWr19mFGJORC>7L9Xe1~MMLi|JDmQ;ph)EFa>enYr%|vAAr_ z{nVDGVNxXl-x7_ADfE>ai)7<0igYF!cvYj^Q{fRK-4ibl#8s`M*w1$QUu>HaYn;}X z%eI+@`fWyKR_mD|Mb^>F7yt?o(|Sw$z)mfo-YjYIgBrb?S3|rrI=`!om+{}rHr&5l z51K`GDR8BNQUIAgD7VTU69+qI%ItWyGH!xW&I6aEhNVC`c$BuEsF2Plmc@_>zk>Jk zAM$S~O-sa#O|B#yh-<6{mxJE+wrSV<}m$Gp926+H5%TSSpfaszWkL}}XRjX6>XQZO;UC~g#f5fI1F)jOSK|7GNmy*Gh-atomDG&0?B$-w zMNJPO3C?83Fj|L%Uq=`h4AM#rQ(1m5iZLA?>qui)0QrIV`HMk{4}n7mZ8vX|;bvhy z)4k|&6O_9~{|sznkJtI<<&6X2yk;Geu;P;-pCGao$An_@uVn)PP5x0pf{jznN3draT?nn;p#}3ZIO>;I=ZxL;I_O*->b=1NKQqO zPY;8uZ-jU|8Fwe`xr-|F2S4b5xu!HT4C8JGC{M;>mu?3bhd~P)=7}oDT7xH~?;0uEYp57vo!>qSwTu+_*isR*sUgh!nPV4VD zTSFK5y>(7KTWTjx(K2%LZ^(&}AbN=0j))k&@%co0(1p9+=@6W~_^qgbUU^5rCW4zI ze8y5WO8ABm?|X=ZLwISKAIMeM@h+h!_*JN4`8(KEcu%jwGa+HG@C|WXShyn?U(8)u zRalNE4`0}E@BS)C2Qx^=t)3?sQ!u|L#kYNaGiin!48lMpU(noPPGhFOaF2V}#)AH^ z(4a3n{nla8c)ei0-94_&BFUAAk1f5RtbCf#o>FdtHc(9J6J_4kY*htko)rf32z2E< zi4|$=18`}foF(8kWgi%qq7^!6CXL(b5k+d6gr{sv=d$W6B}d9Cci&o(Me~rb^>w+* zFgdQD$aO{>C&YbFOi`S?v@9}z%^j|T$I@!hIJJ6GNp1FxSa8X%K?&N1-CP~~7B|Ay zl!ly{``smS{K3BddnF%ZI|R-Xx@5qInMfee{Dvgwb4Li=SA=j6?+?gMf9MT?9_VLC z)+|^{0p`_!-a0jG?jWC)C@&2^QLZo!k}1(M4E!j~Lnu>SUd7*Q*9UkN!3n3QPt}J; z_kpS{D;Z08BM^0Hg5qhY0eeq`%}}Hhre&b$>kxBj(+@SIVtshRFp=$Pm=AlaVBSoU z(XMhf&EARuW8ytW><|T$d){{;OILiDT^Yt&;&im_=`Jof=NpLrow!S$zm7s~K>W=9 zPboD1G~W7jCBXp?w?*!644T=K?TMkh2*G++u7!>BqhX6hp}vE&qRBr4WA=^6TcccL@QS`6ljul9@xp06&!ZkgrkUKXJR4-AGQ+sQ1F;KL;7HUeqLZNBiEL}9zu zMTNM)cdsOu;V_99y!Hs!j`W+J!Mv=!k=SQh*Z`cp_~D(i=OLbMIg=VU+m-CZGy2gH zodO;}fW64!Wrz)hay4m1;&Z;j;au}#e3^#wVBJ8qezL^)(kxb&7&q9?@<*r~-o)M# zAy(p+VaG1>kgJqbK)(7r;eF|u_h@nDM4WJXJgQcVvQ`+2R-BWPBUwm!%)Z5HgnS)h zOK-93kbbmaAny%dJK78tlMjjYaD04Ly6dD>(FTlLFpbS%gg0zH2Sv3Y0sD8ct{^4N zeSZQsFngRrp4GtTT7qqfRd|<;J*=u))7XO*6fbsx>g2TZ1F}`h^*uj0?4@y^NmCi# z&T&2oY_ClrKARgF7s6|^8|v@j@fjn`WbwKYlGJ0SyfEy{C)Z`=iXHIid4l?#r#!Vf_#@~&1I9lWA-@O3`>jWs& zBwc0mcw(%Ic4P?92Plt)xx@7R9% zkR@QQUcMZ)57qT(gMVx8MLxm5XxtV>GS@Vr{XlTC>yEN4*5YlUr|}4N#ik@zxDeur zIAhLJ@&T5~oPwSkZ;FesAWe*^l$GArAeE5|JL~LGb<8)Uxg98j5>XhwV(UkBNVRns zBM5-*-$-*M_V>%!g*Mh9`no=dTd%YvL4F_UazrF7(tW{cCCCfqy|u&`!!L0B!;Ig5 zyesaT+kc2f6YYg#?+F8cqwbnedO7oH_@i68WtLL8ARIE&vVug>VGSO$gH>-BY(}_j z6O?5Jw9M|u23?gmc5+QUy8fAt?w-JULALvILQ5Hubs*rAj)VTBkX=BAoOC7)A8ajA zwIFtZSotIKvPOjSWv_#;BDwP6L}A))nCyFY~GEP-l#&ky1U@mtl^ zke*<8>YYwd!P3!F`1L|28sIxwF=#-6<0qZxkMR}issQw!AIQ{me{p+??+>q2H2(%C z-KgGRe)wMJ-%W8!6=osWS>kPK6`q^?m}^^TmtD=fYv*6jP}Z!?&0wuLwG2~SmXWOS zzLzie@c$IisZf*1U^jH|5=ns6HY%#?r%nW?5CQY2=aN6@qcslaNkl#uCxVFt>(2@$ zpp0{e{B)~vUFwSzAvF}C%$2`76x@=Yn$KC3B6}ew{lpEZNM70CS$N^iEcO&4I7n<2?g5=V(AAbWQmhtBvve|XId!6TXY~dGz%AQ zxPa(zw{NLcfGn{IB&HNfOc&!!3#9-6EdLtQmg&8vD{wZly*)m(d=}g4A znZmbXHEE8F8U8DRxeTam=>-HxG?bLkh6{XKG8;vu3MMZR>Q0CA6QjT?(#_3=6U#UvaMz{6EC&0y*V(R3D7`aS0Nq_rgDygxAKAS^w4fjTN1-xcZj zL^(GWB>pThD>?f94xBEO5TIosZ&cC*PS9 zdCbf{KuX1U+0TJ)U zbepmz0jZBjFIUsoyedgC`z`AA@OCDgTtn!)Xo~%w_0S~|7U^U0~by%e2u3YKx!N#>yc$##9}>-r!r-R7Y_OqaAhA9r^e;I+c&g~UepV-Jjfi8T zW>3PeyjI@>(7aW&6AU>@Ghe(DieOQI+MqN?-^}Lh7(4ceLFA*ktvt{4Tc*dT)i%ZZ zP)U_0o_2sx8BtcFfxX#>t;q+=UVzON5M1J25$031zi05ioXW`a3^jmwk=D2O{dQlv za5sSRLdREnL_hd{lF%CAlQ!2B<7npFegS{%(xSjx8N!^=u` zd)z8!Mz?OED*^6C;lFCbCjg$m5e=%G+UT6Q>laKs$I;Y=9h3uY_Qm8M1Qyy#+Vi-Z zF}*HtxP!=ANPGIHy{ZScp|G0!IV1&A8S4C?4>6L0>jabvm%5PHxzz^VxxLe`^^9m* z0NS#vKB*YmG&yf*!j-;1cCCYhc|FxPs$A7}WOX;3vN5%zsN?V;!0C}%7g-@{8WBPz z=?8XMDz58eiri-U{3x;V6&F@F+VRiCPriMhyA|suwWd7nD92V-BwbW~E{WsO=uvrP zuGlbrg)q+mYcQux%rTN#p2b;nCCw5xqmzfUWrpL(()sw&0|5R?JbC2PIKvEE($@%kpxtn2(=n@c>EVic*eKj&!1S)T?fWvZ*qwup`~*!^_fx zO$o4)6?V_nkz2}e7D%L3eN8RGQ7m@PJ(zv?lVTPxv7SZG#e|oVc$v7gFq~u-oNRMx zzE#zz{@ILNQtPU9%GeDFR{VRStAsZ>Nr)of@!W2QyiK!@ix-W{{QH*Pj|jRcX_N?V zFD21T%bA6kqoFv(m)#|k_$^TDtDcwZ=71I~u21tM=wz8ph@WMQc&EL z`iL+wA>C0xV|=Km9Rq%J!1MJUA;?B(lhjSs2Uh>Hj&?Kk?Jwq6isby?)>8@nFQoYI zk%BMUmFgn%7wzgsU0*2y({bpUshvw_1Sdz?!A!{2(#W+J?}4@`AFm|qs%VYivDRx3 z`9_jEH?04AK9JO)Nzr9`EGyY1>F@i)2XvsN=Y{cD@OA!`afb0)tphnFoM%>a34&S+ zR>a`wuhfv7^kL(=({9{7ys%?YD2K5c8TEp#BITaHsm9Z_0UN9P`hTD$yRJ^%6=R5pRJw)2Xj~eCePWmEc|Bex) zUjLExItW;M3cs_$qH|&_H?cQJl|s~IxXgP<$?>L_mDQ%+)9oRT?>TT>jka^+*=c4y zEzO`7HRnCWCRtIZwqIkk|X(_x);kPA5>y+y5E2noYb90XvD5NNx!{|Q3XXSpUyIRm#WHY$V zuJ22$kKIM$m0$j{5HGHjKf`v5?o&PW7E!lIS2?vDaI2^~n|CbPS&QfOQPB!p`SjPp zHbC$p$egF1@HpAAHjmyLtM?+P1PWd0>X5=0RC+#V3@I|l5c^25NPJ$NVGqd1fDeY> z;1eDArrZfo;yWC6=G+l7a&7m@P1B6nww|;`$i}U(oO)s{#IdX4U+*^TgIN$9|9Loa z3S~8ImAOV$3;1V-X`+;A6y+{Yd3MD+ zF0?>kyhyruppZ!7p9@RDH;NgictsFLuntH&%sTzvxS)YOD>T*6_PL|nSq7rnK`|}K ztgZvV40=Z!hu%6HWNE+#ULGBmP8BWu`ODx4-!xfN1aVaJ#V6T3G7ZML&>KAOKn zC#!&Nd#d<|gBv8?LV9w(?6Gv8zql{zz@|F4wL41_+-rZT1}>x^=jgBNy_H5t96;V@ z=a`CZk}VhI`<_tW9Fx4oaiWG?-&w}Vu%Ypo8Zw5f(AepCp1Y3;tT#x|IkV@8{E4i1 zU*;wOwZT7GDw+6{TpwTG?;phGf1RHDSG4B;S*j%N>H5Wbb@{Jl_aDnGRaIXRT^NIZ z;d@6@m9J)b#CUkCz5qZ;sEP%2u1NX=1|{}Slj}&k*_yq3OYGqqbU-*$w~dMDtd#F# z_%quguP%hOCOYBcW=5ms6~7DB)7#8UIEY1JTK-siG7HmwSWFfxKvk;1us1L;Ch*RB za5gzCWLtrJmTEUBP#CiCt2?P7NLo*sIrRLiC@J3vwjxLu>}R67BOV!o6T7Anwvx-x z|HIf@Mn$$R%c70DJB_=$ySvl4ySqD$6z=X?xVty*4vo7*Eed^HyLSqwtzps?`|Ft86loG-~g>>T5!2q+vYU=rFCT$ zIw%CBTz~`oiTa>U*G`|BF&x11lWdVS)T05^2J<`nWg1^4YTH&c*W|2T6W&KzN9RsTho>HX@K;aqinG5f9VOY+dgdx)d03qA zY{CN&UsopP13HWeJPaIskZ267Sh-K$7k(V*_$-QZJBpf|0F|TXR%Alh^qL|gK;TGY zbg`{Is3@c=O`zdTn!c8df>-*9oPp6}*Pu{{I%gI)Cd|85y2ci>wqX-rSf((Y&9o??aHNrqu0O3^Ga2 z;4uoS%$*}hw(hspZN)A!?!LbiI$zmIw!2?`d`-wO6M&~*mAIydt2jkYQy z^Lr2?6}R5=g1sW0Fp`iX2IN&6qF*_cW}PuhRsG~Cf6*z12pk^i|go3y>9nX{Fv zk+H3rw5yrj-x%T-!~CgxiY|6+Bey64hg1+l848)K?DTEg7_wHHINrTpsax*hJ`igGqYDV*>%X#7?%DMRD-P+2jx=OK>?;_E2uO z8cooHxByCo78aWL#jZd9h9KxKimDH`5@E^b*KzQ9{d-Vl{e0KK*@Dr?(a6NojM2*7 zi_yfv*^Kf3230vL6El04|B`?8`)?>swbyA8q9roG07+=rbbn@H7WP7BPe+#$5@!>j z>lD>$*t9F@Bm`F6V?(I|6X9ofdJu$YHxuo$q+mUv5`1STOb&Wc)*h&ray!YsU~NnDUKqFD)B8b z_ZxE<2*zEy=?T4p#^V&&f}?z`xh-V>jdc|K&ECBYWcQPzy`M#%PT$_2d&=EQ1}xXI zTKxA5_6+riucjqyv+K}G0W*?(o@Md5}>FId;Vf9q`4iPR_HyRN&+SYdpNg%LMCwiuEIx1sF zmiu22u^6ZD(@37(F<7$vspvj75~ACz*1wI}N*MvpYZ%YW>&CO>8zroO!@=y03hIef z8vSM`?H@Jp?-$!~#8~A0y-qP0jqR;t5!v05;{N?n_Y<6axl>uGj=0|$qbp3DYNg-4 z?HDW;@`m(_$5JpW?Bf=RyI^SD&2525ENBpA6FKDWB`#OzkQAlFCd}WP6#q~O-z6~0 zrNF`_!xO8Rn!4VKXqf#va5DuGjHLJZb7gKTUf zKr?}W(;NY_ReD1pm>+wclf?c5_pcx?4{%d;`vg&`e{?tacg~If_rF?R?7zFzzvH`G z-A47R8n(ZJeHMWJ`X2sDr*b=s8;qeOmx-Dg zX`gE8If7EcxNaD0Z!gE-=W-w+E&%p#?{ihDbc(U&aWvA`JBZbvzH71I^Xd>aMg*|y zTyJGYQ|-Ry`u5JI;ci){c^I6bcO-8g0@7=4bynxs_ZPLi&^^Z-r~G!?ntja`15ech zj)hGpuRe~`r-J#|2O#{mWlvsN(X2V6|0GJ0-cQ&n&@}%v@3&(DjM^=54B-Q2qg8j{ zZ?diKhHmw9J8itTNXX_sx1`^rw%us1vb7Rx_Ip&YOZ44K4s4BD^v~K{Cy;gA#%ubH z}LP3EN&cpSsQ^u=5A%2Gw+s5mYfCkWva!PB=K(kDwK~Q#ej*2J27@_nZ zk@3qhYTd4jmIj=_!xlunvNm z)Rr_{YA=E{HuZ+;&qA;t$P|G^nN4WB%jr0JT$cAeR;qbC>xi+lNhCYzy(`7WmaOy1 z0(uF^u^PG?cx*q&Y|=vPzZU;qL${8{<+9{8R%)d?hM~NShtiGB@0O>xLg<4pW0w-~X0^#Km2PC6X;hVD-Ve%X!T)^w-W zEwy2k(5MUw?5aDVC%U4$+NjQHUz6s{@tEg7&T$Q$D-W8pdlDE!l9o*owotq-%dH2)6|2LrpKwMl&e^ig4kTMgn+TD3L^M?g{X#n0OQ_z4BE?{%pD0P5hsKx zmg_xlE=pRr&^fG(T!UG0R8OQKxxrQ>R=X}7gU$$S!ZUm(p3)AgvhMWbio&l!eFZs3 z4zuPx`qBiR>CF?Bb^x*G1b1#O$ZiG;n_Y`@z#wZ$R<( z5UF#$2)UXb{Gtc|2+SLg@cu#HJfg@R(Xp6{`>6UD-XUe91k^7D##<(q<=k=)n*j{+ z@%Ed)_)D%he56la?JkWCLv&TE08dAmU%*zb9dt7|E(5DLs5EW-yFb2!iM``-+AMzH zvLnCsjl8BFP;Mz2eGvW&xOy+*W7R%EtMDI%$o}Vkk^ldzB01{oDrmyU7Z7BS@vrhi z$~TGmb^|5_1BWy1KG*E5+$r1d?|LD1-Tz_unSSiv}x?FESOSL7nOw@ zy<}+sE33#lmVnnTa-@mVu6BUeTGq(II^E7dqqga&Ta$*j(AsA2JmV;?#Z!~(45VTb z{n(aH_m1sR@D%s#uWzK_p%-`S7eum;pVk^ z1CFcl8|{+2$A-#GX(A6-jx5n4XHZx5ty+97{IdCY1w#2=k3yNwbu;!d^T^CTYlEW# z0iUVg`yM(&cHmY%5g)(RexK^b@@l-ja&8kTw9k&QY(mIw_v96{QR-usD9bV+qAX>N z#-c>Einr@Tv#`1JTZu)4lvxbFqn$EtdL22ubVg+yUZYlmj2FH3eTjl9_XQab`fUK{ zPlgJSGIPs|8Z!)~;2u?L+}}!VOzHSbaggsN1KlsyWus=}4c5y12$-NHszXO@)Yv52 zF_7x=Y7qy4BwrH%PF{xz^#E;Zwk+HIo<>xp8OsD7skYoKCG+4xaPE0x7b(M^opj{N z^IeQy$SCa9X430gojdsiZG4V0Z`k`bOre!ke$#ry?W=9=Sa`mpd=WuWK|AI?k5FhZpSOq+g@^h}KO6h zRn;WpN6Ss;N0RnH!^FNnr^<9LovxDl8WX_sZcr^1y+LR^W>Lu8%9byLW>$Gz3Eo8^7(tMVHvw3D&rtsg&-mu|AMBqhtcw5W&!+T0u9g+u&77UB zOwIn64D0_ojrczg!GPaB1xAIDKPbqw603xe#e@bqv%odIo7qhiHUvc9bjMK4TXKDHL6T&sMPK^rOSay&~Z4?G~fgP0p&VdN&pT zgW$m~XWfP+3~eWq9T&9giVl(;K@yEf&XaPYQm7N*B6B}D8|Y3xea7S{Jwl|yBX@uL zZMPwo`2GxgSR9{p99L}*lHsJSyvj-UrxPJHfP=KYZhB&4(uvZB;WgK_VITg6zwKZm zJL3qV9{}UhT{Q-`p^a8T=a8QI3x@S-4T$&Ll2N0nD6WMG{583ryl6+$yM;y*LlDBXKu;H4|DoVS_%1Bnh zTZx=SENwHg6n-;=kC^3I7uM$?1K;#QfZ<(eY zP-~Dm5%{v7;-7*3ot6ykAO?9$sDt|%Cc)GiPP-uhE`|`=SSryL2(Ba&-%lFCh9ZsE zI@>8bt4a+Xl1Mn$)}Du16Ig`K^NsZzFCLF{fk|7-*O%B?`QD}q>tkMCqqD_La7SY%A^nd^SKvzgTab31&Yj|ZoVMm z{wkyhw6aVupf*3PAuvX~(;PC1#4g*g`t#SQ!X1KOyy3GBclgJ{VupW?7b{c7|E|8J zP5-XHW4>uAqA6qZi>76?EX+e7D+!rqDZqg>l%XrBQyWQ#r0<}K@-~TfaLYx!p}&Fk zT<1s7M^H$7+Nlm-DTGA=iT%i0CNpeYt zC)>iY!lTkhs=~>oNc4^5262zXsv7I%IZ_Adpd5D~Pm{Q~iY*gG!_YSc)MnJkOEFo- zd#p}E<${9c2E<8=%E;MMexkp>iAZB0$d6$y9O4~XnTiT-Z?UTRQ2NC;K`L=)O|hiCUQ2H-DRlwSsMWG-LOW z(5dxfqMoum;(_x;-=I6{Rc)y}vN8%Oe7BurT5f7gE=ZH>)-0PM)J?Y3AFZ%h)&BH# z`Ek3gR1JZsHx&Slk-RLus#RwSd|iKQjlz@ASx2GyNwH(fKB~Gn ztk8X!)RNcQ6{he9onll$wMBTJ;kNsheeF;aq7bjPmoZKQIo>B5!g+!% zOz1h2VOUHR%u{{1y*D2qoOzG{KFckZ$GGZ!n;1`@RJSNLWWC}fzVd`bL;U!dCvRP( z7Hvc&htx!)9To#4lzRe$`@W&iCanF>u=Fv%c5Qu;Tg0 zu=)?}?f({5DrWAVCX^2L{~cZ_pY1K02DX1Jj44cINFY{|Qdw)A80a1ldfvIQwc zku&=8x2ipX0g?gKfrdy*DHh+NJt#i)8`2*2`tMp8sE^-@HDy>&(p3|Ts3+nFq9Z|j zRF(UJBJCz&g@oW|1u2lj5hx%!OoHKboOLK}=O-!D?IxHS(Li&;)-l#cDKX6=9`>3v zni^oV63er5nOY1eRK#T!(IZb~a_?P@*BWA?IWg9|mu-&1o{C?dJ*>exE4%np?X)5wdFhn*D{H{{(5=BlGg@=)fdK zqh*!nrxT^%(ey7lQt2hdS2YuXSctVM#sF^w%-d0M)ZlwUOv@ZLxq)0%E}v|UNq_L5 z#-S#L*?PnD7ZqS#xsHu9t0J#9>(#K-T^fIvc9kaedv4#fwpn>`mKBKj(j=;|H$iy5 zVC|@H#umyu1V3-a&x_MX!X$wiq%~os98tw2aao1+#k8r&>NLGyVX3;jHjJ{Sl4E1e)AXi;e^>}soqgyWOFn$0BHGxC zdeG^DjqNyp=M-zT05b}wG>hm9V%dD`zyVpm3VPeF0$r@i1sA1zL?gl|a!Obn};)7*SgP!(ipFEH}3-EZo=g|jMRl{1BvGovX_BwYp*EO}E zQq2cM=OqgALzi;F#Fmg|>uc1U%&7eB2=grY`}}+#cs--}NNj-^Z6W|m&7ZMDGhY1WD^uPBfgIwZUC}cj>2O%S?OM?8Rm~iSf-W37*q;)y%LwJK6D#9+yYg9 zy>cC|{ffvf+w0O8x|_pPytpg&ci2B?NG^Lq0AnYe>*YnwLqz@N)F&QrN2tm-L#4r) ztN+oH-y9nETK-jRb5O&rY}4O5vy7T?;saN$9C%=H9G>%z+_^n7jpnm2dM+T?DRwTW z$FZ#;=o99+>D9RO@z<320BkHx;Ioen|HnR-?ms5}zx&t!#-$uJE!A~(^p6l`7Lmmj zb1P{n=wiBt80joFxDZTT70WtlxbN6fXsbOdup-VO=<}1VYt#2YMm-9}U8p8TK3A9m zYF546^wYSrd)+bH;Ym~LnQWgKkD2%B!lI&pkN0<^FTt~*p1gteUlzaIg3v{^k&=)a z3lA2ClcRWu`;zSChQE?%@&}#l7RE(!VZks!l819K5(nDD{60A*86OG+k1#>?ZugWG z0n_-}?uig7=CN%X7>$jEW!e=tq1!bvR2J>zOGO&`i)G%IGsXy~m5F4ZR5$5{xn?|l z0E*?vNm&9ji6yn2%1}xIB@G;U6V%u5Jqa}P^`(@cEN(HR1~+st+RjSCNJ_{O$=z07 zv3PMl-p5EmNOqg}_4FJpwK0rKS&{vX%N4XQD3Dg|`j4pA^!E|;N&sQ-aQ2~@Ig0$7 zsY@$7CT-nFgieJW>jc*reF~KxMD4Jxc4&yUii>#aEP-F9DC4b&0O?8Zq z`^=BMpF6j|5I2AY*8n@(KrfOq74lNNc3bVGv$Ai_ALRRPb`4G@KS{oQTh|SxWD?pw zsCANgu5zssaj$jX(KL~R)#<4lkG_m=GGa4L7D^5F%Z=yP? zc2jIrafKb~#owOyfQ!D!KJ#uzhl_4s6wi2Q%U;UV9Mv<7;smGb6GxW5tp-RE)uMxH zS90QXf66Il=6jaOBu^$(C>!9P+{x*!w%YI(V3hOduPfC6bV4A5s}*r=TC5MGY?$%% zt_~=309GCq9riV>G(Izx?D=0-`Ban)#tN#yT26rgIJO$le5xHuUP1*n5u2w`r)66& zV8lU<)nm3|AKq4x4yV*tQTlffGpN33)J+MCGc2|t{XJcT4-%?wS?$k9c_Z)eO^^E{ zwggHcwKO*_+0tual5qv&drzrv_O)7UEq+yE1XHA5kL^-uGQ_5J?zR+lQ=}1fT|~?) zR%wOscbnc2)?ISmG8YEPbCOqL$c3un;R>sOAVE-C7cwy1^3PBG(|%_b7Y>lfB1 z8&1Eh5Vku}ta9g`tHzuzQ1Tad{^IQAlPJ?72t>#_1*cg_Al?Y9cwdn$s-h>@a&k$d z4m7Q3(1_?ZE;JnpBS5hCC2wBYb{}y-vEta z9SY9!(+eGXaM@7?~`n?e}Zw?^6N|LQ%=r6!NmR1fgG;kzte0$+Q|)5niFcYyXOp`AO5F4kw^( z#|%_CY?sc|&exn*@3SbcRwx)d$r~O!TF{k|6ka2=}6Zd}EeVpqu<8jITI6WzF zHzP?1#q{-0{IwGdFZl-fE(*+c;W;mcxgYJ*-V0TW^Z94xlxE90|cj#&_^mZs2J z@xhP)2}nC^^OfOi6hNG|;gB{U3x|W*cQUA6b7ZHP2&)-W&Jg1(I$Vv&Y*2)yxur?l zP>?jVCR2u=+%P6U9e_OG4u?`?uBaU}frB8~}y$!0gJ*H&~aqYlI^?N4pQ+(|b*hLpevf596yf z>~VvEqqwJ!Q+fjf@G-sLPh!a(&SA;z<23b+3$v(`9o7O6USCUonnM$s)JEskotHjk zM_ZsIIU%-Acp`dCJmK_E?|W?!?|VEYN2hu1A$fiqCcII@2{3kW*jZrN8YIARV18}A z35Q3Eb}AtN$8yHBz^uwMN46HklJIB(0s z;QcOIOkezXBd||M?84M$1X{*U*kMu8q_$M$SBj7HDtnN#W{enBCKIDx(Ib}idcDHH zZ_t+2v{YNA4c^qw! z2+4;s$F8I$9}R_t(P2#%WaAHxL-|H!LIK59~_TT5ClC6kxM#66*F zDAgtar2ODAbJ^#q_#ixw(dc@Ptg);!$@<7cTxtvhEb{59;H4?cCp?}XMBu?26St*f@3SmZ z?%o>N3PreImja=>xUlY(%>l%tTH2xzaX_xGvFFTL=O=(_qD3zh^-8nq{)&oM2XwGk zJPq7q&Q(c(N0aCh5H|xX#mKEFVl+Z}a&*rvVf4~irgG>O|6RGgYvWa4)0upl1 zgrd1mi53CUZV;1P7wE*P9QP!%F~~~$0;=EK*b(tCfVE>mKlO6YDIUrG9Ag-6*?Y>h zU&;r;!G)rd#n)Uh-=Ph}8Ha73#-U4284QaIeL@N_04~Cpml5ydD(G$Pr0bDkeiiKh z1wOP^VZSWL`evQ4RSEFqSGl)~$Tovmv^)InC^mO;q|cJc{Fqtd6=OYsw>BTx(zvvH zH3oY7=Z?1_5QmiNL`s#Fw9WjzSLVku1|0(6>Og1zZRpZq`` z4}m8&Ia*0fa5`hxQq*%NS@Iq%j{828JEdoG>K;LsFnF=?b~$}%emQ?he|C-SBff|O zE91W0UC_@RzepKHPO6ln6zSOw(E=&8v!lKc>JX~WzfELyBN#8!^G2A} zgHs0Kyv5qxVEICqjGZ+?_T09_Uf05KRXVknjBXQ|eWHsv^>mYpm8o0a#aoCoM}D5o zn$ogFk~!FaAtx4F9hY|t1}@<6Y{?@@$K5wr$qDYy;`=9 zlzF6M;;=Cf$?MvaCheO#i?FWNW=+nse#!=MpdD7xt)5U@=PgAOEX7PGMTlJ}hxKaJ zIC8}qNg!b+n{wSw`Y;XsjEiWA#>RXQMem58dPlW@=XCEo-~pT>dI+-kIND*!Y`T#b z{`@qLM><*+vr&hC;CEabwAFy0ZhB?@$x{LqWoryC?!&x9k=gc(#mj{1GH8{8K(t_h zC$o`oU> zVv?9!I(e}~;j?G4Our}*TnU&{M_39vlu1o;(_UIr)TyWlGTWb4OTYWHmk|^?2Np4>HQ|`c)!aTPxA7Kh?V67$q-yN|)Kbr3BNCfpZo;F%E zfr&r^j-v{_hU5{gUOGL5!-(Y)y~MDUDMiG0yl%3T@@m++1Au7MTEk<|Srd5UmoC(K z(?gnW&i>obznVSA>G~jUNnrqw=g(b2j2YQ`>;u`wE9bw4I9~aUs)3&a9AvD2s_+>9 zhrQ{)R2NsB7 z$2~D(Ob&R-TNv~j)Jc~r53!iK=E6rXa}yO1M$o!XxC1rxYF?6B3<65%yzS?4*?r{^ z@YwC4@Bi|+e)WZJfE%O}91rZ6o}nI# z$=S7XGHgphv)Y83m2qNd9Q9SLZ8?V9(q$m*H$nJ`sd3h)7G7!uKEdQ5B*j_=H0#{Z zpgR&xT|`7R!KBDwaJweyp76Ozp@A5tC<%eYw?R@lKG+BttwEFUC{I*BlH9(}YXFP{ z(YQ7ot4?Q-7@jG?R0Qn?wXr+;RUFTJ5BedTYrKCem}F1%;WwEviG*P|SLi#5LXpNG z4jC7hHSY4FV_B8M30^#)IGzTGT2ONmh_{e+H?`?4Ax%EYXEASk5{Y{@dl+48gY&{f zdn~FcquA&+LFZuQCmKin=vY?au%csm(eZM+6hsuf?9^mf%lYzR3aGD$=iSv*>(wU< zB40*q_A((qHq2?liufJL#{S!jO|f*7cM4b|r4ZbNAUcKwi;ywb9~ z5+k#1myHw7bF(RBz9KR@(Whn;*nBgCc!B3ao82(eK@Mk=R#z!ijgVGb{*{x^@kDt#vjv zAo@2pqicm9K<>#L78Nk4!zXjWYr=zg3m(SvQR$1&k%E-A&#TW1?jF^_5YnpUqJ*J-*y z;_y;fG34F7jUE!VQ-5(YZ&XlSp2^3wC_i!0{6142!dM4mU<}7V5?u?uvT)KUoI|3d zfa?agm;*2M8+^QMsC9O&)k{_?y6^}!2#YSQ({^JwKcgyT(kWJmEu@fxeZpTNg%k)5r9 z0?xKiwrtAbWBcHp%YA(->SBOSA-^F_JlaUwe^8Jt%!;m0^zSvfrF(*oKcCF)Fob`~ zEj0fzw|w$!ZHz3;7%W}wY*WsMdT)171CaZ>L7#tWe3Agp;vWC{bdOv>rp1KcG z04U$@9*U#uF)$27r4^}mr~^lfPOROX{e9W|Iq(PwQRZSI&8*Bs6p|*$#L~qQ<07fkvEfH5FHeo@ zL-%`;95hHEsAj>WM7AjthQ)Mt&CgE z-SoivjEZ=tzMqq|I$RuLTa*&ryn396t~3tNII|AngJyZL`btQUTfd6E{N@7=H{t4w zRCp_@`35PU$x~CnjD=_`bR$?0FXcmD=>nhYBMO%ldWD~?r5rlU$u@2hkc5)CHFCt~ z+&l4M;yjDxaht^@PX7F~^=Ah^8$R^QkhKwBKGw#m>)J{v`$B_hBQ!OXnswrHDEQyFjMt*h6v9u(C8UpBnkbJLp9ch$&{JY^rv zU#257u!L~xV7T`%8T7!Qs5;=Gq`{zxYG zWajCVzft+XvXzClG{{uEeeu&kX$A$)04$S z61Y{cxD277&koO$_}B+cv~q^k}MO~1qIkl?1k0KVW_@6%-V%P2d#xur}f zW6k7rY2f{mZWH@Dg=9@0t?HBmw*YGF)FL^24ll9BEEhY`Ddb{HSPnD^Y_t-B=QGv| zjTwQMTQl@B{qT5(xT%B9Bpy&l2$lO{LTcQ~STHoXMlsj*{>Su^_~m?4NsgkaxzZF4 zo^|g=!^`dwf=}WN`I_|d5B?Kc_#c9yFgh~ln8J6|Ey*j_mCCHn%c0vgK1EM-@b2X| z(Av`)e3RPFnhz&9BgSSE2i9H5cow~rZ>cK53?nF$!O-#kC9@CMB{mzBmNHxVC|z7E zMZefK$P{Mq9~+jBxu;xj9f_~$M`)!et+^z)4rSG1)Zzc zrdG3Q%%swrRnDP?Yjl&dHl4+h64DfVBes3jXN9kxW4P)wA3H@gE1TkU@ZugTbncw< zv@4{T&-K^&IuIc8wgu@&o=xACKIX7bVcnQuEAq7nF7IfzJ1lB`K|eSpY_c_(H0iev zmTcy7q`}Byqnn!504JUSdmQ+Y(|PePB+u`j&AZg$j1p}+r$6yTA$Aft{W|TrG6(Hq z{~)m_mSL^cKCJ;t;I>+&e*i5KkGx8b|477m3YA1_F|Dh&KzT3yV@Gs?fLpExK8R~Kt|}zQSgt+ zzmfI+<}MZV6IqS^(E$5DP)S@|ja=Pa7*+qe{av)%#c#`i3ZoDIaMGA^n-a_bS}JS& zg0!16o0CTdH^!2{+KTyA7jt4?)hKecC@zfZItO;IG`A|xMg!Ty>T>j$!|Ea+u)PI> zSCBBIj`7@>>1=6(3Kz(C5^Pw2vE)nPS9wU1uH9f}Gnzj?IlP6@JN8tD#d=EHB1^)UM|#Wj zme49DC~tD^Q?5m7Om|q(5AM&7+nZ?o__1L$=6K%*Eg%YxuYE4JU?isr@HnAESz!Qk znp_LTId_{FBW&0b`!aKdPGr292;w=nY&7$3JQvE*?M?5I_pEj&jm40=LODAUL0NEb zA-A|g#?k9gE+_6Th<0XG@er1YE%5`08Scsw2+!i*mm0R=nnr;1No0>O)rl936@u>~ zcJt1+mE(U=6!799E3jcnug#Dir;s5b-$$tvFSeX6SPH2w$sSCOq@*jCr zmaMT5GB|Kla7=W~#NHqx$$k`Q4KgTHDJdZ%cMo^gF;nv#QZmmSKZEcxDo@%%ORJQ2 zxs;a}8o`&bfZl+C6@K0<8&1s3>bIlrOaC92{+kX*?;npdKfbI)U)?dvD=0u-TYgal zchng)Lve^|BiX}8(Gc&V-Q!2OifW_S14AKjlVHkNEtuAy1t zPqA!SszG}gk~5rGY3O_f5oO#57FU^gPwi+wR8xYKeWn3r7-2jk%Y^V+;eKN_Y0V(8 zA*#M$_>ri3WFKrq7B}V`UiD6kK)2`?{7JYHKVVjf0&kk|R&=bzW zY@?TyML?uAno;wW+}m{HU~U;Z-U$xN)OX;T{>~3dRLI{`G%0Xs7h$SsbgA~Uq z>Y;+U)W4=}v!0DqdM`41#f^=B{-7*%FCCLQs1bcCbBat&s~?AUDSo~w5rrH$=)D-u z!Gs=>k&0`Di*3z@Ac=1ZsJ_Ggo#mvVLE2KpRU^N9QzXuuTs*$md{@<=Hd4T6`!IJI z_p3YRbr1g4Sh?am-BWjn`%L4-(J-rN+>X!7KG7FiXlaD=DeO zgn(OI7bjPFKtgqy8h46~Pfxtfr=%3Dq`WdxSYZh;Gnw^s648pNv$NX4@3O7(OeE5- z^PLUp^a$doy9l)`EBzD>s#2Q{E1L!5&)T`@N>G_Bc2PUG*P;qHm54Jkm?P`EhI1)d zi*W5^bm?OEg=XV3qp6~dpTSuxsa>=;PyTwx!Zbi!CxVqp%6y)R!QTZ+lWeHk^UaH% zR5_K_%c1ZKc_FAJ6P1xB<^khw-)?PuSO=8Z1_Vk}0C0QOi}J8k5tT4QKXY917N@0C zHl#v6tZpf5%*r9<$1R;5A5as-s$~StROO2HRkKDLC!M4jvs0=ZcES|d_#-#ZxZy1U z3k4?(%#oRXWr9Ptpfr9%7N^5@+*Ln{JELpLGr3;tq44Hkz0r;Z;_WvwjEc2 zVp{;JhgSwKRxfTf&AF>9Vy>)-H!wE6KkwZeN$6ksx z+=Q+O$j|r^RyVt#hNdm`{BSW^N?6z9jZRxhN#qt|JjIl%{wD;;Zqnv~mvuXA2A16 zR$}?yUt{<|mHJ1fU>yC-Z=g)>>jqBa$V=+bg z4A;{V=5SK}NjKL@HUG3IEH_JSW?SJJYKYc45#$3@Ouh(~vTiA)3FOCIMvN$bN7IxC zt^<-*qEt^rpr#Aaq_ayZb+MX7Ip^~F+!9mwNg!7g{4pfxgI13#prG0) zq0^A_(UJBK1UE|TvMLi}{8=C8Ei1_nfBV6AnE0?Isz>s)MSE#D!q;r#q}`k{ZS9MP zf=Dm*mG7L~fO!)aV-Yf+AN91S13GQ&4&&B4r73vp(_8W{rZwCP^v>pIx~GHxr~L8n z4(S}-jBTx4EE$#lzKh#iSlR#APNJIPgnhpdJl@?J2cJ$)J}T5sZ-0(+`Hq+lnbK8O z9c)xTg*GagKolBkn7@!9WcSd*5*8Kdz~|oITt(=l>U2*&I!3Lo( za~7`6JWjzarUQFHH=BBZDU?BH%q^*Uu0{r7#=gD1sC~@CSuyk$&$Mnn7De94L1uI^ z!eQx;WnmdhOqJA3sh-M^+3x-BcKOW=rf@wfwZfpB)Fana;qxZBkBfsGdWy2^3om3! z!QQ9QbKs7Uo2U41l_q`(B=TWu#_opsosN)~zm`W zAU@z-WG3h^IQyO&!&V?yFrz#1fdilbvobDn6I4o5)1h^0yrhC%dK~U9;-b&QN)BQM zg3(2>54)A%1V?G+OEcO!#|VO|TTln?(dIKcIDq=Ld%5c}E>2!@S}#mrdYM#LgE>we zYHLeWiVa!x3rk$}Fj0M!Wn-XDe|$K(FfmufWNnDHzDZxj#Okj8$G0Vl6jyGo`3CLp z_giKsUi_+K<@M3l#j@J~CY)!=v0frvKnh|XF+ zP1-Wjl(!IVTvZBZ-|nTJufYnP3=Nt5tuDWL89FN7k8q`Bu7Sf)kf`O0J5P0sI}qA3{~<0q%POISSE*cmMSTXX&D6wuT+q9_h^h~$S*M)55-Mdy3!|8 zmt$6MTUgeq@7jYsW~dSenFTJ4Hini6Ge#B?^4TWzT`@0^h%U?a2Z=Jn{RX?Mx{xkb z;=aI!>%i))+=__09H4+c{<5Mszjjj*99+kA%TlWnPljfbd73==l1Yt=yljzUkn*71 z@s7pE*jMwZA9m~fhLw7obdYBP1d2Mes;U-arX~5?m)|?xBH)$DcAm}SKiV#;7W)Tr z{+hS{K&x?`E*!WV$LVlReS2iyJJ|E)6_ob}yxnn~Fire^P_~eM|00=P{O*2wmpq-2 zSKHa3wC0euB>g_>l`7oXBz(&9+)?yhxbu5jcU-?ouK3+;eZd}RFa(c$GmEVo!h$#9eu%|9FM)5c#pI^H@!#}>WOIGYT zBW3-V6^7*HH&2l_fKgE`TGaO!0jbz!5G)eB76cV!VF`|D3zL7s$>Cem^9;Q3496yL zqG@#X$X&IVoUuF;{MYmeS318<=rgg+{9|JKclrbW1=6Ap_U2X=Zq7!opR8>aGZzP2 zceDQ;sB^S@3{?*?{^U@c+AXZ7Vk&`tgM$t!28M@YD^c6c8SBK~(iD1TZKMii4)Vde z-F2THnJlzp*G{n=W)8ZyXT_JlryKZVuS-Tuiq=mdJ~!fCf1BT=72o zZa?RYcuc3=uIM1)gVhBFnLWwk5T>}e1on9m8Sfrdu4iPfO)G%SrMucqmxsHlAxW$$HRj`gE_Pw=TV z>}T>q?ptLD%8&fOocxEsLOGzf+<1_hYW+S)68=-CSdj=CMhUB@RFU($DglLGxg2SWwO%7 z9YV5dL%DOgeh;jevS#qA@`ftegxPy55}u`9!W+BUiIIXl9DJA?+H9w}gzi9;Jvoz(GMXarkN_xHZfpUfnMd zefe|qsbp_YK9L`JG4pf8u^K-%C?OzEWF=cu{c`pAGZ?G|P8fJRZU=Sl@LQc>LOmnH zC|p0+6!V(eQ>nDy0?*{mbQX{q2xO(?wRdzU8oYWD*w3M5W^GARM_D39QZKAzWo*WQ zIuE=JMH?U0__-DEaMdK(UG8IFMvWZDknM*&-h8r#euEI=5+fDRXut#K(7!l6v{k<( z-U0`{EzL2@dpn`j>g$lNlk#@XQLuBwjbT@Tjr4Yh7rY54IA=Q9N-c2G4v7t zI>C}9Jl7>mo#5jm8X`OMYi36yAykhvfTnCEd)g}+Cjxm2ft9u=|DRkL^T|;H_M^Ak z!vdaDRM4Cltp4K}y+Tk&HGjLq())`t$l=tJat-F~_AVYF6ggBwnvM@;9j2QvdUv3d z*vemx3}`erKErtN4hC|M4Yph+A|g6A4sLeeg^y~p)1^Ar5hIH_S>}v+w9k$g{r7Uy z6-RHgbOoDe9VtfDy_krFt(Jj2sBC8Fd1&$C;;W>Ork%Ak>R(K?Y23Bi`1l5$AI^qe z8BhuxG??(@bmVJV92r54TEzQ3`hE!nLe^c!P)eRX`eYoNT&q==>l^bs-7)sIS~Te4 z%ePx2#npTj!v|Mao_;4t&naq`GNjK{jwViCVp`Ps9f5&c$+&0BMwd-Hh*q`Jt8nC5 z0NYIag1|~<^=!=~PAS=+`Xk=rP-&ubZ_4TEYh}mSOjC#A-0h8N-Rpxij?UHArY0A+ zTJ2q;NO7i<)#7LSL9TAxbfv40p%O6QAjy(yPh|v%r#inde+vbTF;cpIV&!|zZ@g?Pi+!cf9Ru z$>t*U4wr> zi%DwOlsBWnEryrGfu&|X>+N^}SHMJzg+rmeW0J>4@$CVw(}T4P&47#jpLx(jqn=+$ z5QizN8g!U)Ap3~{D|V{;2=H&2|X43sBXOAPgf`o5!n#rjD|I=UZDJ5Byz= zzcA5SInftGrY3obLaF^HU#$;cJ@{OCqF46xN`_lqCabG?@6EXOr>5u-ck&^Gvm*Wc z=|==r*0TdaQkU5qRz^Y=@ZRGXDX|c5Grve|jdhRL2G%SQaN@I2V~RR;^hmMb8KNC_ zm53E4`@l-yF+c`BmAoQidx>VbleOL`Tfx5#_-1kj$YL03&%}%z8qPD=D#Q3x?uK}- zJMz><-a0}0mAnJ#|KU7EK;2@F@Bl@`b{Z3X>s=zPEIA&_Y!!$d z4P3pKr<1R4 zC?_Qr)!92p4od?Ijud5@YR9^Qnc$wQAQ>;?fnd@@p2$O&yr_IDmbaV&uzbkUqK~wv zML7QXS)Rxx^H8Ps(`FUL(PbP_agQEFegmmwPtDFm(tcS$A%SV}<55^|;UJZCqf|-EJ2~5i!C^4wR z;Buc&8#s}*1DQR6J=DbU>`+v1!^w;>L_geEIBaYjTBh@YR>SKT;Tu&@e*ODK^;?NT zX3*1Kd26o&w%h?CI0%3%vHc?(j3q^AjE9j&ckzBV(A6=hQv!V!TkC!mipHz^{D^aO zWhOhOnh{uQ6DT;Cxya$WMm(UV-)7->H1nf+AN4STuGg+DcHl7@FW16QAE0ed6qZ;5 zQV^Zq%M6}S*>Hn<+8fcLIri{80UbGXjO@X1UVk?a)@%k+(Z_BNhNb8q?!*&#y~c8V z(ojg-8TYXpVB4CS?FPWuQ>*S}U%hO~2e#f-fgM1*Izq)pdv|SS4xsHA1lg>a({_%g zJa!=%wzOkM+Eg3#F^wILN^jL4THQdxpAA^j*td7Mzrf-u*mOxTb@CzFCiEq2oBQx(i%P4J6#aW9>wZ_ zUP&gp1ib^Yza?Ho;6i8%FWgD-@d8|FG>gF>nQ6F?#2&+DrqRA|BXff7j*y;@B5sTn z=zHwKbJHIIbPwNn$v?qnG%HNmcG`J}ThH`bM(TYS^$9Pq6i@y2XUo6eKG@P|1mwKbF;LeiNw24MA^O+ARXLEG?g z+Z`n^LeHFX#FVR=yl_s%(uHcEXv4FKGV`?TlHjM#4r0cU>bgQI#&n6B06X~zD%!eT9_Gt8GXzf^E?_9QeYl*pC?RRD40vGEbKPMn`v@qpKP05Ivsx~(EUvFY06FS{OyzxdXqRjyto<`^-a0pv?;5qXt6}A zf~XBRo!%%r-Huij>7m=#%P>$6j&S;%p^ex?>f>sThr)~~$r$e{?i*w!qlXQtb$cOxi}$h|rivL-k|I$fK}g z50;+asHf0mo6@raDp?dHSh4v(3Zu+gwI$c{0LR~}?&Xe&8DU|=R>e5V-*Ev~Y8q~iG7agS6u;~KoTShHiIn@e=i2D-Ji6Tx1(u{(BlTbA!rl9#D_*E9;_s-s{yL)A{;44W+jmGtuwc7mf z{kBxm49Iup*wzkXI9k1GamTi14gN@A$3xuLd-ioiMI4qBnvo{)!kN4(fHUAmj#u>4);L@9S3M!6Ei+5e;!2% zriqw1Bz7$hb4vaKt~^-LeeN!RNF+m4GpMLwdA@ks_xGsT!kHW-_nZ)iG!hBRS7FjV z(hb`IUqW&`O_sn{1pGDXb}nJOqjvznHTs9D%eE>Vk~81cMo1U}b=1Vez* z)5`qYW7nViI|;6Xrm5{F^+WX`(v-0VY3pF?Cd24}5%GwC4VI280FMX@i>XEWtwF(l z2Fxy)bgtmKT+`D0E12b&)^B_S57MLS)lEhqG$l+}oHhYZ(DQ$bYM~~cC+>fX+TnjK zYX1K~8~k@s%a}MD7#lbn{GYIq|LG^!M?r@kya;nXe%j{6}S_9@?h-63y<>Jab zPc|Lgmd2YFC7TgHNB%+LJOCiyq_$2zWE+yiEpDfitwyt#F2AlGptT`yWF-k{1d;$_ z`4AcgQblo+WFh=^RMR*U%tn}FoOv{D3*Rc9_DRHM7?qH!FAREZChK(*n+xrxASc@P ziya3soN=6uYxjiChHK2_hvt2V5jC}e`@!-w9XQQ`qKt{%(#L#Pvfm|D84pd1WfD$A zmiR>Hy1%G@Enw8#4ae@p_t#3z!sueL9Z<|oSdxk-oyzSL8UVSn@_A0$+ckIg7@Xnn=_VbG>6rEv$VdsmK2XjNm zcxKr{Ct8NcP7;BLSJiYAev5NYq*S04L=Nc|l=D>CeFHxWu=l(Noqf^sA_agI%osGu zdEQ{n>z=zL>=1sFhCqmq;(HdghI2*6`>xK0JxivFd&b6v-z^{J{!2d99`ok)`8(g* z{9n0}|AYbk_mln~6w_*T4{zna#!OwNL=5uSAW(2f1E>#Uqko8o1ov7DB2pF#n!%O~E35smdLK_r z(o-dZ&gegTyl&f1v%FS5P9BDO3HZSEiN2Ggy&4@jaO^4pIT0T!Jb4EiXup(sN)J5H z=*s}v5s6g3wFd}Py2=ic!%|eTm4Lzs{&IJf;arujl7nZ)v@kd$2B=K146qhxYm_!o zLMlWI@wT{VQH&A7s{koeI&0)MN&QwBPNB(uPYf1o2O_zYLs0TPE{rX6Yk>5 zoPc!R?>uIRLy;;(c5?vL3$(ZD;OczwzM!|z;Ojjn9DiRb3r`R?C|~jZKRw0!p=+i4 zdE&SOt*p*4*N``hb0l{RH}m$W1gm$Dd{z4j-hzX%_ZD!yePXP*qu1EJ;aaRbAvml& z;hQX<&~8@llCFNyvF3p2zD3|(GETiU=2#6wMBjp~>APFWdlJ-aC2pOD8(J4ZmMUQ#Z$D?#qxY zNjw}0IFZ-`b~5Z4#9c{n4_1bfsyo{7pN9t(Y*CA>mv9*8b905P7lK>Tn(UjVFCzh; zn=ZBU^XJc@g&nQDxZnB6~FZ`6Gv(#n_4cAJ>0llJGyA#d|1%c>#_R-TrAeb!YpMtmMojd)VZgY zM~wU4NiJneqwg!_eiIxM71Ycn_Kb*a8L5wYUAfR9C;u*UU7X5RB^gV(uVI8xsvBlb zK=3hHMSUGcsXx{d)2MQbau>I>UT}P$E*?*d6JHwZ=*M_yo{vV_>{IOFBQ|7QG?}$u zDPoJMv&A3Cpo=;WWI70DT5Ysh)5Y`-^jLT(&Zk`4$C7arO`qou#YcxC--wd8Cah$t zk{cokclU0i`Ok&?%wq*>2m+6^d%_rPj^dd-!rsjBHeAtXV zfdRuJ*K=ViGo&OgdkQ0fJ;n7Z^<&l~N(NIB;|MjyL_;x>BPqsB27Yp1YFf1{`eti$ z`tJBiXgpZ>^Hqn;nE3WJw6!gZ-GU@=R>!UwU(G|%vSVFq%gVmsUX}~SKK!`=!eQEU z#@-`e2!)$xK-t{M8#SaMzBz&1HY`~Yq_&MA=(=KeB`$B-P z_%|Lp13*0dPMz;63}W7~K8%Ah6tOHqo5Oe`1B(1&_&ynjK~lgeR*Uq3u5O zi;j$b%Mb{y{b4qm5mugSnjXV!)#=78+)@)&uC15`qt||&6IU8K+S)V0KoBoHNwwQ{ ze|@yHl4b^TU(*&arF~ za^%jvKk;%~n`Rp}NTs-%)^=256$THFevoV>9;lP2BA;%s&TTZ6y)laiO-|>(uUOcb zI8|b%Buh6n&h(S9gW+499N2fZwUd{lIbRJG))m??x7xwdx6|yd=`KgQfJIh}Gj`$R z5LBgb8O=Cud~>>GRljwr!pT%-rY30IRPz%@Q(h)b!m<{#McTnA9W|1;=a)MhtO=Dq;P^#c*PwF42mM<-OOi^m@f-EoHzK51iN7-(D*o%LH) zHJX}qS}K9r*$S4N?ixi?R~G_AD!o*b=jsuFQk#n9Lt z8hw{yc>@}w%65Qkv?i-S9W0+B{XuCelXDU6z0g3|P@W~a41TIGTT}5W>j?Vfm#g#) z(Q)DYyArL|vqR@us6toxJGWSccf01@v4JlJXK~23gx7U>=r)7k?)LNFoEJqmn4|Mw zGp6nTYR3HEPVPcT=&@2Asl>Y1Tzd&Id6Iz5o9K+TddDy-jdzbA$E zR6wY~hdB^IMNsXRM8FS{@h;L@+^*Y3t*L28+#ae# zP&8yFQU@dG1|x72x4H=F92XCEgL6&v3kL9hu@8~|O*Nadnv9}jEETQ#x>w$UY;&va zj+*E(3(;Y%^h{4;eFmk!yFPX5d~?mUMD@5(y7o?~(L`6DDv63N#)GJgn2!qdQ`KZK zl?oeixtUYChSM4Gw=SpE$AskTXnJvgF} zsoc0dtWEb}c$$~{nHu*Mp71GyzK1wqT6z3=N@xkN#_fLjrdw;} zDo{xjl=E|kON&l<-7!%&S+UE@dXGfx^<-u4uRTx2=hR6Hzy|SksiehMQfom%#S7>~H0btF z*VMurMdXz7pu!?ZuR8r$*$$O3R1`(Zs7qLpoiQJ6F`b7!cFYq$S+BY)!uX1v{- z@~uuyPIYZfQBQg2q|dTHqD*;br}G<~%x1hfnSOVr{|rv)F+5^Sf1)w}w5I#nob0B( zwVL)%4_Kq8n;I}i(lm~-M(Uc@@6gOJM8cWY&(pA_52u?tq-$KUMCzE{|IxT&kA#>$ zFiQ8Me5^HgXs4-(8%}O$lSEgWJiyk_rjEQcuGc~5Hg@Qy`6i9zpF9B9*rJH+9XqIP zXp=x^pFBX<*rJNuG@hr4+%$0rrNK`a2GYPy8XjzL7ekka<3M>LzqOK3@)enr4J`0=(*_M{t2M;4%a)C-EW6(>oR|pwJ&JfZE z3L{N+RDQ(+tc7%UkDfMC)70mDkv1dS+E;(&4KroDtDcv(5s|(N+*qb8w8sJL1etWc z2MyXq@9zuxu9cVd753*;BQHQFXaZ|F*F|m~}N6)jBo|jiDSepdVPu z-42Gv2nZ%?>ZYlW@=6i7EltVgj@Vxg>JtNq@y7I2T_21cX{;fS6bH(hxGud<1GELP zEvPRV^jm1J7W9U&F0qdn^o9hHxGuKuH)zhUJ2wAUC~tv17+^hQKzd&-BnP5CC@2uK zzces9uny!AGN!0LuKztKIkLS*UWkt=P%;=)mdu_e@G4SUa^E%R4bdGWFyA_G^A#~L zy@oy+oYuH4$(;1wEhs+8oai2+|0q>dd)0m)w0pL78_APAse{z&NR!`P7CqI=XZ z?&k0a+=+edpmc=x?Ec%J_=NU<`lvy7NbWfOe+UWN-Q9`&CD1vdZAk9efoqZ7qx-l) zdqn%ZkTDJYxn8osx~}Me5m4Mi`Z7VkMfOl&Snj-GK-XA7Ykt{d0_BS8cYp<4DZsEa z;56uiZwly>_(wu%`~R5)MFkFoge7N4=v$*noAwp#`#{7@11r9Q1y)0B!vZ2oMRuel zBAB%m>}!NtgM=kyV1+qZ=yI~yX=ibzli^G)$&xZUU08ebc1Nc%mNV-QzY+wNLv&B? z(*o@wyVF5euKxmsM+TUo)0oI10mRU0%($Vhi0%3Pc|kQHd8GEpfb9?gh(M_g`mFg? zra$C&yudlX^R&3@X}fjv)PcEq;eJ{`~(Bmj>8 zC}b1mC+m010RQnV#V0fH?mZmnj3VImQ#AOe@2BF9;H@3{CtNT8F88e(`X~5S9#AWM z!T)Ix{FACzc()5M1=d3S=JpfX4=sOX@DoSs+f@TS6ciC3%V4;tMvL!frxZFlr4-t? zqawyJL(y?BChmBMMU5ZaRTO5|4mg{sCAM!SegI%ZjQ#4uUz$(!%Y&qjzMi&I6(;Zv zC4PVSK%I}hRUKmdWD|Khy@bjths5DmKmvh;C1nV!RTNLFv?n)O|aNN01)e zebb4_|5gpUh;&hlM3*WOU5^k_;+w39^uC|!kg{(CaS77=xpyXU%<(IQA-I3~CzlB7 z<8VZpNOxoS_SJ&9tY~P*sXU5}(E*SyNh2Jb7VC=4<7kTc5fjRfOLa68g)Z zz9LI=!p)$n567T?aXTsx@s3K5Y~qWsZBL>52eODehhAcAKzRg5c5Xf-zn>-&(03a# zwuImcqWUDg@(@>#z~8XGzo9}WG7%h+FQ3-mVZ<}}J5EryJig_`EsE#fz?%Sk5p98D z7^q`nDjn%PWZ=|?;)u3H_0C8;wV~}I&R6--Zcx|9z=*b3JLRD-+29z325O0Qic0iWlbsMk4^}#&<9_v$ahHKXHBE zlex-6cAw-TpR!E6Q3Aj$4&nE`i0<<79Q=9W0e)rT$TtDTLh}+>pTOXKf6VBN2+daB zvAbE>z_*z%eORMiDyA=CHkw3Jqu+5r@NXq)7(Y`jsLP(;Dwi|%VM(?2S*$1O+dz z+0YqbB+4$>lQbxHw9d+ZYJ|r zF3Xl6`!c3+7^}>7!xokD%Mv>tLoUw@g}(h+#SO%Nu4+w%_!5{W%u0lp6KcwZ z;1ypv;913G9Ij5_^e>=1)I2UZ`Y6chDQxi?mVp=53f;>xjl|L7eaUS8j@4wgb&ZGp zIT;05-+84gI(RhI=K>Yi^B8&Yp!W3}b;^p-@QWIYabl(6xa@;PsDYnt^WXYipBQH) z%>tJL^-A3MS~cL?tT%bwT@4yN&ALqcV*rD{>n9)MbX5VCOYlY6f7&VjK!=>gkmTXO z2zEM>ytF;lnR%&sc2Q|6BGPjGITS5ox6L_WFz*9AfiDf&DLJaxm01b@{gYnC_j57I z#-Sai5KhMPLEwr>Hl{9AL@9HFs2|s3*~Jr%J)m8!9B+q0XRZ? zffb_?J>19BfRZ+eq0bfOHSuy)dr1pQd)-sD_97pmIo8U?xE%Pp6nE56P&IxAkS@hl z=Z|kB-krMm9E<9`}7^q!)6?Asni(9VMx_%rwK0R3VB*W5Yv(#rhV&Y%#$P zQptz;BLqr<`L@G7WlOg03@s{7u z$i}4S+zlXso=-{S&csJ?VLgtga#He66~V~0h*>B|CF8)$N-z{`G?|P`Q=`*{Sr!qI zZ>G=Oz(85t|NN2l0*|+Uo=1_zhDfLc1jx=HXL=QHRSI+nqbTL@|8+^kxUM)qIa<7j zW7_OI|8VOjq^%MtfNVT9#$~V~Qpac*%-8303#5D1~AP zD1^U=cO$jevz(*KQsW?QiuZ7aSB?lIy`M!@F3HYfpS&NXom+kqOy<=7{7x+OA>^K5 zURZE$r+!vlWnC$Qsq=6=2r1s!P+B0{xhWy6mjAm%b^PZXwp?={RY3yPO*a&FSgiL6 z{(dqh=KyKD61{h^Xa{PS>(x(Kncq(N8_2TU$IR$mc`>$gL)gT5LYy*_?0g#M&~;HI zX@g&uvpX|_F*_GBFu!-P5c$?|(PfD7G-r{E|9sv=qeyXi{FfBHgba5MY7wWDrCHL5 zrJ{x1x`L9l=R~Go(~;ul^HCvLdy;6Zq?Pj%lmEz+v{jTtj4bQ@Qd1Zye+TAiT@wQ4 zIoV8P^JQVvB7$(O1i_biA)AcTJweuqcxb{*PRNYa=qwUl_@W<30w1}@TBZ`yLOlOZ zu30JGFc9jh)z7Yz2TS_XeN)r$bxO(=p)=~L8wM)P*Uzb~p}1gGXIO7(47mVmS;9V< zg_Q6$gK-q?*;9LROUU|tZleF3uF9+Hy1o6grB-6?Ji!lDrr^&~_J)-!c*JB$P2pkO zw9#6O+rdLeFKcm$Mttca+o?m771+ z{kOkeD(zm$!g>4z&kwL*%nCzh^;jCKn$rA>k*kX}MSg`|jL*q9%USjm#teMiAA=xf z1(`C><9(42+Yru)F+6R<9UHB66^xsDN1>33>K#J5$-p&4+KcMoZ3OC4R9D}DQDc`E z|7v!(ict!5(8SZ93-ydz2PX5?^J%}(D_=4H7|T;LyzOe%Eu_ZibT+}}@jVDaGa8AX zHZw9}kv?C-0C)@f;2@u}8Py8X4sHt+mLU>iOcFYYa3BrS&hwKD5T#hQy0x6-*v}o@ zW!j>OxyydxkZf+Kw5hd~_ADQ@TKf2E*(Y0C*+ZyVps_={p!uX8spxF13e=~;N-w)q zY+CVr!MFL+!kibb@DHvEHo zkkOchwv4O5xA8Y_ZI`2cdoSSVo|Y<)bGWv&V9}pAAua7?;8dkqFCs*l`mvi=w$($q zbF~hYKti%eK5WYw`EV2@4C36}@ZeOxdz)d}$erR!A)kLUINW14eO{ zU1{@|aRbsfsGHJC&=`~UI+8S#)B)t^7dv90$W{n64lj@-loxiM!!M<@f6unkn#OSZj>mWT>f)I-kg$#a5_MnF>WceIBeK-<`^M3za*&!Ur08E+Hw-AMZy=O z&5jM(oOnj-36%Q#Bm_+pN`c-9Vr(AhaSBKCmmCpYsA8LJ!3I;*)*D3U{hZ4UxPxBCkwUZAOBAs&94%UKnZaTDMxN*y`us@O5+^TN=d%2c>q^8L z8db{~kZaKN*wx>&REVsN+%LYwvpDp?hNHvr6YJ-W5!aAx%IopvA;1rb12c=SJAnD` zUZdJh{6_{qVKcmQBJB0TD9(TmTYSsLqx+Tjrceksx=oZYP_2xf(qHmX?S|Rd%9j}j zo*knvm@(L~bW0c2a5#@s>eK0`)xT-SpFmB6nSX&zGFl+JoL?aa`maC33Hk^Gb*-aN zz?>@Q{IM-SN7zW-l(Rq~!rV<&`p6(x>t^H4(3lF1(8RAIIPCnzIn0cSX+t|K(7K?* zKG@q(kMH9MtN7z8o-Fv3%81DsjzOCyD3XcJ;u&bZFn<4hh;2$X&W{!~YBK;=j!-Mo zDVw(c+L`%8A}K3mMMx7FL=7DUesAYBS1*inl2@McuXCvdJT3c%Dt5f@?Aj&8tz|R0 z5V1w9CPaujd30omW2jH}R=0WsINF!>h9wXiyNI)ix-Nu4 zO5@q~=OT=8?0H`VlPoKJ)e6rtT5MCOn9tJJIciSsghH{l13g=rV=wQ`i*+>n`WXvC z{tMz4Zx@o>0Xuw1$In<)7AoU`%jY31I0^U1Agr_3`4>fC|It#|zE{fnPR5;yEU6~e zuYz(h{AlbWgc5B20w*_0*b&^}Zu~$Czbm%5{x)a3Th>ur6E{;cyDt*{j4)32wwUi_ zpg@J}Z!YrS9>e*pOlmC!Zj+ zU)`;I0pf>#_(=nWWQ2zrdDyc>jO{aLGkQ_2Zlu`&Mx;OwF?TT@FFeaDo*!UIVC(}u zOoW)r$RhY~EAc~Cs=xu+uOUc;%yC_FQFy>t;`2rNK|!7?ZE6|iwO-ha!X>sjdAXnp zqY5${pLF?A$ld+4VUo3TI6W&1xGIt?$#y&8(#^x)xYuqwxa`;GfR(ghS)gj=`&5x$eBCLKL;8NzzgA!;Z)GF*EHRwto(`Ql}O zsvHQzU@A!WRMPAD&p%n;2Y+5Z!~Lpw{d;S9y&e#nPFFv9uh)lmmvjDN1=sDNKT=*T zcXCaqVxMr=>7%`KIlI_$!M)|Y)gzC#UlP4@D;Gr3JH&fWW>|mj)#_hJ#Q_9;o?bB8 z*1HJ9Dzx^8x@-DoUi;deYcPMcx#1j*wa3-&FigrEEZUr45xL~<;BhYloD`|Lw~}uA znC`k{TEjXFw$0Z*@LwyY(CUV6H~x<=10JqoFkL-3W@eu*L5q3RLi7(x;l(C#5WoD< zWc`i$;06^6j0A(v_x#)CAtL)BN`*jDWJ3u7`B1>($0pxp^);dK5q)NZKl2?~)mNxK zIy2q&E!#PktkD)Vrf@Z;v@KPEi}tj9>aia#Y<@%PGGL1P09;GAt6=7<7{tp)j2SUs zRbsolZQZpXljMoqtKbp}1m3$+PMnp>=)1$+^;v9+3|?J%J39#zED zYwjs3oSonVk z)lJHxU09jm`$IRA1+@0{4TjK)SA>e#_;0fSU)mHz;1jX6B^wCOO}2TK18%V_n zj=eOerIn{8o&xjwa9}7~Vg2NV7>^=iw^92qpan@iFapB8mfAFA4|1t;JYj1jbsbVP zW*R)GVV?XTvl4~wZUd1P7qntH?Rwna(dnqOt<70ev%;fDGuCpV(P;<;p*Wc$K8e1? zWTm@|zgD&^WGxSgzZ3N>Vzq^B;|+?_y0G?_d&*Bnb0S1u!7T^l8&HgC&SYKPz{48& zwjlznA`k4CUT*-)X*dbd>{c)~+R!)eZ$Mg?I1sJGPEd>{H+7P=|8DwDyN

`&E36 zow5)}I%gDB683(u6LSUoZEs;l)6tFRy2Pffvq9YtUxz(dePrWk;uAPnwvq7Oth7dB zg%Pd9%o~ccDOvDq7QdV~fH-xIarlbsJ!`8=l`TX&Wc!8&wqJ_&Zif%x1j%O|Vl+q{ zk8F)S-IQ!FQ$)ReR=0rXY_d_Et{Hzao@(S`8GeHZ5o8fFZ;^u&07D!Dn;eq`)=|zT zUxfu<+0HP;^mv@DZ#oD7T2wDPg#aro-AjnfSukG2^tAjs)DKvRm8&Mm)#T@U#_aR_ z4K?z|h5?c852HQNNU@YYxHh!Y9#a}Z>R`A%*vK69K6V@Qjfnd{HNm5MT<&1U-{3bh ze*1FPkXwBr{d(*7_9)yywx*H$=&sOPBP)G)*3?_$wxM3@-F1=OAYN;BhzBJ3$+^%F zfZ;oaT`<2xgnpw>LfnJd!NX5Hci_kD(l@m`Sn#t`&`^vbiMs|CEZt;fcNtBDpCr0K z_9Cx6+N~hBqOZH0r@w=SZfFAW+Ci~}(7XC4IB!Fy{?ihpJ(xvAzyb?)k5Kq$SmpN& z3s4WcOfZ4;;{LBiX+W$6(5Godz^+WR{aRBBtIoZc2pS#o^yd~+>A8&EX$vOJ=Za!D%F-KK@Vo*%Vec`Lt}U}!>%NHPJ}Jc zFNPw{$Kv@XdeGU{2Bb0%Ez%WlK=*)7yZ|q7AVXb0VPRQkJ@DCV$e_GwDB~2Ak_OV0 zDTx98U=wmKKAWh~r(?Fm4jk6#KQnZ8qir)Q5|}=uBrXo=Lx8AH*h=R;@|G-V=wwzS z=dkB$$MK|WT2xd<1Z%Uzss{X4rfYbslFG`;NqqAL*Lqiz4}CjGzHRv5gO?!8V!-w*R_}W-T4oU>JQT4ES#@0+9OD z18*#U-`P<&2kkj|Dz>pF#KeE>?I3O$O~bc#K2&IL&o8^~A~@Mkw>I@9u3Y)t`BG~I z%i>~KG@ohPj_XhN?i#Z;H)y~XY+eTQsb>Xma^tkR!9ST4Gk{iGkY+}1YJrc|J5rcE z8ZcIZ5@@a#TSIC0r}Rf#b82pko9oy%RiW93E%#|j2P0_UdCy$zbBCDOo)|%MXIGo} zy25kk*cb^a$3k@t&t2xHaVp-37wqmaxE4}PXo1h|T5bG!GyTJFTeeicCJFt zwPZVNeJYZQ#XcgjIY&NXc4DSET_%ERINOr($Sl^{P@H79Ut^h$2POE}AdCHJ8Z`h* z9Gw3oF^fj*P=wV|NE43Q0l=Sp&Es4|bb6Wg=iN*}QkU9|C1A9xdf*c!s} z;CZgk_D}ScsC(A-lzeG=`u5CwBX@e>_V7r3_LXo;q|W$_nOdXO+B40NtugIdV=Ql< zrUs4|BF&T9B`jQ)wYmQ6hdy4-<4r{a;#M;r6COn-x7&JQMVgeD!V5!4k62v07*XyQ z9@I1^6Hz#1y|9oRVg9f9IVJHktTR!+?kU0}Cdg-UN>1loX_16dx}Gzv3Yct?b(pwe zT~KN-sOux@?Y)yT+723$2hvj3CCxwdCnIn5dX$uo2iT5$v$6<6X6vl@4 zE?Z;l^VMC!w#POUhPMS;9t5)b1bvDxNF7OXb7Wal1Pp6Eki&`ov&SF%97}zfl~h+p z!r4SAjR@Z%$4)GN9fzVyXL)bG%r!8R$(fG&4w=w2W%hxlLw6~KZAY9< z)IZVL54pF(ZsR)%^qD@8B-iBRCw#$SzV^ie8-3XRVFXGm-CDQuHU(u}rdBgrqIVrlAFgP<)_@l2US3C-_Hk z`lGyY?jRm*83!DV;S+#*cr$RkAx?dV48Zcin%)DC74j>aGq3^;i!Yp^oD^3Sz%{_c zzdIy3W)}UO_Mp;`I^6_G$2^qsH(4OIBS_@Y?6GWEG)gTL08ODD%Dj~bP1%sE7Y}%& zaDA_j0BP4+7!Vf1IaZ%_-?)vZ{iEP)w`n*~b$Y)YzyDT0*gPXwsKtxl5MO#|k8vV^ z;mh-M;AFr+A2{EIi=hLw2=J!pe{t+VOt}-0kCsE=M{+YIvtt1eL~{S$Sb}NHSF5a; zP>fVf?JL(muw}VZ@opFFw+nMHb+?lUZV6Uzb=A_kVQFbeW$tQ5ixI+We7g&c8M5Jl zYjt3CK6TmKdoEMbO`Gtsr?KlpLlseFfIc0)gH;{mu9mYg)x{ZAa8@!0R|=+V)QegWo--al(|P8^tdP4PU1dgE!Qf zu?r%K+46p}_Jo%H$WjZFqy6gcP@MGUkGw}TP;%S9oXTK#JIU2_7exkt%z_-UFZ znQ=Kcv#ZH;Hh;7bYs-a4E4^~1efevqc}oi#p~dJiJmq7lZFVSbmt@CnWlZ zvHcV~;D>KSOBbUCe16C&4`v2ne(KkyZ@4mhN9fs8Cvap4$^oO(RG~rl_$0 z4{PrfU5U4CdshV&+qP}nwr$(CZQHhO+qRR+icv}B%m3_s+HK#%+4nr$r?pyJuVcPF6n%JK;HDp@Y`+s~XbF03RsDWAWhb3aYh^dg^eiwJvyoQ2RNK~1BSjkH+6^q~13jWHgrM&|SwC4x7@5@H5g1*rCS{Tq{&?B<3v2k!+J z0InH8iG$4TB>d5dcSb1k6_K$|@Dw(OTR*;4sKX_B8P_OkQ6bJCJstSC((5*r1nJAl>9I#`@q2V+at#SJ|-^~>K)gMt(q1B3@$QD8C#b6#x5 z`s)LudRM)jtWaao+7PGXCA|Dfy5w+cY%cCd5DpwiL@nq z&L0=*ag6&@p@L6Ve;$Z{2f>1OI}rdRqAU=r9+{~5>tRGfoClu5%?@3%yrIBpdS3rlGXI!}Qa5qEM})^U1u(MWEV}mz#wW3axYxe|ui4 z+CQKWGc*bloj`dxTaz4(>VUWbau%)XKvDtDrtoJjlCQe}$brd1cv(9ydh=t>mUa8d zTWA~In6uF@2Gp(?qai^qh#n0pT(*JQj9it?u~2LIjFC^t~riIVi0bz+VQ}|MuWCN3h8EY*;!|D^|Lu~AN_$4VHem=C8hjR@h=2ro$_6xv7&btB5rNObl# zCf3|&V9%#+nSMdj2R1w_Lawx^5w5WnvS?!$@U4%_U$CrrJZ~k++B3=KJIW!R&PfsP zMwfppEPhp5c8}yF5W~XSp@GFLyKvz0O+GE{ys6imy6?)0V-D-!6Qv0a)1p(j9wko^ zZpSdE=9zW#Zc5+APRiTIu4h5sbmKn<5(=_=vewBow$4OYdHqnBYR?qrAg7Za=Jj8v z?InabXhKrJ6IoVp&oqzo6eYC8Doi~xDB`(H6<#JoaDk|c&MtZMnO)N5vs~OkGz1p1 zANqEviiui+Ip0xBzHp4AeDOn)AV3;HL+-f9FxLvnn$G9;V~Ov}V1Kaw6KKa#V zWX&8LdJgA3-SMcL-T%JbY;vU8H@W|qp+fcpg3O7qreW-MrEVq$>3bS|0(|&>M+_bR$HCUq-zOV$Vdq$l`_iOySneHz#u# zcX*9->@Oig7nY&hnVQ&k#NQtA)KSPD-apg>ndR|Sdw53h+V0SQf0g# zWrg??|Z``u_7NRAGyk}eI$o+!XG;y`2n{qAV-S;A!tBe z1-5`F7V+Qggr9Oc>urPJJtderl;;t26bn1e1O`Gm@y}2)%A5Xi_m^ zwio{*%pP*!EfNQf$aO$Py`T6n@6U!Z5nFu3dOzu5-Qe}5w^riJNJalV#DyyO&-G=& zzZ?i(#KU~nUn~fEA|Fmf1Jni$ePorbU=4K9>fVQjaAbvRR+x@G z0lctU6NvXOLzR>_JPuGvfOQcyq+8N$2T5P$OMF{zqQuwOTSjr{sD@q{W1naec2Wx* z^B9I|tH>RlhM_ z;4a8D!d^rA4nAYK*eAH88?e&~eBT1ZAc0L-vGt_33n!X^1p~?G299#z@OU7{4|gGf zPR`u)o5 zeVq)XE;*vY2ge!7#_bWOX6H?R47FBALGWOLUZw|sX&LKR`n&(mGMZ?{`vAInz0!Ct6; z2AwcHN&vUN+aL628*=qwtYD)Xs4pQ95F#5s*q1@h1fDf z-vTaF&y?9ZrkvAS+II}zA7(8w*3cQs9N*XwSDlO~>8i-4A0g2~qHa8MIzaJ!(jO7u z&$d${-}Z{5`uN!pcH6M0xQe0p2orl!}avh7^9Mr7(9WwOI(q!lZHJ2D|&|&+?UFO;hQz z3k{SVRU?DyI{fb0Z)u&06;Du9s2;R_PGdRBgU(!?s3{$2AKZ(*ER59Fb1iw2v)niW z_SwuOx50!Lc7tKMSb)1oCCXPrA5u2msTGLM4(y?FH=PeN#2@mVHPq7K;;9 zCWW3k%=CJFo==RQ^S(Zwq5E;YY4!tuNcq$BBg7-rk2ijyhwuopDMr_I7VD`?}DZivtg;(qw=UC zcI0*(yyJ_kM zk>opIr8wb0138Hb_AfEUCN!=J3Du3HD8u=&jD>Mm$a%ewhLV& z(!@gv`Gf2;jJBdLQLQ#jS4yHH>3JzJhxZBrgaPY90b?Ik2NjH!DT){C9#f*NoOxtI zAK6ZImCz=eGUo7d<8cFX=l(4sE|w5hVVU+&m#)2;UvZfe7<>uL$$hO_X^u7wQ7>t*)@R zErIDLUGa>4yS0 zu#P+NLtM*}Q(2QzU)s*d_A;{Y@exFz`83UrwRzWBlqzyblrwIiD-ib=B`r;#aLYXY zdwmKllu2~6VfeTO zwMl3Z=IEYQp!fxOQAY7r)wH5skx8I}85@OSI0CuC*KcZ@AQXJ?^YUTWl|lLY{WF;) ze~Mjs#H@Vr%tf&$s%XUi9s_z@*Hf(o7HLP!J1TeIX0cDP`$+QqbW&@Z$aB_bf$U#5 zxy&pgtI78*KKmczVCnyNEy=%c@_$|7HEQ391}pz08ZghssoG0{!*)6-2OLbTM_*pw`aSk-{mw6wBX7L*{9#tM|T1e(9hfHqrdS*bMB zHn-f!dCq##Wk{nNgnu6SyzDy7alB+Z&3!wa#`e4b`2HBLZAcIZe?+)R4lNMo!r4g= z+r!O7xXBLV2y@Zyz5&xB-Nc7=gu1AAYzTGX?kdApgtFo8I>T7B$A&ONnrM^tu?Li) zaUBt^BtvG~^#(0iuFdTS^aeabDA0MXLlewsLb(u>LoLvFZbjfZw#Y(GMIc!_a0!wvom>>(B+EW<+_2JWa1qHo|I67IpFyhB!qIK!-9 zqSwOUG5f0EqIRXhqu0n_vU;7tIRGd`Uw<09Z6ZJl_b;SgD!>Z&@2?l}R~p+?02l7x z+_Vqip`_RDV7>!U2*1%Y;txm>m z3T{Wr5+-P}s;pLnz>TU80n7G*)=I0u!`LYtS=%t-fIV1(7+m~OgZ_A%FkDDDPkER{ zCAUo_L#fDeI&MKkVRHm(%!;1yH`L=msY}_f>=erN<@Li;_`zlo_%8kw%3Q&ZcwOYa zw?s1y71rFqZ*Eb5kV3vuwSFN{!pU+@f&fURy%(soH96 zd!wy0m>P*JLN8f%o_`hX*5f`@-l>=2LfEbV?WX^RLASVC8QRsSt_;~kq1$1%#K+H1 zpI>I|u-$#LX*zS%hs-Ds+X}*Q@KlqxO{f_*s*wy(?o!YDQ85EOgvo+T)I| zv1U{q0P<<+zJcmTJ{y_i%Y&^HWUzREajskV6p!&!Cra}jtb<1y8=Iqf#(_mRu>(gl z+A*@;cfK^>u<(M=sZc{n>DL*94Lt@(bYTNW#wn>oCzzQz{-VE~g%ZBfO;%-CY?KFf zwESYGC1$!g8DX84;0>TC@&miL^r4t?i@9WCT=UfQY+50gfiaOtAK+5CKxzX8Dh?^C z#E?G^RwG6za^B30O3xr-MH~RjqKtPDZnrqdu=%5AN;+`xz(ry!{w=VS&89T%s(fn}^_xn|9woFxXa|3}g{WW88@kaR*x>9RB0KS6sP!nGd$Tas^QEcvkMsM{+_W2@(n6=KG zIT_qxb4-aslWR4S-RZT5)7C|FqsT>m@n*!uQ$o1pSOQaN)`m2gU^i9%Xme|jStWIDgi3U9QOKI;==`9{FRXMM}_kN}koIVxg0Gdmwcz=TgZyKr#BiUjps0iLy7 z)gd>~Mx(R?rc)zR1l*Zn&2_R!R9&}<1I|FP@8vkGM)D2I3WW6S5ckN4cH>l{{mNwP zw{h?4w3+hv_gbtWU2=-ZD>1*y4y=3I3SQi#fpEdn$XLQPYp|y*77N6+a+J74a5-9W z@U}m|YrjC=Yaz;WJ5B}pbp<(}ssRft`~mZNprx6EZDDjfz=t@Y-{*XY0P+M${Lo^v z^XPocik<&}Cz7j1rYpOZI&G8VVnM?LWOh4DzBdfIGuT%jTG}vzKSMruSn+gngUQ^4 z!G^#dD0HKd|ycoV3{f#xipDMCHcgn8qZxZQL1U> zC|JWHktH~w+YYSQVl8DT5{odjs{~d#5s;D^Au}Xh9he26V5dOVBHA7amrfn0HrA4wpG5?s#BeK{Xxh>ld9mTOXn+%P(K{8*G5RETU3TuB8ssl}J zJom9fO35k8xXs`~y7t91x6V!_64oeEn;Z=TR{6I-`J-X7wy+p~fOfk7?N9G=wAdbW znX*NLc)X1VrkyGv(=ut?6?1^}BUyvy5 zB*~Dj28Hs((xf(gjKBSFzu^7}pptTfbW-{bEU-fTPop>Q|E^&ZakVh|$4*cD-(Q5k z^T+4J}D)~M=&f5?`#h99Piw+Zj%8F z2!BHS1;O(^5I};8Amx=jjANYLplwrxc|DwRGd=U(%g)%C`1p8w;_~;M9%rXI%8q8m zxM5_*P&3*7rhJD-;FAWWLl!`mfn=l36YfX*uhJW5gA*wSr~4-X9U64nsFJ&<`SXIV z&C}$I4baz;qYsFHwmE1F;26jpB?fk#$Nzv()D{%sCK$jwBi(?BsLf#Esj~{TEo}iE zJ2mxG7kr52IOr2rQGNB=wYrgn+&rUHM z#^#=$M(Q!KR(wO$fz$D(E+A{5>-a-ul{^eM@E&uls;Yhb zu~T(I_sC{)E=GB?u99+FQvUGiH(MY*B>d8KD1`__w0%5FJ~PSQBi?k_!;^7V+b(BL zvOH~{G;^X`OYfzi#}JNA!=@V2d(5jFu5VYIwGRE$hcB$#yBmlusRWB z+$qT*cJ`|z&ve%P!X`W$30GkKp>sxBOb}-Sj(RGiMUf*2A|sgco`zn5pePN~!*yYG zhT6?N7vswICAhEExw%5@#7sr-1wpgs2k3HT!z57U z3t^5xygh_3cL7K$!hq895r(0Atm=6=64X-E-=DmV{%)6O%YXUru>|8<;ZgLm%H!VI z7WiF*tvM^(2*gU_!ab_ZMeaH?V`Z2TMSVo#2!~TdxLzH%b!4$>ITMfu_DZX|(dmq{#REuRi8i&LzSUeaXK3DwY)oJM#$wHf`XU@2Kv z6UGU;RM_ISOcB?|O8E=3CmONO0wFpzT1y-AnQA|tQv=oa^Pf}6n6wW^3H;+nAlU!Q zQz>I-{7rY4F#ZR(@vl>vqNHQBq>t>oTVk`gZfn!FIuo}s!?9INt4pb3HySK>^9xD9 zj#)-;E1EgOLUY?rUST=QqsX0P5o^FEe}61xKMrv&%QbR`mNejD#3S#FBP=PekJsm? zJwj+8C}`$4rVo-al2MY8c3eGDXrP#$0|k+hl2E)V+)y&TIx;=naGca0a%nJIe4qLr zB6~<0D2hGTKbGTHg?x4JGpbOl{0_ZW=|Q^fl6sAuJrz0HZS?XQ@Sbay$%0>t{L@FN zfo-O06qc*qR6TbdmQCG)3%MOS+hF-3Io@Ah*Ads7eM}+a4J-T53Q4;v(FMu5R-i4X zv|*~yh{~IL%{HC0zcNj~<&949afcE{=*zEk*V>Hx-Yuy~|tVr-xZ+#R7XxewHU0 zI$Rhvn1{1^^cZvU+(y@+7%KTX*Ig&mDBYQ7p>#IsNB42&G7u&c(FH1k;*xk~HgTg! z8^7AbMSO=N<`e47!PpvVs4Mx5j*wn*%eO4S8RnDc8p3FN5C-BVEF44T5%?vSFvL{o zAzStb2ve7kCe~fN6?5_@^e0!cTDYXo4UzIE7P4bk`nqT=Ly zr4I~-9sHOUP+6W4xGzUGbh^Zh1%WnYitWylV7>!F>|nPL;w?iAugcjF=*-_=|CD5a zB>EfK_l|bzyDb0b9qs=ucIDqY+W-E4_*WIS{Cl*wWt@;Aor?!Sgdh}#2hZSF7q&sB zwuTTG4FTtXyeb zt=wtTv-+6zvSXYuRs>r1`zNgMCHK!x)Aq}((k;(Dy1&P(8!YYAfSD7ZG_(g(cNqW$ zLucWxF+|7mAvr{ask;Km4ArxG?FnsT{ty`2!rYw@Az+2frUEDg$boAU11RVk>sOF7p}b#fmc5&_Ksp+S0c!f_G;ZGrG+ zgc$)CeoEyF7a=!`M7%Z|-~=<%%$R-UM@*eu6j zC_DfsgA9{d#>ycwmMO*75YS*>)HQL&q{*78^4L!OEOc2E-XfwbsmE7jW`6N1S z&URATeC~-gr$A1d*TawlYL*(u?4}8)jB8j>j^2*;iU<8+ zkrp?wOVtJB3ZIjHUQJ&pdYOUzb6^C0rrpEH7*Y?`p?ETmgb7I7HeL&`Ll-?4Ti9>|bar#xf>_LHs7sHBKe?k5n_dE6e zQBy4V^j(3DzVy{Ob~F7HMv6#joR&zfTLY!Wxb!6hL6BF@E{k?1N~ zMnH+U0JTYp_8)s)G{Wac#jL6Md; zMJQTs(jHCuCNY_KS6(s-NpWqdrC`*a{nB2& zg9VepvXFVNR9#uwSV5g?R6i?R6*GdUZ^3{wj8%9uM!ug>xWMULqxl&=NPYIN=o0w* z%|81m?#5qh6dzEbQQ8*>Az`6$F3;?Op&$NE7BBIAH(_2JSBOG`$QhT)A( z)1mt;tqKe8eofrp=g^r7OzS-8l$!mT8w(gWrE2R^;!VJ*PTmgvnqq320p|Hu=dIcP zfBZ)EB|^*iP@!{CV1wD)9Ic`@F3@Z7S*k)Kvg1#0_1?dLFsXNf1cr!5s$i^`n5=`V zB@8>vkNHSGnW*_W>$VYY>)JH&+_^f^hze} zL`>K^G-@SY%K~tQ89v7?X(z8oipQwD;f~gQ;wLTP8UjsN0L0wYBx4DK#H*1^>R7Wz zD(;w7{Uy{uRvLfbG(J%zY0eS55=Xvmg*qcb<*d zS5A~s)0Q-}Xbmkih9(!|&>B$b5>TR)(pIR9Nw{1~O;)b?!nxBnc8Q0SU9pm`Gpi(Q zbX!jCk0+VN6M$!KEF7lKhb%*O(n$~5CqYdivdQVa_$aemc!1krdp0Jj|Ag2<=$6dr z9m(j0!P)6EP@Sw$Cu5t$ENETx?rj_5^Cckodb!UvO-a*4?GQ-$DO8KJcDj$92% zz9zCkG9_OEC+cBYxVc=>S|MrtlVkhbu`jb0`K<0_%APR=4{NMYXMA#7=^*MT`Sd-= zbuX3fk?SU#-E`5f1h-awKYYB7H0TAOI@%Dt#pA{JQY_Dfi^~G~W=RRT48eTDoax9( zT7|)`Ff+`}0##P-sOXi-a48R!m^r*5K~%o+nRLmBCil$ainT|zcMR%~-Klj-J0;Hp zZ0C_!HYZHm<>daZovS;tbY@hj>Ce~#k8(+@+}OvTGkA6)hc&t+UQ2 zAigL`_N7ELO6@cS??@TgjaMLacD5sVYL;;A7|A=L>if8Fi3hPd+#a>>?sR24n&O+1 zJa2|#84C0b7VRgk;Y0FNK(fgnY$&hC-NXv2j5Z_Dao1g9(^v^Z!D#7SWIo+}BZ@_4 zd>edP&VKy9CvobKo$k%f+tjKbU5mq>B#ko9AU_;sk2RyubA{&+sT9-o^b%1J#a0rY zC5^3sB}D{822XO;6Ia}WBh%cKbRSwmF)9aZ7%&hOrq>vzS9PLip)n?ONw{Ivs-hW` zb)wIeO98SwlUF^74QP4+7|&2@IqiSQS-EWIO?~k?RLEYtS%c?@-!A0Sh{SXTxH}R( z8guHdRM=H+_A~ZIzj=ad_Brn7+%Ez4nH|s_Z!>?FvC~7KA-AseJI3 z)5Ld1&~9*fyI%}DH-)rz#6(}14)ONW%<(inpXBwnBEnubE7|j&b#v?U7QwFZdc5}j zD}B*ChZ|1>^2ZM;jQ`Z5{x5X|Do)>gc4A=@dlOsZ|K_xpBWI2p$ST@QbD5mq}MnYm`abj!o8M**P4b5-n3)f|@_V4T488>V^qbZM~0!dtV2CG}OWKz}D%xbC1 z{v7uAGM;wzd0za|@=J_UNCbMoWQh3^4*i#E_zb-x<5V~N1bw$bx_47U-r2Jfjb1@{ zdD_QG69t5w7L3-JRj7Es>v+Mp6+c^@G>Jo(I$T7)3KFEJ6$qnb|1TOqQ>7`0P{lTX#`k|i}4K&o3cMx&2?bE%2n8b3lf1K~K;SPoiMvMUc z<$y-xnt6-3F9KUOw=jDKr+>CI9{yv3HZ3 z@MBnpdexm&|esv>N7rB0qUxxoL zBr^Xh?SC>J|IaEQM(KM&CxHAnRM&OOwm>I8kT5mNmaZ`nKaCKM5Hk@Ha~MR=nL?V~gTabc8&)-F6l3De%UA=Rl=P z`|Mrw(|%|7q@#+i30cT~qFLH4W1vrvKh*tdQgwy=Ucw=!fXq(QyT`SDC?{=HHbeeW z1upko98yU96=iF434Jijb%p{!R;mw!nF877Dj^k6m@LwMEKqYvN|WW=;+Vnzp)*6S zhH;#zK*J&e_odP#n}617(Xp-JqqmQ;)=U$s{>Klnr|&Rgd9&Luh8Hi9S4j4N zw7>+?s`5qA*=I86#@ireCJQfPCYSF`$S#z5O%4-{F2Gl6&H{tNB(9GtF|82jF-VP1 zOIn&@J(L7|o`Dzc&EM~q{WseDl#B2f^PEpG5H|TVV3GeC{O4E0?JJ={TiFvs%nhZ? zxe{TSgmnU)d~^a{bWAZ%r!cjyXV!E0q2Y$?1GIu7vp<@!kkNKw20=)m={fZMGlgs+qkSJiR|Zpt-M5sbCA5h3q0*VGB5sT62kva1NL79{@=O* zFLxy~jBj@AjYQHCs-YHp$4=B-uiQ6x4`<#>yvw5#g%2Mx3tAgas3hL*v(MdIuf5xs zo;JEYs4tIqMt{|h!O(n|P8cHA=uLnC7O(M;(f0~0J<&lyRC08Qk#^#>HTrhEo%#p^ zV{X!2c2osp&6u0+2#iseTnxq)Vv%S(3&bM|->zjSl$Gn+Vv%IL9tZZixkzxFa;bPH z))IQX0-?SLbqnMUa?xOn(AteZU@n028sRtwmbzc}8*TQAqaJQ6;0d}xHlBrJU#Jw1 z6KKD4HBUC~6WBcvFtSb~-iIF)2o2GwH=rO4>LDK=2I?U^Kz7Fqnm70d1v{u4 zyHjBH7y;k-Kkbg<+xmLq_Im%0-t0c}@MWGy%CD(mHFER2?`&iZoC&DM;j!vZ(f9QGL?bkOW)6^Vh7 zDI`nw8aeTTw&OyLDIMw}Cm#nEEg#MBY`tdWNn9=lgk&R1$b1M#+-gP^VKQdbu!8~- zoFvebXUH^RnsteI28o=}Zt>_*9_1xxGa0qoYR*2*3@$=c$18TrI>(^qo#ygZ<@M1p zBN7HWAyM-u^Y7Q04W+Ga=>{++- z^YCZU`@OVwG$Bs#F1xo?s;-b^dW}n{FDKYj`+6xCCnx3eK7$(bO+1RPyup)vHUc~BcchQKI zok9pbY#glBSV6v+NQr}p1jQb@BvDYZO+s+rR+AWd(g9w08O)zOM8vN3e}I41YVU4t zW@1&N07vnm6N@gk+N%!|4hRS4cp7RkGa_Qj+~XQUj*zUg8~qZmj)jPWBBk{vD)4sa zbU2Tpu5*eYf{!8oC=mBf;~j7rkhZs&XRLI30$Q|QwRs^6_dYnk-m#*{+9~HKK+Qvk zZ$(3!Ht}@utjt!@9SVoUS2=#bWFcWB=`pRxK6W)T%-yZlM zap8S-t*41d33U}CMnfJM*kDm|9nWjPm};N=T7$6Uw;GPYcC)Gl=7+u`U3SdFw6{7+ z>%F7{Fm(n=lkG0~YI<0(X($FXCZx?3qDKdMmlz9l(_c_DjtYsnXF{?L7*ng^whB9Q zW>F?J2oW>RMp=AV+;bNAnK1dd%xBlPko{>{(l=8P(TH=SUwI!M_ae4IHN-4gV5kT#L31+Cj^P>4fmK1Nwi6mSM7eO(r-v?Z)`P&w<`d)?U+jS=)?N3*}WilY@69L&S`C-G-oCh!u#LXx(aAif*tJiAK&~DWbUvjpc^OcEa=IaSWQ4 znIyaIq>- zo~kR#iRH%0SV)LqzuNteJ&V%cdf=~hARlaj?_58>g8aWE`o9Fx{>BKLe#9#W8cYD` zG&;^8Srb|8|CV0T%rYUEt$c5A! zc2AHkko{BcDQY7rv&<18E?P#QG*z;!!RS5gZb>@qSc4KYj3qD<{XvuNID#fg| z`LtA`BUM&buQ4uJQ$}XjZ9S1=Kr%UDD_BxIKw%@=(hNFP;$?0?6|(~+V>yOufu;5t zoU;N#a8tr1qR=Z?r6p{nPSqgPGmPNpyc_EbC#HSOO!{OT1bPR;4LKU$j==f7=sWOSJw#(^xalzfAV~p0WS@}S#zcboZ8gaT%Tt>sMyS8=c>RsPDLY*Ce{!c zYjx;yWvFFmYUO%_t;RP5?F)hX9N0qL=MsIRP%9(mX~9b2zA$Tzf0Y4(MgA7-A?Nqe z_s@V)C%I@({4E5CNBK{)GtK`+2=H$d)&F_-Z&CNs&r(78+G(9gBh|~(8f$D|x%$;C zv(>-O)Nd@OYQbe(q*WGf@lZ3BN=gzp2#>d~r7SFf5lJ{n8QBS8rAA685E+gSha!J` z0GAF*m5T5g3OnN_TKpYnXa1Kxp5FWy7dH-2XdITIeI*>#F z6e#c6@B+IUK81ZQQWJm#&Dj}3`&f{ImlyyMAlD@yo8K=2l!x#d2!*mv36B~Mu1yV~ z`04b-h(?(J-`@WX0OCTyTXKMNh4>KP9|LLw-=?(>iw(Q;!HL@!#pxT(;q(ofV7Gu0-b*PjkM-1*O{i!Dcr=k_x^ke^57 zTsnsuxo~uf7nB-9h{WhwP!IeWX$|2*cEJof%Y3+Ck9T4c(Kr@EWp)3S`Ny?cBnWXN zBxh~c@jPTb{>b}7U#aeRScBd|b90=+O89X)?1mr4?XnjsJ#1z)4PFF-0N~L9% zQ|VqqOPHb)BP(_&LrMo#qE>4z($Y~h$rR#&gdt-~5QTM6qHSVDU~>zt>y$s=Y$KElN&LnY}0&1v=^WY_NE zX%aXEECQg@|QDJ z-`6XFV5>1CUS`7>q$bVInWQc5fF%yPf%b>%mcuuqWUfZpDs@y#ajM-knExQWRMs z5m>mdjVCkO8O=fvp=^MUZY&%zHDmyq6NKPU@G^;$FTafe>udw}z>ch~vmj0gA2}?? z=1M+&DLHEBy>{A~@=N_Yigp!L%obJdwD-;$W|$`@6yfd>eE=TsSb&ddz&1TW#a>>_ z8Z0kLkc$lie9XeTOIJZ!XEYFaSD1Lo%-Z5FSb-C9fOQb96;Sq|Xq>2Y# z))7u8^x}$!gzYfg+>ClF`k|tMba@FYW=x10N={I+G7Ze4_=@5b0u;k|4n+Da1W{{3 z5izn(yv$!<>QUZXh#LMNwSf1I1G&tkARmKAZ2bLX_=YJ>ok>rBCjfA-?$#KVpzY(E zX7DSBBJor9gsx$mPBB-zv_5UjFaU?fR8n^4Y4PZR2`%HGg8NY{cTk~7FF4^w)ZPZa zH)Tes!1-D$VE}zeXB2(pt$QW0!l+%*qH9O|q|TIt9ZP+fo0uk3HcCY7IVhelm-K~l zzpk0+G7Qq`q5XIx|Gs-~1N#1VZFkoFZ4Knu@e}iO7~RR@G-2eqq(ytRVtl zK2cg&$z|A5>UmP})J3l7y|_wzFJB@@fuXXU4u>Gy|APFNzF5GASws=g+g$WpPVAp) zJBHtBbsv8wK6pdHUUOhWyzHs&&a!O+VMIP?%U^au=6t8m_uXdkWjQ^6+#c44`Z<9$ z$fs6uZ9W%jiZ+=rd%>L`|0Yq*aPmF9|iucdlxBmYNXl_8vz za)#V~k-I-mF?)D{r+o)edn_Yn@A6#$Td8jP=4DW6uN{r$>33mIy4GGjniFHBsSr_Z zF^^3293`c8423!{o9xl1rt-dswh=|%NZsL@svMtNTzP%g@Y2LP_b-}N(M$GgCA6`W*zmhhi3ARbM1?WJA$3UV|{qE?>9CFwoYO75(eP6s^&K6gBOro z3`s$#B6DV~9#|C~;4`4>>wW+XJJUFfWms!}V6%ZB*JUFjqFxB8`2ik1^o^#``{|6$ zE9wyIms!%Usr9r&HMFHsN#8Z_J~wbI$YI#d3e-~OYBa08s`=IQM%5Y1=IWYj02)kz zP^gM8R*_TpmMFjq$M+9BYoizEw&wrG*f|AR_H66E%eHOXwr$(C?Nwd2?JjlMwr$%+ z7rJz7pZ|`y_dc9`&U%;;D`Gy($T>&mmm|mceb$P)N581M!O849bZ~m>K3ZiUSNivw zp9hYY-=`mSliId!oOxJu-rQR3!_JYhO5C5*TJ^4x7Xpd0wE`>i(Vk2?Hn@a~89wKg zWSJRU(%L89(eOT>ElY&5wxQ)EtG zgEd%=XNOD6(GUo+ssA&^2iwf*N9AD$Z&43G3evjifMW9kUDiYxEJ4 zEhNv1gO_ofF;z!d5C!5uQY)y!fpe~USf<#C3$BRV6BF((6YmI$f9hD){<9{q> zxTEtEQIFvD%3NNS&l;T8CPU&|(_O2=*s*`cWVm(@>mSrlnyHg5v>D`QEx(Gc?2}oYqriBK zXxa-HQDh%1)xa*T=~aXV^kqJO(8$skOmtT7w{f^Z`#V)>H#SX(^}6!fA;RIAKl>u& zf%Xw)_a(bxT!|DlCc9mFvX2=qykTQyTtDdRPtkVcv~Ntye{=&kJ`(tL_I*=F=ZB1B zFv9DtBgN_vY$UcwXfVBhX?G)?E0%VqS72jil258o7y|JuC^rRm(1~_v4=OSYb6G<* zQwww~QJtEtw5wL7?ag98@^q;c~WjSJ(gMULj;4q(b~tp}Vy@ zJI4?6;|C_*e-h^Zy~FiCiP3)x^Z(aEtjib5P*v>S!|jjE_Qd1_6i}{iQE(_OF7>yi zO`1j@a_)hsfyxMm8xLrl3Oo~V4{c}a9 zM9Oo;J-^zgP|9=VJ&@Y4>RRdeRfVm_QAovR!MH%hJ&_uJ)p$;YuV_kK#XXZ6|L<`M z)z@Nbk8b&t_vi(h3OVkTYcGJE!W0rv_*dDUp1lVwkJ}=`g81pDL*dF&j-MPzP z6wKoYazHgG=QNnwJ{QL1$p}i@zNiz^02yo=vn{D%H`u;52CZ?bwNnO6>MkKbiJmjU zih?EExKA4+d$?p_&>CaWxg7nC3HL16%s2@qHA;Z-Jz~!f_;O%2tzZxg$mu)JcVbR6 zgavESp0Ce)6%4NKoU+<)n5u8=8M6lie0daH+ZwZ{1PJILB%U^E4crsP5FNs!w=~9K zjfldew|K!D*n+i#jbri^4Ge>YV8I<{3@!LsFbP(v)wAT8{!IsJ!2E5vI_^j@aD#Pi z>Rsg8aROsuy~;uSb_b1z`}Lq#lIST9%+ zEC5rqA?AoGV;(#4S z-qG5SJs;r98OFdpM2vUoMrK=+)-OB}J}`q` zbA-NKKYPj%W55q~1M7427IeTKP==Xn=9#<409(V-*>}r6&;c7@_L6RdFd%^0ZTb?s z2Lbp(CS;_9wf{ze`T~Ff#(*NsPWshAiPl|WC%geUSO=ze$v_j#VHVsgz|cM~-~sD5 zk>FVTlp$lZIi<%y`MvINMDq!H$nAeu~@F_p&RPaUY*Je+sh#Tns`eQEH-PGR5v z+j5-qK^A++$VyCn^VPB2Rvh7ddH@C%zsU>W$4L7g?p(hk!T=BK4bywfo(?eYAl%7; zAcroYJz;OirXx{dtiwJwpaIi=$dO9~W}q6jg^73YmT^EHAmRG{Lty?+#X^H}+C7Wf@chz2H>~`WJ0ds4wM{lY*7`N0amlmaULN@7(Y~zUh z?9MEZ+b_bIvcw6OH&=q7app`FRdJ*(z<%cZP2p|B)p2VJrXcLCO)RT}=Qyj+B?y5f z0)UrR?<-^8ESZJMD8c+HOpuX-t;AYpWM!!`;|$rOeLTD8u9`E0(XX7C@?;x;2b8&b zv22%3?Cest`(hi+9~MWWZoarT>l8|>*>h?4 z#Uofa!tpGk-x(}(dqvMJnrOk=?)$!pUcYLh$N7xU{*M1sD2?D&QD9Ku{NBMcgI>V+ zI4z%?h3rq48}Q?3x`_>n~%QCK!JTJS0q*)hCP6c=8WYk zMl@()5c$^_@LwCM{W}gY>%vd%`X2&h zpi68#YUmFMs(85Qa$`#8cVc{Q{syGyMm~!#=jZQ6*NFhTb*618`TnX@%^?(Rae*%6 z5Ev8}QbcIw17(#f@X}Ch5(G2vW0rJ@VSS4qdV|oCE?l;4FYI-3P7|` zDssK0XIR7t)Tth8=e?^zl7nduL#yv$c1hWrP2mAgQf1+upd(kp8`t5mWxoC4A|hNT zxwN06S{}R_no__j23%NqVZMl%8DalD!nv@lbzhp7~Gq*|b>#coy zH59hiMciU22pnr>LXsUrnK;UVJ?oSf_|(F_DDtkHPA_fJe1Vh4K?4qi^_4hU(;I<7 zwpZ?ziFr}IJ$Bd!T!EwOVdLz%q28#54%#0v9FjqSDjk0ptJ|t+_@%!Q52ufuWev|s z>>}CytOkn`Q!-BrzPW#FT;inSe7qgK2v2M7C5jqZJvoognkug>xvV*T1>i6cO%|I) zJ=mja!zv+Zo0KaT6jvlukR%-I^VY`eilVQ(iI{4qNiZ~f8hn-ZciBZcnwc=z?O+?L zB&HbyhO?L~f+C{8P|@KG5~+uiS^Nn65349g@{Ti1C#*VHWntHBZSJP6qS;x+>0*)# zq1DR+=A86Ewv&g=CTvR&(y4kzjiEw4ML+LkX{2Up>lPPpDhYPnOeg&`l~xS1m_Jjm z!|RfQ!`)}>r$O4zJexhC=`$mos7|fU*NgfY3ZSy=%<`PZ+$J+_f5+g=#Kc+ zcB~Gc8RE&ctTAC^^OF4u)C?caJ&#`v@lPOllA60QeoBR+qE_aVX>l<{brV>%zd)x@ z*FI_YbEu!LkW5&8qCS^3K^yN-Tv(-VBdRXEPEuxW{`LvYmMA{nln@`UwwGy98&7Ve zKsv-Y?oK(QUMKq_*M>L6?dU{hvRVp~`aEHWIb+9dPh9tfk}|(S*%fbp2HP?fz$R2B62lIZtuLz;#Ei`dw(PvU4z=EFHxmS6Y)$f>+@{y>Vt)RuIs5$;t8x#+SCTbF*rl($m5*9Nk`jg8?V)y+mkS6<%WYj4ng^EBpz*054Ka@(^~ZQ;0nahAxdtv+eD*53Y??;uYN$u^$`8_D2W0P#X34eYV4)q+j4txI zT5KMx<;}O8#>JXpRbg}8;vtzL`h#}N4@!3unyqZ>Q?TRgGuiLxE3LaDd(8)(O9Sth za?kFznQa>XLR->FDolH$teZ*v-=9r0Ey^EJ=r;%W?v_gRETyel%{6?H%J(JtQ0cnK z_r_%sX$fhle>{u%CLFwL4WL^KE7{+?Ze|}alKEm z98YJlxOuCk1cs2yp2}AR3TCP9mR7l{?PD~V7b*u5xc10;M zUj=pf(w^q;LN;S#HAh}@XVG3(a63`YyhWqXd}B~Cf8RYIr9MhOJ4?yA0zx^4%@OEIJsR^HaX0z+A`_u8eJ9OP3WcDwyoVB_- zZB~2mK5hVPq6Jbg>W}^yxZ`J{b}8c<*jo%dEVl18W%d&;tEV7;=WoocLr$9){jpjp*#Y0kl+y#KYu95PDakYcT`8E8CBJ8E;tThiJ%k07rf=<&KBi;!K zB8x%S3fi))EqCz}g=ubAYi9HBO=|CEG2PL^r7mOgO4H20OgPXJUxtU7Q+21WHbZ0E z7v<)YniLjllPsZ3PNfzlB}#uD#a9t2T|MikUY6%}YB$G`*AN3eP1Yr68!Qf3QxFTqbBNY76sBht5SX4`ikv zd^8Hbk*|<#VYntPOeXT&lm9lLGL8vOn=(`d@?)ytq&C7BDXHfN*gU&DYCH-lIpnf+$4r60KT%;=gucc71nS~lUO z*Y=)<3%7i(j?<*E$=%Vg=x}6ss1uP!uS1{ajU@Pl%vemcTP2mx;jTfcq*tpqmou1m z3oa7e+)6sP$S9R0*4$3YAzXBh0>aDfX#JI-yR|r(WgJ$L6xfG%F-i^R4^8tanOzis zEU{8PAyd(0@yi35YV38Be9Z|o*K*?Z{wk4dgW2wjzR~oczL;8oAcwQ~kG!`ubUb=e z%7EsU8rbrBvy3TS>NDp>vfA0pD{_ZEC3d}TJ%QNVI`^0FL*-1VAglj~*uBOJal;hi zNnzrZ4ln@p97i}oGBB4iCf1^JnttyRR+_Li+WGQQ&{R6-JD+`(b3CdWG`c9*T3Q#| z^49$*h~t|ha9(3`-MdWFX5Rkbc4k-#p2h{wUE-DcDt8mnseA%@ktuh#u&-l<&c}&U zhW%5v#^EP>)?%w*;WVU@ycju@D0O+lE1V6XaW%5VLO%LKRZR$$q~cn(F&|9x~hp&22AoRQ6ruH?blfF^&Rou?FlrYXHV@T8m=5^38s_BENfaG%(8 zxGeQ4s2NsOHIP|Z%0~xt7w(u7Qssgjs~GByQ)>!frergq+x>xM{d0Wv5Q|yNviZsm z#DY!I=n23U`lVFUb12{_k!HDTwWxIUAgHiwKxv3OF8uaVpk`{GSHLF|@QnjZb6Q1i z7dgbMZVuA^PNmC)tGcL^#|_R54wPT~eWAF>Wu6ARAQ5$g_^70@e0~P=(xMV#3B;Yu zy)x*UQFZ=_wKb*C)n+sPB3${=q@RR*S5?}3bmHP7w1QAF7V`X&&={!1*Ab_v54-(G z@2g$X1d>wz>tfLU zsw++b>$%9a?AZZ$81Hn08)#tOmu&g!vA9)peI{N-U-)XI6t9L83^y0t#qOkN?6WcQ zZ@;{@OqMiUju$Ab+y$8shyTJEa7>{$_lgCu_|t8Qh=xzG(-wJtRhD>p&qG-@-xAHw zmpMijQ**=QBDPTMeH}R=5fjrP+pqKT0>I!WiClM8Z4dwAVGI3URTIwu#xEP#q}pzy z6Vl`IGt3w@A4ymFMwwV9%7-&iqnh;*IL8s52BT72e4a7%#YTrgeizrZ>?X-~?qUtn zB|i!K;s-zQ2e}L9FSC2Df+^lkQkAMALKSZO4;GTAx$H1^zBhEyeG3C0=b!0lcOxt> zoswxu_VUFK5p45!gV<-Tf>TkhTxiojHEq`3FD$a#BzWJS*JwYubAFyvu0fAjNqNNU zZ*;mVjz-Qh;MOYjq_4Gf2TgNg?aao6XX{pw$QZL8LFoK)?XaWi6TC*#)$BP`Nu1RwKCLe4PI7{ z&n?SC3!(sXfL?33|AWwO)(hp{+oLP5q~fWEqZ5vLyK|Gu?_LcyK_nH0)1bBKbNg@%9R*GfRgPN9S<(A9BVYF=prKYx1*IwNXWtI zX=5VzJA@eM*66)Q2J^!PJ=Wkgwe*Zl8-;ve*P90%hRxBFQ4EY=e>((?<2e$?Tmga$ zF>T4zM%$TIg_k~b1ZfbfuTj{q)RmJ1g&2onjt zE!;ps?mtgofrKBVw$0BTwupUzD%S+S>VkFt8O|{vIDzRJ%L2V2M$&KYh!wW2A58kZ z4kzN#3$-+@8+~%tF!*jbWrw5Nut2#97f#yXl~GLyPAA3NWqkXqrABDF0?SVa>MvtK z-dq+W7>RcI%=Q(r-~eMZ?(!v(PWoJ0D5`0lD2e{L1ypR`77WK>)N)?lI#5Gp%q4s4 zc$UnR%T=>e4J~6fcn`-YRlpTxMLi?mbre9nvP7X&0K6ul<eV=i&WLs!EXd7tz6YOZ1(4sBItsQ$pob-Nqsqi&(5Wa$*?0GKSU!2NIbxW{lbbJA`$&Ik%T5m$GW+y{3(RWTv~V1or#6Y zf0jBJf+`p@lVe5^l%cwwJ;CB|OXaFWtnK^j5r>P8&e;x0tVCFsx|=M?F0u`!es~6t zG{8xrd$$qeb>dW2e#MIPoC${clr4DSI)z|REcQW)+%@Bb)(?xg#m0rG5kcvM(&*c$ zMcEGH*yVe{-o~(hQG5)E8!Wi3qXf(hCfv5A0~`E=e(6YuGl&qoZPJAQLMM6=%7zn& zfZ6RaCV8j*9n`gNzD|#iN~vFXe?hF$$)I`{HaqA+45QSlh-N>D!Tp4>#wmKZ%b5@- zG>0(I-z(1aVk%xI9+w1eO*RE}N{lhUIU~JXmn2qA zTQtlWWX=I8rLoNP;A8oBpqgnLZ~cKi{I#?cZ0?D`)xIbYLSLqPT_DC6twSkWMwy1n zRo-jdl#+!rt!>B^YloJJnpUn_x|yl+saBnyDRxGfR%3V1Htq3-7_=yw``v<`aiL0d zL?5cjYXK|x*KIpo+8(_P29lIg%KM|0Lo@9R)g)6r6>l^4S{T>jX~f%Z;)Y?&$D?Oo zvHPCLDwzI=9NLp?M=Hj3pVSO{%fWB7YRBvaSQOi3?caoUm*!W*k+aYt$ke*tIER0f zu1+ZYOqSrdP@^!`LEbtzBvMb%l?jQ4PBKMF${5qEKbx?;$ppR{!2YJL9-Uk+cjy!- zNMJ*xm$4rzBX~HB78<(AK}leR6;9HS`B4IQWlMrBEC2n`4}oP&1&i*VPOaVxixh}5 zc*bb3C{zU5&pr18XNbC0KGdl4XD=S{zXFq&Br_DzMl8SyK#5`zakp>lL|Zc5FvMLa zS%ty258NL$*tC!A1d(!GV2OjG@l#?j#W<}s61NJ|p$=EJ8-$t%lV7_Yr=F`G19w1Z zQ<^Usul8s3r;Y=Fc7}G66_t5GS%hkmw{wA=F@!=a!r_UwVDQieV*=5A0Y7=Kwp~`e zVxN#>HgnJ23D@fkAj`Z7CobNRPG)J7*#>ev%#&_qj&fy8Zx-A#%dXrf6>f;+SDkIz zhW3HL^omROd%Fz}|JFPe8=69qXO}c_>0AviGCttflXXV{V}h!sIQhufD!itOoh6yw zZ7GLLY6r9)ul1fgD%Y~|j#-5OA-;muhu-$I0T*OVhb=}lb%R%KQKJb5ZYK!w(hwQ> zlwx^@-k=#@o5?M$0~M1Hy0!uzj+VJt7H8;kDw&vUeZKq4tCpm>un7j!|Sm}R=>suB3s?Ab@J6R#10p!>WSMz&HPPCw|-s2r@^C7kc0<( z#}**``C-jsiepc*k@>RRm0hZ5)Or&_V`2;)aBz8$r3Aaez=ZZvu#Bn0Nnhytw?CiJV%aWZ)BG`KAt zq|A`Miex=Z^M{5pgC`nX;yeL=Z%{1+M4tvHX17=zqG#>wt>PR`cx0bvdiqZ>V{-Hb zM(IIYTC}KI+3aW%QIJ?LkH!DrQH68yMV&4>>tbol-?Pe%A5BTfW~UEl1V6#G!EKNR}I zTA?xauv#%^hRHmkl-T#EJo$3P*)^pH^kb`^^p!<@BIL!}#uV3wd^4+H^_5_(>IaW> zwZ97@OWVj;C!CuU>Bx2vurf-*sJR|^^ryxNHSR$XJb0tifEmi;ynvpLakmefJMAh8 z_&UKsRM1!&p0*hK&a4VECRbAy*A%_! zqoDeZIuAK_FR>daHCR##Jng0OvRQ)@VmIe#SM1{Q4elMiherCYdpx|2)M26>Gh^zx zT2OH_M`vVppq9@z)G-qlH;CjjFGlGP5$7r9f}}5OR4!)zM939FfzNj?2{9Bekc00r z!uC;?=!bwim%u-5>`x;gb9@U4Jn9% ziFJ9qlWVskP*H|>DdYZFrn*$!L*kc>@NOs>nQp39D_%lUT`g9iP^cjt@hz}DJHL|y zUpS>}G*lW3XF}j`<5+hXQYyunme_}EqjIx7ESlzd)}Jn}k~>RFLkthj(K$ALV4`eK z)kgad+W}N$DAx$i+{T z=@|3;0ct*0Eblk;=~lNjXo6FU7T~jUy3W9qWawYpux$W+>nosYpB>UZZ`tHDYSd4i zXa1~1D+$hU;*G+!Sh2C87H+Ju7_bFo8`cHIu|lYk4d4OG^OnHD>ij*q_$ZD6-*K$z z-VH-Ou;i#ak6NgkK$}~6%OSg4RI;iWBb-hE&o16O3D>sBEgQeAe2dDj7e=zxxJX`K z*~cLYs)%heX{rqTD20fzM;??f(SipqrLdVOvX@}ZFosB$JF+?;1Ej25IOp+}QK}s@ zk9~t7otL1O9|EKhZU+a=A@IOp&QfJeU2bcZN$1TA$dMl*bOHyPb%v6> z9gV!+wF7)a*K-lL5Bgfu@Eq10ty(XnCAw5meem4)O0EksRZ;Lx(JAC+vz^bSXJh%a z;T#q$b+4W2dQFR2S4&ojZX`Y(vsN8$3;dHjS5%i-Jrt>TbdEe8OpEsrONw>@mb-~( z2p3cy5fNj_C5Stg6mnyQ`Y^AanPNbRIsMU^d3OLgc<#i2F@x~CoA_iq(@wdIZJFol zE9CVim=yF`Y0A3nSW{hTOu-pW!+s?_*=VEA|V(@Rz!k0K#7_oEWIRCd% zT&MI&472(`c%6?y*1vr6PZ$k{Kadjfky0QQ$k{=4-+MqCYfjODGBq3zEcJY^TOquL zNiK0yQqwMFsRhoHWoRwh$_~_WAB6$92qs6M_oEniaiPf28JjZpPEc=3XMT=OS)+RC zj7d>9P)DYt2`ZC-j!T|i;L*@+SHP#l89lSznO%H=ZPCl&)bp{ION=iG1G%Ij(wDNi zUc@1qN%ZDmKf!+ME#`K{{ptivYx2#@ovI6gJ<}2Q)HqAKyVD@U4YBUzwwAaReM}da zuWclgD3-jEFUBO@q$Q97uY5vsNR=7~IexOVWf3Y$`i)}QjMK8LE!C)pyMD)noik3v zF`=&U*s4g^YBG~-OrOP7<5pikGB+${#zxq=S`IqX3k?~&Y5{g;OIk~BGniKXCWS$$-U{#t=>|HGedb6D6=Ka31rJMtXPBne*GER^tF;<0_du%@J1xUjg*@QfV0TnG{G*kg zSgxcVS=!7Av+m=RkECI}5>xBS7+wyPq5yYS*VmO1WD}R33nzEOJxv*2K3T@8hiuHh z_cikh(%xqJ3;m|q?M4qzl>+E%FX}Uv|6U1aQ%J{Amk6WMl{baEv84L7!xWg#oLO|U zt!w;JD`lkRlQnH2aKl99LgbuM3ffI+%25lPkDiZBhV~&M zqH19_dEzHZLNbax{FNKc5qD1asQ-!`Kuphe{O=vE`GeGS_5ewn=@gLzMyH1A{j@pi zc_Y!WV>dDav_;y*_Xdi3v1P~s2F4J3Xoy;R=_Evl9yG5CfZYZfd{62roC_Y1z;+w^ zH~`_yMhc^m+fQwTsH;UzmyDIsj zPAE4VV-)6&+l$FQ700Qv6E-g~>sIa6##cl?hG5+0PRA<;wO$N;6vx!St0z%@%QNHJd#srCTAL4K6^%UUW#Hb2=;Ew{RXSBik}~S-08Xb|X@a==j&b z@_B}KT!gVlTiHrfEfPO!;jGT>My5I+M>?YmHF?m!^9Qum1}&)E8Aw+O(5wL$<}j6r zVKXzVyjtY0s~ION%}}jARoF!`l4BP$vUUp7?N4TSebw0D7Bq%SrQrdNanKk23`ha$ zq|icHfUg&yS;ms1nHMWYS@iZfDhLAhB08Q|b%DpU(KQ%JcxSaq;;x zNnQI3*7z@ihIjhpn*fw4mW*GzQdRuiRPJV5iy42gjjQ`Cp_gg68|+I5uO)O%8t~1T zmBkzIu4DTAm|DwJQj2VZXw7Jpx(_kxUxxl6gU6)n4qKD|_`zfJpUT+&CpN?XSH`Ad z=IUx?Z{hM!qi>k zN<7Xu5XP=x(paBY3M^=7Ye08P>n-k*kxCYCS1g}+=PwDV^_J|oGgyyEtjUn1?V`Z| zA;&^EZxL8*0Yr~OFE)mAjiZ{?GF+P)L$Yf0vzjC>?77B9{JE69b3>kC z`96d*Jzx9A3CT-nLZaZef6~p=>uKwMZ9@i42TT3`-FMqoa`Wezcy_hB03C7m&~eJ1;CDx?L7* z2S_wM4`)M^aTd8%lhniMn^;m~jGahQY0P>cGLB82)%>wi690nwhC#qpng6#pjf**k zCLls=3*pPA4ysvFW)OR^ok&rH67d-edz0Y(1E*@4-KT8-f{r}RC4EQ5Xw zGhZx&_&YQJ#Vrz+!GIrgcdU=8Z-_@XnAD){L?4 zunQ;o0jVrjS$w<`c6&4dqwGMu6S#!wcV{+MkD%`Gh3qZ(fJ6EbF+atD9H0gBGZYES z;XofNU^v|D9SmXog+@isIBo(qV88^pP?nHK5f5Dx>zBPiel%4@zM^CXlb88T7S(#`QsoQf>ZPrDjn0_VZ-#Ud(E!^TZ zr%n^)Pk7m|ON%BZ*1vejI>fm+lW?gqcr-gpi(9+03fJC2FB zZxqR~PUe4S_@aQ~Ug5d|(*cE>>Y8X0Q|>~6i^JJP+oxFv@p?m%^43PtQsF0Qp zH4U-{ZI(XvE>%dV1({uG>t4ym!u}GkVMZj0S)uBfz%VeH;_z2UVK|c>y|`?*v1sk` zkZ1{P?BLm4nePbsE}WlvdUC83oa^^pbk0wX!{@USRV6Nf2qQ6Y+eLsdp+;zg-VdHynUQU}hB~g9O1t_-_#XN^3x9rgP#3}fpzOcHHYkWrcMK#)t;&CGQ>m-2c3yaMkE5a#`#Venwu$$x=*h(-6 z8k0LH;)#u*qMtXc`4@5}@sMN^O4*g-M0l(zIgfaO>J+{d=dx5G{_NTEYLQAbUT&uf z(IfsWGjsYgHf?u}HRKAIVy2+>3zpr%Z3^DG9>>|Q4QD2=SD;G^$FI$8;a6@ibq~#v zBYFWAtC`UaDZQtj%fl4w263(0xV_`srhmj{i2vA0>6(uGfIv(yXyp6vM(YV!?1=qz z9a()Z8|3h=H5pRxm8A&ljA5Emq-?=Df$%)tgf}u(41FEGW`Al)3Dnj6Gwkt zEN<_sNKM3HaG&=`}7->hJMAuv$?S!sXppHDc1sI z+_zUv$_+{~y<>2@dQZxn50#r+mhBxIk9R#2Evyvb{e> zoNdh=WVODcstn2cUA!icNBuW$*T6{rJa*>F$*Uw+1;+2RM|ih#nP7GB>syRKKq_d9 z$q7LI_;HN#pTr5*|E^nE(b>Vo%*92_)5Pq*WXeB!(k*UE8iWZoOyTnVRI?g~W52&W zQ=Kt*A`)j#4asWA51YZnG&6Y;;jiXzXnNlt1k#=5g@oxtZDV62eA&-&D;z&4R;B84 z3e$uL;S*i!f>=k}1t|zNp)QXPB$lYPsC7oIu43rK`xK_5+5WT%y6#;|<#Vv7{?2-h zey`3I{rmyepI5i=p&uhiZaJE1_s1Oc?&r=6(c2yc z7Cs%)$C1k*W^F@LtACN+QI0zJrbI0$`^Sx^&$ym>PIyi>H%^{L7#wzhFo(}zzu|ZpeG#zo$MOBr zWZo&KD#kg;7Y~S<2@(|QRCVeRa9;FP*TN9XlyfyEh#PK~1M#sRtJNl;JZ`wX`gkU* zg(kWYB$%^IK{4FYt3?ONzzRl5f)+-O+JQc3)ahG}5!b=&!u+U-^ZpZgq8k?nJ1=}YwDJA-~}!V)Zt`bzfbHZsCb|G zatT^d3p*o;JqbktE>G&F153?w;&vsU|3btq+iG1w|BP>-z!y&y{q-!D1-+DZwoYhI#J0#9CYfMFNwR#t@ zbNO}>ubOF>UxUsF%OnBj!p<^d-a2j58y@%D3SJU3YS)f3ot)NcYC9%3&t6w$S<5~2 znTV)uyh6qN+psqjwBr9<6gu_9*03YAnQl8vP4}4Pg5|^Y1$KiNvyc~(t$=6!S0WQb zu$b$JWXdK`o5CNlgA(quL$E`f&#+T``_z`u1Qll#s{>~?nh&lxrl}mfo&gexQrTV@ zP%pU->`Z!UeyflCG;p}?h}ww=(KEbjb&l5qZ~theYprA}mg*cJ%Oo5LQjAG9B%~Z| z!~<^>);?pLS-n^R;dMYZ?@+6|`}(YTW@(oAB7$idccT?8Ao4C`8+(Ui(ph>db&4h& zXRL!`Ne|T}bZV*dD>ZZ)$(vJ}6>5sS0H|Q3wg`^(fflGsfl>g{_!QHlETp~X1*p7i zmOBq^hvX0R7I;9-?9%=9aT@q-AkSAk&izj%a$;gMGZ)!(JMS~8=@HR~gF-%Ut@?C| z6@GAVa1d~P zcW`@maC~}jad32eaCUHYad0KLysE)^(U0uGPS=m!#KOr=SDPtDad7sm$<9e+o_yhx z_heOZaR0o?PSu|I&Emzv_`zb_{GY0d)=pB!N>K6RGgLBgl;aBWbo9$08N;K!XxRqZ z$=T2FbtkG~?5=6Dsnw)XVd}|5q4*&XvYhY&8*cHRY4C2*f4y#Rt;Mc3{C` zr3dPR0ErWV{zZ_i2l0{?bRs95DlO^C3GyP*%mtBj4iY;GI!6s8))iz&0ipP>1!T_K zZDr+q7Nh@<^W3+&>JJBJ3kD-cBNIzA1}k?j1``KoGX`g~f4b__zvngA{|mc_QL$54 zP{ic(z@bTI#$;4PHSoM@ms(TR&n7~Oo(nBN0_k(uQkNLhm0F=&>zNlNlr(-f|Ms@X z)j1|-m@`{*@tN_vKI5YB^Z)$&7p|As85$kU6x9@sSM^r1FERLtUzCY{LJkdhfLgdd z)GWeSHY`|FfEe>k0zurYt-uJog$=W{h$u!14kmrpa6cC?lD<+K+<~2IC~v5$V2`f} zH&LazRNr>xBj1#bl1_2yZMMMO82Q$G#+qq9_s4dj!N$JCD60m0imt`9fu)MV1N8W@ zM^hgI{<1JnMn4P`d4;VTL$eM1Cec~E=RAK&^EoH7?QGFB>$k^|WINO6R{OysZqF93 z-?=JYfE;1HkffpGLixJRpE$4$9}6^Upn0UE7Wi_r<_X6oyOwLTD~#CU2(TMJ``6{* zLz74Ka?<>|W;7()jqbpH@9wyVv+9e#UM;FMt2p+5LVADHJ*8y!&K{|PiE}6BdF3u8 z6D3xKbT?YRv5IJ6B$2;?wM>>^LuPz96whmO_|X-GyID0pjr>}-Q#t8d0OXC)&btsNwisv3OgzUhJ)ts%91j5>$x}R~f-3l0B@v6Wvm>tOyA@Y?l zW%vR)uz*xj3E=0H$|J@fw*RY=DDz2iP1gwMl10p7*Kf)waiw9JZG*053<)v{UGFmr0v8pROd2Gq=IMacp#=qWYJ zT;T^a?fhBcFm$R%=`eJfM$v*)=49hE!9v)kQRqz~$FyDuuxs>}_+AOn>wjTYD7q$7 z?}hu=^gU-U#4@1#fIGmzyKae(!y@Cnlp=EURrj=mE;=!z{Pd$mdN2;;JS#ESR!JBM zZ@Gh3U^<0%V}KoyJsW!e06QQtu0BC;QaUE}@_^c-wCe5h0pFl`=JomqfdJoNV4}W+ zVxTyM^mc$gLHVx25ZK?*1@xVObyw}zcq$I<+>)UM zq(OZ3I;g#?sDJfKp?wWop?&pansLA%x{*XA7F^P!Fzp-Lg&J)!Ob zj;mlzkR!ZkRRLv!ZQg`9%rR-*C4R9Y>_j(*((e1l=Ivo#{_SF%6iMdo>9arU z&W;ilo|$yVnKY6J<GE#`HB9k0( z!bN&L`(|Pl8`GyM0IedeKymOP&bZDcB-=KLb+rDOj`Z4IFoaE4#N<=N@+4pfA}Yaz_qZT~|3L_}EF`vYu)dA)a6IJ%)jTx`5hPNuegaekjG9(g>s1}<#v zPvSje!O!h#WK@HJ)8XH?j^6pv+HQEUV%ZW@4({Zxt7}J(n~|N9i7}U2RIA?G^*QL^ zWfdK*juu^)FCWDpC6-j~*e%>Br>>p!9;ClL<2ZjBhkLP&GA!CJJD(pd@6)cwPNSTF zG+l^I?CDlSo!s1p!T6pfaBuL9kacd0H)#vY3JmjiOs<+do{xnm+eY0Z05p}H5 zSz2#PbyTvUEoiauuv<~VX}WUqXSUZ2)SyUm(o1U7(WA9M5zm@u!JXdh_3Aq3vB&dY zbIulVuwc$QUtSp+^jH~=mBURQ=k2p~im&BJoc!C-BVa<>o0mbrr(>Mh{=c> znSbWJ`|N%0S$plZLhDwyj%;ZuCdmZ7nk|zX%OA$#Da+QU5L~Tp0PD34otwPtNg0o} zeDc>oA2T)vwnxBtfRVV5yL!~!-vdH^*4I{~0_j|2zttn<`j&aTr1N`OcgB(FjRXb5 zJ4bzGSqp+%oF3NSY$U2U3UE&;-VU;+{2ib=XI&$cvUn}Sso5I#HXSsAG4e{cJKreL zpjx}%wMxvvPVLQG%rJ1<&3MfQpH56{5w%tk?O4A;x3iie9rI4vL)=faH(&yA^O0gp za?SDN^z)iAWiohQR*L1?)g#1*Gj9ej!#t_aWb;7UbPHfA0bx&o?V-#!yighegC?tj zCh26RdJP#-t$}b;XNd>vQ?-Zy4*sP=3B>*^lZ>qx8v6c3;|uW}Pts}SabqjGYTc;%rqhaVGT1H+7tH6a*hmB6N}}oN zD>l0c3x&fP#a<1aTqZV8F6Tbx)#|H)VV;_pAESrbbO`lo<;fhxdF9#p#)UC5bt@du zU0w$Y5*uFzdwtM(A5Ng?H`mJ>@5dW^BUev7lpC#kz0*?HGU5{Hj372sm=d`UHv~md zj|}?U*+{&1YeR!Grnhck8)vRINM+rDJSc(3H(L^#-USX3-ep{Te$Q{t_eMLx(=Kzl zd3VpNk$M^ z`tA)M()T^=Oq!$Sjw{^mC{Xw!Wx!)3nAvB^5d?lIEkb+Smmob$$NeM$2Iol(jHAj< z8Ko$U8L9LqE|nQi)|JyKhMKGXgxsHvjTkkCdsPavK^^*Xw8H}!I!6JW*P}V~@pPe9 zd$87k9&luexy(J8GHQo3IBjygsB23WSGoeE)Vy0@P7!!dr(`?e;xvmExTMkQl^xwe z`D0rLV%F33`~QlGDl|s*1W8ri77^e(p|ZqDS>%GMgox*@MJrqdlXr-%0|9#&v0&9ivF@;{352si!+i@L6ULOv{YuI!oEwzPZiTv zC`imXa3exvQ?ng48tm`F1)HX6BkFplIWRClUx|;{Fpp|~7}zjBZ+qZrm*d=xvps|2 zGtBS`Q6o{lz~49(pUc9`6A3G;fgArjvXk&FQ@gAHR+m3Ns$iB}Hh(M~1XfmB4&e89 z(3y9r&-DBgKed+_%)=F-p>a5irBzR4^Od`+B;SL8PQWZ4dEM~GEoUvC|Cy0O#*c*q zVRCD3ihQ2Md0E*4uNJysIxB{jIpOHxk88|{mi3Hj-VKf$XK^H%b@>52E53k>p-t}6 z91U;7U_F6fur4xiM`Mx5V>5-6zwO~5d@k##sze8hcUcU|YbQ&e`S_ak?bHu^Y!c7{kP%nU2Ym zN2QV-A2`+14w%+SM+?e7>4oG6#&858J4Udov!mt~MHd)oG3KW(IbY&l9w{eEr{q#) znvK#_2dt9+=!%O?GQpep_EBq^pmp5JOb`3Y-os9~DDm%^-a*q5aVrM`&$ z*XY4DL(%!Ywh7U14IX)O6ksB*Qa4^B6q_pbf~zs29IQ|o>mz2N5XQh5``s4{37=^571J%YQ>!2TSyu_knPr{(a+j8F zg6Omn01e7SJS-7WM_yfpT5VeE+Rif)b26kzU-MhTQP}Xq`}$;btMvT!sNZ7I$Y!=; z0J6V;)OKR#ayaG%Qz7PWnIV{pW*>0{)C?aBjKlYQ!7#0mlj+pud=b81$c2fQB8UXv z<7X~I_;eVK`M-zvt*?F7)d7v7PPZsn=ILl92-P#G3EJ)Jc7)-V>GNS%_rj>Fg zDp`=5uTqDv@poe<@d{satzus%U!@9PCED4V#RgR$)vOg9+;kiqTp|C2kc5Do0Nf#C zL&!u~1i*hGFo?>SJ-MIwO8@8e@6jvPe~ZBUkDFIE``3?uDI1Y9|La_vqAcUEAcXQ> zQh&ae-lI%-sH9&&F4UQfhY_y`kEX1q_#MBkI!A70{6>6&Cp>S!Z!|veZybUz;%&;6hi!cODMtAqL3r-thfUss+fc& zWZ9zh16tYPz#1m_^cWx*Yw)VLvx&}Q@o0|xaEFU^3NY)t39s;@>8&>GrXLlWNyk7>=DS)R?*i2Wkz!k0u~H8UV>yoxLrv>v(rE|#eP(+oNpnbHWp`N6Yr z3D&Q}N-vhaHk+|F*;(JfXqwHUyskWLy`&M>fYeKmEe5meoU|^=zYIB?W>*Tpj#A5V zMGqWu`Oy$wL2&pUesuZr2#8b3kt~%{FC&(X8>g83tJa~ep18qhbPN0%eE5;|&FHWU zqS6c0@pZYwGpks=fWhRXAj9|kk-dyIYas4?Z0R!S8*@8pihwQ>Oay@{$F$Ce_ba3 zS|rUpR)%;>bF7^W?i95qIs%-O-(k7&^WuXcFk;DDz{0qY+>o+^MOv1p(X`MKe-9OJ z&`7k?q=^hBfFD^;B#d$d1cS@@E9XN?AYVGAbT$>^%k^aFRRdNVOzwR0{3d%i9k#kW zS?|n`_+M6i;(mksi-e%zBB$Ii;aW^^Nr2i}2pRVIaE~WEf}uYSx_`{_t^=wyeT$3U z9u@8ZOvYc=qV-Vjo}=}U>}H|$Q0%Uw^^omulY46nJdt}F3?P%gDGv0?JSPMb$vj5{ z=Se?j1aC_}=LCPG0P}+XP=JwxQGAbwP(rg!jHiS>O(d2I1i-pU0VRb)$*B|LX<*UG zNo3F(C`csGvaJl?R{M~sJ|Pld$5-hcJ19SsYm%T|^gdE3+8{XS$-r%tK78op0Riab z{zfEf3teu$V}Cev#j!SNkC6UKg!iGHna=^TFB1fckPcisid8OswjNtx&O5b8T=*6| zhJ-UYGdt8CG5vxBKsH(gX`D%?zWVE$uUG?D_;oP)hS#KD&z7(E*kKS@@D<^Sn4g3D zTOpVhz()Fe@td|=alAo0GT@p={yh4(qR;95f;ewk{Tc{8Cf5ewmL`T4@gu}}{Uit_ z+=eu)nsAKF9vNTY5JaH?K@Kp6nP0!97l8?w)#ZJWMxf+2J)lKRq!t7_J#`}{RX5Umd!N`++yX3`j_2!#)Oh2DpYyAz5GV@MJdgxkhq2sWeV z-;Mx>BWolyG}3nrF`EDuwJQjL!kYiBXVibcp6hFB-!BM7bEbfRr2a*OvXL>|S%_YZ z>uXem}2ALFY9eo??(?uL^6)HNz~GZ5I+D{+W$NIR==0t3$OEFKtH z7~auRZo6Y}m*&%PcYU?Mlt2aGxfGv9eu`#h@Md$;?sC2D$Py39= zC)R)LakG8fcjpls$Ot?ayQh2Rwf{?~LH7*3wM^vM%V?Aq$mslp302GTj2rqyusa;7=r15fP2P>Nk`E)cQUHT~!y-lv$|uDS z{zW%Hj;4_wZ9y~8kD~;E?M6F*v4~+r`!0lvL^2m_B@!v zRH>MYUnjg#$0p%C54hfj4dU~+;p>4pAbFXUG z!SS6)p%|*41TpbeaWnAG5yBj=M!9Tbm&hoM>TxqmC4bn`N+$~c@X;??!v|aPWi7`B zyHv|Kdw%tuQVaa_?N;eR_|dk$EqyUQE)5Yt)1YgU#$E+vU(Ri6sHtx`!hjfSPAxAx zbn)~incBZ_(~9nfkysFK`W_thh^kfFK9Gwvg<=zFV>T=EgSpNzATU=_6#gdEX)h8c zDKe1fuBqLHaNNw??v*mLB1})48UrRF9SH=*Ur={sJRSok7^uloIqffFxlYlTn)|(@ zZ2qg-NWvF*lv_aZ%)UddwI#QRO$>qvXM*MXBP1-sn|6`4naL(_QCS#Mvury2va+IZ z(}Oes1@V9!kS@jC} zCLWbN`8M1_g{yw1x>ujogZKoofWM*RUJ+)6c)BTf<}TR8DE2vje}4VPphPOSFF;;Oz#73Sn*Cd#ZNs%W&)(WeNeL|%)r4|QF_enLG6y*zvb7Z#KS25K`NmreQfk~i5LIx%|bLrTc+OkU}a zHz{^ouGeN6AR)A~MJz4+RDgBPO5qv|H;`%~NY;t&2(+?DEmB4GED;=dEk6wX?x|`$ zj^W8!422zQ=M^Y|=zJP`)o+~Kr%@}$rz++pE~|JP_oJJcXD3{Vm^oj?^*f6$vo9{F zeJ_1AIT6>osP*Lo|I<><~iHTmoIU^S(RX3ZeUi09FKEvmFx4PN{NZU4rr{rrL9_63Ed zT**%NR;~kZrV~f7EW0hP;GT}>VswX>6yue}unSt3B+yF}DVR1K@}RaXuE<_oUBiW& zd)GK|nOpD?$T=$YY)+L|Za!FA<7=V`Cao1$;gvK!mkiZ_s6d2rtDR;L23Yc6Ov=Z_ zaZzgr%H#G%3>kW75zbe+M8(fx<@@})E<_$EPm4c_9uxM3j%M-2#1v9wY`|Np zMe_z8U2D$4ve;uw*dHN#VK!Y5=Mxjzpxl63pBeu*5sQmMuJ zDZ`~MnInjQ^QCx?~}MO=GY zK}H{jjg}1cCEil~NZA01$ArNciB(a747?q}DQfi=R}&xyH+qtwuH5r$Q+H6=dYPv; z`?o*MM-CT+y8O10d9QPzgXDZy*Sd2GNY;-wBi3lZjl|T(_3f_m(dgK_*JgpMrfgm0^|W1{4vx$iLhm$XtMwC%axVP}Sv~gQb5s zuj`iv2doyhqg$;B-K!BOZl(yvEh)LUx`+|Z%nMRqp&>$3c(2){L8Q#`_nQ z=ehiJYDX?NlZS?U!EK?rXE2fN@m0cc@Z^@6XpS%9k!dvHO&>)E0idgc!w*tStQI}?f^vP1k{jD*v)pt)a=~_zj&Y6 zg?^MSfY}}UQz+@i;281M4x%>^E|*N@I9smo(%UsgC`fo<5jYDDPsQJCjv8cb1FJmW z{bklM?oX?|hCJYrJJ(zz^Fb!fDe2>SIY#o~2TY)S=Hub9Lcwh1QM>?RED!lvZ}B=V zm#(;C7*ogrfujf7s1adY$ka|+rBKVs14C2)Z6=_jsY_HyGr*&L$`)-FKtYyFan~}N_vW-==*pmeL>CX0`oZy2T0*=P3ra>Jf{@hCg95YeXQ-f4+-=J~~C^8FnYD9bxcTwvp0guVp$eLckbsF2-u5$b|sB=w+w zI6?`L+^(7zu%w}12dk8saPO;th7EZ}50tM%)fELYuM~Wv6hb_Jq0Mz*k(_~oj0Cfd zy9zFz@&WEQG*D-sw{TOXN$gfoTkh0PFj{PO=mU&c z&n2#&Fs5&WdGv)z#7Ol{636d~!>L{S`tr648TeLi-@{ z>&tJ>{8io{k>hs6}ugN=_fVYlTcKJ`0lqS(PQ!y zh`E+NscDgDb!5pF{W3@Ss0Y<%Cb)Bt<&N`~ZzdGRjIz#z*~Z>c%nS={avR`+O)Uwa zb!wb3mI@X6lc2^i7Sp@Ddm&2rb!7LaO!}sWL4wMpBS7@j3)mFRsPS~E$B~6v8D!N; zmY2~{T}Tf4-gs=n1DD?0s_kzQgPeTYt`ohl*Un(-PnVr?Q>v_qOd5}uS++cWqvh$H z#18^vv3M@L5i6_6-Iuy*%s!*v5fEhw5owgT+M&%WGNw#BVJRipTUYh^u-JKCx|WN( z6@|`^lx3#YMF5m#o{J?5D)A*#Gxcs1v--3@>nPwZSM^Jo+wHU=OFT3USpn9V<4R(gl9h9Ln0SsiJEfgGaOqATHa zI{-e&>HdXX-3^H2-4LLFJ{|T}!knc_6pIxhRC{xnFoH&~|< zJy5u3a-IGR@f$5KfNE&FKZ%2r&x98#x0ycZ*jDbki~bAaO2>}ay6ZKX)uq|G+%+6U zra)KUB$5ZrA!0_;iUd|qm**)(#4rW!YScde^(8qwPXz@gZq+=p8FOu~`EW z03DS?yRR#w?AFx8w6(dD0#WwzZ~6VBQ_-HzJ+Sdu^j+~4@W$w?sZ{4aH0kIsej#ig zG2&bPxYv~Uhz|WX*BopEya%*b1FP4B+~a?Mh~7fZf@BOtXkl@NblwP0160>2-f+A- z@X9N0d^D+XrMG^BbH9vo8%j|K!%O#EOb!foc0P0>=lrD5Zv6zdzR6%d6pb6qKGIHx zBPP)r8+8oX(H+NBhh|%8)gz<(}yEaHi{9|b=xVS)=g=$0q=MpP=B8-=l!Pb^Ly*NcZNN-6M zvKexm0Vl~D+Tw@e|6H!+9)4bsP5+IB4z0=&o(01xfXKA6Y>0g?olCk?$b2 zI&{;Tt4bUsZ z_)YfC-olt6K`mn5fLm;i%LzWGfm#=|c?Z3N3+DAGC|o}ETOii3iF#$>IM8=Rl17BE z3X0n7cL-_a_2@B){DTszI$jv{jO3G&<~c(hKNRNFPwm+QhN6d>t*_|MtX~x34?_03 ze!h`5$Gg3-nPm;U{apPzJ5xSFcs>+~oFbh-J|rq_C>3t3wh2F+Dry}aP?0HIj*bpR z1xhbRHwhYs4H{+!Sv%S)#*~|KYN(OxAY)XZYoUi`58JBR+cA(PO`dQ5YUW1&@dR7J zUw|~B%33`iu4DdPRNv@$jQOR}a8ESv>U7wD+{&xQ%YB(V;+1CbTOjQ5&I4ZjIjAjVOz7iSIN(00W&w`S)>iSVg z8h>c@X!MP*jGZy!_=6#FWKe!5!V5I(Op+#3i-i{BgZc(zF3e;8ozh%{3nLO&Y{EL+~ujPeb}qW@46N+;o#LcNb)6L!5m(mG4pBlh+Y zj|*BrxFcZKJ3Crp51bE6p)W);E((ar0K%&3M#lf9qNBpO!=?B>X(e zrv;eE=V0HzN9F%LvHpLH%Kxu}ns$o)pTv3;pVQ@hK|fdlk_9 z#S`%*HDifI@PFBnef^0jrc<%LOB?F5+--Gbcz<~Ng0&4X4HFM5MVbVh1M+xki!0Q) zsjMXZW0pt?~kn)Xhb{&c_i75@*SaXe`l@@VB z8rCP%)>s%>=GgRV`37ypj8?X^dsvx!qB_SqVH{>l(2xOO$E~0@{7j}ym6vy2!FC-? zEsxvdAA&lHxR5n&c6*$KqnoU-*&ixPONp@Gm&c)RZ;cRjycF`k_EYWF%65QGCyfgF3X-RE$c@^+-D!7!H#3d=s5!hFy})L|c|l8EtbH@!oQSzh?@-Z0#wg{b*lT z(8rzukk5TJkX(n(m-}h8^Emf-Tjlx9(l-}_Sz(&LAvqV#hhQVKQ#A1X$aLFF2d@xEtFUa*9yc03-HN22sW|4W>L|tE{UK1x$axbG*UE-JCAWdJ{#Uz( z^ACpLp`ntsr}}~$4NEF~H8q<}9JjG4PxgbTUU~WbVz7&sWv_F4ycWK;7%LlsZj0?i z))GpNO${C5r^7+zVKpezNqwrBhjq2CJdgh%BJOYLojf4z5&*zdKc0A7>3EZmfJYpY zcAhAwUG+!2?&LyyRV$JmY2=u!XsxU+=v`~#)Sds8a;}G{Uz1K~uvR4FR^Hv6HoCY- zcwR4>-QKR3qQp=e25T0G{jn&`)XEEkIH;SpHr{@vnM?bM{wsr)u2svSve;F-e&c6* zIqiB(hna&@E!af0?Yi2s+tb(@=({!9O3AkR6GiUpQ?2%3#-KOLH&C8eWK_i^FOD_g zZV=Pc$t3MQa{5Lw$>JQ6;c0Hqbk}ob zND92BA`r#t<)Cy29{6c8QTmDNu0%`G7|Kc_n#e}tEGUm~k*7otDo=9ZESB;}N-grv zKIzb!1V57?#*)cQjI8v*axj5XoouZl>2oY(Slh-*bu@?Us&>fKy`vRwU@!NOv~~IY zBN*#KKRr1t&2NlnU!}xxeS5!;AV0MbTqofb)d2VP&TuTqRg5|3*95pvORwhkDd8EC zRw@b3m+ccfCeh*NVjnC!7aujRvGif*V%=>7iJODRm@Yix9xWE%vmrH}%vlI6OUwE1 zd7&B-!Wz!D4rQ>q33x_LIUn$pkAXbFJkLJZ4?4=VVaf}ei&v7}Sk}Z#c016@+wg*8 zsxzX5j--P1J8*v4`gYwqKf<0nKHPgBIsYja7EU+AGoLJfm;cD}|37YkDVqTuoLv6Z z0E2E1C+jlSY;B8m|mTlN~tHmkCA}LQcspYn3h+ zWJDU%779Vh5?+vRL>a^^o7EP`jn2vBV2le-X}{kg)srSyRcN_Kh=sFPoRa3_J)%gr zgzTB7qpUSkOjqH0$Vq?c_(w>y;m>eG3M0V4V;s54!k1XkiMOnLhF-(bqEtbnd1|r` zbXZ;TP~u2!yC=wGD|(GOeup7)TW>pNm_$-vjNcz$JCV9_>1_?;7sshK=MubN1Xa8?(UIw$Ee=+nP)jg5KcS(fjbI z8Uy`R4q=+-m=<&y2n|I^_ zPI@M`X+|;Py?j@+YHn_pxC_bXCw;H7PwJAVD!=)PId1W* zrl4Sr)5pkU?Ot60@s2n6h^G;zxcG0OP$7s=zH!FFW6>VM?)pRH18D;V5M?w#IdteWZ{bJ2! z>4d>Q+4e2`!2BJ|{2fC-&NH6rfM;O!zdY@`Rwz~f88Sirk30RplTdK}AMzHAQf{9A z{RZ{l{Dv4Mewjf<4E~v-2+*3g-CrDhh=is<1*)*fC`A-&NIE3@+Edn^L`n)DSQl(R zIc#i5Xh)}Oi3rYO!_&px&06ho8l&+;zd;M^C~5&Wx)TE?(5Nnho-49;}8qXShz^m8+rQ!*@V zO23_%QMQr}XbBt{BPodFAzz}oMC_DK`e_DN2X@ZOvLJ2^%>80BEHSSoFfDQKKIOO| z5!SdwVr%L}`7?{JJd1V#wQce*Gdrn!0Tm6;2MR0>i-7oQdh z4zl1((Gl2LV&O$WNzsk*`e{KiKP=Y&JsbZ`X)jB;&gBev^^1Sz4{+x1o3A%G7brF+ zGNvL%4o`=pg@6gb%ap)qh)D-&+hB$VmDG$!@I&dXj6H0bD5^)ezj**vORz?rEDQLu{hvQA)KCk#b z@q3C{rCRvCOqd>hE1mpyi?|lo=^k0eWz_C0QuRXqNT}u!={yC_i%pCU@7oNu2|)oX!R|NljW^}nMcCj+j8!C#AO zVa+19Ab>?57(SPe-DGLhgc>YlW7K5%grB#Nm$7WvC#ICx<7oYE+l=&63Q?MsS4Wrf{3GJ}n2BK%C zIi;NaSd>A<;0vchey4U+hEL7)T6tM}I=Yzl{^;`i!ZpzHnJ$?zhVQS=o+UrKbSoKLTgwxtT2~&PM^qDX}G7B&yeaQYerjJeZ>NiFr*k?oN z-Uok>w~s?@#G#MI)i?r@(3vT&y7riOkNGWXC7Ys^~-OzVv6qm?;ugl}9T z0jc#uVdC;y5kGSr38)vh4P&VD6_T27e=-Y0+EB~q|2c4WI^97(qWjH=F2+76F77jQ z9?y0@bD90b9xZ-?!suB_O(+|1A`uM)$n8Ocun-tod$PL<`sLeG02BC*a7CfgG zsHS^!L^!yfxn4&8)F65O;J&|EQJHAzvOt)#JC0|Sy>e5i+_)RMd*o4LGJOPOUAfC! zZ}2l&%YhvZgeq0YkFa~Ll9HM?X>SN%NKLWZ-i3cBq>M`&f|nbq;iO-v#A%t^VG!Zi zK}oR$I!rK5M|ksUCSx7pZMkLY>Eqt#;^JQRI0`RBzk2KO9+iwRVf)r2K#tg*h)*!Z zwb{EWxGp%hZ#~uu6Ln&RN_<2zM1gqZ1S(`Z_7O`#@@x%zdC+&2mmx}~Gq{g*>YOC% zsNu$uEwZL6P2NFmdm~fdqa&7dhS&>!4VMGFrG`AKuZP@BvqJID!*F;nm9pvUt-iTRw9lFu_92hF7jRfLm79*fLE&Y_^WqKYCz zXv2xAe80~TYF1>!6%~YW`IPnvI*}-7g%Wf<{V(#~bb4qQ`)9%E`j4-S4F4b2=f4(Y zwU4_Y`pE6|@5?dwROC7_b~P~Q=mTLY=3sMToH~+pv@kX!_(EctSn{v(SKVFr@mo^* z9*jbL9FAzjkn=yTx}RQ^U(9RDvgJY3SAT+5yEijdKR+3(dc&T3SAPg0{FUaV;A%CV zX+xkfe4__6==jR6p+dqi2-Q6b2jbK`M0b@!MliODu8}{Ttp)_upQ+$JRCn`e-kd}H z%0ERu3uu34kwQ@$$CHLq8w-j>tBGciQqt@J#Ig&+oiQDANvAc2h9lc_Q8lRdq{VcL z!z(fC^-+IO?`ezamWH2WItG#2s}9LUwrQcZQtzpYZIpzsV>*VBa!~Kdi)|E#`_b)j zi}BTlV`JR>B-K|Q`Vr}2fJ#KYCnDxu6+VM;(@xr>IFuIoERDMT8Ambi((o%xt7y^~ zm7=d0r;1_~N=0NCW2KVc#jF*J;z>g=%asyHBQey9&`lHQ;Y$HdrjD%uj`2OghUZ$q zWx_C}mTl@VmevMcRCU7&WfYcH)92(E*1DNv9$?MPu@^uXKOEGsLK2nJutE_PW_nWx zAWRr8Y*?X>;&1Sf#fqEQgVo}ri7GX{X#)7h563k;OJNC4>|txY(MElY?Qv?oF+>TP z-jo8~6Nk4OR_J7iWn@#^0I*t#reh>BS`FkAiKH^C4F}1^_*%+lSQOg9GTGas%6DV2%7gRDLW>)C{%+dMx z1~%PP@loXhKuj`49Kh48TrxG%tXwoz+RQ2$kduy>R;K~mCAMP+eud%@*Y^iPfcP3U zAP&}Be#ZuU8|68*&mE$N>iTN{5$vaun+13;%5zHJ2h5xJjyt#?$}>C|Bc-CWAt1mJ z0)(C}v_lN80wYVo6w`+Qo&u4IQbtmj-G>UHj#5UpGZXq_MP#)ufpmA1))zKZ~$nkKe$3F#-y6fC$GB4Y2p5J8lEk z0&B2{qNt?PB>E@8OiSNZu^{d}0Y2g;yOKausASx(zrEy>t0~2@uH!@~xi+qkHtA%afLrqHw7| z=MfXHv9_o6QB!44Rb~T`9&#tvP_CV*vL)8I;;gY1M<9PIfcuf{{6WPz3x;3yk#7sX zaN4U75ZYk|k3;te>`Q{kCB0?~uz=x{+2IDCLH9`doS+pTzs3#dhG~-(B8eBt#|yZG z@Io1-h?nRS%clzX4iSnfZX-5|2pVTJ=9+U%Qf6yPil<1qt1`_yejsD=>r3B|hZ%eUrAF$)`Kmik6fo*N% z%mzY}I1mG=0y<$f+pMc=frNxiz>n}@$Xqu>Rl zM81nM#~X+l@(19-+7e&K(n8%`sqP-7l^auUS40|7w^0+<^(v||XMg;RM150G!?;~n zV@~|YheHNq3?O~ar;U7xtDxQ$to)fwR2!*O2E&}$$Musyfkk1*MvXZkov^Q0V>keU z@-9!+5d~2o#99SbcHghunCC+Wll56&E&0wsjX8DR#fKe9yx-jaGg~0o_9#lWP9hD%hKYz7LZ8IpmE}O?Sw>Vo%w+Gdiy#(&9Oz z!Xwq7JTgtHwDy|4!Xt(4{add-X6=IN_6~C8p4^#FTA#!lr<%8Lvx1dU&(RcwTlu`@ zdpvE-VQbmE$9oyAoDhZ;>6t1|ytUbzUjYp-Sb@;IlW#38!L*-7apAL1g$rA+c8dD-lWI9uDdQ6A^&PsJ6@{Qw%9)av z6=pM}M22wQwl(oqvl)<4KC6<7Vof^c~pL5~{ zwke%7Xyklj1TZARjPBI1Og#o}(0?ku?zSntU|~`4qOxMUqi~4RVvrcI*?!RuF|}+c3^@cauT%wWQgWCDpb|}2 zz}^$Pp+e6p`hCcm%!E|(OD6ToFS=rf{z#Lz=K3b{gJ8JyWH#GTB09Z>GSY4<0S-jA zte@{|C6dI;x*bdc2EQ*Y+&G89`{!?gOyMSHdOpe>w8p-=MNAQ=(x>S};fe8{G%+bX zAhsiTG9y)2z)oCAfySSR5bX7VgjZz)vQ;H3Lpf(FJFY!Oy#one*s2@G2ncLk8UK}i z7DF5HaJHCT%P)s^45e4no3A|IL*KF_!LSL<8{9HkItDRk z-LohNZ@R$uTOjojd@x%F&8bUFf61#5wYK#oXx^!ig|&ySGRDr7(LOt%p3JfSL0Wi2 zDWm*8N1MlhE)k0M%%5Dhv%^Ioko-Pz@`S`7i0H|#y0>Y(ilqN!5~2y+u|PWQ%&eF( z6zB7geM*R=6bR)2WdN*y zx^eY(lg7~HPtnlvwl%jfp%6P->GV#>(vl9|N?b#tx@U40FtN;)KTqfdMZ}xyu}kv^ zS>wwev5oz2VV%qR$(}g_zX7dZd4?~Tcv}@$q?GZC=aF0uBE*}O@{$P!t>0mL?}vt$ zBUZWN8LpJj=I#Al?2^#Z%Jp^=TVxi%4J6>#Mz9>qhie)2F=1;d_)E$d=Q6|TN>rv@ znnoM<=!75_KO_i01UYRKDgl{puT-s+1lmp0 zHz9g)KBAnfxzk(%5NFbK&Dvr)e<`)MAa83%dJB1`HaTkbabfo05;!BYvW55-v{&0v zv0i>bP^Vp45)wdXzcL`-E0nU;3?|qj2b;nSKbp40W?obM#@hy=p2uE(#OtDl-Q&8A zE5uDprmoVJ+7z1*@zIm@oN@qp*W03t?uOER8tQKJGa^{BAtNk}p+z(7R=3uRlro=b z#FMlp5yH$#&s03p|F)Y8S%)1il3-fL*4F{sV{+}*aH83irt*__g=Gy+nikwacB%an zjc8EQiEHe0LOw83xLnflQ#i?863+D?*3P!mr#G_ymh2)B7optdoXrHrd|x^8_~s&ZE5SD;~;mVD-^Ob}Pt_?bqDL z4xIgO!4ChdH$UF=647W9UA3G!Ie7N5bMx@_K^ z4Nw0OvK=D~5omg8Z=m`EQOhLVuW2Zv6I?IzdTFVk52h44=;(-8Bb+{?TECQ_uFG8kmc~yz?1dU*;@WGmzpP(T3)LmqT|+Jl(q04QF!b5- z!iuzoGJ1raeV%hleQa)Vpd4B<&=TabTx7mshgrOa+Z)ST!=qQ8R%J2aQv-W=I1+P!j}#K*%QEu};!w9yEP`c~QnLdOwkWNe%jmt#pfisfw# zt1HGebHr8d2XLJ#pK2^3{7ONtG+t*h%pV<^e3`q^^{ypWfc6Mv=ih`8l!KyBI@R>-8UgODoDF32OsneY$Ev6VtUW$edr<$kRv$0C+=3^u$C$mf!yMqrm zLDOT=tkm#(pHOE~C>jPT;SC~h zf>aC50Zh1PucG(VZ7=17pPjm%{dyhjv1g7Zn>(BuokR*G@rUB6BB|~f2D)F>oMs}& zTSU&cAQ>3~`#tjdrNqy80BV+HlJzg2H!=?@=BcAEUDHXi#tr z%oBg=*lG*Z!lx&X&CYA@kBoY$T93x~4qVkdga{ka>kPOeiOp%Ton&C;zMj-k*WQX% z0)H99wbNqR0O;r+LX0WA0Dyv`*oJnGDk}ynS?#$sp|33xUy}n!BsRO-&x3X`?a62u zoXOym@SJFwZwJr{dk+jn<>fLqZJ0C25;Q`ygk3$h2+Vk;M|+nEM5al!=>IsxZN_{WvfWdnn=NxWQb^9?Y$m$x!GOrc87LPA1vH7@>bQ ze)~4omg4_OykDf9Uh5vvz_xW8DdZVn!O$jvt3@pSTQVwVQkeUB2eay@h(=jtMGc)5cGc(&UGdqS{|L=QOT4}ZK-W^S= zr`4Z&TB@F@dVW=po3uAs8W_?}N(W2n4w3O>X7_DM5r8n;M(Dt>yb`i<-_c1GaST!G z_K6_LE2UB^)zrs@FXhnS2l7K}Ws`HmN9f6Kq}$fBF<^2l^%r6CwY~gpiHnHjZ$jj2 z@q=NJfl35w_YsO4Rp}?>nzjt=g<}~hbA@M=7_#$mATf@hhNas4zkofBGh7sD*>1?b z1)3MulRU@*J4bm{E{+T{ZY$lY9z}Jz>F2ueY%U=;=iL95PSUUHJZxmLxZ>OK> zVpak*MCa@spO6YwHX>b!dBXGJ=AnUJE9HIBfTsFzZcyZzK{g02<2(JEW>g*1}atWDuD*C0yL={oW^QL!-P;B*i9QX zE8Zuy$YEUkHD$B>HUpBVvV7B2bnt9@8`OS9q-{nv=lbd9a#Sh;w|9X)UpNDAlI#Iy z0-Deaj!fL*b0(wxk>%M+hqmr&`G~f~o2XD209+Mpo(i7D1j;$S)RCG-Xso}K@7Fy5 z-8JjYf`?E}F6oy*GmKm@e zaL_4s6<`Zfcg4i50@Gtk4=<7l!Ev-&h6F{Fd9@DOv-Ck3{Lvu+dD+6nXHJ@PfD4w~ znK_A)zd@YxZO~LFD2s;>2F@vWxEo!rEiGYpz);Om5_)eg! ztsZ=~hh>DZHoMp!H|K6hvuWB!eOD!$^4dGqc`~_-%E-&jSq;BHv1G1e4xCh>6jWPC zG9S1vHHnZ@-_9b+a}!ryE?YKH3yHreW7*6q&Q%Lmj$^gim+Q1-!JdEMoV<_Cd~u^Z z(Ucq(V3LeU&GMk1tYpe)nVVReE%h*eKSq;m-IkHI;Z>X}$y~pEOSYJMtVw^xhai=} z0x@NKjjJ0nNt<}3VYUYz?M5GrU5{r>?5AFhrfV!k4EubyPy>>WnNU;2b=y>_q!pVe zsd84;#+xWLy`#N!n;Y*i-Ugq(JLLPj3iK6>XOU|dGWAyT_)haXZ~rTk|4UdW*gqLu zqWaT_r*ZQ4K8IL8_5kjEiqQUtR0nbKtCIgxsCiYW!12>}SJ#1U6utc%x#{7yxW@5k z{!zP_KR4R4&|R!L=uc5Agz(O4kCc)4V2%d_HujHgTFYg)`*s1acPEEHqjsmxTkXao zQn>3|uPE7)TN!YzL&aJDN@*5pt0YFka8ta1Sf0E~RLQhaV7Cu$#ym$)3El$>j?n~su-D8f&v+YtfFyx zT4vI4EPA%z!h708Y_@f7_XJYrx1P^7f&t}wHutY>?X~)TflS%!o81ZxJh=Cw>TL+p z;>m4QPm?z}SJ}au78Gez36zY2-1ofxp?sV7|1a>?0yk^PEeL5-7jHB2&hSFEIuD2 zt1V#pwU4n@K>IN>ZsBk)NGn>D$grBTQ4o_<2lo6p=^jEY^(^F=QQ94iwQXG%RUeR2 z4+7WCkw>u+ox`O3`i5`M{Dx2+Y%_y2*n85`4b$sbuQ!*YGc(pE#+~u6FbU@7s?o9j zrmIJx@s{)>LoJ&1Sf68zGwK$tz5rl{ zzwXlFj`~-WMEabgr8(udcs>67uklJ9ksr;)^Y1!@)+@d-VPI6f$Z&yxd0rCC^%eWu zevmLN@==e~1tR656?-B39x9HO5DFtNjSxz2tqR|T zy^Aa@LQ6e$#!beJT^KZ1KPM5));BpnYLODhXJUl*uc0M;RdmSI@`6%V9M2Sr?{nbh zKCf?j_Y&(n%;#-iyeMn~W6&g~$;+=O%YRUm=a7~EATQ6MC>Nk8kD)BjF-D|au}9@_ zZ|++@Ja6uMBAap}Te144M|n--PYL^(=P=^iE(~EEV)x)xJFdDd?b{NBpLHN#*oQeE z!xM-5{C*YnIPq3Bb2YV1WpQv+DfKw!%PM_kixqsp&$8;^ha|1k>3!UjRd_G0ooN<~ zuv^tGheRtGTAnc%1w1@0dDb~N^Swg~Usi6865d^ynf(7wk0bw>P5p3_g z)wnje;3newls1sD9nVIXwOplTNJlMinapQb=g5L+PoPF(s!6yU>0OpIdEHu!VI}I( zElgW-WNz{2&GnfvKYML#jIY}NqPA{GS-IFc80)e3v$_F$cQAoka#qcO`4zOfuPYuJ zePj)c=GvEdpmD1+;{h5+b922i8wKmyZZodWN>i0}`!IVGfo+w8Y!QfEOSlb`Cz)>y z0EX}=gzh&Xt87ERaryH%_fA(?Q(*QS56Wv;`%cem2k{ty*u|8;F#(#w%s#0FAPw7C}C*D4zlTV9$@dWAeX3ftQ3UakYR= z@H&woz_SWPMj|<;zpQ?hLdW81njuJM5)DUoB2{|I*Q?(j21dnjbp+(}Dd4(SaSPo(8z8 zCA?<9gS@H>-1TYx!fzt|soRA7(NM69<_hp0k_)zTpwO>n-IaD_U9t_q+L}n)gDM|@vnhQuzGrQ^6u%MAJo5`ulO^?gBS=yGCHZ^f_dy~K4fJ|q zBf52Ym3qT`5C*Kkx<39mC<1L2)`5W*U7ab$J=}v>K63Z!lmY9AY~}=lH=Oo;mA$rs zU}|8Ht{h#tc6a^3{{Huay*m%S`abT`*kego8RH|0mi0X|c?ZHrktiHsM}NhmnB+osth z{;m?2PYsh?Y5odOA-U99zs>BK#foHGM{J5M`j~+-y%d2HNCAY_@CS8eOtG>d->so# z4F53YOa{@YeGRGARG2gqhRlGJwI*b1cmZ~++GFB^q0X6FQ|W`GSBUL7D?qY9{PmT* znW5x3wpcNs`&#qtfZLeSWmbHY=}kBvento|xe8=>|;NkfPS~iS;f(`kI@NNX2Xs{PdyK z!@c`H(VKL&&m!GU@=rTQvxLjQ`__)U7V(-kF=G=8LvhrEN(2$D3lYo!FYiMXX?s|c^T(FjuhrZRXhQj8LD$B=22cOYgy?>KUrk~P(uE2> zx1=&&?pJYzRH?-?gzJE_bZE#Pvh8!}#HbI>$uW_!t}S-|O-Dav8-6#L@A^n)#!Y$K z0oY(p?Lu+=M}N)Mb{*V~;HTxZE58pwr+q(*rKI0mCZmpP-Z&vt$1(k)msy((`KEeU zN}X^^;Cam8nq+wV*h-=3CnX_fvQWRc7pWS764TE|tMm-?vA8+el;s>Q|I#lN_+(@nW>u5AeZpn2`wZG)ojzMpYO+gvUZMxpL8w%wcHCrQ zxTCbTMol{Po0xsUZTPn}A$`0e8&~?lP7>y0`;nnjyB&ibVa7)oxl4KMp${R17d(oF zE~W_9)N%3&hocM}x^Sx9Y$(^@GC^m-z?oyDuvgv&;k@Ydv8wAWTYF5yy00Lknr}+SaZ*Kv$EK0tTtGSm*HaMx@*Ny_8! z;G>lFRr;UPgrmH5FKVFZyIZoMu1?T+yB*Ifav-9YfSc(VgrAJ@ryz2T=05rNclI1w z`)|`A_6<0!JN(v!irNUbKu&k$=b$4awn<~iwuq#gt#jC|5cZqybF!_Ntf{+I%}xQH z9Atx(JOAU0tsP`M4jfU)*t$$9e#AiKq?68Ze63WPDL25tjPNe77BiZIaa|`l2{-Gu z!Bs&87etvKhaH7}&!#sSA``@NF@e5~?u{+83)M=UJjAKuZyR{b)+NfZ9x4x&CO=QL z1_+On5JYOnD?jzJo6Wtj4*aH8@2K!1!}dt{#vEu(jyiiuE8Mj$5)p?lAVxNb%n0Q% znP_Z^1fVGWnrH>yQb#ogevg>h<>~4)-MM0b@G}BnC(Ouj<7& zMMX8c%XZ^q?E`$X{rO?nM>M%%*TdT}xnbNZ{C^@QiTMI^OMnWr86lU3oChSUoay(R z2pY+ZB<&#Mc;}4mMMH4}0@& zjnap8yN+;RzTMmMY^-zj7+A2vN4^eymFWkxC;Yk9CJNbxZ+qh)6Tra8=uY0&SayWW zKfL1vEIf$qH4b#Zv^Rn4*sI6Xk~_frHm2jF9-s81>CR%xVh2KVL3JmOp%(&=@||ju zCz)&$8=FP$${p&LdR>Z{=0M}sKN9iJ@S;T@)* ztFowqdA2%0R3KXTtT$;Gnw=Y}H=}1_z8%m@fnooh0XgllCPIDYhG-l7uc?-F!b9!&&YRjJn^E(K z8~2DCx;>(92X)`R8qFXv6&52OaHRR2^N)90M;;GzRTST3ql=FzL^IEp^ zopt96GeSr@)(^{WW|h2QRTHF#LEET1td1SttUD~8UEYb4p_U61s5)ssdBjBF#6;7u2|$s_ zqy^x(**LD{V25Si1%pJhnKA!8P*{l6pW z3w!G7w=ql(3(EH2EW?EMMgVWT=}6}{WYxK}J%tMQ2dXU^g+XQ}#mXhnx;3B$Vlel> zLTJpsopwIqCVh-A)^2V6JBV?<20I*eALy$Qad#c{fxc^=eIYnr$4Q5yMqH;hU?s6xd(z=@Wac?0QT=H>+40or! zyjwTOZH0Wq!n5z$hJ%TSq8eI@#BGIGg;8R^rm)EEEBBuq>4suw+SY@#(_+6SvNQ-O z_K$-8FoE>T9luz7tYXusSmRo8rLLgQ$9+_40mR;%=QR;^N1K~y% zGljL+rIaD86Uvudke?Z9)e_?X3;jU-n8H_BjMDhAj1?D17L7Wqz4xjg+Lpd;Y=FcG zI*%^!Hi@nu6e7*zp8b)`38dH?X*Xx9mnP}t$=206q;Y`>S%Ca2(Z*$u_OE5KHzxx! z$Bv+taqb@c#scmYarYYXhuP_cbl4H3^3nhBq@c(lc^Qh736GP6W1$w6G@A^h(MsD)zUy~!O^s|8Yw9F_pSv%Sb`g%k{a+9N@5+35P&F~@;i0VcS3Fm+Q00A zR}7NQs1nbB&e61Ngl>4eV*u7OJKIaTVcCR^zAe8zhXd2*B zeGD=-a3cyd&$9HoaOIR8TFK6NLt6(9S615fzidQ4OP}BLH0*7MJ-g*ml?+~)fN-9< z*B)kFU14C(B02N9@kbe%U=Q=l63)SKJBxfwBf@t1=+|1SKRv$;FJ;-|aj+>b#&=3) z*w1q8W3C2ho6y4ZNA#Gq*w<)5*$MZx6zqH|c@i>z%x7+^xg!R)54+10AGurCX^p>g zT|@4~Glj(>9Wt&X{!Mq832~Saw^#VlGh#Ng{2zFXln>;9&-hNA$P~*=9;v4@;xrSY84P$;J)qQ zKOe9-S^e@2%>QbtmLB=WT35C*dz!|cvBsV`>wS5;8;0to{vd0gp`j?p9VP4Y-0Aou7n@#v0e4qx~bb{wLF7%5t( z7DQ_T1IAeHo`lLcUb_Yf@hSbm4lMQ2w+#UAmffu+S1uUNQ*7F&`MB4KXPg0Jd$)X0 zOu=`(x&wvi20_P6$kk{kheXn6I1V)T!Jh+ATp}Ka#Cpw0H;njusQ3`bp?mi0{~0?# zAt{>nm~Yjc!XnN%Al7#eaNu?shlxd>#ktUB<87Meatx?9N4=a3B&zrclzkW~z26o7 zEs=Z}PB;~ySQKDB3&U}@N7`wHQ5mKwAg zlh1DM0t-OT6QM2iO1a3O4=Ea(y*JM+-wEYg1u6TW3QH+y9aE*Cu+> z5<(Cqs6u!DZ-R&*-!p_DW7@&FA_;6jz)s)#`JATsQW_JF=KZ4M^;d7I(NrQYdyC_M zn()h?`o3~swq}rK0kiNo67D!tZY(JzNk6~SnkUH(6?lmpr$8+S8N&5YW>`5hjmKD7 zBtx?yZrrr~k7k$&COG*O7MOWx&3W@6#hB z{SdXLi-^p+Lu65W%$&LV2$@P^_ncm%r}rxlw2Q?+v<&)!5&MIv6g68BvXevMeV&rn zJw)NDBj6hAKePjE-qzaMr&b*J-_i~||3B2^|I6!>w6VATFKKy_nvF7=I=U~zHx6se zxa&EHqR-DGNvQ!L(OgSWIx0IT_+Yg&ingGlQQLMePV#rkB7xo-7G=ofv{OofdvO=f zOEXqj>yn>oyk1WmcaND)j&>g(N7FrDHhOCey>W)Jm@Yv#ve<^IHQCJ-4w8e4TXAP63V6tz2+iNR1vROD1NHvT#-ND>2g zNTqQsN{vjlakn{~%2vews|8$+bZe0z6f)U_w8m0xSPl7SiO%3x-0tB@(2s*lG&-Pr zxC^4dULMH&SxOi>hV3oL3s)+2o~ST9f*jFw0{9Fyk4v*+U=RrK31sdvaN<)BYczAlFG_ zean5~p=Ev?J`HuL7n9?Lam@E15Y8U+w0U7;D?jG`^;8?M|Mm81Vn?N%-~X3f*{5@i{tYr7k15g&gsusyDgq)ac>E| z@-aw~8@fL7Z?~=?`W%Ai=y)39JU|wwnD~+@sJIH>{22L`Tb&LARKzE|&I^aRX#`rg zQVP+58MNX-ix$*v(%~8Zfjd{M$(YodW(*v|q)Ye~XHB1cm#Qw2zK_E>jY2_S4o2Yt z&~MN&u6wVq6cA5331$Rs#U`-RpGhV-d6L?I{N4Nz(amO2D{RY5Rs)XA9_u^F0OKaD z?ylb0p7J8J)zpzB$V)XhrpHLxDf8_Z;@2d?uac%_F6JTS;2F z*y~&`!4yd4?NbpyZoc-~+-=1X$L`1ItiF(Np{`owp{w*fDX6B#IFLM^RFa8_vu(2t z`QDDvpDUN^&+lA6$2U(jZQ{w1kQwdRuX^=pQHWvB>=0T#z^uiI?C^*DC$hxQHT^Es z+!qGoAnZ-V6uvXl?Vr0ZFe=kOaSYs&&Y_R_+r@BJHSRBbDJ@v}vOavm-(Y?w)Yx}@ zeQkA%gf*S{Ve_e-jqjcn3rN&q3ZO9(Z#~2zQ$mwP%gZj#p7{=~N4ymp71+*PfybH2 z$uC)pCmRG#)W9`X2^HNaL>E7KH=lS7WB3jy$&}_D;cvpGB0A0g7uA=DQLOavFZoX- z6R4kZ5F%mZ_7T;K4toNEi|HgsKM7x^dPtY+#R^19_vwu!Un=vjIGta{w`lme`~L(V z0iP<^R(^9TU@0+9iD(ud!K->kNmwhOw3TF4SXzBkxugd?Cgi$lJwlFbt{*USZFLNau$%hp13nh5_OTFcv z&lIq{!BxR<@3nIFqagCycz1RL@8b9;Bz^Ho4?wx6`L z5VQ~~27-7tZulT;W%qxUH^K5(C-BqoW_`B*{qi{ePo`PJ#`=GdX9|XnPNt6kgLgJn zkp0AJ(KD-T@-3RGesob2@;1`gLfK&mev=m+EOp;3l-^7?8#hzFXABh(lMjdS`z;{l zEekK?wKCjkxYgNmclY{$m>UI(Xpd0EvSQ%iWoji1*Qm^mEP}VZ#>Pyq;sF(yxe*No zh?878ZR27*@!ZMKmLwh?(Zmx#A88DZWiz2qJIk}EbkHPYoiN27ihcXeoL!@gHWrqh z=O7U2Z9(rruHH{Jhc5%bNe08<@MlAZMJq}7ul+k6Hy{!opIIuN9+ZfUuef1FZi?eg zOTXdX&k_b*q{S_iQ+2#)&E?mz0ZbFXZ{2Iks2YSoo0p4H%F04mghbpT5eLj;McV~c zSQ|55g|+uzCs|+K1IiQP7)y-_wp7|h3(z3V?f#)ntw%m4ol2SMzeE0KPcE_15t%-Z z%GUq(xK#MR-;+-s{eL+?)gAw*;saT#Hp=T_D85r1$+Z}$v`97yGh{XpL&cbEHWHAs z5xNOnsVPKyMnEtuaVU$>oJ2k4TjE>942Q5qF=Nha!OzVE)7~9dDSqIU>(#3HldGMY z-~0R7)fcfk;(VIDuuz=u?B5$m4!=zxXrNrH^yYt2?T!C!`n3Wg8?F>V<4hDf$B}tr z(3~2!HA#pOi+V#&aIXWvFG(83W1-Vage9zVFn=r@9RYzwic0uv*GQ+oMR*Uffs5I~%b zr)P*gk;Qig-!q5UI%MAy@$RSOI-mwQ-RSo7cif4}IY(^#v`bx1Q>DXkm(JvkEBfT( zJr>A_;|YE91~AAO@SCYy$>y>Om|bS0CAL}zIoXKF7|Hy>A5DJgLD}+A6Ab)TT(lsQ zlOmY8KtInB7y=j_1tw7OXhVYsac3vlT{#i$!o&@Z1$Oma=_0sfhqHHP_5(F1$;UUJ zBr*lnXr-kW+r&?Y#ukiR%2g^#0o02Y?(NJrAZp(AriD;b!8>m-?d?or*Wn#4^ZjwM zF`CkX_5i_BZJS>ylm~#c)v}8>|+YnBD%kb{khhPTq)JRP}W?3lEc@ zpZR#@G{*0o<$ualae|qgt+krpL1#bp46G+RKqWT;>Ut-jaZa0~+Dql^l8af>hD}cq zIKqq{Rv!qbsJaaWRsf&m0xPu+ngMp#O9~eBfAdE&uh?SKNA_NkyQAE??WXOg7S-eX z!<*1e#L|}-9TuL^T~;f70p`{jHOFbslsS@i4Z3)^Q#k3BDy#zzN%G>jD=ax4jYH48 zg{rv(i1I?Rv5y$U`;B^|KW3hgl-|Gm=pOvzy-Q#J>fFnO6Hg{F6`smA|3Jz(%~9d#Mawa`!Ls`_=Bq+gO3dB zIOG}~QbUAryfs7>UW}C&oZr(bBH#r1JEkF4% z1hBl(omwT6$;Y9scfN404za{%Y?z#$+>;UTZ;q;);0VElY}M7tA{z+7!CsGw{g(WK zBXiXif-5B?A%xhIrhd64Ntxp9>v#O>YHn`kX8w6SHSYHy^Fw^uzu}~|P1@t2=8$yg zK=(`BL!!P++=HR^P28iT_D$ZyqW+*Av_lORNj#3{lGsBD^OAIy-q7CTM}11-lG*z} z3i7U8oicc|e%3WH36h7@2$04;X4t6&6nA5(Z< zVqUT#A($EP6=bE0Vvw9L;D}X;qKEZukPIZyN;J^O)Y2SQfMbcBj-!QB?bavtNv{6_ z1!5zatsDnXMTic%FoZf$GYm1|kK5yep`rJzunKUJSsPmol2VTYkkmZOL)&-QzMBD^ z`c7ncbSoYDaPk8TdBJgjI~+_VIzx>ig=u>Ta2Nm)5CtFxsR9T9l&qQk*E}F)964jI ztX%+z6=%wbD`OWQlpZO;rQ2c)p*OxZ($lK|?>mFBH+yJlEbGC1IGpjCh@fU|AK zI(vuPs1jl7&sijqc8U1 zspBXnq09p~ONZdA0m&EpVlKuX3@G102zR?Ab&0#~&^7V6Y;NMi%6mNhIG2!fc9vD`G2sBfshaN@|;dg_&Gz>d)LO1#0E;_no zdP)v;?wW(-bj>OoReLN9i>!25n>2)^*~&K>H{s#KVI?ZOdK)!+a2xjd$4axq@27jl z7?4IC;gn(WDgc%$)r}G(SPYD5G~`nEuTQRd&q{+$Bzb#3>ofU5p>!|EW1-Ja>f0+}{72xW>8(4Ucg+aR5;c2&kNY-!5y_eN!H!OnXdx zmKr3`@l4;v0h!}u8~aQ$q#nX_)`2-eG91YL$hP?F5P?D z)5P#QaCz0_E!$(AuH5wbuRJ?%$ zEHq(FvHA8Lgqj#|1NUq8q;XFV$)h|I7zra)Z(7~>$tjN_f06mRQOg-dom4_JNxO`V0axh7XF2hvN`B{8O zrr%M+f>ND{0ec#PfNhcx(5>L~rxTRm>{p507JdHRl7AzQsBoADi?Q-jx}10pB?CPM zDj;dGQKDOeO9&Bi4%gDh#s}!~^QpT>^sMw8h^W^5LZZlrT2-0RY~8I$7Cp;zlq#x+ z8C^*+6gGez&8Gf$ZKX2FBuC1RD_?z0t(+`Gd+(EALlo?y!4H3d$TNZII*9x_L@Jp5 z$6CLX8ORiGnu6V`)4MZw8fmspiRR+JJc8^SV)~}dlYj3O(Ho1?x)n$52KirWym;RfVz>GZkYkM-TsI1Xk_Cr5D?21Xir6JGN%^*yiJs$`WBHgZf_OFXVSW-s1=rHO*ig&kRD#1>uiHGAY4|G|uA* zJ|)SoOxV}MG!>OXmcaQZ5&2lpm?0Xg$QWa$geM8fm7e^1P;XwYr967$Un7xl6)hOe zDot|wh9?nekr+^*bE<0bM8^zCTsd_|!C0uZ_1rI~jnG9>E1wSuchJ_tw%KWyoy}jz z=ntQhMIVTW#Tj#HX5-Z<9CO#_hh_~Te3n$!i<1fe!<{UrM=S-mS${A|$OWdk*=oT* z@zsen%(t4g73OD=sJYRf^kM@bE}QSdM|mAZP9DWnn(Z0+3*#fcnhD&(J;km_-J;0N z4pK~Dney|ybkli~^ZaB9cZ?q&7J?k2vRUJmYUq9 zh;gQ=m!dA2x{$a(|K2?tgyb8 zUt{;%Zj6B{x&OYAa7T9saP#o4TM6IB0I+<4gE9Jaj+} z_FgR-mwhz;J2?yJOaGNPPs}X##q)&FBWT%JdvM-T3yfw1Hyjw14V`t#Dci`TW1YWP zfLxVDnXh~*thS5?g@5^wx^yE_<#8Ey-=`Pj<9QjTaryl}RGH>)mWRaH8dU++-;Gij zCSf$~3QE6nz5@YC1H6S5v-@|6Y;l9f={Pru$9a#x$nf`gbi#(swRt=sZ6boG9S0#i z**q2tI`e-nTp~c@^3m66tPA;=W)|yI3~TjAWdk!Vk=dZ0;a6}eh1qIp{#xB;DzSJS zHsX-~$&N2DW1z`X#9W*iI2_W(yB{HW^oD0mQ<~}DdNFp`hN4@4zmSkTW$21d)p6N% zAE;?Chr@2>8i2eX=r8_CUr_m2i?eD6qO(F=<4bkxcHkAerIQjYtf{LjCr2r4qS+() zA#BsFfiMXoOEYq*;-u)xDxO3XAqj->?kgSDv>GT{P@ulp0nJz*9aHOw8hv^g*SN^+ zTTbiu^eOzx&Y_>i12f#wz>eI^ACnGo;ukerBYkC~?*SJdxJ;dr=sP8b7&B(NR5RwW zW+xB)+@ZrkqQFpN%42Qi47-=;!MQ6-y^_~<^y3v$i&iailhxkz%pyk9EILV^y*qcy z&`S5eLkt0IpGk`M{-P{eA~cGht|yNI7Z?6OFWZ^~|E|Ao^c=f?;mC^qvUaF83BTGT z7DOXO8Skim$|u=7qf(?6BUbkMg8n0tqKMi0oLdV1;R_V~yy>VDBC?$5+~Kbd!dnIO z2Qn-Hj5+vGU0_F?uT{k%nxWXr56%o-l?;nJ#{?g8X3J4nscxCs8I2C328B)hd7eR1f@c>Vtg69s?4XAojs_iURd3=VOJ=o9-b<%BO^?zEtIHzBNPWV;lL5@rTD zf9*wJK9wj5r73(J=PAJ*u-3ynTK#43oTdQMklz&OT$Qa#e?I(HK4N4xAxYy5m0#vg z4Y_ni@d8g^y>SZL5dL5$?ce87M)uD^?FeMt(C={T`l=^wNaB%|ni7}ZZW3Gcw^M(% z&}*~LdiDO5m0*I}(uJv%^EF_eWO4}FnF-@b?mHE)sc%gFm#hhVOCQ0t=PFN9f_5a+ z@;e4{gCFKiw(M=9;sO)Sj>zog%(ux>q;K~)1#xOJ5hC6d1f~R89L^^Q1^lQQXS_NrLoSia%)R}h3TwhXu$aQPM zhudk=3jf0BZpQ5Eu9VHy%iv_oP;yxu_jbe&I?>+F`G z)NcEUK`HdMW{usCN#x32tS4)j&*hUrDHLcvZ;v z43OX9>t-|@JuIO+AtUfIJS_2BzA}@8HBPBgL(r=<3gVGQuJ?SO3gy%Zx`{g=pz(<) zOK<_|r0@c*@I3)z*QJ!m`Fumgb*N?1o~m`Nm4Wb=1#XsLArEJM%iJ@ShS*$;AF2Oh zB!W|>c{a(Rc&0~DB$;v`Z^sC6Smw&E)Syb0RFCyJ)~1KrpYWfK#d|7)Qv(r7WVBy@ zEJe&6Ssq*t?sSrIuX`Rj3M<=mIW-bT7dH}}OS0cfRTeAPuZ@7tbI4bnaHZFj$eI9` zgGP*zOCf}k4Fxzd)anv(nBor}alf2?PcVm;vMPr?c{^783bAlLvO;NNQJlb!cX=Ay zZFMl@!e@P0t(%B8P*g5fSD7}SI|a(vRoSwc3xc&I^l%#(4k^Yzz1z(Aqm^kilhG{S zoN#2bp#l?5dVCl;ugav@X7E`uIeBooR#Q=qGA&v#7m3L2XrRQ{F`$Y(6mBMTX5EF^ zFhkQ~5bLfdkRDm}?JR8k92(ja3GH(wdeb8p3Eo}Xe?~4Ynzv$XqkXyW>^T6nc=7T! z=xJ0e^Zh7O`LCEAF{CU9gUWL1=vNJMB6SSa_^+jy%ua}_**gC+UGJvyu=S_zF-}|h z+^wi=IdcFsmAs+4)ud}E71l&Gl{ywDv)6>c&`lvYBA+7SEO1=0$nS(I$$eu_r$Gg7s#qo8Ma5x?>2en4ZI2 zS~Ry(b$Q@kt8D+Pb5YywDSEfCC~YIA>*G+pBvT5MhAnF5G|X;JoGBuZORYg9S7{n_ z(cYQP*bT3*oUF~WJ~vxgk=%rxAi;;=&2=KeQ|o0(e|E5EVC%)j{_PeqUFBa@q;~kT zyAp4z>MFp`1~fiS_1@+4IbSZK)BYb4z&I4Hhy)UuISikWY3ZUBXDf%#&5UYa* z?J=U9=%btn(rzTYB8}Lh;!=Q!(^p!G9ipv6wivI(3$Eb7hd;VP1R1|WQ1cBCc(Z>~ zwkjq69-i>`)7^r+8$+utbFw&UZrhyE+eW0du*}jDg&-u_dTrvRQTT)Om(0LP4U1Rl zYLcSAKzeg^y_36;C3Uu+UjKuZCe}vVB>ko(miN)*_P5|olp42=c7q7cZF-N+7dZENACS;aHX|SAF03B(XK^H)NCF-^~;wXEV`my?oUtsF{Y{!Gm6SGRSat4 zQ`Cko>Qltyxpl3lgq*m70})WK$0!Cn77l;JPHmq|UHSV%J3;8V9@$~Mk~rfw(Clei zz~Jm-vJbVMbzzbsrb%zwN7or_Q|^r4i&Cr~pG3ztF8x}vqI2PAY-)#tre`lNywNJQ zc_@DbQr-d=zww@@5=}=)Tq7O>ceo`<*16LZ~v-dlWM4yIfyrUW>hl+YP*ZKJxVzMj{O}wA}X_71kc_u z!;mDyERhs9(x}qTWw#91u(jP;D>-2cl!Gb&LXzRNQ$*(pB$Y*{{-f{w3V2t9slPjx zh5In$avD?t1q7w4KY^AeNV{ZyQcsO&Q@HoGzBalQvpruS$F&sSM9c$A3dr zmY1=EcB%UpRS1#}h0-rb3*!0rSwkV7LP31z8bRv^(AJGw` zpA@fbHqxU^;ozL)fsY^mr&qFM4cfoyx+tP2t)Sn`Xkmtz`2~^RpX6C;6TgHr&HYRYQ}SYI>MrEcleTnh5A>m}{I?O2OvI zK4IFQYb$S)*N!1e8#-y8c!u*6bnRE-HAM9m?*JN+_L4K3OZ;vj+@&E$3R9JC#+?sfDnS7*if z@2A9+U7t2iZzA#DkXzg5P}jrZTdZf0*h|mQ zsfrLgYRcsStEDbO2wn45rr%DtwA^>B^B3DnWxm8H;SeL=gB9q)CEmPlD#<@s$@{#m z)<#NrHc^^4IufT1Xms`UGkyzHh8QL~Znu67ZzooeAIxaqS+cqBBLAxUWVt_aFwfg% zbuqtArgh}@zLDr1Ogc1Rxf=Y_4V z$b!o$m{VIDh?wq_3}qZOD{mm4%b;HUcEAQKre^@J{Oy7Gh^iv3CIGG~;*OEPnRz&GtetgJ`I)LkrhC9yq zE@I{HANHLiLvZ*xHKyVsP{-RDVFw}>yRo2qPT>u zhtl>GGs*S(R`*;og-O#Xwi=rE{u~^2Vw@!`w?IvS&3`1kIXy-xSY!TErT~sp#uRh| z1kwR!e?p4782)J>Fk~Wt9S9@)UKGL*a^bQ1@hbygTCtONS@yDNnzisuYF`{lxVkH^ z{QElRmbN|HD=2nIX>=DM#kp~-S#Yzw5{~yZUF}TvLZ5NGgJDqfF+qSI^ zXKb@$Cmq|iopfy5w(WG1o%OEo+k4kJ-@D#(YSyfJ>Z$o>j&YBh*SN0E;;&zk6@{@@rGx{3V~!R*|@*b!Tz zI?^)(ABx{U1Up8jH48!(&@N#i1!%cb&#ulvuK0n_(;5biazn&?Yr+TI4cy=2zz}(c z>ao zFep3BDLddl=5S8@UU0^<%n?8Xfc=1?j{FSH>|-SDZ|h63bGz*XFL}mDcvRr_It6YS znReTiy{GyPuC9rQM84rnT(?CWEyr7ovN;LXhE)Xog#3*UXmK;$b4@dugC8=G4Gx*P*fQcHxi`k5y{zD>oqPptl#i<2BWRE3cQ&MUiq= zcmLP9cpdLW-K^GdhW(jn4>OnFlv+dBp01%fKD=iq6&c?7l`}X=XHo&dSnYP>edHrU zyJ_O2q@lw*2n-#**nLW$NnbNv(Qfc6>STb~)LK;zb?#dsl8H4pB8A7R_7;A@7LKvg zBhP9viTL6|t^~!e!YE9Mq5hi30;Nbj_QXi^;Oxm{*wIS0n83!#ol));wWfXRwg}GSb(B;v0$m4Dl5Cq4nb$)gRXR07=X4Co} zn?K2Fnx5$H^q#g*9}lf(@Qr2-4>C)e^FH1BlF69ry^Umw)T=pK#P86!HK(|6E*Xb6 zf;HP{JryBpVq^wgI=Zc}&4< zBuGG2e^n}nku`vY-aaI}lswV5UwS#G^e*W$Q30ouZu4NtTV88mE`k=k_gAQmr z>xcp#Lmzz^hsZ)x?q?~L^%P65nsc3g5;e5CV%u*BD#d`f<83M7{1RIASqq=$X5`f!^k8t1)m;(B{5 z@gXGD?2t5Q4q4N6z%;#~%e8=2yxq^)+TFJSmq?vq;A!%>7BqZ`p~Ns%XD!E)1L!51 zGaivnjpS8usS&B1VmIQ^6^U-xF;^_bO#0gt>?~#`*)M3%dHLN4NOr#dsP4 z8JlXW;s^wWMw9%UAPR-$bTFNuWBS29HKI!3xS{1{^PMc-Y>+YWj` z#lg0Vf}uPI6Cte`IGWm$P>{>~beHBsZ<5CKL8Kk>^Xlkd;0OB*#Xin2Uh1=d$>gewLI4+$#BG6L-f0aR_#s2bx8vDkw? z90(q&^Vy97EYJ)H!Hh|qEYU(P|4-!=cg~&j=IlaFm)%m!hSG9U*^Pa${aFhLN5Vi?`xGl?{I=5J& z$USbdW?T6=9En@tTa7{(rjAZ;octfMWYTS zhrLdL0LyVVVDgd(;hd57z^0U6P-=gyUN?|ze~!rFYosbZ`b+g|B${Nw1{hODnk2jW zP@6v0me6Nw1i|~`w64IJ3$z%=eb^g?lSz2#qA%huac*{slf@GjA!P$0Y*H)L!^UA` zhemPQcIz^QWaG9TT($Juj|Ahl0p_YV`s~e{KIzY-23y|M*`{v?g(+ceHCfY1OtBjB z)OxkQ_QDVgH-@Ea#ZUMHN&*|L;waCQ3Z4CMe_F4>un+z~6iq*C9OQL=WZ%R=baC`K zta(#2dJwsY78AfAB1z>VA4Kq~!ujIGzmZ?gru*^f#ukzN37J{I+vgAz`SvgAxM*<& zV2v-k)dK&ge1`r%xl$*0I}>9Q!2gxg6spTQ?s6dU3Zs7tCbo`Sfty(VYHx%T5=znq zK_g2|%eQKda9)m7nP_|Ux_bj>wQl-xR(CO4+z;D(^;>M< zEY|~I^&cO0Ae`l)F~JEqtDI1H-znf;#XQ|(?aA`Qy2U-xpeKCpxR_YrD8~mxjAw(8 z0Uf?wNOjCb&+mtab{q14a<;B-$+Erdou^r`a1pq2K}pVY0rjfm8}Ff7=$ zNR_7N`tfHD$D^S5_-Fv3f+alxcGcMl{r0#hBb`V(prtt0llC+)V*;JnHAx1>w)g`VAHtZ0U>WtX_7|5>?{ZK>LMvv|OwH8r z_Qi)dqpllFvj-&4{IcIQ zSKDS1*<#{*va#e=NfNK9wTw+hgNm@q#;2(>iMZ^JqkgjDh;eK41`wQ=iZ+Q5+%oLd zAv*e;<#@ETU}Cvi_#~*S%8X;&3lNaO}(89Tyf3!qY>sz zHk~E4U&U-4@o77SWE>;9`N%m*zbBU>DtrRUl!Y;t$^*|8#)S7>|B#jvj6+B*eH5w2 z?tt$p6=x9pSJ|-Bk{(GJ5Qf(xc=o>@&(N3E($viP9YXl9SVrW?U$S+tsp+$fpVweh zFFP)=_#78N%m%ev@$fLfWY3)z#P`IC-zq)o9n;|Ps8%fB9LtPfL??nXpvTQzTmRmw2b>~n`Vx6(DOsD)f!8|sL>(m4>t=KKEnCJsD ztSr<#9#|Ls+alr_P)G_TIuDQ#mYh-I1%P)lbvvxGz@IWoys^32$Vv;nzk~Y zDB4HbZ@BY&hY9PvFdSK(q%jc&Mr4H$ z1p1SLbApe-Sb4y#O{`%Efruz`w<}vqqZ4Ajp?btpnJgClN7p1 zhc~MRGEfY}ghN3TxQPe(6Hs~-VkP(k(r3sAoY85I)sg#5&?^}@01R{k?MgC^UV%>> zOcM6%L)Ov17)NiuB$W&z_Z-)EpFvRZ$ZQ-czD?W!IpVpGJEAf4?rhVEW5?s6%h>bu z!$*s^3?WH56=TVfv=yt3hnF77k{3_WRE1iZhwjX8ojQz6o|VR3kH?+O1PY>7CC^Rd zrRrj|>1r<2SCQZQTPEHKH3w!toD&;7IOU`+e@U*~W=C0<0}|?>c|zi&NAy*j}gbGN2VR8V|zze?(f?1uVfAc^j1^o~?r!T|P` z!QK`PkH=?xIozJJA31I6BoQ+|osWKo1;Cw5E3#X;tsmE1EEgh6t=li5z#W}QGS)X( z?@YDFE%>Bzz*#FdeJ>gd65W|dk8o!)-%3aF0K_%tFAK`kV1j+b5;p1@%1>tO}m)Gth_F(E;g>OzO-@Dp{ z5>RLLI=72_v0=58#ky<#xlx#S))3{9)k<|za8x&wXmVUqPA>a1)eUgW^Xzk3TfEJS zwD?$8>Df1kw1vtNEFl;tg^T{kxqo&1r*?sh-hkZhroz!o;DnO@<3PW~qeH~>_-8G9 z=7*nf{loNIkKK(#HEQ2BV-?7OO|49mX@`J4lEnSnqk6(^Wg;)T|0$AO5Lijzal#hM z1J}V1yoP%tNZR)$VE>uVPreaE^Lxmsy?LMPbrJPCSNzP8PG={QW~520-F>dr(zS$> z(p{5a*$fSF+=m}|NFj}O=to!teaXM?;P}4-M4FJM%wlCT81qI<&F1n{J7xoe3NU8! z#3E*_D6|~Wld;(BRzt0cq}DPl&BrK9Jo~ zCrEyVLusGEFK1kq+m}8qlth|F+97H>+iag3i8Nm0z;k&tTthTaZNlx0+-+&T*GQJ# z2|Q?FV%Xzo5)0~R{`3M`YYzx?Zdj%SiLo5hzu>xcO#eoX^Oc89TV8K8`M7d)Q z$H)^#6kYp;4^3jg8TuY+meMN?U!M}gP-=ic&(bmI2ya`!N^by72~Uk>I_;F6++z9G zc6182(RN|Kl?ium#9V_wF`F$_O_)YrWN+LiYQ&8ctxdyVOmQBTXOJXw=8HQyl4{`Y zk2{p5%SXJ$##Xjynm%0bN@9D1I^@dJOa)Uo%MG`pGIsLg5up_w&4#Q&Yjf);&n`+& z{t@FPXL9jFY*uN92!oR00J6#z zmtc+yo!@xDoxw|^u`|_0+cwi*(dZ{RT)+kHRLoB|SL{Q=MLnuN?;oIWV;5j`(}Np0 zlMGK=^y@{ZXsY!-P-eXNkw!sQ#F!fKz%lRe{fI6fvLDs6j`EN*Nv+ihE4)HJj--zF*pGGRw{^LZRevWZEM)igF&~93xQ;a2yVSanz$*~EW0WVV z2a%>_I`Z{os;x3&kQBibCy+oS@=*Ii zcVXzZaoTQ>u9j##os?jCi?j2iBE^V@sE#H7gigZ9soUlGAIrBW=sq39PceMkX7o5Y z$n*U%&6qAx*kpe=62UK7d=~OMe`;nKWL<=xj6oozkTDE*SRYSQOLhkk34^N*!F#`P(0|(JaLh45?AN;GDTDzTMoQP@CB}&L+)~% z`0#M*b{6gh59)!a-j$Muug(H)x4W)&_BKm^9GiMTSi6$oPu>`8CuF)T`bDXw8^Yzi`hQwlnGt{(3g6WdNSr2p^nS^V2(r0a)Hn3Ijm*3C-(%SBZHBK-6 z$^u;eF$?&&^P<11n!hU#M<8enSx7E?9RWDPIXqgu)1RLXAkUv`C+6caRxDx?w--N&bfe}$efGt<;a z(|n`%o;;cJZhJf9GxhlRbJV;AY&EdtbN>>X5EEyo+MABD8F$?j=t|N-z4INKQ2JS< zmlb-6)Qf5-IB-fzF&f|h9CIIB5CLWu#TW4gD0~J>AS@+Ozx|sy=ygOPA-q2x#=Zm) z@Huo2+=PFO1ZNX?4H|bqOpSbuuwX)-U<{}rSS>dcb~~86UQE$93?o}4=^E9+F|%Yw zUjl3#xlt^U=13MU6FA{mgBNNNAb-HOPu;~0DJcP2f| zeR#a&8-mG^XP_e!uh76q`s=(%`Y$n7b&f#=v{Fe16)_7f$V6n)EVX-;j(pn~@{2<+ zmt<=TkMQL&@)b>O=^<1$cJws{^ZYP2IA(NpCMgHsp~l!5w^;^vuVsaLa{~3&M3zRl zk-;^*$y~gEgl1lCO=aFfIles2v|RFez|i{~0&iapzmlU%^daI6+>&Y)xwvUd4UJN$ zKSwK_5|l6z6J-7Sd&hQ)q`gBuF%x`~+C&|G6~PO?+|e0eb|s-m`v%ztCT6#IqgJ+7 zvezlkJgdCNp9I=9B&(EHI|;)o$}4KpW`ap(O%M+(GoQ+YC0Zk=>-2Mwr=t!-khJB_Ln01tR;Q3ai>osP;(SyWYZT zPVAQ{OoT?xk(3t9ts7I2wvXhh^$jzim3Ppm6BC3v-8aX8LC00aHBpFC+8mvV?)7Uj zW-@af5dQ{ih`-|Kv)FZU7?b#A*J#IneBSm&Bdlfv^|E3NHKX^-0AZ$`xv<7TU}k}Z zzQPK^T%dOKg%mgn5GH}j_4}kxXQ5;tvG|2y$(+7Klbq~sXnTofsUyYX2N2chA6$lj z1HPd9_muYok|EeGs(a)#y`$HdbmrjO)Pt#S#UqYW+8|wS8a(a^gv~0)WG8Z=xg8Et zkw_r2us{8l?=UZEEygz_ofR8pEaLWqI0H_W6Wgxk2rBW8BwO%}#r<%& zUUNU5q&lucPEF^9KlF#p9=ZeTJ(wn9+UBEqRrN)w^{whlUb6yW`K|1c`imDxSG$JV zL3($R=rQtJF4@*a@y`K322wQdo7`>OG>PxU2EEiW{9JE5FzzFWc?84hBtv5HL z47f#%tV+4wJ3s`|vCOCeW#t$XSoT}*v>LmgMR0ma&IllQhyrtH!{^ntp*ge82Q_xY ziFwr4YSC#cxY`U3y^)Rpq<|tiQxLp!0LLqY0ef9!Go$w@VS}*y=6jF? zZ|jjMYt%R_qv_Srx;^RSp?rZ*7kF!(ohnF`R07eD4~&s6c{1G>I)6!lTIu1ZCixsW ztd(o9h1!u9(7>VEIFR!4OaXXa1k@6G;;9JJg89n!AEkqraefhI4{#L^cvKh=x8Cj| z3EY@k@BbpONl!;XfPer3n*GP78RtKx+W)DrjEbtY=@mc{F@;`+MkmFmHLr`Mm?JL3 zB8gf5F-h$>ysl{}y#FJA8}yk1tVkd>PC|wehxcMUYck}!uWLtts9}HL)s5+NhOGIy z*75~zlVDIY(HeShs*0MfCgeeB!Y=boQ|jiA^1RbTOmG z4|DKX7=pmEZK^C0MTP76NBzPlYpw!n`(~>+|6d1^H(%?euuGx>mg|wa@shH|-p)*z zZF>b2U3+3QG1T@8o-hgs#K#u~!$PCi5WXA=Z1m#YPoRFoT|t@D$b@)BK9g<6y-JE8NK zsBM9BKgujc6hwVdR7qBagRHbzo~a?)xHsioDXt{6N>wQTWfP>>5&t7^@=)pXT}=NC zGrf@O!>6jdjvB_4Q%>Qq6gI&PpOnx263;TD5eEMnL2sM)E=V(aL1r`XR4x^j{|1Wi8b~^0gUn|F|KP|F|JvX_2Un=@(1Y-*>|c)wEqv zRnb1=;sFk+dK6$>s#1*-b7#KE{**$}+TW&wrOl)TXGZo@J6klyg7~{#KmGV3-eh2Y$0S6!KOY+Z==?p!=Q(-4_4$5c3iNhkg5-g; zMAGrSCuv*4juefAM#4?KH!tvsq=SA36q-=#S)z9cg)jD6bo-d!b>Qk3(kUZKFd}5{ zwJ>~s6Wm)LRCr>f8l-NtTBhPT;#$M6NaGe^a{s!U|j#g^g z)AJbszykA$W>q^V9QRcOY*nl>DqS<{Qk~>XgCHY-qt{~%!XO-0Gs8G3Q?;?9Xv%Iu zgQBd>#L;SI z6Lb_C!h6>$w8((3%5L>LX3oml&6%jZWNx7#FGY%@EO3jU+;iiNs$RegBu_N^K*xYUnBN zx277RXRcP1^3Hl=blnGKIpmBH*zcw}LSu;p@z=Vqb_mCYKUZ?7PPOV2XV?&j5ujCLAhdH@#d<+XVh*1*Rrqd5_b~tyy@*pwdHRJr)t~jHil!% zFVgByze0VkUOGiv;>FN&0+tO~s-0#9W}|eEb5Hgm=#p2KmsoOHrt3|?drnLmNYoXi zrw&`X+5?YHxOL0tlcp!_P?HBN==-W;iue3Tc^o05!hb*~TnoV$*=)Skt0T{%v{?+3 zvEV0P<{a>4s;E&ZLG$v-P|l`pvFrXObz;8nKihM**DSRyYmGWmQ(&*0r@TcBO_~8?G6tC927Ur#*1vMhjWicxA-S2(oF4^5a(P(*j$H=lBd!RJi>0 z>n8TRm@sL-NK>p^i#qg{JTHNwzf?gW!swMp=ti5XI`k{p5?`#xbvq<_mPo=ima1z? z8JE9HWFF$s%r8_D$!#7v1DN6wm&l@O5)6BcB|C(!*h6}mZCe?pU`n-L5f#{|73j$t z#jb4FSL<`1z<R*2o{)ICc`^TD+@&B6#RW@;Q`syh={_RPf;JT8N{4G zVt5u*SP)Ja#1~7PLD+{wMRY_3x21Q}ezoE?5J8Xw28(aqL``Nu7 zJsm3f=jlpGudYq^r{y>_Gg471dLRmI$4EddwA}3lC8D}1v@k3o%?+iroinL}qpY7e zPb&Bvfeecyi@OH>tJj_m5mg&?bFjYRIraN#ed6U~Zw=koxuQ+O>1KE!F5*B^qs3f* zRA2-&;3`+Yn@PX#i7^73@X2mzNO=9G)*lj1+j7ApZ!gBx-HArXeGxHVB!WTK34;yg zARO9>sU$fI2mqH>*3pwaPY3ne5@3p8Y|;sr>}}s+&@<(S&GG@*gE&g zy*y^yn|JnH#a1;wsZ^Q^EIdF}E2nm$#$%O-h|1jg}lM%3(Id;uz@wH9h0 z2bc^USdwqUjd?wLE-D^-mje)-;jy5!bttJ}dfGhjplAWo z%ay=3I^EFcuq{U_`YGQth;d3L20^4W3LjyLLcUX>O@xMRQ)f&q(aL32pg03Je+W7A zzYA^%AU9VBEtnbk?Z!)MfVu@%@}N=3ENXMhM-4AVi!ue!r#zD`jIE8bI7Kr(-Hjdn z{1=|E;3lkN^2f}wQ`+;?tQ6AX7BH?I)eK$Zj; zyfp3_k2SjiJ$F0$WZifGP?l|G6>V-x&9P#(W$<3bM_6=+b5MhTj~Y7Ya?}8 z9{J(ff5}2;<_bNPOv^_UV<{}q*mZAs2fv9xYbsQrZ*!@gyeOY8&UbHk3SNYqV(Q1! zwMh;O&E$SS|8-~;HVVavzP97f*Z1FV$NzvyvUM~y{~O~eh{WLVUGIw(S z+q*@nSj*2Tp!rBOJ5z%XVnGqZAsL@R3}A{ZGa!3`x-$wfn8_TAtw@P+w?_wd?IOm| zzvYAp%=aH-CO_v#9yNhFv+O1xWqD4vPPt#6w7%cm&0zweroiXXhgyQzK&PPDX?NoT z&k=!5_vIw_pd<6-5qq`t=V%9JTfZM0PY-O@LHUV-J4s^=(`^ z=%`Y$`dwmWrl!dr)zDYlLxlbrV945DesF&8zSufC84Z}pm8-tTglBOk{mjw4mUz0> zaf5XMjeZ8f-k82CaqfY!?0jMlrA$BL$ z!^eBg{s^-K42+s1I*XwyNRv)1lL>q3Dn^(f2c-r!9GaVCUkqbDZeT%}_q%(*@eo@H zjfB_CI4w5RW12~bGYX~T=t9We!}ykX`2#rJ17k9{gcJz4{hz-s{+smDd7umMn+ziL z0%?`XK?YAppY7YHzM2(c0(JcX&UG~v2aR?NR3hh54u~p+?!{Hv^D0|T;S%F;9Bh+H z25gp2UE5~mB*qecA4gb3inI$kE)^%1b^c^rjh>Oi?G?L~6!JKBzC}M(My=f2!P=`7 z+(sm7Bh_kRW8Tr|p>;YXzl^Rt#WpR!Q!Ofu$ByHb)_aYgZ{R8wmRo<)9wm+fvHQh$f(PH{ z&9Voqbq%G+L*^cFJb?PzAWdSO3IRwI;V+zV7hAE^=;jh-uL(PHlN_Eg5xCv!`Z^zw zP&`5qO>vJQo zz<;#4>2f9}W^>DXdOZ@6DP z^rBfQ2VtqIP!%oOG4TFY&3OXX;WaBTxEA0m6vK;N!^^}Hx_mITdNH0RJfXb)U(5km zg4p6nF*A@|aOiJf=1Gj91lU91^dtpe1V_jL@?v=0CM44&dyFi@Zd%8ggNoQw(1}O( zGK1J)q$xTJe9(ne7N7e1B-`-dz3-dvya7@SddVX;5#kGW{3`)i<+g5ptbl3D-(|z6_WDnlx?V6-RqUJD{e6hBsVxNu-yq z);#e#ce6`_J^RDtLb&d4jx>S&Lxn;2Na`U5yO3@cbViaBEHG{KMh9S{b8~fR(i$wl zo>874(Ku6J8JMN?J4CZ4k)UQGY;4!gYf$8DMw~7NtZCbWHVL7oue($$tPztM!HqivV*I+vQ{gDtEFR(jSnPhIufESVe<6b z-vOT3JA545;^1jJ#2|X-Ll}772t2;LW(Q?a!wskt2x0`1qMZr({fy#nU#*mdlns@r z-H(DY>QXHl|2E(|(gZ!Exp7_bmNQMRc~@1^+!1svdbvK!g%%ix=Re8YV1Ajn)4MWr zRH~3EQOGeiS{M{<#oLoY>O}$S&r^3)u0igGj`#_tTy9Zi?}1iT0d;E}ZUG3B_DjrQ zo>`F>sjrx+H>9U`revj`9!ft8T7;g7>Wc1aP7~|yHC(;z2YjJeC+M?BTnk<*GQ!ce zQV;9>btpS|Cz@fo_e!5L`hoO^-7p(di1pQnvkwWEzUE7WOuTxf0W;xEZCZPLAsOfP zPw@LGgBweVf8y9$gLeVE$SZw2-xE>?2Uhb#Wvh#4VYp?i+)V%z-yuu*71~%W)yj>%rZsBuZl3Zt9D`u{s7ZI`gesP3l+rjYP zJ-&j|@2Y1E*t%hN-3@1jLa<)!lwqMDdsGE_=zlyNG<5Q;Mmh=}b zeHQw|{jATLu)HEf(fq@7I&y+JRB@mfJ#VG!UZZ`p=lO=@hbAvN?>*$bxM+TuppXbG zH}lpx`_@;di?G@j$PP{$q5`+Zpt}f|6cr1Ltl*ovFUUS@4Fp+$(tywQcbgX-4>kTp zA&-XuZHS4{%*qDSn;7mVv#^zW zrG8U57aSq(u&D>qU<%!Y$(EO6eJs=`v)NyQZMizeHHNTZ8zfjcojah}On@Jxt3636 zzPk!1zC-wt#FxmM9|TRw!x~^5OD1;DW5aY}(HI654JNRVCL)FlgkI?GyXD8ku~omU zIaQA65RK;HaDI{}S2yC!KI@`4qw!9P7w8t9$jOECT*RL<>H#Olu%iLrJx>R@yzsU8bO5OCn_y-ZnIr30c zs1!?83bCocRm#*##YlwJ^@O~%h;4g=WFtR^M|bVwf^1+*TM}G!ik&cwkprJYwyg^7c&r_(s*$(lNNPy z@TcQ~UAoK;kThR;f#d}TY*{o8LH)RHxPTo4$l64aofSr370jEkBB7 zi5-|PiO1nr!}N>hVDvH?Gk*6t4c4OK_ZYWJ*S9}G)XS|yp_mcm7FhVCFf>~D^wHI? z!3|7^h?`li(NU*qvZk${XZ7t%g6;<rQSo7r|`&n z-cCETnQ{)lRxp}%isvzn_`-)%5~I0X!pkXDj*QP;lu-hwS}&icR$^ot*6=my6q|A! zxN&CzLl@(l#G4dxz}(@Go-PZ+Sqw#@8H~XOUaK|@s}oi{%w{|WQVTVYwKL=U_+yRE zc!%xU?L+O2InsQPYzVVM9Ge%Cm)K|ZVH)c&m`F^=~1U9B? zUGVp;Xd~Iu<+M0n01p=Ku}o9qVU4w=?iSLfz;xQiv`yVWfwC5=p*BN2oJ(pu?Nd#K z(Os*|i2UyP#bTh5Xgof&XZ9KK(0Lt)N{{l&+7Yc5L-t<1a^273gEo8eu8VPv$ZaFM z7W`-X(RyT;8#Xd{R)yYiU==7yVQ^ zC-pwmj?L!ULQ>?~Os=r@V^O~XK_iz-)_hvhba++}CR;rBgj0Tbr(r#=XHye%KxUPt zqiERo^IlgVw)neX`{a?m_>eXS4uWQ`aktF5Sag#!&pb7%>ivS5sUOj!NTrnN!^^sA z_8(s5N8$7z42Pzp+oTLBXXh&hAt!x=l;4Z-e8axm5?7rN^#YhTMliX1_#e}e#0~qs znGG*(s3Nq(5WOQS=1>w(io_ChYa*d|H?mxw$%I4yaRC5 znc@)IV7#Dy1lo<*ao(^A3>tFTawPxa!P|L-&SFPH6eAfunK)WVoi9Cst`n5HFA$&F>&?|;i`~9fhF>NtKX{RoWwp8* zHlX3Z0wk3)f~dqZyU$Mfbf@@Kk4DmE$@mdq>i8TVsI?I?XL|&^W1sBq|K}1K(b_k8 zvOD|s9MxtgF>L(am|rN?;}C*$HTZ)9O*)beJfkFX%&(m9ydQQ!by! z@YwGbGDm@aUcen;61V|7Yt?7`pwje6e-VO*Q%A)UQf_R=>x1Ezu>3in@Q&C^>ipXy z^6Rv5!l5Z1OFoT)3)1-+?oCZ(U0}6QYwrAEO?^&McA+%_1t-Z}_{g=P+V;0LX*;=` zd0nnhUR1Z(}tn4h!QWmNBz)E66(rAey@+p9rH5fH<2lL0JSeaM~CLFBd+m_YT!C( zczz!R9R*$vUXAb|vROeS9LbqccvZxOc9Zbm$v(|q{WTg|VRg~R#`gzrg?Z)rajzis z2dvv$&0(jn2ftV2#|SWM$eo2P=uD+ks84u2>`N`-9acIAZW<#dEeB{N&swzOi!C{^ z*${%$j+ag_tQ{>~e@MzJv`*kvE_|0zf-+O^&)#0xZBIP>g8cY0ePwu;8I*G}Na#^w z(d6#j=I^rVwQKyLHm@8W$IN&U+V;@+y=xlz(kDkPkp%q*_Y74zL-wpJFGP(o)pNOR zFKI<$OgzR3af)KWN|}AB$iffF`!xpqfG49xePgMWs}3)Sw@$?J736rKjN~(7>JCEO z5e>Gq8Reac+)lyXi&lz|C~da~n$JMIMO>{Bm>)s(IUAm^J}A>vUeYZ%4khi#u8>ar zgk5}w30vA8cM`y-6u<|h(tpTXav#A*d8KhY?MtO$TUd0>f{QVmq%$}oHC7`qlAntV z`s?&6oWnj0L<%t%`cz|RFvut^aY`o>`A&K3Y5>5=7r-L$+37|Z_Jw|RzDlzp7tQc& z@rXHbH^0`EKE^kW=<(vZdP71u7=G!)xRz>WAW(DaIA@@wUt`Je`01{fGr7U=x~0V* zW6R~4-th5+;9m2P&vwm3d~T<^L5Ey3kK6sMDQIUmIbqS9>51MK}YYzo;I56?30SA|mPBe3v_>GD9b)gJT##L{Jx@Z>+lSrB#Wi?O{edA66QGpwr7Tg8>k z9D8PaJ$xKWrD~$qQqQ=vpHI?V9C>fT94p33O!E&oA2$+1?8PQ^|r! zgk*kacAp+w>Bed8w>Sa9yeIJSIJPnCRUwcAV7@X273pKhiju141Q!|MgHMarw*+MM z|1Z+sF-W$x*%n@HueNR5wr$(CZQHhO+qP|Mb+6WH+}oxtaE zWcTN|w^=)4Y5SqMRrxOGLJg;Wn10BD@W^o5iQ|2H z9|&FklM$#cs9F~Gs>a}0qu`PdA$JX!g7%xDcGv;3j31clj(^L3!)|&YM2E30cd_Y%ufr) zMzB3qm<{;QbSRB^ksIQoW@}+7%uf|gpDoPC<~n5!(9|{{=iOHOm5KG*Acu46sI)$8aNKvJ# zO>(W0SSQy|oT6pV;pFF3Qz6Po&?{*E)S)Ac$g}czSh6%gfXvh@`?Ly z&TtOa)hY3l%jwCbc`25ej@o2i6M|y)BREx!5U=Ei@7D<&CxN^QgVw@WM~SX_j@zud z8rUY^IiIZJxyI`Vd0GyER7krcat#ZA&Mq6#Qq#M}Kv&qJN+KL_QO0gYB1y(E_e6#-nsGd7}9>g=Bw?Klgx9z>h z>5Q7fa2nJKw0Nw=*LNCO$?FF3r3%Cnl!ZUb$LM~%`e}&zRtF|J7hY&K%-#@?1tz`- zCAj4V_95-c7!6G0XkW;#~nIA54hQ~h}VzP{pUkc8l6(=8aWPe%gEwW6!@)rT+{*aQq$kAnayr==|5z zUf9OP+`-o7uWhL^4lEtz<3EIJ+|{UAmNb%9Vrp`8nbqy;;`{OWhTVrp(tVa} zFFR5f^@l;kC~9Kt1+P%BkC`_E4tb7duL75*-Z4Cz2n&G*s*NCT5f-)ppl!Y+4|Ww9 zdM`wPh#Sy`qK!I6@JxX&Bq_kY2w4hmWG{mB1Wn^GJlNV=RuL*AUQtyd#pZ|sdSB78 zy$E#)S!ZGmVis-TzV{&4#_GX}+_8R0$=xziGYQEW>9e3lD*4HGGAkqzgJon7eHEsg zX&g^FGczmS^Mraf!Dcay{>f>Q$)!zRqh*S^yAMnVIb<)u=C1hH3}~xLb2K2aForX} zTRzw)a-)^;YZZ-y*2W1*I(Vw?X-X3|gL{)fJf_j_J67ZBx=FlKxAA)rGgTO(PS)D% z6YT!{x1ZUJ>=tMn7_6cBuUTRl+Sgqc_BNYDgDG@I8tlrH7VBK9zq?&4+A|lItIX|x z<8l*L4`*B@oYyT5C!D(kd`cgv%+nCmpj9)VX%A}#*>glMOe5RRAK|}`+ul`SA1v^A zjW6+Z<&O8qH6;=c$PBmu&j+gv!Hr-GluD6SN2z=h&AW1E7PwmKLj{%@NBA`n;UusN zg9p6J37c5<-J`35nE<+lRz!^;CVC=oK!tid3YdZyCY4~cD-d9U3V7kR~Gum^*KOOn4N_jNomA_2gcJ3~y>wdkZB8KPxl*2Cykl;PBs67XJu(-`RQh z>pF|0FxQBIcUB%^w{B$JUAi?@w_?k~_u^I)UAI`3zl9fDCg+5_4!+ zu^mAty@(Dj>241p#+>4^Q6y&3754$sN6jG&<|E>ib9$1Q5f-Ufe2mrKFbFU_XH2A6 z`oJ=R0$^cFt?tQnyFHrWynmgA6=P?b@}dgL25zbfH92?(y+)#`}5Ob=SN1xs&#JwRa{5s1AqMGwyLu9~S3+ zPaZbsZjTd|gewi{z9$S;2n*F-U?`M8CgMeGh#cOKfCuG1JWN*Di{!u&UN`JTVMvvr zfnY>9i@G1Osj+P|65;A@AoM3K(UtN1@vy<8n|y={3O((Fnr1|{&I`P;eVj0DBpcLl zq=O5&C}w03>ZDuS7QCk#mY#hmYI2t*u}l5h?12hx@%)4VYb2-o*730N*+V{HYkk{h zDCOL^0WQ>fDCPXM;Ez^GHpuK@Z>ktEx0vQrdV+nlpOS6>vXXA`+ByNm`^H+m)dJ=Bf6+x_aVgs3IadZ=nw)$!Vn`jAlKNk>oh653H1SJN{A0^p<1P4~pzITjp z67HyLvj-RvdPGGAVBPoz*taP6xkmrQ56ia6`k_gkI|zt_lU0;kBf^!HU$YUxj>jie z5ff6AuQPG3^_!v8SiIY^65 z@VCr1n3X#xn589U+vG2d(TaaP!oVwx0+Yr`e@`yU-;0lMY)c=a-GO%>{9P{3X{QQk;XYz`mH{KR!Zy)NZ87 z9UaCsA6(gYvmNFwP38@2^GlHj6_CG&oAHp@nyd-%?9`)wlEXNd5ay?gFv2fG5X`Yv z4w@8JHJb64=bHJ28_AqWa-E5|YAc$}Ffp!GqWwV!qe%E>`)kWd?AUM{ z4kcq#(B4mk5AFCHDd80PYs!?KGZhWFJ16^D#|XeFv%@LF3n@d(BMT}yqpw4f#gsZF z4*WUjFd45fX`v(xLO5J0)c9zzKqpjMdfa+E4@s1gPZ*UHoAEFrV`f;p6js2J#Wind zOlyx9{PByTR6f;UgS2PZNg`t9Y0@OAMuzX3RT!DI|ITG`1A|cK(1CznTxN4U$EZ=@ z%vMwMT8HFPqVSB=baI($qqkEm_YE8KfdHSbLs@lE+#)PnmtITc9vY!#xv#&Eitril zEG5rVIm@LZ*ID@p&uMDB5aanAmp4hW{0Wx zw3cX7j7TxcXnRQwTR4Qx)|e?OX4+D1+^oz1;e=ogXlUqwN2;9>ZQK5 z{o<fJ}j%r$g{d^l9iYe_C2qa|bB)c?Qlf*ws5m+cFwAh}NAt$WZ$OLX{&RK6Q&ex_nw%$Hfw3 z$$4?vuqO7^Pw&S0idj%@%+%b|$P)>ZxJHAwK?mWnE;=tUf#UB`LPvm(4rRu}v> zTdfvstrmzPUvzXQEKRBi~x(c<2Hb1 zs*L_Hn+?YzPjE54!5fEZJ0^kkDU~S)&~GECL}np<46_sU&Y+xk%)gGuE)V4&wcn+s z>)&q8{15tp|241_cC-6lLXgupwEPxm_;*41FJ7p1=XC?$Wa4|&m%zeMp^j1IHFI|{Y;BM8FQTU%~m@5=@zH!$<6$2 z?Jj_n9*rM5JR$*!@D_r-=1@}DIo|sjl)#WM1P-(vBN;zmEHJJ|JNn)UAZsA@U4gC$ zu;7+fGJQxHG{_(W!WNOea7ctCgNc10f7r-(*=T0vEV{Y4RAHF3Ty;}~Hexp25LwYu zc`VssafMx`Qf$VAr`e%RPsMl)X+{4iiuvAPg0lR3D<;QPp}v&*Qhk(V${^y9(RyXG zDhoZ=a=K-a19g$tHZ7!aY)@JfBs>4l=dX!{lZ%nY>f_P%sW%M4*JJ zr0!S&)fgo`D&EP9*D%zoCqup!sO+-WMODhp4{s6u%)r%hc}Zh7DrOg}x~k0%!TeMg zKda+N<5!6ofh9DGm&7=iG(0-y61{NXVjK+4xRMD4tF%MhCs&p|l^f+&7MoHTCqf+i z=sZAK414O(M!q6BltnleZ**rhiwpG=a9U$I{2*s?iQRpE5w?lABz?QPmVsdWNy0&q zU79cV3arx(eTV3Orm@YkN{vJp`H?yw2_FPcx+xc?u;^x%lpG5_(u;T1pNLp;bR#^+ zzeZ;pZ-`O|3r_Cr^cC;|;4gTCgTwf7^T}3kqp$>OSvLObUaeX#IB&^E`b>ba8eaj% zdNx!RR@^g@M8d@nDa9o8j#PH1!=J(u`ad3=T0q_ksSs{xwkHtq z7ltC>cLFbc|3MJTbbYQ)`7Q^?zDeW%xt;nSoEiV?d6(9=F*h-GbP_Yx|E~Omt^PYa zqM~VwsSNkI((0Tgoz*5Tu>d8Vd2&J|LmszRV!o&)rojoKpnxEuZ*7|;jm4>5D;O9Q zGz4^e2PhUc3SN$&bRi5Hqyd3%A0G!t3%&z)`wlqZx^l@Dg9@Skz1CoQlG(UBmGc|+ z%j*@!Pv7e8J_>iyXX=HXQZP9_EN>F-#Pyh^9C0-~c0$%_2 zCM#)vaVu?0iA{NYRED%@4arJVk^4=aXFt=URmM$AL#N(F`dC&udPO>88+^q^*tt8v zg4S_VWhQeEi1g4sQDll%%G|c5C)Oxp3kL{--ctV;FMg%P=IoW2V5K4+ZvJThE!KHN zPAVS!1Z#<@vtt%0cFwq#L?T?&xw>^vUHn{>r3HbYtRi)LKZXV8frX0h{I8R$k`g9E zjzViQbw{xvad4_5ND+n1d5|nk>0% z|HO7H1!i=C?-J1Lp)^qve%Q3^F0ODhyfAW zoi$Ysx;n}a4;$95EYVM~Ac0{~z!@Jwq}KDjy+l}7LE$lQ z$PUC^t%JdD*Y!)fFVynS^@1Crz31x%7JZe`^n>L~?jvZ5#c>(LPc{&IqDEr(op&L4 z1r8OmxRC6X1bo$Bm9CPj2oX-eBH5_+37r(&1qKLT3UUB$T0A4p?D_>Z);|JkN(f}s z*ocdx1TWKq2xP>=JcLp}SgDwxWYr@)BH76K`I;23k>*FW-TVYu-ek8RQ|1Ge&-%U3 zbmFZGJt@z67(D8>Ps4878%J~c;ek2{45J1)X8?NpAUeVp+lFeS6BfxQ%v%gY3V9~! za|byXS=nxE31gn=VxM2v-?IqqQypDn<#!(rUkM3)PgNgiU5;$2#9`D@0^0C{UzV*z zQre1Xr0YF-a`sl^kTx&*z50BrXy$Je5OMN|O*&n_phLF?cGf?tGj6+2qLVU9g}96L z_$$&VViNuTRnU3xkDY>HhAlXrx>S+ZPtvqqn;LjSO6p%+okJv z>ziI{+I^bknauSu!x{^uRiixU)cgv zNXp?;@lE0IsX5GbdKVb2b$Ztrt>oC5yDy3HUA|9=**$-8#s12tJd-%DkWr;{UM{O- z?yy){`bI276(5+9=axK~oB0$?%`T**vy00cnVaPlPc1CsOHeE=!b>Wcl?%%?q=UQI zU~y6JQUO0D`c8qNq1t5ocmTjaj6l4WLx*25aZ~ihLM^rk_H+QAp*pi1sFVmJDKlK!P92;HDhZdYkp>;HK;g0)9&MNdxwr4LWS?4MM7``NzK0DG=2T zg@SC6?G*yrpxTA{&A!crU4p!~`^iGRME#)23hjlvg8;}yx+4c@g)sk?t!AT4MnT|?_{3(1GXhOF>SiF$2 z0fS8|_6ikfM5YXC{X}OL^@H{&Zow0j_!J-qyb*?u9MlrfgudZ|yDY_;@nHRLI)gPKZ$ls|jb%HV+wt3bJ95 z=<@IxXC&@~lEtMHK?J2`_WX=uCt-ujhrN;Z!1M{zNexAbfyl^od3siffVL_lIq7C$ zxLPcErSNdW)>H@-DB`w#BlCRX5js^h`|=2k)@9VNVS0DUM2;eZT0&`Uayrg+4sUS5 zK?SBv`5VGRN59&{6)WNsf0Y+%%zL_)kx1wC(#Q+iY6!Zcbt(y4baPDxHba?)&u_?_ z=6WHj#eo%*OpKm6Dc52HAoLOh%NhKWVNUiEyPC#5x0P=D*mY(yE=)whdL;&Y5k{D7 z?aikrSCYf}nrW*ANaUZ*cv9kln{ihs8}HHcN?x{eQcjYvu`$dpPZ!Z2KJdrQi$bd$k#%ir2O#6gRJ)muJ@~D^CR2_}jG=_e< zp2m)K5*yP-Ai_RsKU3~p zh%|L}&S|Il3XN(Mc6;edp&1Gq5X+M;Jo=Bd#q?tk?>8{Do@ZILWnb9I|@}FfN*Nc86OTI*d$r86aepv+@!rj-=VLJd5$Y zRS6ZAsCj51HLGnD9+O0uGt|C-U zX^hR)fr`*s$0T0mzJ(QaBHEgS*bW@h)7Fh)C$kz%d`yQ8O$*tU#Ga6 zfufcQn4P#{O}cMPyPzX$Y%VNX3VPM%$RH}H`EqQcN*Ae@jsrpET#OgjxrxkjnVj+M z*w;E6uAvrq@-fv{YaYH7lDCo>j%u0jBMNA9#2Fc!k!M(;hSD0BB;NCL0yR5*&ZOm< z$fL9uk6#@tWG3?7JpNE-%xCbfe=IOg$`U`3>>G~nE^v~IVy?Q$YzY|ZDMa3gTus;B z3S58R?gaLO^?B_cQxAXVN|N_U*LZQjOvNo<8OtqO8PoM&QMP38yO&At*AzDYGlsh;A&xA{o zI$#-O%1_vZtn+9d&l-r;bB3GO8z)>qs8s%0p+8ZqTlbk0n-@`kj?H;_;KQ zq{aX?{OlkbZz$|e^n~B2Z4?2@T^W26Fc*Ml@wNaP^>qI9PH#tpJ8zUjjC-#|4tw;RnCtm<=I4ww__@-6f1oR?C0< zJk8NFvd{86j+2)(M~}|!ONNsd^1FnOtx>Y3)02s^W#1uIwXe-jT)178)42;;=QxxY zr%t$VsvLV}_sU28sn(R-CIV%3wc3Rc`qc4r|UxRuD-? zuvBaT+v-YjpFEz`iV0js*DIqGrs2t(gH_-q_&4OQI5*VpAkWF~fPW->qJfBQ_w~ka z??G1XAi0i|u88mKK&hrBw_+K{mzLFm1%_6GzSWG{TzYJBRxXJV-PA4}=O zSSO0qIC)+$vFyJp)$@t5CNBh37#0)h1HD%k1hHWINxwdi>8Ok3)bFtctwqiH$yj(b z_UflUK$G2mS^o&!qFyq&P~(O69n&3I3(d0N#F|(wVLhs?LnUCT<_@id+$0&kSANQ0 zRb#F#jL5db+0i@h8gFi$3)?lI_7O$g0ztfSdf$?;rzYLQP`rS8oh)C%Z)&yIqGZ_+ zX<9nAGe&6~$>$WLdibq4t$}}B-6Ej4nO8Zv7SL+g(*X$340KwST_&$;klcZRxkRH? zRIpaSVZQKP6VTt=8yFnOmCZ5L1?EngWBis;G_%VX04+UJCyNIejFHS2xTjLLdq&FG zeJbK3FFGIJ9C;7R3s}UVQuJ0UcUe5KwC}kX4W{sat_1Ls(+*`x9zmuE%(ehmt>j*- zw&}}zgCp*e1Ak4&I$qj#n2ic=MGP`6!?;W8ZayM!a|aXXMxM~O6wTC~)SrY<67zW7 zRit=zhK|N|_Mts53_F?Ip3l+E<+a&XRO9t;PZUqx1lIMhWA_3M52K7shHnjcKe*~1uS*+(k((5XPb2VM(wT*I+jecip}o%DTzA2QZC zhJ8b2e{lZ{UfD8vI!*7436PBlLz#;?;4r75orsdH9Ch;+DLb_IDkwyw!q>sR2kkQG zrHvdBTX*|2i`3W+0yYuB1~*p?N-{A4XdJ_!Xp+)!5oQlf$pIMRbV^BZ-0*?%)$YEG z9Unmo|5NGc&WCrfk({Bjsl1_rf$s8!e+nuI%tTqaW>4cMHmhpI)%Y$Q-Z8sY2DiKF zlTmXU?`ZEGcnwS&Ed4&>*xjc_)j)-O^?(7Xqyspj&Tv;-NVg@bVnZS=3?hCHM@j)~ zr9HLAd@Rc^-C(4BxZ^E8@l`|Gr@Bq=H0d)DFEHHU#OMMWg-WgOuqM@!HlAn>>b&p0*G+=M z4f1>s>IJJ2Wqe9Jl!0e2#We`X7{HbfbNKrvrEtGGJQ+>uziq_WFyY7U}pIp5IiLhnl& zhvIP;@)EVLvP(tBkk%)AX;-6{@$>qhstd9Zu%aE`fH4R5-?fYXgTu#v0@41_F8;GK zOjiD=~KVRg|Kx*B!6j22Q`g&^_xoGgUO=|! zO@K!*ItqQ2Z#};TgcBmIL^|62j-afdZ+SmRfq{Fx0XF0SRMB9ukUc?HaA^Zf0b6i} z7>N8J_&!O2p+WG2AUM+CGk5|Lpxrj^R_-nwA#>N!faKn) zR$z7&dT2@Nv}rQG^3_W8sLMAZQhEoKKXxiK$Y{ODbz{%y7u(Jevl{LC&#*jd{# zql6#s;V8vYEN{wGIkx!x;3d3;EvOkCWjJW-WsQlA|=KQBr>E7o@ zH+aa94-U}ai_d`Ph4oJurA}XAG9?Fj-`o^G?J~O%01PUfu-a9^9@hoH$HINSl*pO5 zUkRX1L#*>yz9A)BBn7;LzXBak5XQqCdBhRL`5BcOV)WOzx%pLKt4Ce4ReVGr7{}V% z8o~oH&1#J={c?}|GOgjP(FW+V_$hbrsuO;LR~L}@zLN^ARElO|9nuAxkns~Mk$&qO zrGZl)(hUN=ta}(g69}Ty(s7BJ0a>?8P1jyIy;;D8f&-%kr)9xoWNiN4Z(3xUREPk9cf877OxwRYpH3*y7-elBV2z%D7uwh2M+*p zVjDoKp2(0jSmzr2SE4L;-bBMahJ36qXR#k~ihVWe!0sAC+4_2l=S0u+W8;lv?P9!T z<;f#{?fvroYgRRU*mjTljZ*dht=HB6AolWKX-8I3)coJ5)PKjLzGXyJ5mk{sMG~N) z<(Uc)kR&OU@%r|fC7L3%m{Kx9GZ;@# zX(%(&6$OHRJNd#l@_o!C-={~-8hha%N}}tIzi5oGGW1mU(R8W!TV%<@^{Im}Y#=8c zCIvudAr=Y7(eQU5ld+ZSk;YJ#4M2c&+Vl70I8lJ_Hw1LI6~&di@Y7JN#o~Z4vuTF| zg+V&##H|Lpq1u8kkc#!<2?jtzF(3qm7{M)y0pI7uN$z{VnL~r>A4tHt!a*GvZb2U) zYQY?EcA*bxZ-F0p4h6ozw+6gG6!O1Q9r^U90gI7YT6K`@^BF4cMzK_uca)DrMGYQq z8E9}&cSo1!W$<}54+nbSMd`3T>xLVG8ydxn8;ZLIIJYuY&S{byOPz;QYfu?(aIjW`IH#?b7!s>FP4m^}7zk(B zgqbs*7o}wwxeP2Y7S%j;ialfOfi~9C)4ukHqo?f6muP~tsU{J;Qpx<9Ccfh`GTLA` zZGwPQL0dnBz>KOa9IQuB%ZsXL!bznRyR-!hQ z#xyQab?;u;=0hvAwo=maSY&FLHhs}zgHdn9MEhnF-FQ&O3SP2!@Kz+ENwUB5lx(~=?0 znz@vPM7!Yt(aTHOzHD?5<#-{-+0oP5s`#p@3DY8!q7p7o58hKv~&>-5vmN|iD zLCw@2Ll&cFoEkgv_RvhkGd&tZDM9K+ZpB+pUt-V#-e}LaW9hDxD!g|Vlw_~c=Vmz~ z##|TKRea!?^-F=<*0ScwRKh-OV_0JO%vAXE$b zj$7b55OAEO!)&xU#tw1cCj*z$6 zD8bKN-NsHRW1-mtx|w_&`3Bim%h?_v++q+dHU4F{OT#jRl)1X7k)OV6y@0}c)_|VI z@SgAeWKV5LgJG2R8_J=|z$u zNoClH58dr;HgG2qU(>8}S3i}aeL#5A^B3`;<$lPIthW#)2k7I@WTmg$nwWl@6@Gj^ z9FqEBtq`=);?w5i?H30TL(~#=rPNXB3k%RkfZvRR?}bF+*|-}Bh=98q2ic2;1l^;B z*fEBX*^J@|{vIj{pu!!|YI5u8wG-##*dELPI!e$&M(&y%^zHof1)Gv7HKro4Ddw)&S?K}y%ZYaq>6Cy0LX(VfQi2Vr8vT$-+sZkDcyZxWU2SqeXh)aREo{nT5hOURLN zE`i|4s61oU3!f}C=fKt?jC{k>)#uG3YS+63sn#WIq57RLB9pekZV@F(Ks1~?g5_L< zCz5CpW?u3zw7ZG8WZbp;k<$sL!TGIR&p;u6v{9N1oOO%`(Z(PyJjkfoSHjG@fK!MH zPZF$6ML$ao{n4OBS)~|hn?qFPDB8H_0oBaX9Yn~(d<=zqMU}{s8Rja`OAv&t>hTr4 z0V$fbAywEcIEbb{ig(-}z6K0#AjqbG>YPc*c@Z!0YDR3TIIpd9ubY3YieN999#Ue5RbGgzb@%#HrfAldv@oI?A`re9Hy)H-6|YjTK>fw}l-z!wT9A;LuCOW;ml zz_A!#p}AfM^bPe5qV0NVKo?&!{USSb?Ldbhr02@I^-A4!<^FiPd~Ut{abw60!yVme zZnQ{;%q4%P)XxYqi`*sm{Zgu{*ej0kF;Xg`D;J2*3LqE1DHAgk369vLV3Mjti;Dx; zTBr+R7r!x2GUXoxqOx!U7aSynrH?NfI2--!ZZIb|8;l z?2lOw&F5d9S|Lr~9Ef?!^JdFtM@>22PG;d^6Y#8ptc9Jv<@iAb^fEp%vJFbgu-1!3 zjE(dXh-I^B3I!7LOo=rCC;!R#5sJa(a{7M4YyXz<^Y>lD|8X4rEARhH{?Kr9yTPhF zg_ge`mZHhhXayB_D2;4>sOSKw1fCZs#8@rF$~AMH9O4TS(O(GB*N=C?&6IdU$^C4y z-F0Tm^Co-z_5R_~^#{I&@|KJJ>TpD4Nu(!YTR=x}B~Cvx6bvk}lsj4BNq#^^>}6UD z-R&WY%}bl1K5`?^3f-WbMRO-~WA#iv8v+-+`BZDPig}m*YcQ0Lp3(qoqmHhbK7;mF zx0y2ow&OhNy#LX~S(>SPm0Obe_^#XvyvRw&5ZTBm=JgjPeURbBoY`r(M6&yY+|JY! zkzjYBLOb)>wZwkBSq0+ z&bzrQq(i!0x!UE)KXRW;;I(Pzx30%u>oxzG3--6V9u_Xv)Zd=Ue=#2&|GVR-+P3W` z3!JZx|0k@_+rm%shvggnK^(2p^(JN)xK2t)XmGOdlytcwiG`Gp-CHtA6mkV~|KItb zzP+7OPEdM-x~+HUmJv26masN;7_?86g0$vZn}S%RnRQvIR?({u;^4 z#e)*g1fDe)7O>uDhsBO>px2YW^bEJoc)NG#^o$w2$%C}IZ#;7s1BiH?5Fcj4Y~$Ad zjHWQz-*NDMB7%{iVTSMW+@6%}YIP+>3I+gq;U(y30$FgV33LNDXk|pHHz#}9Ia=J& zjY`>m#og-(qQWSM#_xas4-Wfn97J{V&D(y)mZ$jv%QOcH2 zkGH4)to?5#yEeW@&WJhGv>RyTc2JKpboeK0>Dp42n|#`b{*bR8FMpizlqKeer{LP_98wz}rTflJ71^t%Ql@E^xHwNK7L#b#+u7Oc{LhXct=_&=c}t>6JaM%N zwzKvUi_DckW|`suG%!1ocBOR<32gjxls9d^4HLQ&dHaTl05;+bWBZDtkUMybK^Q19 z!$=Mxd=+`PT50qRUWPUTiW5n~JgKu-e;NcoIInfTx!O7^D6Z^X-?Sj`o}I|>wcYqD zhzh2GLEl3oV&&T|S?Xux?1?Oqz82Fb`D!=bt4T%LQ5mVM*Bl%>!eNls&G4iP?_ zsD12Cr_ejg-eNF~)xqvE6>!VRnJ9C+@5zpTtmCYl1pvOqn0{VigG=YVE_gxXo}mc` zk)&5&({aG2!m8Rc;Vgh`x!}UbE|?@5bQb?!gjndL6I1<(4y91?lpJ3w_=9mxxf;AF zHLejhIp6C)}E*@vp>DFP&e`Z1C!gdY*d ze-p)9Fhd0Svee&7iY$E>W_69_>Q>?KyT!Jamba*>1lcmd2 z(UP8tnDp~1`x7FyQ08Uafzk3okmmKl(20zNuz{3$jY2}J#cs+UrqSDA`GYG-gGsg7 zr+eNnSY3`|FMQ@}E4jp>+3eKYt!-9-8gJdTVENqfTe*N(4M9w&#uDLZhxmfU5s3{2 z2k6?u?Cd;4@GVI?xv*{heEWAH$vb6crZ8Xb>@5n?UhkNr5-S-QWZ)hV@ofWjiO&!9 zJ5aa&>+XRy(?9<#VW7c)(LK^u>;rj**+Q7I--o2pa4V9-Fm1bFIHKd6Ff^S9gwT}< zDq9R>N(F0YI=vH5EkEOMXH*Z8A>tRIgYCDcOp7L_CdD>CsnVUgGnRgo@Elnp4JE-w z_Thu2-qK+XcDb2GDA&Lf665xik|VkUM{wfk(@*InL+qXhrqL5=!fLTMSjUFT1Ibe1 zOr>z6x&Pu6Xf()j*MTi5?*7Q!U_=O9Z#K{IDZ(^klY7eBya1wcfVgQwh+_CX_mgM` z161mU87(&~d$nOiXs(UwG2!bUB;%}UiJCv(9jF8H-yJL@|DmYG3jD<>HrH?c!1Z|EsC=PGRq%{lybBH!) zhlt8lB(#;XXg<#~LJZ?-r!h#1rt||OM@(0NU$n^fsHR?z=$8n7W?J2(3O;9%9_{q7 z1B-rAl+%oq8Wb$Tu*8alk~L*+8p9-Vr5c)pgB1B$43Vy^Lk|b5Y07ycVS*HiZm%F9 zCn4sQs_AWpxu+N(bJOmIQSE8IZ&&=s7d*3+AR8FQx-DygsV2~DQPCQzJEL`g+PUhS z*9674o-p8Tu)#=C^wagY7Y~;z)of<^-E#P(id#NP5gwZX@>KoBdzE?)lgY#5DkeTA z8uJaxAve45B(S z@wcSssx>npZM@!Kj~lObidE=V%k=eAM7ao!ZnwQ?{ctwWW|uALNVLQ7R@^!(?Ja(# zTovAX$&+3@@i_lu6a~a6k>Ew^303=)D)}*WPPt3{Pb#U-rc z!zLb4ONnPUmAe_-`8Lx%RwpjOO@aW{ETCG?x{E(QHMS^6Bkw$N@S9E!Dm$4XBW<^Z zbsf!XI$v<_H%u z3}zH`hH-2hJJ#;4YO!l>;FO|3KU^UNWl+>Dc&D^RB<+%*Tq!%|&ZDQHjG^E;aek>2}U>u#25a|*> zm-M>FI;oIgcmn3Sl0PBv%qm>b0{7Y{@xo&vgWG@hAV=oyktpq+ZJqswsW;nd?d3DI z#&zYhc6D%HjgO9Z*GqX>xEnN*ow^6g6a=Fx_(a+sGvqy4rxqrlsi&Oqs)vMEjX#2l zKE&}n1~9o}Z}jSqFj=E-jzFL(+{`6RLpTdi^YbuGLVmO%`IH2Wtsl1N@6rC_8m>IIwc@`f_+8`m01-_*Y*Tu3A09XNj} zp6vs!8#`%eIki8(+BBqUN9Bdh9>JbuqY2g$3JcA3lDqHuIiI5k%&P~;`zzmwcDY$4XDYOV^<^}&$wb5;{pW{;9SQ88 zhq?IBJg9COA+0(swu=FJ)Hm(04!B8C0E^%%WChYAcYGKdHhE&BW=zICXF}AT4Rv-0Qk%{6xyUl-ElAiERf}>4dP>w92<4oV>I~F z0vb+TQ7YJ>^`fKj_@GZ0X9x%)i7_@!mvr@tqOcgwkI{)h95IHGF~BZoXniW!;#Q0C z3E_xqRfov*Okt@`SxM`P1UVw5prNf5^7AAxjor%AqS1&Vd9z_ppcw_?jjK{i=49Sh zUEJw)1>IORdv;Xwm~;msYzClJh@?RE{yLO#5*ptAKN;!uh6|h}Ttf?*TKYEGl$ZY( zY3~$Yd6#VsSH-q%Cl%YaZQHi9W1AJ*wr!_kn-y0@Z=R>S&pG|Q-OuUY`(5mt|J`0| z%r)m)bIdW0l;vXU_wQLDF$^{sR@N66nU_B2@;ypoDtJ0NTc}zDO<5uL<))N&T z<-A^$>ZGBo{n{TnNm1J#R2LU_rIO_f6R)z{e|C+{s7^xHHbK8IiFh zjqT-O@u*@+$16(xi1zgER zU0!jHpK!A>Uy+~Rjgd}gs3J>%(btZM!=!0}514XG%!K96L_LxbV~u}DBZ8GGK^1{L zFKMbK(vq(Ya~9R+q=6jLy1Ym=mAE7WCC1#hkH^}|Wc*!;C$)TPNf5Mx7z<0^2)S@# zTXelaZ$B9UPOLg_P0pDyCL>By92=QX=ZxZ{q_CRT=VGXh`0W{JYfAra^m0fyM%y={ z#CjS!VS_jo*MnCJ+f`_Q_p#Z2OQ3u9xF_0h8^ZJ#PK&}pMRzok-UGeqw&570xrC7L?KmE_IZocU&5rfk}mf5N?j7NKyKaObOR<6wO>s z?`d<+(AnVXp*{&O)gM2~e^C8^Gclls-)anyh;2`D5o@yX>EpnTP3aq0%3w@zolc!g z3O~?9Nw_k5<*Ef2FunRhRCvumhi;D$8@6l5>;2I?%lv8C5!|HVedu`bv)*ZCxJ8{? zhcIOo>RSPSeF?llPOM}2x504PV?;7%jH0ZaTwk7`z5(d~RcL*N#E}ypzK8FUW=Ktw z1t(8o?~xSBXIYzzth-|Ic=MY5?I*%?A^?Nz(&jC3qcjBGUD60b;nx`dXS2~HWM}pK2 z&Ts^RvN7hiFhKOafZTtD9}9)K6r&g#Q4eka@rw!Z%L(}f8G8x3xUa;#N94ML6gy`E zIEUin_|7nqBn=TxFw2)s)(*chl6$ z5AUj}l~wPqDK?0w3XI<)JB~1Yw0Nnl#EQIPl9uh5X-9;p@sc0W;RhnG$N@J$2f)20 zPFB*s*SM4WL{nVt*+kV3Uo$X2VP@E!-nu!*d~cleDU4nbQ=KZ5Ys{KgJa%C?PtzXS zZjc>TXX4Xmz>_ET^KKsYN+R*DiRnkh5=e*2#gXV5Y-RK%k(AXL0$Z{Wk9F1+DLPv$ z6l2vMvR=~g9mC`B#vJlJg*B3mQ$W%eqQY?HpJudmnvoKyXR>p&_dSaJg{}+SbBpCZ zDtBo&`FhPAIU`(1KSIi4cmJCIk)aFf|3cln)gQ$8JCvkK|kB9m_v8QB(ZMp&V zofLdfF_b1EZQw4AE+HV-4(srYJ36bh$RK|0%*65)`V ztOmU-1Z1ea5$`JcA7LX?-rZjM`nemJoUULK*?Eq~#rBrbQ!-2qvhu52UWjt&w`H_s zsy}EQnMS>J>E}HHvzTusO1Q31)oz4GJpzMM#_jO^K6NP>_0;n$h*R0p*ESk)nj6z4 z)FlLuJxWAQ%FYZBMa%nLQn{LGRL@pXpaq4t7#&qt5zAb2Ns6|{MVob}Gqfm2QmZcJ zD4A_bDi4eecnqrLuvNLk1#;XNXQbII2XA&&qSN>uecDfEesuRcoxt%La1HCCA({Ex z?|eajXpz77VeS%D>umzp+d;6LJanAfZmQ5s7A$VfLO)m z2q~QTi}~Ef^x8{IVk59f<&(ouIBNt_Dn&edZS3-QsX)u2N9O>E9?JOlp~C!q{D{K3 z6U;S{F7Z#rm*w*ev0yg@muGLH>o9Y#5ZO&Mb2W#9a*9QkRHZprMvd_k@?f~TeP9p5 zMEJ|Xyi7X560(haQ|?@}$yw4$d1|CZ08EonX7vtN`*8G6c)7Z|>&R$Y4=$W!7v86B zuF{9L4~^((Pp$-6ri2P-|I4D*8X`?DQ7fP>t@(8z_;1AK`4oZ70Sp{OWS?>YfB&eZ&qGExz5OgX9@K9BEkyfOtxRpreg4&;eNR=TpMk> zfh^dxLh}gZOHSgu%cpV2PUJ z?i=s%lG2MwaOcC)EVP6vVUd}{mC!)lXX!nMV$iZ~^j$`P6M){6zH7^N0a_<_z+Yk6 zcPjR_zjk{C-$>4$cs)L-0?F+(FzvX4y|_~QlWzmtazm4CG>DBnfBbQH>02Ze7=KpC ziGNuk)BfKTvc0v-r&plUHwjmdf4E%+d$6q$G za5QqUHgNnVVCeMO)HiY#`uhiA6MGX|V-s5=kAF1#|9+S&DEuL;{{Gm>&7#CK6&UC# z3A>g<9mwd3#~qa3I8t4-JB%d>({f7HjiNX(v>&9?&d4Uv2q4|yG9YjFeo^#r5ze}n z!+4SX3mF^Qf{Qanf?^MvMb$JnDhp9t9d!~;5frQi85TvTts?r2lvrWGM+x<8`m4(F0B-gP5jy(g~t*Qf0_D_d2kTvPn*R05IcuBkF z@uPYNS<$@1Zf+KHQ8Yp$5-rmB@c~j&axFcA8kuKsLxL}mQkQRHtxrh|Nu#L{(994$ z0o<&0k|{3-HZv{)S>Ua1dPx_Zts=f}ezUl@s1a}x*MH!%+x5td?X$B&`Pp~-_nRg4 z@A3J6ZI-m1+5fs<%Ik{A0th^bAgnah2)2Rf)j(}uprLeu@)7uFzd)s+Y4S?53aMMJ z%Gj_{-od^H_!%UFB~jt~+!WAw36{zuly=@rKV@C<@@y{oeZ2jK)rYbR3imBg+^bni@ArDGZ88hWu@L<2mm2trXeogB{gN~ z6pi`{W9^^Egd>4>1GGYGYj%%2!*V%^Gk}ne%T{nl!}b_rix$zo+bXE{4Fx&oGJFOv zz);O!JKm`>aJEHm#^lODP;NPNYuvNg=y?N1OEGdOEM<{fXyxaR?*i^uJvG1 z1^RtgFikT30l*Q2oEd*JLVbF`voN1%D(gE1vula!by9eTc52~M;JuCp$|UwsT{5wY zeJrznkA{&SS(bI|I3UHD9(;=B5!o?8M19d%8%>0`moth&ZlVj|@0WlEEEiftPQ4mP zw%U1&6G=UB7($NJhtV7od7i~2y1~|2N+x|TSxCw-GjaDA8zVAjj*1MIi8q44V*awx zcv@~6tcDq?R5Z^GKvSG<`Yup`wM;27!`rfu(yEOWU%iO}XtrdCte%KWDsE8qfypcP zh(3OF1DQLow|pZJ+I}#ardK@wsSu9J)d+q}gf>$(QFeN+q*r`tZP->e@HUr77hy1bwo!LEoo~u(+uOUl3%Iq{03Zu^ zg8_#b+1zrcr0T1(&zLc!mwm#d>zFaZXGU*}gWfAHZ@Uk89gu0vJ=opbznOGQBcz!n=lE-vs635J(g25=`K&4v7sR2=Ox(*y~?7>eY*GbUwQh}Y9iT-nn6OdbgJ z1-9Q5_R=wgiAdq62hyG1rbDhrW@@&-aE}Br+IpFLwjS^Lf`TqqjOM9|;4M+5Cv&pe zFNL#^FQ5Myqy=;>Pxt4$E5kYe}cS9ey!j!@lr!6j6yj2D) z=m5N}j>8NZSe|>yH)31id?KiDuQ)#@mS0TV4Jz+vq~k=4!QV669%W^f^P{f7@9G`3 zge5DSWAheuDMG3tNd{bJG>yLDpQdL9A^aYgGcfyc$xsQKlrnH(O>;?zFBUxa-jiHG zoTxzvpEN)v$p$Y(n`EI!PPr1;8LdoOPFspNBlOX@bJMMdA`TO^vGZ0?Mylqd@ zQE)qqjHf=uj~5E#tod||M>U&SOmgNr@o(F~yGvAYs4c_LM5;L_BlaV6;mkx>r)r~4 z*_C8(SE#%N&lWjuiN+RJHb^|-@sX2a#vC0uAeE_p7Alqtr5-?!wt(F&ysFU3s*|=7%yC^W=A5?GhGa z3{OBm#CA@-rB#e>Pd6&hIRRK4$vJO5BEj*4zpdl;pdQC7m*I=!9x$xD4~O4MHU>k5 zKyiW`r-yM0)2r*cPtN-%aymW@-p+WDj26%i$sZk^Ps9ICI-y4 zD;rX8+dy;;U^B!kzuPTdxh90O}cLdG2UTv> zY@M!S>S_-<&3&KE@}#(x9)4fd&Z6PS`4Y5=)*hdB1^vBMwUHO?61!KP`CLTvtB`sx z>}}F$M)7)9KRWyO4vz+ss_G{+ut~=pL&?xWKqc zd%{s->7TE@2pL)9U(FM8w?8s?ht286a8R+rgSYrmTTb5~((JXqqE73|rINYCTz#E< z{nOnO&ko6y;WLdS{v{Vm_&)&YA10swDMXvAI@T|VBK+4>%h2i$n+>DjCBHL zgpIthTuaW=Oqy4ri$5fpzY>M>_2QdwD@h@*Lxo>`9&VY}T*s3+_;uA@*Qv^;B1BXx+6F}@vyuC^;M!3blapzbX-$LQI86w9lNE1N6kW@hQ;-~+DH zKHk{JAHMv34mtYYQXi|GwY-{ln+H&nG?WA5`rGo6mHMF!HfZ*2WtB zC{R({ycSxz@}pe@vWP+u212k(fAGmG^`YIRL(P^g-j^CwUk7KImTk9tOvM+L+FsR+BHILIvEI)Bzm1H!4%1EB*HYj zjE@GrI}jwQ5zV4#V652Q`DIkO?)xH*mry0&5uL&fGI%f@yyauq;zNBJ>Qy};)AqM$ zYLI(r2#~stn??L6n z%sxCabH^XeJ0#H>3zPy`R=^hE%fIdR|E#f{o_(OM=)3$SW;1PE3~}IqC)> zi6O2Y~fd7T!_YShw<0q?v6lah_71R%)tpRg<_%WC%WDEr zu8~)`m~HLK*rrjV+$!#soBFa)O^^H70o!mXYYk;NlaF(?W*TI*I&8W{cQ=Voo8BUm zC@vq8E^Nm%(E9tLM&$Pv$H2p@Dc`vawf9gh%rzE^P0l>iAbJ$6Zca;8ghf)P$mV>- zhD2lu%!?yn=@-lFebP_e5?eM*Gs0!qUUTd@NdiiKN3e!qHSrEPXe$0= zVMl18v9VZ;SAX+cCLk*u*1D2#9~m6?-Ujd3yb-Xg{}tc!#1WQGj}@yvL( zyZZjxNoDS(A-iAjdVc-1IlKG@RbrkAWb`A_;d?I$89i`vvk?%-d`cYA47Tr-+TUE( zJq0w|UOtoE#9vvgsmU$@mae<+1hflzS1 zd;J#h8m~d`Sakx(77`K(canHwNDZX61bGq$O(Y`{234ee5(Zr)WD*5}@bCn7QKUqY z-Q>Z#gfyCP%7iqkaEktrkT0mOU~UW>1v@S7 zs(lmgqJ7ltr8{o!l6@7@PgoN#Jng8R0mmq>Ut=lUkz1&ri65!lq1`ClvEeD6kvUOc zG5J()&|ii7xLVcv^gJGQ$lkGMx<@vj)O%k}6mQ&$U%+SH6UpC0xKLlQ`BZMc`WEjH zb64&dahL5NSbXEhf(Up)Wq^vM>hIGbchhN=LG*CfMEvy}Pjp0|L_aAKOC)IloevXB z-5;te^6XGSbYu`qH16=}0&{|bAOciP#DY~^(@>h1%^Q=@#sY*ac~rfX-&}Tx$g-yF zD>{HiYfK}9U(K}V*%%HWe}rwp@+|Bep4}@)j9eIm$p{`9}EOjb-PL_gG&qP{^vej4qmM13j% zbKWRSGcWqeEz;=(IAA$d?^sh;CmydSmS0OyD+Nd{b`zE&XBfogA+ah}m>EEV^ zTP5Pj#CgEd8YI=Q@)?`{W^F>Ac+JyTYh=eR@eX>>DZmo$<0S~SrFS5HsVO$itx!FO zWF5Uo=$oEC@!Lj(s0vfxTKi#6$Kzz~=;Xl&rhsakzn0dVhHEuI)67lmf#$1=ry!(7 zoI+|Dt2#Q)$}?D}y0KDgluwY@3o8`0P%Jma+)arMTfQV7?5$_*V{vN56gEzE6$({{ zK&le#YG#Vgwp^;v6uykLGk=CrWs9Y6aTKj)mbY|c(W}@=n!bYL8Z+79SlT8)CcBVF zWqT5BIa(UdI-CL(&nGXNE%4RZx%jPzVEm#hehHJ+)@Mxo+`ZYQ)$t%f;`?k^Ta0OP z`(%Z&*8Q74@YRk_n+!H#-( znf!)d`pGYO6|Z{<9%AG{-rYhbsQQFV7b+8E2DXJSB4?Kq5VL`SLE_|^`VOss2!UK7 z-H&{v9X=qJ6Fj^iaR%~`We?O zc^sn!8`6jtRPAlaKDlY%N`t#+D*+R?5r*bdw<7%x3q=kAd7zLs-F9$pDR{ZFg6tzgAt487@wOgtN{u z3EM6f7);=igm5(#M;EhNh^iP?py}rQv~4sm?IP3etgSbQcgOI^CRAOqVohjct&?UpblK0 zRAyZU``v+--p&{uD>{T@X=?x;rE?`K^>j(7Y z*sWSHvGx8(kia}iIta52K*amf;t^VUeb@RS*HO!7tpG`eED9e*^3tUrUvRl-6b{(4CZsp zd?C+M-mFNU3$b>0^*qV6w4n@bboMs{7x0!8A-DC3 zZT~W#NXgoRn`?}~8GMHts zN|18oMpb}7P3c}u>EZOWXqOu!9<+U$`d?rfWg{FAd)(H-Nt)<;+S*Zfb-1vtI9S!9 z#MMH`wt+FS)`VuZL%R9i7!2HzXq^D|EdikG`G{*H?6xB#ZVNAabnUnkFFX!|>~p;O zwgEMC=ild#xz_NkVR+1(=58Vf$#3chE@XA_oqOw`W7}U#zU|PNTqnnO>1?tOT(x%E zo;|;M*{3ut%xPSoFbADkTQdSjJ@LY~1+(RbHRgn5Dgd<1dpc^7TIR3p3UIe&xw?bc zuL^O=C63kXXW{Z2;)Pg&c`rnCYq1Xt5vZH}H-roh?!2-qrETbUwqnt}b_p|94}@Ay zKMknQ>#|$Yw2JS@z<@aC@OqVbvne=IOiCEqvdPNiN8D?60Xw}wWJ&YWlW!X|cjl1DHl94~Wa&yn zJ}M}_rhXELmGL7f@9sZokC)5`$XhZNF7U(g+t3z#L#^M5PQKC%1fl4?P{iQnyRXXC zIrtmyA~x|#+gNNr-^r?8&(ueP_+#51$Z0Kp3 z7R4~BJV7bB*m?IJKK;D1q=|IF@f`autFxp>>XXuv00FY0Y+IN+Aaz-ad6#R#mlTiQsEUy?iYrQ+*@8Ot$l1}1EZV;Xxj9(U zWo1`X0iM4G`fLxd_vAQHIf8mx19u`HxkS?I=Rs!r48`*3c_nqaCVEX?VK^ZFC= zy#`aVNU2PLaJ(eQmcqJP(LkSdV746H{g!oLhYXidyVQZwuniMTX4~li4c=W!7_}&R zw<-fY%(H$0VUBp~29nopT>BJ$fW&c-M=l=V-a!kLY7nSC?UotwYNG1iUO;LbJ(3`~ z&%z@>5zYOeyg^ZgXI!^Ql^s?1Qt1w(q&vIYeT?@mIbj;VYsuH5vW-rARU=kqECzCW z1s`KRk;JzJ?lb2u$#)S)T>&|8@Uy501gvcMl;G#v0MfkQqz~V#ZfnxxxZg`63{%M{ zgU|y+`6W5+w=l@;M5qET@?Mm2>duY)|hUTtNH$CYOPlO_V@g1zw zQu#qr>G@0Ka?nSjso6<$yfLD#g`r>DaHB$}b23C5{erRfWn%0F5Mp^P<{P=D-vvq} zjAXl3H8Fk+gqVr-I^9>ICL{;=k>rg}RQ1re!`pPwH-?6zs~Y4+w)hQ4IbI5tJY~~n zo$h6c6vPoP3>kyrcmu~h1o&X4mSIr=2c;ElOzv|>9JJJm#=fNlmr2q-9_Vnmv#ONi z3(DcUXA6Gn1r@r$;lD%w@d#(a$6Dd?`Hp%2m%0p^|IZOG`zd1bw=+P|#L3Ru)x=oC z#974M$i)5+qrkrhzkHqS06&aRu?-+F`TDggbgu6EF3<5bk?R4ty6_(Q#DuY6q zp~^A>ivmar$EtEv5$kDgrr zuj~Ey1ywtD`oka2X5HDMK1t1^ktmFy&_dEZZ0WZCk#LS=2u)M#rf$=^U$9-vl`Z}Y z5LMV7@B=oJul{Rb(7tr`oS?Dub1>8USKoU%L_fF0#v+-#-HBDNKeY{?vojNWK3?AB zzHr@SM>R&l8)wDGr#h$(b6_5uXvaAy43lEwk3FjolQYsyv=Z$rVXB#UFZ&mr{44;g zc~OWA!{_Zh5(9SqwL5|*p!-Pdj}E$e9vXP*YuQO8UP7P~c>ilI3^%Y7g&!9VY95!L z=MnPO@(ZIY)I6=Y-504@!qI>)s`_qUsO$6sB3eK&Q4U%IkuenYAsmseso6t(K{0WZ z=wn)ekQqQz2$01o*lA^La8!5n`2!*qKsD;dcV$5FiOP4d0I1#p1yJ3CgrIr?YBwxf zO7^(KP~Ag_kWGEdrNPDeRu_dCO2n8;(qg|yWY01>+~~Y!n_!t_9y1uZ&au+P7nf|7 znym3I_5w^wh|hw{vuBA#>G0=hv|^*Q#;gG^#w1r&VE0+O7%7-CiWciYlFe7@=~;N; zN4JpFWlD^l{)Gh{pBMSy6MYMrpSX8J*mrMlJ zPhqDW3)jbu-D!ch*F7&Fo7=5d#LF_US(MPaws9-bkUxca>zB3VxzX$-Hs`F30)Ir^ zpCGrVFUrtvGffrR^@(itJHyb)ItOH_m_qle=%wl3sDpAV??|wtR+g(bbYATKe7)f9 zo_~0OS+Iw%g>6rVf}|BW?x1B#W?*Kk&al*MX^E5~p^3yS&KZq~J$*!Up%MCdx#U{m zH#@$Tv1JBv`AB7;Q9+jItihM9!UShi1jD4!DbU~Ec=dQf$&%@S-=*BId2P;{Ml#AR z;qFkgcUEh8Ud#n-2`M#(OOa1X`S|r}qDIAn&1VAY3eU&f{3G~`_nyGoMcerA?|$#ji9%Ae(X!DPEf7AI^@4uWEXN`!EgWjnns? z7v8_8Nzlr@7@rg9@;O#HBj0|-&DMR%Xhg~&)-i%RFPaKz3FeWTul#|AlrLA*QiQT$ z?JmWB>)ec_nyw^z7IyWRGj|_OL@yX=$MCMLwlHnIQ|{{yd8_C$f!AQlD3vZhqHzKi zsxo;}ZQnl4Fdgx3UtE)*Ghqc3s8(sp)iu~OPD^~eD6M)eXMkM76LMEP7#`1rqQL^=AB!F@JDFyV-D&JbLMAfl}%$DW^Y#8HhYxov`_^bLDX z=rV~7uVqglwh6`0RRNEDf23E)H+Xo!=i|HXU)r9r{RhYWe<#v^YH)?f%Se6F=zLa} zZ8ue4@=z;x+NumM#)L%T0tptolsikTw23Vg>sLeGskn?+0A;3Jzz(6EANZBkRbRzBo)ZN_eY(RO;%Kz8AzMmG1)e;k^bx9Ql)F){PdW&UhEmmO;H4*T;(a_{!(NY@6B z4Bm@fpx>_=&X!NnOp*FaqJ7ROs9+X*{36|3udHT@#*l_Kw6bK!gnLp?l?jU$6JqdMHd@ z@pnt@`Inq-r_*!uATA|b$_tmdIYVu6EEtcLxzl4M8ciJGxR`kZ{sU}T?q6kqK6`O=e@Q&~Pb$BE2HQU!2g!f$cf%FRNCatv zFjG*V6^amO3l|cEiHuCoatdJ%Pbg*JNu|Z~pc^?ZDx4?x9?HJKa}A6Jk2DLkAXMT#kBLPKrdj|+-q3*o3F0MP~2 zf1*lIASM8!i2~!Jz`UPA@TSnX(;T4y19{6oM6yR!Y%e#&Y)FM?C_e;qRpA~S9H9YI znl_??N(sZ{JcL}KCAvCmGeCJ(bE~xhbA;(ByHwO$|KQxqLu}8}o|X^2LVY$IV_@sJ zNp*c%eD2`0PMc(%0PR`aU-h&8P^&tTE^gt3RF)SWXoB*$XM1OA#jP=~hE=ywYDCKZ zxye;{>F?pObOxY$1Q(VOWNg&mDTHV>A@2zB(s61;zBz~GYT zc+-M;EZj7Bv+0}Hy(aUF{AiX2gsI>%WsG~i<&syF6ysz!6-XjvO9Iq^y9~nEICEec zQlpABE)%-5o`7dcEwq4h(AVF*R}Q$yd(EnkJW?DkBzq8ryqXwF0w8Q7$mGQfX}?Iq46rNIjr;Rz8oizLGDRs*@g<5DFJN%+zkJnf zprMS$%L^jJoEH>3F@6p+>C-WL2BpNrTj&RukDXl-W+QB_k?6^@NSGDn%e%z5#U{kf zzbrwMmDGF3wVG$6crh!ABGxQp3dC>-QeH$r!YDRL>%}4TkO-*~8nIfNzBYGb+yfmC z|M(*|e9D3>8a`vA@h|(-e?bMY{!>KrzbIM%tf~38OHs12l-&Y9!aJ+W#Ug`Ga5FQC zpcWK6{2tmkHDzqMKMD$vB6VT^m_(Mdl%?1snc^>8?YLGaS|pSZ;Jv{BrXFyN4n$Hy zAgYOm4yJAIhPx`)9^a2=$X+5<6k}8jC^S?WN_ACxk-jB^V1}YV18otA5f#nh0*wAb zyW)a4`rS-qh0!nzb$&r1EG8_*?zS){EII?#0h1G~>fIy&HhURis0VqIQ#^fe6}hbnrrraCcY0@G5Gb+CS(X zYz*cRvaY?6@2WPDV^1j$W50+L{jganH$zaxp%Q7(IBHX|xJIdU1<W-0lS^!;^uv0s&@yx=RGXi*j89aus6fywX_^_8F+Ch6n8=X@Qg^({r zvPbLyp#p46gX)m@?yJeS2QJ~q-=*kboFNT5=8>+X3+~^Nx5S*C8W#H)0C0^6j*8l6 z`oX)X&VH=pH<*reS-%3+xi|VSC|ltl4y{Ii-fU&!Z~rhT17U%))GQVw6tEM7(%1+N z;8O3$;(>>ehzPa6BL4%z@ZIh?TAvV(`pe_}Ux>W@gU$302>%cL*S`T*CRzZlj~^ps zyN05je-FXZ;G)W!q>yMA#E28uZ(&lLE2A|d1eiU^T&;6Ts@vP+`pu*JOC>@G1^@#w z(4&)l^2i}!LNh{8lX$w@*lIcWwD>+t;?RxNX+o7CYWhl-s_Re?d06yCSoM8Jaaz43 zxRj7w$7yI9iL7UCpLAs$8LgUGaI3K0vhP7?Bcj7VizQ(t8T}*HM-V)iHJi@BpKjJU zESE6YcVh14Pxh~Kyh3@^XGy#G`TO@%S=|4==>7Xr{I{~_zl!G&MJ*|80Sumk^J+%8;}V<#swn{^Z0W4%I`BtFklt*OuK(g zb8MTi`+e}df2|F20hfYH!L4K;Wgji3_eVsB$%*8J={OlFg@zWwo5x@pX$=LH_D~

hU>jF8`UCnXVSf5L*fvd~rwV2=Zs+zmPx zTHg#Bq+N(VCGSO}nN(6|(n~-6b{zNfHG7d1+z0=CNCnejH=+9i4A>PunQs8CBvHk} zWK0xcH8LixhZaE_Iw0oco2CAg*7VA?hkp`FWS1&&z^fHLI3)AQm-!~YMNplG-t60V zWiC2eTAUj~qo_W_uHn>PwN5&jd%CS}rUj4D8S3P!{0#Z>wJg?6hlO>=09k2NA$qA~ zYt_}Rqr%Q=8CUZj+2n}wYGBgukD%$|$k^@uEWUOBl1VA>zar&t_>eYnw=l9Zb2P9w zw=nw0j3o)f-=fGzQO6ER0EGt_dSb*9e!-cUp%cVb!1A|s7V6sAmK-F$Do-a3`g z8Ql%s&`LG2OcP(mUy zL)B1Bp(Lt~^0ib3GN0tYbEF85LWSAd5E)oh?!_CFSZ3}4A6QsxbMx`p!09+>Y38(7}3WS1fvC?H=^5$~8C3(}eg);*7?QSQZkGKvGp-J!=O&XLF@&^o4+M`~ zq^3%RP#=P8hhh2b8=`TOAUt91EVI2+x^ zpQlp8>Wci^C)QN{5^H?_f3WuV*H;Cd&w1X@Ue?N4yQPMvrl34L*#VgkKSclpPdY$y zcE1(&e6XGRy!v3KoFFe3e>OJ4RPDtb;?q<-%j+n|>DqpaPj~AJPCmj=@uy8p=jUrk zVAuo%e+sCgaWH~XZA7q|BGmjS)eq4G0`-j^vQd;wH)lbWpp5o47i-%ymDm|t_my41 zS}lOsQfZ|At;Jl1sdX&dDNx+z>w)OYZTVK&(;9A|{6M=22$#%RZQX+LlXio1W(=4% zGBrq<+(ZtSK>3)=vSX-^>67_3LjJ|J*ASrka1EJqYB8pi>})YIjpdHxq)N$cZW71i7EDKg>=G z!DT?N{T=epT>G#_uKT%_N}u`m-zOM_|8Of6Oj4FJnPAuDn;d^ht%{T^qnR0n!eNw;8)j9x&h>3Csi z@o~-zw~QGg7LsFdD9=+?Lyyx{+i{N5)y(#GH*nal2A?f-7_>NqJBhwbuhCaIkVcw{2&R;vDJY&cBh-9B1G-3^qvSS#QA-FAE+(KiG{=h4<+iYp2A!Kh$TX)Y3? zUEV8k_2I%`@U(;*1UJ%30vZEULP+d1vo8bo6rCQqNvIq(7#ti^w40-(7-av4v~P^g ztI^hN*w{v6>y6Viwr#tyji#|}+qP}ncG4IPo94dlxA!^syLa#IJI?))F_M*EYfV3M z&S#2SX?#?tqW;!NetHmEo|;HrG@6iaW_uotYI@I0sJWNEzKp=Ru?@|$g}`tq`EjHA zpw?ui)ol1v(P?5Yw0?}VE@gWk0N*_1+W3%9(JfaSRfh2_XjRq$S^CHiZQPPd)?Q_^ z)S;@r_M$&&RKYYX$J(m54KAUwqIGZyQFYf{MLHo*BC>Q6E>e({ER>jLw>toM|o(z^coIcafXX$mUecgrDrfUTPB6T`6DO@{A z9&4%mQfXw83z?^pnDX+<6OBa%HCTSxi-Jz*WTmcay03x7%=vj1YTL|1u9MJ z8DE>dSsDH)No}oP9@~cVJ;VI3O(n!iOk6iETLvA&9@*YhLMgufn zZj z-TrvT!|Uc{=X#af?dcXT7Z@NimtgP9__JM4l*^A{m|<8oG+V)LX^+89XC>JlXyA%4+4|t ziqSb+;rzK3@b*@v+uAs~uf%PEp)7bR!-YGRs-iAmW_}%6c#QiHUn zE~>r6c9&|cG90yd;6b(7y15*h`e?*f(~c8`E@6idy$XWu$?PuNG^JQKXCiLMvHn`m z_yJ2$SEb4nILFUAp`o+{w;S|e(gWz|3dZhlS8obkHO?S7Z;km3RgN}jjfoNvqntV_ zVCK(HnR=?cKrEoQrolpmc%Ij4J|C>KXcqe!EqN*s6($C~i@ zAsm2roZhBAtrB%Sv0B2v3N79sT{d7z11~sL$sQ4_u4+Y%62QDDUz=-YHrlFQid#=l z^>N6&AfAdbeO)#Jd-laH?2xu&J~P$SOj(f5ye^O*Q>w!JSP47JOSDROsN}gZg_U=) zyj~+OJpg{8*aDVdc41X?iRj}T>R}U_b`!n(tZQ0ym=S&+t7!jD3ZIW_Fofk2zeVmG73nycgCY-Y>0A9_s3i^9``yt#`3E>4&zsozSq^0 zOZ*~o8FfY1l12OjNk@Gbw|2h2Cy=&-uehNJzTtpz`cX;`@eO- zn~ZN77p83AhU-c=LoqU6{Hj|z4ih*wBc3o`-(6ZSrGl6o(5cJM+315BOXUVCKBCP% z;uT$!HC)k<+7U26;v8N1i)>Sj@3L!kXR#o8?-1W^qr?G*1e$vx?GnfUlZDa7RQqZe zu$IV6$sz=Nf#J+zNk4zD<-MLX=fMC==iWcw-1^T5&|j6n{~6N%5xbSlfag=txT|0l zNBMH7M+<$kt&~Gyo6(Xq8YMI%oS?rjM(xN4Nfk|u$NOg)>|z*(+lgyY7L$Hqy%xp3 z#2qCLBY3pG@ObEOnN0DZczDPn;01vr#N;FXy7Mj;Bp+Q%*;cy834#iO1V#isd?D|H zHVT3y2XT(7fG*M(M{xO|dqPl@Z!;u2YDoN!kTYK=At^bib2P~+P*I3co|V%FqC>~9 zlSu_ZAjg-dN9SuY#D3a zt!?X&x>&g>SSV_^Q8zLTII5q_fH#%uqNC%jVx(hB6l%ulF5+n)Xz^AIM9Cduq(-Rw z-BK)~gtfACgOeg8_}Q&;o&C@InXQ{i4xKa^Qb>hgdIgAE>k#@1I#6O$x~ol&3oo>c zi*YOIJWFQ7^*C6VI1}5GT3#rI3li{A!pzc54?Ca;gxZSK=INS=q#wG}{a+7IMp1q00FV7U@S3hD#u zo+1Ta@E8<63*Il@*DG5xl9;#+*qVPHSjC>4u7{e&y2;a?1k(sOV6ojZW_RC5vL%@S z3FQHN6oZBXPBq1kKV788F)R<28&?%}plu8uBs+@==#b>N zO8KCW2j!uKdKr9%o_dXw;p%7IK48}_%F8pI5k=|}a9>vIvh8F{rhAOpDqdl2j;&u( z7ko%z4dvzYjBoW}_P{_zzT``kBPv)IJx;NHJS5?390PO7=dvMCbg6F$KM)8xgbSs! zra^K_A_Z-C-Zgq0YX}^(<8@&E`ZxrUOe`J197q>fd;I&lgX8}(OHwkkGXAfc&s>pDME_mKyI!{3vU^uBL>v^d;UE zwe;~Fro&W9pVvfs(?b#FyEL^~#xg^hq4r1!{4hwYz&$-2tgnpWXd+Vk0GuF3)Hy76 zMnBp;5io82riZWWH#0nk(e#B zmkL(xwG9N+E4z`L_nItjFK_!@Q(7y;HZ1mEza)F&WD9>XXDQHUA_;FWvV;-6!m*|!8ahC@UQgWa`NeOWQOV}ZN*6s4{Z zHbVXMd+OCbpC<4Z3k&xj%SpEXfmr{FI|(tr0fE4+Yl|hPx!U>qR?0R2k!ss9IkeF6 zD2tLWy(n&D$ZNC{+7ESfQnWj;XZe{ka{@W=4ZO{d4x2}KuDV;9I`5Qf@(dLDb@=&v zRPgjkS^Trj6i#r6gHP4^LUk25XNSN20tB=;n@NY`HJThH;KMizXHY-5jH7mC>E2WL zYKqJ?h|cHre)r{V;Hhu!8=&GZ??lt6bNDiI%=Vl?&!2yy!|baS2E&HoWz8&}xSFo) zNy%S2fWMfz40ibaq&v25Wot>?b}GYF(0TB+#2+0AF+d0fuPxJT8X12R;3xZea5y!2 z9UC_+$0m6IpMV`n>5c6tSK9V*SK=#<50go*v{hj0I5BA-G36t)7<#NmA@OOer?M}5 zH0>4%r)YUPd6e7*sOaFOByTk=gvYF+f9Qg&4I?2~$Y-I-LpcJZSEfNwOPNKjYR5hN zU-tOPq99ipC^)wNvEXbBE&jV{1pMQl_V}0Jyke$+)`%43y;c|zQS;qA)E6rx&lfy1L&B+0HZ5HTj0Z zq80vl1A%G35CO6N%%rME;NxnGJ(+iwinOn{-S)cFsr0Akl zC#s}fek6z!qQ0A$HCfb)f{Nuf^2nmzuC*BInjt$?fRYA3>8c;DD(TTWJQee&3+Fsk zL=@R{$XLC3-xc(W?j$E@J<9j5aWVaOvN6m}6z%cW;g9euZFak#ek6LR_Cg1`Oj>bV z`4wOK=X37im{k8%6p`;>m~CwROdQd*y0mDzJkNSlf4CogB5H5HTaS@Ff|xyry}A+# z4)+2f{OJwuJjR&?HPrZqzcG+nLQP)hbaDZ@Kuk`lM3|6B7+nEb z;ia*Ly|TpO{b%hN@7;hTpFN3D*_eKOh%ru+7}e^0Jz-IpFqjK!%HwGz)jnUJm*yfp?VdmoR5CE*a_r(< z(``l-c+0X2R5Waiar+mruIJ# z&?Fmhpsl6B&FkEeZBt@rpfKH9I`a9Xu-9h_yfpxh63ft zjQlov3V6iq8S1(Cx`L3W!#%lc$xrE}p)Kky0ymE^bwh&33n13;(Id>hAmK!_VB=mt zE0SrVwI3D8YSVRsQ!-n%e`3paUx*=wXXAzh_S#Q;=(WY zB4TD~ETHdTZ1lgY9_g>AzaIP-U&t?-CE#gC1iG_Y>z4R10e;nrX1XN0%>q|2enu@R z0QpVhMrI|fanp)5@|VEpY-NJ*nMVNwsd_BwA0R&|_tJIX3=DV2(o<|YpG-`qeo@(c zc^}=2k6mPh#Wc%QNTwBMtKAC^&5U9#ksJruLVyrOuQyOjHtfa2JoNWvGle0Aql~s7 zCm)XnP@vZv%JmTSE1Tz)HGj1Ot|n8DCx=Rk6*q_9QWyHWKolu!4Frd(BX6SYfDU47 zuw>*&%Fmju*Hf2u^)BC}i^vk&BFTn$&>UlE>K&BkHg zzMu4u)<4#A)auk)9k+eUOc_DdR{t*1IUtODG9wUbrrxd=UZg;U!VU{@VS-JZI3;eH z!duF?Ksyz0IpZTj{9(&Els^V?QeLAh7Jq8No;*N3f@lS8DzO>EHTA^Yb8QzQCY89> z;8rih1Yq?IEoZy(#DTroWz!o1jyu9u-?%b||H0!*2J1P^+bGq`f#md&}bVvZIH*#@7_Wd-nX*j z-9!J_O0)^X*H!;`IbuO{#un=@rEur7iD9fV6PjKK)bNuGpcgff~KkQG%tisQZFR$`hSaY~UAJFY`6Zum>$n%xB0;;=&J>rbf8S{M9f^vY4 zs8hud)E+byFWH@z(-U(SFyDO-h39EWk^?k|KXd2&VVcqM-x|bU0k=Tq+(z-&rG)#& zhsw701o-nt0e(Df&11fl>7K7OJ#6fJDYy`BEmm1hE{;tH_s&dm*RQYehT(IjKlExD zJc)d)#fXJa#KW_~WBUCW$jQlnf(;^mbGxu=!z33mn{~Kw$aMHJ<@V)h{e^4m=hK1e zyHq=&c(QCFQoSSFp=>%ee!vT#WblAUt@A~G(L9d+m<7|wsSMWWDw0Ch2TGY&&FsgtAT54UM?dH@za8{&9RS>%ie>ghin0Y zk#-R3a5Td^b`V9jd_F#?FgM4ldb{Ef5dG-kc58~uRog6CuE?prOLkH{YaIF8_;6IN zm{C-&h@#+7ARv-pr(k_6Qq<8SbfS3oZ29aM`xT-4K=wwbORf1vBr zaC3H9#<X0I!F?TDkBNyi5%sQpe!IBGDFgm$zSRaM=_e}N+=9g?~GHW$yoN;ww z8vviD!|L!k;-3(-s>xTKTj}gI^~HWQC+=$=7_HSaPch9(&Ff#gpyLI}^jqDf! z5+&I*pnEfWTF_&?EIg=)eijRs*))F)ma3C~Ct8jVI%<84Lj~HAR!1FKpCJ)lL0cigSDHGg)*xYfOCk1R2;S`d#Do2 zlF6jjh0*Eld0QiatX&k89Q!ddtZ%ZiCZ--HOrRIneuvB}#VNIh#LzSNd8Q>(Dk<1T zq4fl3*k5dCyfzXeY2FN}T?FPseeZpLF{eE-7Qh9c&lpj#L5CRNm$>yK(X_oi9os@6PZ&$J_@6kD~@ate_3BLuQtHpr}3S}<$2t%3cdwJ z&qF@4OH*?t;MI7>_)>dk^~2IEZR$hWM`ylXna7#Ph2!jDnLB;N=b9vr^IXBsB)R(? z#j_zyj<1)+>YYU^npoWoSp7B}{#(dJIPC#_1+_mVSg)dga-m)B!L{u%Vp6Pd3)VWK z#yX&UAh5DeNH&l0Y8+BdEl((_zJd8*Z&0lg+2j>&%kiO%x|i1!$VyQ-!Ch1sXIgu- z6n|7UT7P5yNHY+;-i(Dx!tS}tYA%9Ipzmv+k&Q2-lx$Y-eoQ&>fkY+}6_LX9lKPyT zzlls_DTl~274BBM< zAiMIU9UE_3GNrBFo|LTRicZnvtYx4Ol89(4Y5jum)CI9WNcm+I~(vMROr zafj+{0{!Ds64qnQvHQW_8}z(?PrRlgrX`I6C;9+?Jbgg_AM&UFJ#G3cN?1kK$N&km z1K&XQr{(4GB4`-V`a=a!QBYt}U|NlBxG)&nlANN3%jB=#@x<#diG-z@!~!#; zTkp5@$#);@lCtBrIeX}GvgjXf2m=~-l`(&C=`2-C>q(aJsBDT)JzW!JLbKCoejJz~ zg7emeF|)4$WM-5}Os$M1?ao?QKy~o$ShamVfPN5$%J@{ZRdACB%`06?n})1#YEL9t z;!JzDLpho|+VF%XkNFmc1es6*o6-m^&4N@k1EXvQ;lCkFgCv0RarHN@q@?VUJOTU+ zsK7Gk-{VSunjQNUS7mJe?@#3w%`XWm52T@!WnWU6o0noaM#3L2E)oi&1cl}d!ORnE zI=Zm%Xt;=TQoZ`*bgeL*gE$=&dKe-UaQ9&xz})pT1t7?>xqP=;Y%Jd>UsH2H7!dBq za>jKtd?+s*NwmS$Y)r}O5oJK#Jo`ff;>OeXA~L6S9D?}ni7-;>m|=N507lt!L{HmH zmI=*+I{wv&C4MFX3o^ds&PldScAmuGWGXTO&X0Lod|x6@b{7O>gHq+8H{#*bP5z$S zCbl2`Hxfky)6?N21KRcXBMXj$L;SmCw!P8`E8)KQ=?irH$4xNmdII z{mU9Z3I`!1`?{F{97Jnz(eM=#W2PE(^czOcAS@hUuwjf~!NHu=OuKs!B_L%OSw*$$x`#p@XQ^nL3{IHlV<0cqR5&(c&+oS z1;3IKy=4lrlTF~|z06*6U-a62LDdJw@ymL5>fKDvZHW-E!mAhPSpGHmC~(1_W_fPS zbITis$(1h(EW0w2LP725ay70i1Lb97EUEfk9PF6GLF+9ndXGdL-y;ugK*#ruH2$%PK5@cN7v%PU#Zu?m1 z691{H*T={z1%npW9p*W7#9$^-X1c`}*XW4K-jsxSAN~k0PUbu6%_WtA8sR5lwP_i1 zHGYt?WHMbDJ6VNwh41glyNGXh@dh7=%f19YAw0iFo5bVP{e&BvUeuey{QB(YDT&T`_t(V53ptr2SjAgWu(|V7ur&HI?V$!GuX5C)n-eMgwa$S7A$&T? zlCoHM$bzHCN`0NHuQWjpW}Q}B99pi0Yi>;AHADD4(XgnM?*-?su=b57pasvP3U?H- zB7d?IPKxzHLE{uC!wHejkcEmX)_z=79%jC&+55PU34_%*3`y`V9%`M!67fXYJ!8@@ zZF~e%;Zo1Vp`SNz%7R|-z=z51eZNw4_h9S7j;~O+ct4pAGA!;OrXt$#MRXM5%kOn* zsc?E6ZJ^{x{s=}X|7Xbwy8H7#{E)r^+J6ot$qTNJX zlvXmpdi?hGcE1%u2i`NsdJLhhIA52Lz6%))tQZ=`RW>r%hi2qx#?b0;J1N`bSmd7k za5K2kq_UjCSxaB)a)TU$5H4dsMz^ZW$XvH6HQ7Qx{0kIWUt zZmdrtigz2_WGT$7WNDJ6ra`IB*kQog% zv(7q?gAn(saJW4h>t&_JW`)=6!mJZd{g?F8Ki>ROndx2|QtOY4zdh^+u#6uET63Eq z<+j8RPB&xeYG#Fr8p3TFZ%pq7oF$r6mT6+Iw(FK1tHmV0m89pGRG}}0XjC#Z)T@lE z7MN0n!D^6KrI7p>$Ifri>X96A&{%LOzCpC>X+nluNK@l>M;)UAyb+m2X(DdYxpxe0h|w8wtO1nDNnnD`y%MF6 zY2`^j>2v3Ws1E6bS*A;aicb@Xo`toL^_e$`@b=t;!nT>M&+@#IaaWsjCfI^|-$PA$ zgY^pm3R29RJ||({Vr+Rw~@Ye7#fi~;}(svBv|1< z*O6+WmNiSLwn1F->u>cZKWB)}{zzHVF0w}8-d<2A5DtF2-Q+9AH$0>~zsc<{Yju7a z0p08LAKmL8_$vPy7=OFaf8|#H=1voUD+Hjx6#~Q6{nicIHeI4#*cRkFqj|yQxCA}$ z?`+n75#?=-7lE%uI=&K=#Y5Qy=w4;oVt&W^ojZ7wA8#i-Z=dh4!1eIQ@R;%H@Eq`9 z@xt&V<%IynK178?Xz@J9oV!@Fi~;)ibIE$gl7*4jC?f*OS}=Vm&I25@ss$$p3PswU zBMu4LBj1mC-DV$bs5ITy;C8f+q?n`*(t7VgdvEhCE@7v*lC0Tmncy(v=@5TtsJDvd zTE=i}rj2AL;BB!UWWu1m-t=Os_^GXwx7E(bMNU*-S$(>qD?6r4iyUK@R)1Lajk#!h z&E^zkywYgnmDsXf4<-8M8=XU672bNF8ovSH1YLSOP=NdN#$Hk1QsuwEi^Aj2at1XK zr{kwWnM~~U_1zGofK#|!s%@|{-G%^5MhjRMk&s~?r_k0rG2(Ev+UTERP{-O@B>IRb92LNum`A#r^9C>eP+I zY?OQl#@mVXr!R2hFGHueUj@rWYf1^r^rZReA))QiiHfO9l0yOW;26x^g2kr%<(d8+P#fY3%J1lN% z#9Jff6@|8Df(dxca3n2Y%tN!%C7gW@7 z1}P(*=yW3{UMSUC(k`YH|JgV`p#u$9@WWHy7@G1{BM5ey(q{lBRwJmMrLV1pclhyb zlkmOxS}^gG-(|-mxPk@T0YuCUgJ9$?%mI=@)-UNn#}G590i~DyN9p}PB(Ku?wtut6 z3}ww<1ED`6Did4t%3IHO$(*@^ZP!SEkO%zz z!|d?*H3UR@4>$~lB`EumgD{5Pk?*;}e}r~JC(u(F2ue<>&feuoR4UgK82}qEQ5%R4 z6>ZMhCDBs8Rx2gg<3`WY(-;VJtlTaPm43<%%?-t=EUV6Uz;6NvB{B2aaWt`$^v<qi#YLZ6T{Lf%jLV}trN7mXq)TAzaWq-z8v@-s(P{KTjOOjGamqi9h=8ve zR3dnIQyjUR?@~%VIsL*Xd!J@kuFHYT9DDK2`_c>Udo#egbIo=5G)9$#2{ZP zXIh}c5{eWyFGpeCLMknPUOYhDOmm?OzEg$m z`+4p)I;yBN5H`U&%Zx`*Jl%cd{!W0ZbNi(V4?{hm6u1{`F6+sml!TX z(K?7o=VL>5V@b~T_MbwplXbDPC{zKom$oTjpXaSzKL6;NsN>s_L+#EmUUCg{7PNOd zty)bsRcmlxSqB{C$Gj(RN%dPl;|Jd-&qs@DtuUY_IUa(}*gAW#$2i{a>LKup)ajuX>E>*EMR8#Lt&q6#+zITB}t z$5)S!5sbxs^~$qMGI6-JAIj6@b0Sx{ZD+o#lr7B`10x>#8T1ij zXS=S5!5g02LJ{x4TM~sEB;tk?ra9QfE~KM9+*oVp%`~+BQSsVrA|`R0`4v>3?V56h z>>j{HdT$$UUmH>f8XpAe&yDxisZF&zFacN-)Vi&UxT30%PdTJ$-#q6Qy=1*HbIZE# z=&O}$I)&e@Wqm#JQ9c?0x1ulRo-7K^qSt^ z?)bVLn>*g_XfZ)40{LO9KXyNSo|rqDmIO@I_KTUkeW|923^j@0JAoi?1r>99Xc90F zdNnsj5dbfXnhGgDN{5#{Cb5m}QWVthh;VnD#L)W1GsC-c)n zst}I!cZ2?Jeu0JML52h0p&0n{??Jjhpqc-SS~50&d(Q|t;B=5Y61S?VD!b-+-)JZg zQGIvCXEG>8C`(vi$-a1AtF7*0ypixM+aZJ~z|Rf&T5MVw%Cp$V&Ft`ba(u`0t=;P# z$6#w{dnh`(1vQKrr3n#%7r7Iy)BCP4;--hEM#2?VWk!sDWsSY7@laA*TPu-Q2t1f1j#f zvV)lMQ!-u&B@(@2xcm*ftcd;?teF3^YW_W!{42`{h>%9`1)%vC1!p%i z&Ioo3oa!u9UQz6TC$OM{2FwfWE@%OV8I3)KPz<#m+2WqaZ4Q4;GnNCc8Sjw+<9(Eq z!ZfZ#^}8XgDJm9$(vOmP%}k+T8u_0`>h4ERbLCSyuN2e-0xvfuF6;8DRd6=qjF$$G zNRw*;N700R{nO`~%7&-KgC+$kG>G?Q&ze4#f*}?rD z>;y5LbSMG7Cp7SD{Co8k{m)$3Y2VZug-a*08Azr15r z566)c4CW$2NsR7Zv&@Jm+AyP1to=cuD+L}-ej~OU#k6^$f)S@J?Rk6xq(YG3-Xc$(<%mQR`L&B z=P?^Hp=1=ZCD@S6o>EA={ZZD0WLx&)pos;cwup%32X?T6l6$MZgdBMvli>!4ON;lj ztU%Vs$s$2=s{s8*$wK$s2ve=Ks3q^Tf-(|n1mY>=oa|xwAqtt}q0CuP8EUIb-KU`H zxhFvR577mwu=bm6eHv+pHBmuv@7PQ!CTEs0ajrDPRGcfH)PLUKIfTz zt&|B}+ly4SS(GETS+?NpP&6!yVMZ@-AGs*flCW}fCX1XI6xwQd%XvsNdHTAs&!SYV z!5rnk8ryq7;MxRxEn8ae{-%LgZh@Da}k=ZRdawkj_xFI(u6a7RLU26-($uA~nnf|~4W`FxS#aDjjEFm;6a zRq?ENS*5y!ut1Ln6j~mh$P}k0I1o^em56++PZbEhh?Vbn=tGo6&wWe3WOb43KHxw% zG;S;*y)^F@hAS)3)N12Cc9EPow}`u*7hkp&X%I>DQCg}(>h_vidtvf+>1f#dbM>glS1prCUU<1M+h&DpNg5YmY|mA z7qAIr2L6=R2~(BgChjEM&3wAPCpC7Wd!{r7)2F^G$BYBhiYEc~M*{6p4mt#4)QV5@Iv%pk2GE(8Q7{&CCK zO5RGc_gCX$-l`&nA*CZpo`5BH;V3to7^a+$1nQ(uxypj6iIF|WTUi$k=C{Y&z2lUfp7dv(Vjycl(t{g?5AS42{Ae?tntSjLegGBnU{;A_{k{*hGb; zU_6Grq+dk;LsO@TDz?2;D^poO^GrkDIuXaBjdhkMf%EpWkuiR%*!U6BPxJ4q`V#-J;`A5Jp~(|t~~#!QSD@@z0bc2guD zai#zM-8Wd`mP2S(YA`nBAq=LN=&b`xK~Ap`-2E4j!rE&d77=!u2mtd_>VdPVS5J(AtFvKX^$N1d{P5-Y@-EQ_3YWsdjJB#yBV=Ke zZ}UBUiBhS(B-y(Cb%0p1Ip|NcX$`Uz-Ej zr%UuP-NCiyrzsO7jxnwx12=-9Iq0%g`MLrDGSJQ0`iPTIuA1GzBRb&enA1fuFZ3W3 zPV3!dAeSe}J>>|fhMb3X!lF2+oEU&g62Av0+TB_NSV7K2a8KIq!RpBJ?NF?Rn(#Hw z3z!a%25HhYMd!`P!wzA#a z&nq?1%(As$xJL9-S043y1sE-oZ9ZZ{@B}t9L^1p|9Qf)-?xr)2s6S+4K2dGHP>V{R zR#c8NT*oibq&o{YLbkHr_w4riULN+3EiB=CEZlKjaOQh|!F|oZ6D!oc9Ckd4)tLA+ z=wMB5awttLJ+9G7AbFtmVeD{LIBoG{*?&s<+PF;bAbo~qfT@->Q9|5VA+ka9k*)UW ztX+FtO@`<&SzU*aF-lr00zp>(T}Kt}q4%)Gg?B_yrYa>9Gy5lxHbE9)ZI}4g15&@$ zq*|bBdoD42rVeL`mvji!-b?JG2wQQxkV(5`6GsTXEz+K&qTOFFKZ&tHmTsbC@=F{> z-$3g!f>w6pbaNu@ss3!z_E7JAh>od90UAF#ft!5mu`+(t5?5GA$E?U@qxi9Q{rZ?R zcIDy3H7nE=CVrL6FLu=U_6vajo5UP>O}!p8dk)+KBvr&bu^BS2I7FdmNZYQ7)DZP! ziF_w%i%8<4@4y_%1J@GX6=lFvF2!VV(S~^lHV%8!9i;q6hrq-0Ec(9Aq>I3|~^nKbto-L+H;WS-!27QkH^Okvfc<4!OhR@XpL+3#ofksf>0p*XO8uLR}v-)zyg!gPC>@Dk)LL25Nss#Saqr8IX39iH0NC+Ef%Oa-!a7^YM zX3=Ab&E%!XZf2}-6S5lmxwNZ~e(yBhuB#o+1ABk&eaJiFot(uoluXWN^$J$j;m6%T zM?FV_FV8is(Dv=DkFdUXmwpJpgf*4VG*D$iT0|A6(#jGW)u)Sr3&ceM5?R#+`S~~f zXq5ho7kTJvzz`KUyHXGHr*!!b)PVoV8~>H`bSVFIT{TdrF+SlFC7}X(4WItCuPJ|g zm|jDLMldu_vSpi~I&GVii{$=_ML5^va9arC^VbkxF$2$g(UQ*^^a3HdAv|a7z)H|% zlAXDzNVm)D87})hG=VQq`uCirAW0Mv(r}4pVtuLJW`7HmO37e}VDaFjTsW&QuG42h zP47g(>!CPKf=E5>K|nz@UMYuWm>zAVxH_$Iqe@7dX!skdb8=O;x0z}2lbaP6Ijpp4NZ^q*n@AeD>!wI z(pZc{8h9NKO)>F~(9#fD{!?9Z`?O+Vh0=AImB=b5D zQpCR3L_QR&_2~z?F5%!{Cjx{;Q2Q_+`!3}C+8iK)5&>uiMtE=@i#udG$_*Y6tnlCq z3uH!&`bhJRFnwf;Tt*ey_arztTw_&TUEj1eQIFyATY&f-IF5yd4@sNgr+i1*Ce5FuS8g;r^&@zyFAK1}J3je8`YUn7k zSTbuqHmOzX;?f#sbcyb!2UKoprfpQd=R@BxcM~->Im~EGLA6ferVr&>zB(=`X`idC z*y#<{_>lFn^1fh!UrMe*OlMd6h!JIzCpOI5X8=NgZfq6kZv0#JCRF5z!qZc!BWOW@ zLunmd;KQ4stC#DL{g;N~_^3FQ+)Sn2!?Cxgk?^7wcR&DX?F3OmH>uUTXcSM-|mOpi79m z$qcsuaW?2_lIb(A!{I*^Rno77C{JI!FEkJrM}@WAOf5>FPZ%HWv(ufvk|;0zN$|E7 z98ow`Gm;?I9?T?ea!25PTED-Ku@+mk&qpwJy99lfnDNV7FzdzU zj?X;Qm#||~OyNxnJ`5HnlbG}m_(?kcs3R28o|AyYV`t@Od*aJx4{{sZGHNL|TT_il zn(;Rp-JW15gmN2#@1LC^PbFl?zuy!ofwVe0k)*qwzT#;r7#m4 zkmA$f!uo=)f$64mCGb8uoz+~d`LW)Og99A1Ll5W?z%nw1vK!p9A%!iSoo=ZfXA|v? z+qD$AV9UVR*Cj5QWv!J8N5eFyJ*!dfm`Kl&;L5ycad(>9F%m(f@vTm@x|;LAI+( z2GrDVO+Zb>`K75Dd96TA1#mMblS`*Jo3#!h$#WcUZ_iN8;b-k;K>kg&wF&Nu|{@=9wNeVzn>c;#G7lgI;NZSK|siV0z@!_|=6_k8m&e3{vG^ z1TU{6?=YnQqq6>C#PnY%>pyTCAkH_ZhxTQ&Lc2*p|0^K@f?Sa!`tj8u7DjA9cp0N~ zYw}sB<9Fl4JHI*bvg6@4Vx$+u;$w+Vt*sKNcNv9Wfuhet#K~edlyWwXBj& zQ%nUwYo{lQasz<{#em>a=>ZfGQxG!n38)8z*9NlE!m7<&!QtYRF3(r{g$8O{GPTcH zoC9!@?*q;~4+}j$vM{teZbQ~!FY`o=j89DJR^YDidt!+2L?K|^xSx~sXoJv7=DtKEwkgcJQ^YO7~{WE|-WS%0rq2a!v`-ckW*Kj*P$aunJBWp5 zkxG|<6y(r+3GP~M9DX8gjOR0`f~tOFSBYl6m~1JOMt~^m;!HP}j=>4JLqPs8E$&v1 zCw34z(;k()u5^Clg#td>HK;7PT$oFVg^)IWRHHOWn9U)MTdKrMV6WAK`C)nVI^o!@ zC9kIUZznj2pKJIGL@bQ{=mc5*Ur>wx3fBJ(E=BxfD=u-vdF|ZZ%xnPtT4Z2WAu)`Y zz8{nL`Rl^hn0ZVG4Vkgn+O;!?1I(w;5V?f@M-d>3JgMGeKIXS_Hh#vwHOcP4bTl*t zjMwA($^*Wk!srjmTyocP10?8)p$Vi00RNUkegT=jcZ00M&AI%@SumBb4uT}3{*cO~ zD9xz?c!r#Xwvx%8I7gG`8N=E*rQ#hGXul;70jZb&7j5qtr0Lde3szOC(zb2ewr$&Q z+O}=mW~Gfv+qPZl%$xh$-#Oj4jXB30^Gpq4#aK#W1f`hu z&+>Oh&2jT|S__X3oJz4+$hjI~DdZQ2lv#%q__Cf`;wPWvaTJy}4oi9mZpTx<3l2#6 zL9HEWSRT}!8|AgSfMEm$%gU}XQ^IIq_!28WFKGTj~0GE#s!+ zTV9i^MCq4viMiIl@>Gw`nxCkLRxcLuN9nhYbolr;_Na2D3_@C586gfxO2);mw!qxlvk2H_9$Iwr+8 z;3`M|oh!83U6pYK3XwM`_Z7KFHL#NcGP+&TY3d0H4RA^3Mn))%@;JJ9e?2o%ZUc^o zwtG>li~*$Eh{8mw2~U0%ktTak<&ST|>_HmTPO;CCGo=ZNBMbIm0i)`3U9| zS$TeG{rvWDqIjFV>uJsd`%8wn3};?Pyq*q|4#a?#JuCTeXe#4Pb| z7bRh-0;r~ax!&7SX^AQA1p^>Ek5w;;No6eA6O=i;%kh$l>DCjS$P zS=!c|N!lWwpj6TEm}Sd`Bj{Q0bn#hBUR;=@O}dxesq4^l{K&CO_pV90yDZ0cA``BI~LUs2pc6pg1;`sQ<6Uph>!errGGbtt4!}5 z#HLc9AiRxmr0M{$uvFX)tUA(2z8s2hJz4S~ojRMbJVC@xJz&(JkV<6Kppojgv7Ij! zOrmfQ)zFARB^6Bkm!+yylOdD}eWeLo){fNI4&iE}*5BVrS=i+Klb}rLv_=5%{Y6kq zbXsEo?0#A-v}l-MV8M9&!RZM;C7R50K5`dyzfd6KHx>w9y(jwqDJ--oMKl8_Q~Jy^ zKK6VlOSDA>tv&#AKS%HbjMBC}n0Yj#BYoy?07m~Ky7$O6VsH+F)=rPw`gRXn<^+JJ zKL(8kpvP5_?t6E;BSP9g_>4AcOL*fDe0FE|ZE|$jPpx1V}QN96h@C;W~@(d19 z^7N~s=o+X|euH!=d&1~eywP3H+r?Zj+tpmJgxI!I{&`ApdmTtYw4M)<#;;#XI049^ zxUG{%rVu1w6WL}~PTb*FR@&7MEv! zNGbSXM%LW76)z!VKv@jV_BydqCN{r?CSh(=NovSd*N1xw5yZ*lV{~BQ!%c$JW{4KWYNoN!4# zI~W_)$Hc{>ou_r*yxd}hE8X>pWG)KB*aUZ*?13q*T|E027jurO+@AkpLKDDZmF8H> zWFzHpIf`EG9+bWgaJM zK%`1CqRG7O`jj8+w{-lY24xNIXfxt!^=9bD0=q2x^l)- z*!l@EfYm7>#h^aUlw zr_#{UG=8{Tdx7aps}NCxI3#tLa$D(QguH8*ogDCpu*xV`j9u$=zL^d&Da*@Cs|ADz zfhzfWwh5M8S2<(`+W38Dt0$OZBESHTDua`Urt;GtB4-dR>X)_ipR@`)^(%QwWE%98 zaiOHib1;diI=0Djni)M}edqTfO1M8Dm`|zFyc<-CN^#v_Pm&8Spq=P}Is-Wc!d?>j zcurt)@?RA^yfsaNmSb4#rk6Z#i3I#+I58T=1O`^iHuA{i)te%5GE4xlb1K348%mz> zrkX?rYcrFOdSt4qW@Rr(6KzeLgZ#od&^Kl-rc{XP1G$>Plclj%Q31m3!5E;9uw$#J z&Mo%vQV+>CjM|F%IP`_g8Ij^FO-74sX*2A!iTkDDjiO8zG-GvCVrW2+NPxK>{n65N zRM;~?6P8SsXsf#@`9sT5BQp?j>YbQOJS0Gd8e<+go}#8hX@ck7NtLk^Ly=RxE4u)q zMw4%HIUYg*by%Zvp!p&4dVJ=>N|J_jJ%PC44oYR=d&D(9?WDAypw#=WtCZ?Hb$j7Kaetj@|o{?w-OQYgY1)k+J=f0>pVfu?{=8`yM zEbg3)WjL{7o?5CJwWIfm$VxMRDPeIT~TvS@&$ziMi+_ zZ_^hbjY!B;IbE$nWZFqq($AWEH;JFE)hrxIOho31LwZm{K(TSdyuH(!IG+#&yu3pd zZ~I%k_gAtv;UC;TYEEQfbxKUpi8_2m@vm85(jg!pfVCF9*y`LNB?7$I3=qpJIy1Jv z-gVsj!~&%3-ywfy$OQ(WMH|vdEni~>6leOd=9M(P*gUy|ty{I(!8?6^>l9f!5$apC zKh@|&uljK&Uk_4^=TkyIGaG{bXevrWY%kWgY1R56sXY`w)tIa(y9b@=o$=(tN}&;y5D-22 zr?HnMTf^1uys+G})I+4lO%Bj?c`G$Ud_{%P70S^$eg_LWKar~kgQRu}p&fMTU&Wd` zpuJ5|=~Q9PR85V) zF18!Gzy9=AJte@AdpQ7}6S=b#>}~P(b*5Ja1<>O(ldLAnK82I>Rhp3@jo6YxS}g=xE6OL zwE}AFssP&-^8OiQ;ni@=t?F38ngden6o-2(b-*nVbeC{PxvH5lQM(W?I)_G@hqn# z%IkZ?4MFl8LJj_u!%6(FM4Cj)8&H;AX{)}qM(8U&&dovG8y6AN@=f@&ozs5?K88Ybj!WjQ_&C|Rn;!4^ik-`7a<3+XtMIBQer2k#BX>Eu*#mE@! z80TA+uQnL~g7Y`|8>KLhMNl|;{x=69{AgrhHHqeg(oXGGKrz{%Mq6{pozE!RoFoS7 z!;f_336{2t<7OrTd02rk-wd*!S#I|-ajoe12;KB8N7IX$58OZN(L%%xex#S_z&Wzy zFaa~6{h)Z=>rNFdZ2#j*(qszLe+%`xzaiWSi_pQl}%L=)$-k(8DAFslb-u-VkMWl^LpNkY0OkF*cze0PUk69^=DKEG5E zr>Pw>#5p_PRDOyWp{UYf{X`Z_A;QCObu;(|^5AFzH75D>H;>%hG%F;g0x|Ie7P7bF ziA1b$ojo>dcH?NX;=2Q8S_g_K22*dDG(rusQ7VGG;W#-;8(gxl*7;glt>!>=Cp%>q zaQx5s%^E<(j)Hk&h_EIdK`MK%=;O|aR@9jIo{$6^dP!Zd2&U!zp^YV&d4!w~qzz-) zZO5;9`99YBA1{M{7xt9+2CZMAzI}W8$Cb_hV4V7I=Jj7QuD>Z(QT&w57egH~1()mP z&2^=_0omSlzg|oTOONHFgX0EhuGP_yZH%Xgfodwapk z**!jawDb!0EwjH41Pn!qs#F6ovR$-^5h?^Ib7lN#cp4`{->50!p%vKYCP0%w5Bt8H zz(2qoOc7srpXx%dQ0F|ItqUOZ(kgXK6yZ^_IVW_Xw{L`+v)^QyTv)Hge1y_Dt0*tx ztU5LLTA$%x+zxMQDBUP2eJjsq#OSK|EuHyiP_*2gCLuRbydSExC_LUxyNxAb>qna- zWPssy2l4?+)^m4O3J+gS@5z=XiIpijLV*1gQG>9798z=uK7Fa1m^IN31VY2G_{Q_5 zbG4{C-NsK)<-HezYu6)vU-jz&wLrpjs9n_;gNOCx28PnqJKP-pFeOHG({*7`w-Nq7 z$hT0@dSE@J#*LsL1s-w?ZLot(qk63GN4am%f6Z0+#;KO-f6?cXf0Wn>{P&f#f4s;q zfu{KvcXs>tg{rKntcf6P0I~eggl^&MC#D7`q94M`HJ>#|$MoAqff^*zO4k8(qa&!_ zCmu!e&hwc*;<+b=orVNAuBzr~vV5R@A+(+v8Ir@4_{NQ8ZZf%IYB`lQmgDnzOXo>%YwyDbEBRcMmqmr|2u8oLX=kkGT8i5wZcTrkqTC=%> z)DzB(!NqIQ()n&6@3pTzxE)|Z*mz3Ob;1@4=mWXL^GP_+!VP`sxQ&lB@};V zA)@y`WK#RLx=;u?t7SAb3b$VNcewCn~$}k$*1vjvLZ0ic@W133pZoM}q$tGUH2eIenq5k(*ZW zT~bZA%4A}&NuIM3EA-g)CH;IP3hxg?XN_q&g0ys>R4h!>(i{e=xN%d}K*lLmDGV#N zdbJX>g%M_l8R4b(PaGfyLP4^Fh7@C|%wN?JnWATbC3E#@W8e`yeds&HR7-owGCEH4 z6ngYzmg~~t(hz0L;TceZ4(3i1k+u?i{mo{Q99?MW)lWb;c}V4FVp56+I3k*x@>P|n z1k}iet@pa%;;v()&U5wc5XuWqYRcLm$n7c@9Z4i-j`a6g`npD~UfC!5Xq_XK43U{W zn8u;HdXUD)WD$RovQetX7?qG7ct-S*l0?2Q7Ils;{2CRPm@B*pCs7$Y9%N1q)C3)$TowdA7p=4BLiW(kNd}n#7hUU)|=||Bx=CJsgzZbs=k<~*U>5I=_ zCF4FudJS~iax!rZzJk%+l45Y2OXGg(2p-$%QcyFEz9Bj>OWF7q8GI~j<{H(^-Juy2 zBv9~RR?s=UOzzi$$VKP-&7d?y1PPRU0q#-{E$X(GmPD)g{`(2J-5YE=NDIL-oS;@O!Mt zKUcfIXT`2z9nTBWy=&^iF7ngdGo0<7nK_o7HEU@Xw-A>5AMjupn-G48&0`qg7H9ls zZYy;}v?=J=Kx(+3-cp*;jQS&=yaf-{QR%Ht^5hVndnQG1kT)V3F$Cky!#WycRO@{Ne$xhZ*9ZLed!hjRVHrpAwB40R zc5xZ4@8DL=`h@OO^cD_I0<0Pf;BEaCaw{;+ySv%)UW);c0Xl3R|5_Ogs@3&CWFc z^!;GX%5HeIM!+-2eTluxx?9^~|FTWs%&`4@di_Go`Jgqpt#~WJjy3p|n5$}^2f}8` zwO=4Cu_w_^df*uXPqh&8pSoj%nTcJt2p)8p^AveMAcjoWbm0;x)NFmbz(IMGVWU>N z)IoR@QFXXOeQ5)!`bI-4cVky}-xThSx1KRrinp$9W79ss-Rc20kDa^U}M__#@7o;5FJc+ zT4$>P@4RCA2v2X^Aq6*PFQGuyI))l9JzFJHW@&0Kjr2<;r1wvaIIvT&qO%^$OMdBj zW?C&UCQ=#Uae`iDbrss2j<=<8;=vCeWSpZK4A@zD4!Eh#d}$1ksnvtSO=`i}$}$rr zPMWZDu0K01R!63z`{ExW4DY}VICU&qs9upqPr#hi^Q&4+U0 z3ExJlQ!`s0TpBjOu$DJ`dvNfg z+=U&Oa}UJ!qWlC^5vo_L)aEXw^pBau5Eq~@+^2zHHzOj%r_Kv9eUQ7K^(2EI6Ux6) zBFiIIbZ1~{;Z-s@$69{OskZdOP}HX`4mxbSw=zpq=CpH>z>*AiKpK5!xFO%WO^vg) zOM*^)Z=$yJEDthfRfcaqE~S#PFzuZ?Eto*Mh@-i=EHP1Q00nk(tWlcfE}1S;?rj0m z&O-L=Pd%fq+k~UsAZ%oT`{dFge#THQ0e_rslYU38ve1KXvH-yIG{S1k(?71rrH1Ac7yj~tb){HFC%L5W`6i4uZTv$U!^X6!IZ&5Q0nAH!5qa2bUHwqiWI{d zKFsp~zCKy`f)DXpm0-;gD{oVUmW89$gV?yJoOmrT8+iR#o)%^^YK$B zC)7nwJNSskNP_}p`;U=$oLHQi9d(bQr9l1rxO6zHCgn=fz;$>i8y{!8*@X%aZ{Nl5 za=*B+9b;G2TwvV(wXQ%pmR6_|W6n{VDbSqU@^|ottQ9>E7;9&>7ZODM&-~oC+c$JN z!}nZJGBjZwf!cmyet$&lOVFjqYnIk4zsA>rtiqhROiCPfJhATndrXMK+KlFv_pD`x z5EKSYn~jOJ_K{7ec0JaF4NuYKi?LE2K5AuoFVv(9ei>J5@b493GV{Lzxz#zs%6;`p z{qhM;5S5VJ;@wsGEmIEGa%Uq!E9H^PG!`heIy5NE3{|k`y=KF6Rln4gLBQNL$0Rk| zUI`AxrJR}*sJLtc;?vdb?*_cHns2HXL$OU`J{Qit7JqbGG8CFSp5JOW3Z$N3FElM% zo39gPp$=-j@?R<3&bO`K)MuN+%Fecb8H*^g=3#=qFRfiJXGSGtU1nHOL|qb}8<7g+ zAoP!B+x%G{&(3&xUmBld;% zR32Yg;=^6!UiI-|_DuADF_1oiDYnqZo84;^LnAy zM1g2`TQQ$ZTtY_BWQlg5{^|$Rz}A5{~^OHQqZ>iYc_Ch!)Dg0$$Irg{#Cze&>EHV^sGEr`*chN5*gQ%Y@B3{ zjW`WFp>UeC>}UQH*c-Z+{B4(|$G3mvE8BB`9T zzS8becvAl2U_Z#Ki!ULeK>W&NAu7f}bseNJLTO0cY2w)8*x7$*S}!q6 z=3NULDYEJ2U8IQ6i@$bwq8f9={z*b33aq@$aI8VVdElW`Mi6hm2sSjsNu3S0SN(M> z3@==;=N!qSU}r9z7fIqo@F0ozD~6UGZ4E5=NE<~(Zdl}y4W1!6HW%{ti9^SQQ{eIbz5kN!eNyD86my`Vh(_tO z1Ixq+#ky#XJy^0a5Al9>ab}Clcon^E>8`${!rPM$>?SphJkLZ<%?~k7heS`nV@gvdcZD>;0(ZZ7NYN z53qlMry?iua{LQ8lK=Rw`=5g6-`dC`g|WX3+qvgj>%je*ceE8GBq?T<=t>bP5cs)D1Zo+B|<7kmMUwbp%3yBc1D<|8!`(CZQkQ+U^rv$CjZ&%<29E9R=NG484r*%}oz4-lO)-5m z!@N-_5T%Iy-apEF=&H@ENc1cG!s3MKVh288(lJeDTMx zwcuWBp38xJns?d?5Jkx5lxo2KlHb5Fk|Z@^!`T39%WJi9ot9M>V(;WwaJUF~(Ve~n zez-j`0^=Gggnn$o$?qmd6CGLTUlKE3?;ovEPPmdh96ZThl`pNAh7hkOZ|xh$-VpfOWA$?b!m?v#~ExWx>Zd0NyDC* zeg7*Ek&;jtd5X@B&};n}F52c#Si> zh&rnvg(XJDbX0@HC@^Z`$_zKI7ZBYwG5d@X(HfWwfn6X13+TQB&8o^pskLQ0sMXoQ z+L=+AE)5l#uB;`BZc;UW^>ZoR<6eE@Tg5GllvNuxT^~SQ?^g})kV|&|_*X=to7p3p z`@)ds|4|Gv{Rf65Ws&&dKGL;QjkVmU{KE-Q5mBYAm^V?x5}|can0sBX7daIprA^vM zxw(nnNDy#QrP;25p5y~h!|V{FO1>0F71b}NX>Dy?t-#k<8QAM*dIbTZ@RuU3xcc0E z0Y7048|^5EsM@rtcAQZw)Zok>T6n;@PGV%{%i@1Aq;cKKWLRC-kkD=Y5nVJ^NTPTa zP{_$19pH!$l0M5cl2G1tJgG_$~A0;yF=WXsif-|=f#k_a~UVUXD_8W|2 zxv!=f+UF^^n@pd8ZDzCjA?vv{&mlX?4LJxtAlmEN7}o$1ztNUu2KWuHKROc_pXk z0&z850}NAq11kt|2N5(-&7y-kKVEe4)!Q(IuSwtwaj^-erA7KoR$tXaifmZxruz&4 z@x$qMh`P-rw^VzR_VPo))q6ItS;%+}W6RacY^7!jtjRpWJbId3<&+*u!u%^Goxq!q z#(n|d`b)m|-}@c>Pg4@}e*!=fh#v04plPl8L3xX|gG>d4(7&WS7o~F^j0~rvsNF8= zoJ7)D^>|7_t~Y?U2bXvfr7qX?6hCd0m2t}`_-gO&0=owVMKYB}zqdCr03ECW<`8SV zovg=cCZ-ocM6OKlhagnY>MC=;MU@*Z4lryG%#*Q263nSj9M@gpA`{Ct0m^39{%PI> zw;^#%WKK~0E5l7?xT2w(;AS|L6edIRYRal5tGB^JMrK)SMWe(c`b$hCExJg$#8?BB zg}4W?20p$dDhbC0oQowaBl63v&0&rqfL7Xs+pY5Uo_*(a!&y{hs~oR^Kv@Kl6GT=R zYYxkeNFVKD(q*$PMd!C%v{1Z+rlzhT+p(Ke@C$7;+xU+nL>$Nrv9SW{kIz6T4{jye zD>|%4g&bP)E!B@WgW2LHp~6Z6^Uz-Wq=y-NATkJ}G0cT73J?BY8JEs#Ir4pw_bJKz5CZyFaG0IOZ^X@u7b8U zCSR6Sfd7zf7Ja!;f8Du|DLw!W;t2>Ne7Y9inuCoP8a^(T>=?_#!EV2SdW&7bJBr}1 zT`_bZVq!{G%&TcL9W9Sb+`jaa6q!nFRdht-BCz<(mHD%l`t2KaFygJyB%Jz$Su!kt z@a?E6leBduk!%OfwBIkJb*gi~QzE7(yDfMpwVNl!+Ez-d2%vHVWHgEw!FROK8CB^u zw${PAPk*8k56V0TYjP`9nCZP_W6B@A3pe{xHteiV2q=R)KZ8g9b{+B4xP zp&Up>!?XO~Xx9Iw^8EW_T2PYyE1G(7Q!%wu z;YO6^65#KA1Coqb63(y1$LBZb0cTLM!(NM`Te4(8 zQ^i%T3x6Q#H6bZ53{US49kv2+PYmg#(mx#D+kD~HVdu!X zosJMkp4?+kqg}zs=CX$KhS`8i){z?Q-*TWFx@;8s9a$Vx-1g^pm$wpb+X&AQ)%Pw| zxYxgX-+BM~4!MaAdz!v*!~Vs^|9kBGgE{X1#>4+R3i)5K%YO&qFAME?J~;1YFx6x1 z<18`=f-byG>=6ju`TcnzVhV6u_=20gS(>84C6+bnV4om(AdB6nPaQV=L2FJrr3JkYTjCEIknMlZ730X6A}{{49XM>39!hI3Mx?x=nAw^iegtC z`bmSqs5OLEoot7Q`n%Ow3P{j6zH5L^u42kIRJvCX{@GKw0`E2hDoGbm91u$z8dxh( z^^v~geeu)B+)1#lZ6$1;6F+!R<48G6%!@~b2Q>_^L=4N)(uzqrs#%v3gyi9g|9HrS zkl5sGgJ9wfSHuNRkz|p>NvWs`5*EE+55p>3uD}pB=j*EZRwT72^{d`|)AP~xvC-to zemy-YERqZTmnLM_BxD4vKiPB)T3hx|cA@?RSe(|lMukErLWO?x*W|vug2mOVI7(o) zl49`KcIAE*H)Lc_?sW$tYJ}KRJu#s)`W0I0rQY#fk$0tIEr}NQ5j~W=XQqgZGugA& zpt?b7QooIEsN;Ldsx7S^negp~R-y&`ulcQhUa2`wSG=4gO9Duz*M9UN*+*77sn5Gm zwDgiPc6~G`$xnH1XQ^-%s!^t5^Crl~7bxY1mVM-r5@r&QpnUXj>8{n()b*gbryqZu zQ9E&G?s#f2h}Ue<7Qv9Jtd%OUl>k5~k7rl%llcy&shl8!0z+EQ=k4^BCiKrfc=GBzP0x(*01a_{Es?+h5vcf zle2O%HMbGAGPX9h`Hx;j{F)^qKhjWlm^ce&i?!68hNWevlJu_e0;@TEW0ZmzQ>@&z zI7|B;hjByu_$BBouUlD)oZK|n)?2{MxI67vqr&+_j7sIf(7IF ze9%-5=>|-Sb;aI{vz>xv&|Xv(Q-BR203pWqb|je~66l!-KD+-w8)S~Z3NMbuheBzk zgV^w9wSJE}MF01}&7RLK!9i0?a|+u!`eNh=|1KjsQ67PPV&gg9zMxmem>4imz#$G? zP(ibyp{Y4sy3hf`{P861VZ`%9+IJ6a_>P#&m@}7PYYe zX31CLafQqRtE_vbs-jb#+XYRp{qmOA>2jBsY2LU=K=v-z0*FTllLu^;R}a%`sCsjV zgBriSlDQjzA-2q0*yd`wJR0a#;P_TZ2~+eDypsWbVR+^$P=`kdRn#mz%7HhmUFb7b z6T|LgY9u%K{ru z1*`_6o=8ZP0qhT~_Di7k$0z~<464$yk#0Pg$1hPll{5LZ)&2?tN>{xq>ZZ+7aP#s7 zoMnxs)z|6l=GD2yO^pNIyUjI9b-dxv&(DM5j;o2Um#ydB#5n7n_n5<#*A4kMH9Y5$ zR)wJoBQB!75~B|3AyH#4n!P;+9(==`NK&KolpAt%AM(93h7ZXhpRp%g^iRsYc7`s} zJ#uwlDmfGZ6clwe<2YK#461S?dFqIu`qH>T0`*MVh@^To;s_4)%A~=sKPiKt>Q;%u z0aPrb1`Sjg#xq2aF;vaJ^OaH7jOw)eSc%@U0=rYLVG%y$`*_8lOu@GjuW1oJl>1tU z-ZBDxQm%0kJ`@px;^m0wWf1m=<%z-!#1qB(Xu`nbQ^~><#HHc~jKrnl0tFEiiRq;g z=7{M95st(RiitAgQ;EYIi0!h8Oe77Wi4G+UvWb2Z(d!}{#FxefR(xaW%R%o@|F*XM z2m-)=^XtV$e+&CIX(%_D>(Ac={Z-t>q~h=G>C27mRRA4BYqhRT6i@DJhuS3cMP1fI z-&+U#1|72tgI2jdgLq1VR=G7}4sbGS>2=RE41m6-#ANC@V_637ZtgXC0t@g#?D+wj zU^V@-yUq_xr-7gGZ5>o|r-j8#KS*P`SC0ZH1zN+70SGKW0`uHgch$_+7O_VF6e7S5 zYKSqlM-Mc>!D`#M%9(z7Vw(UIm2nxUyP;RdHK11$>SJj83hHBYTNmo1f7=@L3XTsm zzyMScjmk)_Uk^Uu4xP$G&Jqastq=5qj6N0}iJnZKE~{4xv;l3rnIB98y^<^t5)MKi zIeRR(9PV_R7V1Od+ms<)3)~_e!-!$*l-~D%gCMyWM07Sted;-8eSUna-d%y^=s5^| zQVFPVv_!hpNj(*OtskG%z`V&Q{^57X9iW)#R>lDM07ozy-wS7=~wZ?-70BHY%XoAJ&jD02_kG|4Q|mK6z3AE=eTm-gl? zp;fWBq)xck6cFXFtADM$Ubwvk==3M0&);rnmF%^pdm;%q0%ZAb?XDc9J%NDI?uvkP zM>#=s1y&%x!MG#6;YdF7*r;5CN!<#9@(u<1^X|;ZrtKkv*4&;=r9K{Uy@0TgKT(2q zn_Oc!lRsf|y8FmtA07wP0=ProUSi1Q` zGT=y(Cv4Y`&i*KadFD-?Qd_^y|9Juln5nucML8w9PbA`iU;44|u*2s&(gX-CCNY*~ z9#|-)#fhE`QWdL3(XV8Y;>6@9f&>dLXEK;%D;9j`cDb>J+dPV%+ zv|2N?TA&e#QeGV~)C7}L-Qlq;CCy=C4Z%NtU~01XMxq$-d45_HsyS~%E#f?zHge7B z*b14~X!UWa6-+~IIpX$h&VFZ!h=W6EaYLawOn8bK)4h;g_7@W>zIhW9n0G1xYD^&W z-nF)|V2ej^R`MaimK_Gyg_#bGr_31b1V200Lg)gc)S`a7DovHcRlB8HVu3Qrm{&@a zGr5ApBbl3Qhx(cR!_4whba0sU!V z2V8Vaw96r}aJt6oGRSKWOZy@%LQOIgYVY~$_ons%JG8kak@Bt~T?x?r#@ZFmPY8al zFIY)7eQbq86)i1HI%nwl)XG>Q*r;**!DiP0_R1PMZbf06@KKiI*_wp6GFtheJVVh?k zwIeD@G=wtb@VQIFHkBsRISMt!4J?>q*!HryQj62z;pP07Hw#DSDAIA!Pl=G#P<`k<3BUTJ7j~fK*ThxC(()-j%nEp}RfA~Zox2n`)m{npT`;Id8 z5)8ejdD_fubu;~4W~ArJyxh{J*g-fpT@%B3-nsyB%67A>xGoPz|L`QGayNjO*Zg6g zxy5F^9=dn2+%)(B!$ar-;Kk5hF9e6GEu%Mu6UddbCEEs92?-1b;iiW}EI{9BTt^T| zsKjA0O5U((N?U2I+o+eOA!#z2F7~wsGzizd8((TN-|RNXGs{8_I_<^Cm?j)kz}OMv zr=pV(*J(w>T+wuLXa$b0TvS|`nS!_sOCU$zkcmPK{VB<@7oz$)u8l0E6=5Oz9FoQT z?Eaj;4ft0g9QC>Ogki< zUdEBL;)FE$BAm*+O);m;=u39^WTElg^Z)4Gr{5fYE6f)ruvu-34O zK=)1rX#x=IhKppDI?F)MdK7NrReGTwOW z)mK0PNO=4%9wNo5SdJwMnzG7tdzW^ZCc(5b`MI3OXDWiqafOWW%VHww7&*_su_Wjy z0k;82>K$FmDFst=j7hB2y`>^`a%c*$Oo0Y=x}mg;l46e{?3uUhuelb!`IvcDtvThT zBiO>sezo>iO(cE0^2+Hyp$pzvSnf4Rn|KXBCWC7og&69~Q67(Zq2zkG$D7TqR~?%Q z&KluZpkwSHiS_N(AI6h(WR}JsHbQA8cD!Qqvh|E6zyu~nG+}|4#4uv=vgsQy=&}p| zmeI9fha7!)NrHq0~lkpSC=PVKCxXbHRFMQLC zQ>lT{;kgX<&ar~xssM`URC(fHlZ+e>dhho;W6KL?V z`*PI#IxUd*d9`wxM^=Zkm15n0tQ%u&*K5m7v@< z+#FAP^nsVS(+Lo_cl8xFWqjExeeSpIL*CTR-IH6KRb)cjC0F=B=0=%+|L7B|;b^G~ zE@N97g4gXBE8;}C3GBOxg#BK4@KKmNZrS`Hvr(oMJUrtqpp?oe9;r0nK1^ZSS(m^+ ziK)zZKO^u$8kGn`uW@nrV$t>#`99q4Y3J~61We)-Rh~k^WC5IsZ4?(-l|~YI;`)%g znyqIw%aE=gZWMU4xuVG6NdM$yS!h*Ud?dcmK3r%)h6Vld!-~J3Uo1?F2%V~I4$aYj ze55eA8vSTwzf78gaQ=6>)kFw`a-kmzQLvP$ezh|*0_(g9>skH~(Oib3Plcr&&hhMs zxMFnm;0nIDE;Y_^nZ3=}+jikiNjcDbxQSWzjtIKon*5)-UcZbC9DjJ2%OTSXi>11A02e^;Y^(IdFEu4g^=zi{p=*+XB`j z;qxu-mZ!DR^V=VHN-eMJU!-_yA8f#2dO=GIu)M;@0Tu3ftTl@Y7VI#uIc;DhH(bz? z)XbgN&z97#=ksTg_4j>yZ;i5OzD1inY_30erXf>SO5l%!9v}LE4w7W~_T@DYZv=tI zdPT0;YkOL9fwM!{uYX+J3ag1vHC-bx^^MiSb^+->(IjrW)b-A1{^9rerlNBZm)Pow z{Vez;#k)pGXp9QfJXY7#*(vAxp5ytWZ(QiJuk%{yKwOUE!mSgI8*0ySVfayCtKem1 zv>@@kj0k%hS>Z{UEGVb{*EJ-dw?>b?L*#5>SH_`}9q$-6*VkC-zR%Dcp2cg|DRu|i zK;2U#c`rcygOAFGD4E!`Xte&v?J;amT#JD_NY)5&mJKDd@bbgOmjmU8 zz#gkUM2B#52P`dv@H;5Pebt5q@9+X-UH`O1J%Lq@_fz(_nl*a_6==SI6!~)`FMx4S z=ZH??5*V(%CN@e)6Rr+C;72vLK=w{8&v*O{Z$Ck0*9XU=lkZK{QYhkKt(eHKy8vRh8Nq<9}58uEI zAUx5n?!d|9Qf~!aF*au&k+!drQ{dnk@}5|8fWxf1z0uU-hI})B$@C?RwM|Jkdl}Q2 z60cRBy@x-hr~$;d#5lx8c!u$Rg0~K8@1u!AJofcR6J*`djgJg)(QcbBufXz(uz?z< zRkpN8NcV|i4U+7{8|szaPuJ__`@El5+erSBxhq|=<_1T^UesaR&t>DBhh)VD9V-B; z9oFn6f|11aT+F&6u-0yCUOJp@W-HRHE4vt%l2Ta1#cz#>?p>EUB)p7I>>?KTGJh2& z%}YHuvHAQd<{Vbnh^U&NFPn9bAj)%ba^n3Yjsw*r!EslyfuP^@L?x9!tVp_3+ZhVB zA0WEz?h&WSG7a(u7=u~S*2}qybwbS&ngf9u<(bjhgr(i@oiwD2>lnw+u=xTtBriCz z)R15R;2Vn@_H*=`&XXDK`gtri$Z_n;cE1g#r3>1Zx%c9_wwO>5j3%v?=+pO_3?d&O z@mnEAkq01-?;+}S-cKYe6+QCPu)2mAurXcHjVCBAQ7JE}?Q}INsYhigdA#s1HSCN?!{Ru9=4 z*}O*Ww$DBvyE)9iMTFd@f?Il`)TOx<?Vxl*3VTw9}B{W5Y#ll=aEtWgMFcy66vE?4(WMOT0oTW z#kY9NrL(8FN;qyv-=j|0{XU72ChIahY@9kqE9epe^6xr9K+t-VyT_@tEOH%$CKpO3 zzM-6_V2@+Q`7@F2V}o+!4t@um94f&z#Yer%jJIQNGP*Z3S0^Ej?3FU{FI2_5IM zBi$hW(=ezmcofC3vSqNfm&VEMnKDu%kbN+l2g05 z?^9OzfhH|7CUK}_e9|_Snuz6Qg$uY|!zbk_5^ql*<;cJ+)X0|Ey0U^dwV8nUYX>tm z4(hZ|^O{Q$Q`Z&p8uU8<}NiKuU#9gz7MS-hyyIm!{Tt|Z8VQ8)6kX+-CptaGp zGqO*pOTuJ2aHfA%1fI^J^#^~#4}F-(3+;BfP;DE_OTvhATs=p6gjCqt!}_`0bkF7j zb55~_@Z0#-Nxg1ZrC=5K*Az5j$4>O^7|ciw?~ zsdd?CEN^o3cJ@#6*qn~2N}LRMgO;UJtNVTe9@m_~#GNV#^0`~v356R)c0Y{l7yKy{ z$P{7(s}QhM7@DSVpEga&IVu2T#E(-X_BL(Q0axJwhVZ-Fe9F+WkeurpsKZeMg}+RG z*^%sSm0(l?E4C-)j4`VS3=LelrqzmAHihJAmieu^Qg5U&2ajtqu5UiI3r!J6RA-d3 z3L5~V^?YW_uv%OmSK>twTy*-k>0s0P71$@mR#(lXQW#~wrN?6tR%__-p~EspC$1d_!47?DO8>UU0uki5DfKjy*lkn@cysSY26*$ z)Gw83A1$L5{YXD8MP^?{I(J@`NIpf{r$CXWgdpFH8WoWlqs#+%MK&5{`DBrn#ds5* zC#sG&RaGDpp7D<0;sgz^)NiL^z2wZ5P4H$}Vqg_94k%A7j^k4-kszm(Q=)~uQpq&M zQsadTklnk|&S9Oa#w#&Nv6F0++fV*Bzq)uD$uXKibY_m$c!c#6={ zs{PIwyUD5~pBG-rKGCp9TZI8|`yC+Q&MQur_1R{ADJhy+{;Fl`dIj5N#gum;>~m0+ zjC{g4NZBSe37P1c3Bi+#(~RhvDM1--FZ7}+x)Ew+Ct$5S8XowZC$Q9F@MGP2B71xm z`l)yg8_4WI4-N7iMD?kcNLinN*t~uZt{8vVr!1ydOwh=Om|?UXKmMyZ{+uTh2GXl5 zvf|S19j_lX9}dEcKaz3Gh4 z@bp8+=c_MXR;?tOuu!_-8vAB#zzm^0XZsxlwV?;9sX2t_I$!QOL#=vQyLF{<`A_zL1UxWBlX2?J4 zr8)cy=?({}7wwJ*>5FFnxlmCTRHDyWj1eK1E)eWC8m@8=2`4(Ncg^sJk983C3&>z6 zEWw%~ao4oqveXtM)DGZ<1sLBVNZTVBh{k3&P+-H58-3jn+-K~T3|N|-6ZTW&X9_zf zZVyGDGESRPeZQl5qG_2w_7O%9w7jn)hOeDkY z8(!LD3H{nIf`5+zvUN*``F@=IPq$EH9@5Zy5xe0 zJpg!$5n51M|Exntv?H&|NaL+A*6av87Q+Y%pbK;ppc@CrtRGp;tRFgQ+Lb-p7HFx# zd__(F5p1c!@{WqZOq=KkqTxScr5hDx+67-W`QrR5>CUTk`{yL!9ZWvKk)#8#6&yD1 zK{AtgN1i>xs4!wsC^3?CCyHfnv{AkF%I-4RhJ}!F=T`F>xhJ1ZWI};Ye^Be#5bbVM z9o^gi%DgjOt<$!LPyZ3?*}5-_)#~`DANq!QZT`Y15?Bx=ak1~%>DUs@$GZVCwGt&U z@j{4Cb|B&r8NJ#50FU0Av}c6>jv)I+f=_f%#Hlf&2OIa`lh=1(>VvE?_JS&(atEhN zb&%C2IwEjT!TcHG0VbG7OV~0N7{nQj%&Uzq-L4u6tVKC0u7xQyf?jC>YeZm^8j`TZ z9cjdx8%)S`u`yU6c}YpnUWr6EHjFgG&UUU_Vku-U3KQIACHp{>%=IxZyNw0Ig@pSG zisQf1^4e{y%A^)&Ai7XF%D}IuSJd&ts&1K$BVW9T1&P|JTt+>o#?$0uOs+m$U`_h4 z5|s=80YcZ6iLP)&7|Duq)>88_t|7hw*SH(|TEcq4SgN8xvPuT7YKMLL$Y61=QfE;J z&g6~5Lj);C=Oi?c$rjI90k2radkh1?f<6!ZQT*LVd6A6=`JuJ^@B1<8lqg8^i_QV7 zl0PM+x7Q;QR*?<1Q^ zOlpk|k=DF?w(6(M>euZKcWo=&ScCqIucF6{9ks}^w0fKBC&!#0C@9Y(bv>m)ii?+8 z{DJ-gqAf5Z+?|k;p}8#p|0fREd7qB%udpioQa6`^EVCUdHTkmdFyS9Fc(ji**C~fF zAUKdHJDTzn!A8t`H$G$gunB`WQ`>|&ZC5n9yBFXJT#cB}d~|0zgAPp$M^$`V%Glxe zne(pOZGr6XL6C;PgPEVD_vg#1i2BHPi@Sddb6Fj}_&{$1R*n0BJ(3X2qaK3oa8rW= zN!*0ySJN)cS$lcD6q+|xfEfO?;gG7+Q>)H@0$2JwDYdQ+IGK>w+8f@O!ZDGvJ~+^M zaRgD9*izAC+^Eem)pmjUGQ}>jtDhcrq1)DVhJ24fkk6{d|}1OvI^ukFUsi> zEVG@KgNRd2Se2?FJR0{M7}Du^Tj#_8%f(pDc&te%t*r|w=kz37`O!(|9ACDpKnMW6 zydzCag5b??x_(Q$$xBCU7|V0xt~pAEx3l9~NpP-R!G>msUe72FjlT_J+;cAj5M9BA z3zqK~16_(ab`k!XNRVj8$)Zm=i}1%}o?~aGPTQsPOZn)a=S|n z26mXFG30drvXc@)$tZMu%^r4jwp0&tT5XUTr${Z$vrIhJyg&fWu3{}qg9jJjB5E#I zD$8t+PadUcx8S#-WjlqPHx^gEIE6hoDO)7R>iiT6Xa9MDyJj=}o_Z^?#Y)+SsEh{bCwTD}Xd>N~@k?Wvw z^b_l^n6r&jau7K$XD|JjO`X=nXRlEoACWJp$3ZxgHB9$mTiz0_{AjQUY(eqlUcckAUIz%=8n%H_Vp`Yc8gm^Tw>|6a*ZMH~| zqzqInsQ=u-^wd{|)0WEAszT4oA=4}D_&k&-a+>$x6TwWVC|>l*WL7#c^$aCq@P-mU zUbB?_cBPxPHRj=QNvxmytBq=o0N(U7`jB!066X6}35Y@xbp z7_86aBd{n|c5M4$V|5%affzBd!XB!PY^fIXY2oD1B3g8=SDonIjQu0}dG~UeR!(J6 zeIi~RQ>lM$!$Bp}7);}Sc9B%ae{~!b*W|9jt^!;^{@xms(e;KgyLwy(*;2%T$zOz# z&RfLc3(bnK28+9lqAwG$&#_6zrc*B$!+r{{D@tav)3h@pCF0ok`^!aYM2EZfy(13o z?v70(<{_8nULp-b146{wQgJ-bXPNw<%-|$-%3!oWa+O+Sx@*0p{BNVI`tOBzjt1Ps zyA6>$o81$p4O1@=dwG)jsl>~%?(%u~(=YnG#i-h8z(V$FX7WbG-?mp=3n#BLC3BbF z;CwRFvY|3{!T9N3_j2!$P4Be{S#5JNnT7o-Y)RfYe9#nL8R86pInrOncgUH;ALCe=fXtH)!m<9z zbGmz-vdo1Kb(eb2Sj^;bOs`7EArh_;Gixt6~w`+I%RI*SZp{ z3?Vg!KTK(}x1zK7uj9_BGO zLT~PjU7psS$q_sD z27YC%p?N!zv*}9Dh?ZuIS%w__%5Zt&o9)={OOoqrz9#}~bhdeYM@lk7AM6Grj=$$13@NwHfT(e)cLk=|l1Hr=>(@tc1}su3y`noM zvsOEVpTY_i`?P~9ck=#KmJz62LE8co8}J3ao1yvd(VXLhq(o=2jKx7os%}4(>(kIq zZ=+I)o65Cmy}9eun~>5=B;DtMg+s(Dl-+eml5diY0N&nD}F@l ziZ`3{>(BUE8zz}V8WT3Q_nb~a9e{e}+;c|^0oVtQ0gn<7iYbM@WjhH5+Ntz57{4QW zB1w~CFvMvEx4${n*%v0PKo-Zwn@J022kAw7z`>?s#P}) z$|Gqpgg=nGjvCjY^;8F!gQ?|){;g&2AFDZ>dKG{j8LwR2z_JDZzu}vAM^^)iitmx2S9b_po)~INggs?+c7W* zTN`?}8hMP=E8D4C+fm>bnbu)fR|G z-=G5|{}pBk&n<2n z3PXte7LfZMoCC)WsEz{#*Ibzkp^udOUpldH{!`@A{-6O`4@phX%@ET+wxKx##W>)YsCbY7Y+m{}NtL6`xJ+rnGEEut55#P@X4v;VUHSl&u9@ zSlt+|Q0FUi^7N`v{j0c3y8Vdq!_!unC=tL$)iQx(>r=P__F3Xx*K?L(;e$MF~ZAXU>GVa7+d- zk>+nW?H}5FaGeF(3@xO6C+yDyv`*hiRc;2!U>!!n8@18+EYM;wIP=hq^#&ZqE{C%K zqJ!&vTctLgLb++fGEg>`6(q941AEQVbJ7t_wEzP?-^P;nNQYk8n*6U3eJ(r<6HpW* z?KhR^4^mlb5<68%NmqXWX>l%T9nB9lwS;(TZEetG zIHzo)oQb$Gx57VVw=iqy@svLz$Ryt>hwylyflH8KW}|06j$cl_j!$YCegiEzGDK;k zTVqdPQ)5?RUt83%I>5QY!NE;)S!gaW1)1VCQl5|PcFAvW;c=tGeXGl1(_=C^&I>}V z!fg$Zfwb07FzVItnrl+o6Rdxak^dBU>g@IqfDgH=&S8n6a`-` z*?_s*Bv}a~k%Aw`?NjhJdO=Afte4e`WF;P!gb}$2pWfW6T&Fcp*g*1!N#n&6_d?uM z4W2kLYj-J^lgSu#FJ5>29nqk%#E>_!bTmg{lX-D0cY7n7gD6L8fI|J(_*mVo#JVLO z6vd~EG>^zz*Ky{p36>0!R+C}5vNnyJwUV}V`ik_Tw*DfvOqF5oK(--&uqU*0?fB^K z#@C=(kdfKUv=Tc?=n6+5H(6NJF%XV}B} zhy?>dVALfNiVAipuf%CYL9*#%JSPkf&$$!t#UkZ$!11rhL&mGW!~ULYU;4+ncFF%; zf3nI+ zgMg7VDH48wAZ@5FC<}iq+BVp>vSzid3(`#qMO^ z9ZSu|>hqpO|Lq|q6WWiKC+~fJ>VEG2db?`%_p1lqCERf`S&E6Wn`%pNF_9lg1Qi)* z2&O?+Ck2qEBR3#BA?qg?N{z|1QGe?~Et5A95A0Ef9Wd*1Qi@deL1D>Z?kXOKa%LwI z6&Z}6;~qOGhX&R~moZWnWjO2H(T17e$~%WUVJ6HEG}Cbo7-@z=%#NYn#|O%AP8wwo zf`)O1Nr62`QcCW5f>};c&mpPQ{lq|{-4FbMGfPFen;pnz0j)ob8DL}iCy>)hx zu6zqVL(|r8YQG;1B!s>T`DX~U3EXRj!CIsFHqwkf6s5&9Ju1lbNDSCG-<*9yDq4UG zq#jeCA!boLY~u%7MFQODlx{zHNR27@>-y73CFLp@&P8r?UyP3R2!;z?wN6mI7lG-+ z%X>p^1B3~jsHq9tTptJ8%BJLJFr~JpW`5n*{&52k6(dfyOXc&+b;8T7s*_U_i@B*M zcuoYs*vh4=k!I0fki&M6{f`d;lb#4G(&}AORYr>Z3}&ej0+b;I)X<)dc3DQkVD$-{ zN**QP*wtD(%Ydk2qt5+DCPib0bkN_n03faeI!!9ysok>dMHLf>*hGJR{j}7{Gc#?$ zBr~lIZR#u?k|@;{!;K$XQ5|@CjgP!oIOJ&N>6X51J%olQX1mfNe_Zr_u-?kifi<~a zFl{f!DaXdul>W)K1x$NciU_zaNb5D(XLV?6u&a4nB}Js5(68NvP^Ki$tcKXsXvJ7R zCH~Z7>lZaLHD$I=Pxjf4NcES`)(;zjxo^AD-dU-@W-8LsIK7&4tRUmF z(ba=FQHPES%|JL|rK#1~ll6a@hqX-k?7Z6%`BC^a`0m2Z$IP~1C|{FZ!hl(o%+9yy z*T%KiUHVw}r%V2V=)`uq;eiGGMqj0eF|XEb*w&l_hZK2w4sTn!m8$Plf(db*M~nZXh=8o z#Qu;%kk%H3Un4kF>fJ|*UB=vB$9z{W<^4Jg25--mWS_1&*=8p zlk6V`;2CsOsU;uCR2FmJ^r^`C9hZ`BeO#o?miquRAkv#aJdVSJ*oI7K&-ZW4O>ZEX z@i-*EGWNR}rMepDUIolN?Q?I2;n)b+HQ03x{_x0q+%-WTDx(dy*JS81@qNh+xdXWv zRk&pIGn2T*ng6X7+#~kqu^cv_Jc%*Y^Nkn+uLy%0Vl;6XEf;+YpN9nr~uR zCJ2;AydA!8R^E^Uhx#qM=I?s~_yzG_{ZnfjEeV!pn;VhebeDfZ*!cs_d%@yn1x8=V z3ke)?LTozl{|W>zI?f^?T_mm>lsgGNA7TW%&5F<}CCmqY}|K zTXPulp9%uu|6N-C@8n)17o-0q_BtrKzpI*ygNU zk~(@#UDd!!+9dC8$t7Rt{pTAW)^c;D%3_U9A^By#5_y&-o|Ws6^2HWA1nV;YHgy~k zzvYG*<1k{5V`X4LneoQHTXzd#muHV*LRnp2!rXJcOwsjpal!2#<{?+v_1S(X7JEW_ z4a{$aI9LRGtl_JXQ#X@eMyKe{z7GXdyv&sYw(KUSXmJ-?_x9K=2N77;Dnaxjpe&pB zWDO@Sqa})gORq`mOk@+RZ+L%5@mS?g?PX&f^e!;XQ6S`9Nq!We`n_aSOy7m~i@ zQ3-iIqCtEj6yEaabXnvqI{@YOckD#a8j)$V6+-(0>cZk4!RT($#BN~*?-*2Z$0jY| zq6Jl4UoNXkarQYiBD##8UQx`Xyoc-UZYwqU#m?8bq{9cZHD zS_}liwV8nMJ<|xpD;EVn`-R-2)}KlPOeJy{RLxU(+ehBUKEeN0hA_cs>HqvLL!kfB zTJ(QehWxj9|CXTq@9fSLTNNC&e?>j7O@obyQ~^7`29gM}Ccd`ng0T+1l{93sqH?8r z+P3vt$9haB4s|WA`8%@j1?PcIPCb~*$UMi?X_)e?X+be!I-|Uq-3-_B`*r*Cmh<)J z$82pc2xch$pbcyY=Jk+b?A`hXtXA>{l2lzOz<=iz4y$8&EruN6dm7nC*4I6S zU-Zqu#i5+5YqRy=tw4ipMVU6Am-P%pCD$kkre zjq9|sRn>E`lF?;p+hkUa()BRcY4ZKepIGXdvkBh%IQ?2FU~-4uO2v^pdDbYwH|F)g z{OU4U!5+uI&c8hoN8~l#B$ro7$17ISH>bvGt8cZm($@K_S_?3qbCj+sO#v#+>yC|_ zWfDIrKe+t*iz;v|v&MP$0tXA9UsXISg{dHzbr6w!m7{mgrI!#jX!W^h&hkYAZu%Z` z|J{1q%A?dIY1BD`d`7L^zYz-K?^1^)Rw0T7ztrF8TaGrQB|0oq<{h^jaXmyw<;bm? zzZbAMq%d{G%cqD;3lkkGC&2Xs+lC;@k3ZqE_-mQ61vYFL(gSgdX6&!+-(>8<)l1Xa z{dMd33>sabmwxMCFhw^xM=_xCh?IV0nYTTDy+!T`N9`9J;`%mFa#ineg@$7s3Y^zw z=@Y0$Hz95PpkH*R3s?w)CmqI-{T)&4VR2@#_TUjWVqN@5A}Sy_190xm_lBu`m&G1* zH6p>L4dR?k6jylXFG|#7)XTV3A!X5Niy!9`dnQUpT(pVrBO*(hURzMP|boiujVHSl<(@Cy{&IoNwRK-9AM#7!95dqCK`7`hVQ zQy?oikgO*}UhcAmnxA?GCa4}FkUrgWAi)6=Vxg$ew#5{rH^emP(d48yJ~@>+z^~?z|hnKX0D4y)a#|3f7gYH{wD1J z;rRTw+Zl~f+5Y_ZNyhIVx6S`4OZUId(cfw0|A8Hz@?Q>!@)g@|TF|JsKRjwTV4OAz z71+qZ=i-~BDT(gJZQSD~6R)n#7}Sp!kVwN8!j9(N1b7)kbBc|07K?QBi&mbwpXxn3y?w@X?@m3Wmb;p*;C!G8-)`r*rzD9jrfwFUwT6eT z$0j1RF35^ihi$c$br&&$r_>hQ9cw=-mLBu)=89bqx@FXzj*X^r zUN_Bx_N1;rqsp5m*fZZb9~UsW1qWdTl;`PFeXV8Iy|;CziX|B8oIaSTvP$${r?4cU zkk@$OET{|utG^U6ytd~>WL}2y4?$+3wjmRW^9#6YP7gVPPqJdC39-Dx>vOH*GIfMu zPT}~F(D5b^Yugemy7KAI5}m?mV1wCMv7zK?6V=+TTIzV0=X`eT^vqYQ^~+j{Yr< z1^nwnS?PNlJo!htgMToC{cl1Y|B2uK6nWJeP#(G(uAgyc;`XdwsntpVZ{taIo8eRm zlDJEc6dMZtdTc4BBQYlOkdq8^%ggyG#wc4Q3y^8yHA}6`I&fuONomu}F=f&HU(!<2 zQ~`>Kb>xDS!QcZcbEMb)zrC@iC5sFn4;IT(>wc>AuVEgVdU%4GA@FxOGWPu|>8;|SFzR}#81_9q>8?)X zEXN{bf;pyGf93sXWD@DE>LFk5o(TW>JJ6YP59tiv>Lmi|JJ{=mJ6im?JCd!^gV8p< zQH5JiJb^(Bmv>x*GY@=(^Oq29U*Np?I~q1Gr3ixqc)VVvqBO_eSdFDq-k$QP-N}w* zBS{R5PoXq0%wj3ZdMn*!$0<&u-~;hO2A|~w6Eo6Hc2_S0c08Fsj$WTbF7!B2 zVVjL+8)M{)y0;i~E~<{6+wIWEJ;%(B&D|AR)qL7m}LWEU~wr~FU%R4pUHF=KipZp(pV^=o{6bjf9`(_DOhnIk3WHWWW zQm9TUHmWu5VLNuU%B?fYY)gp=N%JWe4@28F9Kpez^ful0I%QhBMn(q7=#B^qzdHcIib`R8NM)2 zSP9`7mzSdzuj=v#{{8)jC}j zN|h{1V*08BK{4(zcf#ZiDXnO7yOEYH2{$@DOmmUQ%U)S9Z=1s2T6E7&*~N{zKP_U> zPbpaiq)E4~EP==*6!c8;S{?2OXIWE)=$@d>z`8xnmrrC;8=h#O%sh-#0R0^v=c{)a z;inbQ(_`)a&70Cp)kU8(Rf5_pP3@foV#eW%s;YRwh~m5M=RG{$^2vEg4?_qj?0nW&!5^|rEapjQbRjpG&OqPO+jqEQE3g3|*%z4?A3t6LgV!LP)kkxpxl*R(H@!p#F zS<^m~y;UPsW&;jpRQn$j-@wXc!Bq))G-hr^~y0E8A39 zSi}_6#CiGWgS+m0JU!Pur#ORh>Z+)Xa%~g<4dTRw%<*+7$&#fI(H@s{+4oEj5fP{C z$yTOU6yTjPwCm(gnRm(et)A+53Uyv?wWNW&Ap&%-#EyF-aMqwaW=1`H*xSwi7&-#o z4Ag^kjX--A4Mj7)-{Jq|#h-n@S$V(}If38PSo-@5IJqYvZN&BMQmZo@EJk-d4sQzm8_8neCAZ_&ZBqE|b97z=p0N}0TG zPmHb5ARng6YU)7Wlw{|czu0;&fuTPRL#McOi<3bV<^$rGSmNN7@Ou=p|7g+d)#r_< zy>(|2*dBFcI3jtEbQclfG8aJQjmqg~V#Vc-66l249I5*|2f4Gv8fjw688xUkUBJZZ zeD_G=VN%6|=@7Tywo7mGXn@*`c2=uumJb^?Soa6Rc{G5YGE*m$2`GwHG3hbp24~h1K zK}|JrJJr!+%f0OyC>vz5x}?=f1V=m@uDbYa|7LiruG$h)V<+%@j`LX4$^sLw+x-5m zigxls>Ioe5R(|0N%+A@tEkD}751EFx2ML#R=`I|sad?3{P|n=Zf->%?Y_rdMw}?i~ zg8+RY$LhU@bSIU#tGYJe*=XN4*n@yV>|K{--#5}Dk9?B;(q)+4Z z%fK^U9Rkn;woeC&HF90SX+SY^F!Tz}l)L-Ia5oD-19MyF3q;W9aC}QlgIjxn4;r?{ zwe~hrm(5qy-DnBQS7capVTC`8WMum2gov~i7E^84TeA_d&0r^P-FKBIBVUaVpBz?w z1dLz=a&&M)p!|)7uACe3K;uJu{5Ue7=Zmz!@>0nA(ha{lD*t&BS6Capi=n{X2=YLg zSInX?pHZe1baM~w7nk$!rWr=C>-3JO=MkE7sMQ4}U3$(T&tanv&Mq|$N>j%XTd@QSm8cp*gwQ)T;_5P*~k+?n#+!3LtRz>s1>dT^q zEi{MCjpZ_-R?Bru9 zRVRKnU085_W=5z|6jII6{x6D*yGB1|_u&Rl&%0;n%4UCKWESHBei32&>>sI=|9BV< z^9 zbl!EgYv6LV<0}0ogXnb0IEMcurh1-t?3^eIIP6xWdTKz$e9WVnl zWJ5nF2Y13koM2^+I8$Y^_Xq^5Nb6)0IcceOI)MSCG8&(K*H)eP^kr?{*&IJMQ(jYP z4$2|5(s)>}lsQm!&&f~ZtGKW6#Cm16=O|!2Y7FT_u8C&7#~*M#cfyA6fd=jgU-^RN zJ<|*DPhjL|a#}sBUAfbG$9qiAv3LTjB0Ewaqf;dIQb!f(4NQ@Grb4Z5`~}=;dg0@Y zus${xk(%!3&NkFBoSf?DR8Igu-(Tt6d1!0uW}0&!R;*xN9j&f@Y{%Ohmgom4KE_vfEI+Kz2C`dFRY~DA+Z9XYW22IqPzhNzvVSZXIwmBOe>AlUu&UHI zl=L4jG8n>rW&3fDqf6X7EdGkuqWqhRAz`A?$8KF!o=df6?02Z+uRM8+byVRFG08{g zs9}{w=wq~z2enfte732v1?K*V2Ik;N-8E5Pos5xk|15@56=-tKK$_4s;nc)Qw`0pu zt_!spxmcM<{p-T5FM`)>TF0lfs%+f!L*q@m(yLWRyAIp=sQ%)sgI7_sy=w;3w-}%K z6)yYY-2BQ=%hG|e&dM48w|_Hh$`E*0;=ipCr2o+&@PBmLvt_3H)~&O2u{3q2|Ndf{ zYHz>Ki8%86S;fP9!BWGD3bZ3+b6KjLep$SgSUEuFu|x`1Gi|6MC;5o~Bj#b&@012K z&ipvzcQi_mUt4##ar8%g1ZA|OR*9x7DlM(2FoAV*t-LPGQ}H?RQHm}dFijor{u`8il!3J5rB}uVGI&Z`EbC6DmnzUUT!llE={R?$lNidOl50*^{+fH zmQuN`v=aT<%4_Yrj%&lMI)BOpTQ1C)bqQ|6pcR%{6j7*d3XZNg7=vHa{%;N6Ob($jf z(R{s)_s)mKKk=%2wi$)$%9CHztTsWpQc_DKyXDUL*7(lI0n!O$-LmD~`-%@ih7gbB zE=H@xsN88I@XPVz$>=0$YK`(Ep*? z%Z1J2SvP;Kye^IS0Jnz>lA!M6ali`o!?W}pb(H%=Q;e4GR>-P$dP^P{GbBQ0Z-lxK zxKUJ-QoK;QWKL5J#u(u*K&l(l7@-YD=l750S4H`1@j`7=fLg+!Q8(C_K-g3ffqqOZ z+AzOBCT30e4^R_89j-2f2>{X4HD2g&SqMBq0|HA_9}iHso^Q1uBmmQ7A85LQ>44KQ;)#{NTP zRJ#ZH=c>IiA|c4NF2vYRZ`XKW_ZkpDAKnEJmP0K8pfBtV1r%del&LM3I0(=t(2t4* zxjOsbE+<}*u*KgmC|L{79RK~2gx%a@f^!00@)<-y-}9YBAV*!(@c~XdW@Ca}_AG$> zE^dq)uoJIPSmjspH%vhO*2niBH=JW&txo>pV=c`*@-dg^w0Z*^o*IaP&dXiqhjrg2 z?i<)(KmV<@X`OP4#q`~HI{!8;{eM@9rTd>?jt=&=^e(2h4mL(ErX{LM3Y(0GeiHiv z*@b*{6>`uAM#f-ELRDhPs0qv?C{8FGkCK;DZdvLmU%uuXC3b72Qh!PwUpsHEJKwfm zsJlFILzM`22RHfOyz2+SL{b%NETFZ2<|>`0koMPV%DRe?h6JQPBwLVk;{Mt1_wFM} zA>_10gn|m`3l?>tTL;uI#n+Ap01_7hovM9)rYq}EVaE(c@ylxM6ia7ls~GB_EwTi& zfThGMFp8-smQJyXIvL`EgxJ>h6IqG_COksnQfS5-GQgD_sh6Wu)`r(CVj!~6mpeuX zGW*+q4Rxhn$(G~m!t@Z8Va+D|4gdjVz3;|8?f$ZffNuCf&pL&+y#m@HYf+(u(4v#f zMZ#-|yqx4w5lxJ5#m_VksuHmbZG1lP^Aud^0FrUBgBL2WveL(R_*K}f-;B8*aTRs= zgB|*vV-at*LYeb#o-?_ldlFha@-QhN5ZPH7D@l_%qI#`EAT=@7C6C3*09Y%&ov;u)P-_Uswtbu(%3SN9IULKmOyIDfl}IjrCS?$X6Ot}#qk&!8ETl) z=mgYC$@w&}#g(5bPvR*P?#JtqpeKr(cN_nk zZH5-j*AmM24XRl?8*!+XhypvRVT4QzLG>E>?hP9-Eye4ND>D~7+&4UxX<~)c$nN|Y zwyHRE_oXvj@ZYszOyuar(`^gK^&W5Dr+~g<6jS?9;`)QPxX4?MV7MlX(YTz^*k09& zum-stJ0j)GRfMl8Czqg{bTL!qsYyn@P5F{oUiYBs95CdH^<+{nh)y`-Eul5o%ex3b z8xqAHS?J4EcgX6dOP?!f0Zgl@JoI_~6cQ1;)s0Fe^0;)j%&{oam9fo&?lC;30wgmN ztAt`Fi2v8Wfy^M^yoK}c4(`D}0-0?8pNq)D)`mgN0r0(k6dYVE?d|@{R53@@QUOH? zwL7k)bVBqp>7q7pJE@Z=O_>##t#ekD z*I9`+V)a`w=De#ob`jB_)r=*H(Y;sKh!6V?kDjZb?-1% zh}l9$3dP0QIBZcW*!E%6@HU)XwFM6NPRr|dgZd4g_1#w8=iqkjp^?1?XBLBYmXEr- z%}O>iRX3w?n@|pyofAxLswY*5MoY9pjyqqo0zxKRd`og|$2l_sro0*ADZ^~8cK zC-TbyJb(-#?OB zc;haZU3rQkx65Nd<=`Q+`=fqe%f0T0Z$^wh5fYtk^YhKfmt@bhpd})jF;GCZDCeneNFOxo1ik(MoXk)8TT*Q^m7U^d{eX8?x+?wH65!(V}07do_pxDE^l1(r{mv zX;vo_lp2N2{I!DG+ORF!31ug|3X*)MZYwM{iko@0irn77sXFB9&D^-Ga0u-RDtKG7 zdS3HFz=ec$c(sw-+VEDodAF71+rVgpfR(_P@AR0f*9UbkV0g2J;@@B_{s#-uDfDQ= z>F;L!8|0~jk^Q2g7lJ)rR-?Ti5BKRlqszW0H2#Gu>vq|}%{K7IQ`Zd?;?M3o7_PGq zJJG=Zf#26birsYo?ku{){CDE`zaLQe&%V~OxBGuH)c?x{PxgOz!%zEtyWz#}z1&}N zdpJ`uQ6ND9krHCi+{9F)PS`|3Z2PdQd%>w@LH*-oGjgC+-3Dhqjh!J-UUgo!NEFe^ zeeU5;z50#I>Z|&z`tFYU+xlPX|8{b{Q{|YV-~aV~y=43RtEsK6?XIns3Gzx#<(Sa}BodTNp~t8Cx7u&cUjWHf3WrK!wPmS`tg9h+$VQiB^%9 zTN?6_x0@4FLM4_DW1p{zj-n{ZiLx%yM%A=BkdgmTL)ElCz>(LhjkeEOYlzNecaEiS zSRTsH@e)R5v^ijx?`eqUWV@@N2q=&CWWOt*2&j*KVZRHe_^gQjZF`_AkDPNbmj;km zW*1*XtE>RBk*lggTFG=&0My=2Ypj%qDb(#WhB4Ihs$*K!oI7X&%3=s>5Af9UYGWSM zoO@{)l!i6b^{QfG)$UN$_3C0|)tswo7*vP#YhJTyt~G`k)VE4weAS%`X|6Sg!D_se z&=IQ-jMe!{Vl>n*HHJ%T)=Fc3)kM{n$fIA?oGA^P)|{ygAJ%w@puej%8=^z0-IdV< zG{jV@-NC8z)yIU^aFxXv)*N7~^VP=55i!!lcW=XZw&H{H zK)a!=N$#D37mzq5^)bTmN$z=re^7y@Ldy|5^}yk*TQXRgC=%@*GjN6eTZKR#5MM86 zh?4GW)tj3#a2AI14epDD_M!;J106-qk~6{y#s;_OHbU*r>N5a!2DJy@fZ+=b8gV#a z)o%(vSsRc<%DN6=UC(9Uiq($@I^oYvbn8!CJ7nO>)DH7_gp;M;Tnuqk=G>l2tZ#@ zyn^~j!3BtJB{B8J8$A0JVf2O_$ek#I3z31jeGH&4$X-c(qTmD+b259%pzkm}5_|5T zj;LOdeUae3M7Nfpft0uKpfk__BBqEwF3>|5G$bk#CAp%sK5$S}n6#gJJRYfV4n~^5 zJqZu^U|X0i@jZ1+yV8WKO9{*^x%ymdR|jKGB(LlszkjsBfheyCAcQhTSRi~oy6e6S zz<7JleuVGR3*RmT@46%dF#n49mOl6kxQF183wLeC*{y`?NbH0e{0_q>vG*gG66PbM zZxein_?9795a|^)coyaZ9>fzFOR7)a2r0N9+yw?e>_isK2Re#`C0jHH(Y>aq@5C7G z%eX1Ohl;s1itud&1t~lXd4HnQOyAHx zU2p;NTTDK&og6%%L7$EOG-S$rs&rGNg&rPrbH8BbDq~i8|uM8t|MRLU$ zB%U~4m;mCjMmhhG6h=|rLb)rCD9#O&^(tYGx(EjA=;KM&kMq^UQu_rnRNR|)WsNA# zhi67rqGt8ix-+Nhw0h+;SUmi3cjhy!j&h6rw{^8foVDm-}&afb8IKAou z277!+`233YJUig{G}Zsl+x(Kl?0k^X-|AOf8kYS4fM%W_m=i1w(|-8r_sCR!VO$h? z<$gb~XVj#mUnheAMa6IOTbSV8yHvM3Uz>lyK7xmzhyBCRYx&D4D|iR zzaPF3<^^-$U+C|~*ua!2!#nz}S4`?b&JT6{k6evU!i~_cdmuya;4y7q;~csK{fy%r zdbkb`O-0Hut{Zuzqsb3|76BMJBi0BM?zGTRVxt)oRiNLpL}2W1wj;zw#(dm+$e*!!}k`th?yy5J~^7j~HU-+`dqq+i*+G&-i^N1Za+kX4kCcbv(ETnNyAkk!S`_ znN_|(x}Y&)>ahteqJu^6L(NT4CFa3Nf*5u%bx~m%5GC=&89!jucfj05P5oV|lynHAR>VW( zfk6x2!S0_YN_9FOQ?h%>k)5l23jhYLk3FMb+*K_E(UU&XJ#=#@aR=Dr!l?*?R5%Z# zIi*#A$ExSZe}9fg9^2xPO>bnT`G-VbNZp-ps%QuPgc_@)r0mQyI*`bdhBI6k3$Y=^ z7NbZ?L)lDu_7Jmqa0?F|_D#8QIK&<$NSt6}_+1C!d`!AZW?}BZM73 zUsD&f*VG1H6eN{WS0Pq8OP8@EIJ#d%Pa$$Z_x}n39b{u-(w_=M)teZQ5*8w;-qMc@ zT<($`gh?0nPuUuEDLca;4-4{=9FxFtWoZZHr2?~3$}nC*OP6n*^6sfu&?JrBLRr!CW$dV z;#o;{ZnH-#2FzHCtjXIjjEb@=h!A+KDEQL$OlWb*U=3&Rx|0qUNs3fjn5ls3WH$!a z_kqfOLQ%Ccw`jqLOljbTmpH@dOSr60tKk|ZMult3X7&}uxcq2M?T*fC0UAU716E3e z%7&(J4gzk4iL1U4j}oR{a!c0?Z4hshe}9t2-;!`6$(-#hScDNF7oH(l^V*mS=H4X%JzccJ28% zr%%1@5dSh_;iOOP-MI?A?=~@A=KRPA(ZPQ$(C4sTV>|o2_c1=qicb|G2>HpWn_)d z%a>Z$AOz>HS=&Vl_*7q7D@{S3(ibpOF6Xvbh4d#6V0z75(fKd@v=yY8vzN2aQEAOj z!r8Z97OuKq5ErFje`x>Zu-i)YEh5qqA+&vX6y-47!6Y!_yix%WJ=P{Y}%98@lyZYvDx!c=hNkvbevf0wEVL5+z zFD$*S7wLEMqZcyrw$H=AbQ3w;Y!32&ozH0#{i%6S9u^#?5Pgqv#$A73M6r4J3yucN zQiakeDwh_b~Cq4Ja=(eYf1{(UkdHq%1=(02z)3sV`tDPR& z!y)ME$CE1=jE22+E;BL5=v-0iEMpBKryW&4i#d?A6*PjWoxAwFwPr^Jifq5*rG@4s z+ZS?-L{*~}3i>`qpbf^?@)z|Hx3*i|A$LBHhK_STka7540+oUX;z~M$ulP2HwKh&| z+@yE&V6UCri9*2tJX~Db&TgW`*%Q&PsRM!gD4BO^$l~gnvbA>W&To=>Yj_bp1%+(5;eRTtI+{UPwFe46&I{k4;S$#*;^TN$~fGZiAQ=Tl@2fxAc#C3(ctMW zO7*W@M;@~yfd>IHVzL0Y+E$jHhIuTz+Oh+CV&r=((5Lb2 z_;B?OJ7#&LC!}5ZN{7pI_rwz2=g7HY%hy~h2r^rfWl7!Xo~%HA0%g}MH;gaayU38I zFtR`69aT0aYp_JI!59=gj7pdyf2t@ym0d!-Fjq=WbhV`nLY+}|dYnp^M=ZeI>mW1@ zZT*z>&Jo#5*Rb1>mwv;0bxD9F636Iud6CH;#!ur~uSb-7=)-qza zmhqPB&EdfX?1?q_i@Ie5Ppb&F9=SS$Z+jZK zqBksTk!%)Dowg4gg=ciHSF3*q-| zs@IRLBHae1;YL)McvbWi5`x9FP1J*LeRQ%WMUv-2Qy4iUnSL~%;0tILI62UiZ15Zf zOMj%C=)g+FQOoDu;bO4qM7IQ_`9XNGRKQNOa951$F*eNm!2P-ZIvWx5dF!bc+Qz?)cK*Iq_pHt{kXT>kS z)iS(`w<$zsQd;=Pe5YGH9Bs%z^x`1cFRXrfA1Y~UchN||6d-FtD_Y#=6IjL9(wnW< z7=JeAAm&y9f-A1DD#rm~4sKo5eeOyd_l}aS-YzJ{i(Ms&-h8)uRh>CpwW-L=>@RC! zwh&=Y7x9u;r#&#yQ^$vL=h*)bCr^w#a6X!6@n2-_nd@b?!Rw>YKaoH;umUa}a~p&# zmegdBlIKb^F(&zvm+2MFge5Oh&m)iIC8xT3v;ryT_f%NAFb(1 z?|`oA@=s2!1REu=3eD}FGofhtU%W1Ic~_85GHJ*i7ULcZ(Uw|W0uxqfn=P>>z8=M4 z<&tg@sRABaCGpNCQ*l#qm&B}iYiiNN#LfK=t-s=_c|EqB-GeuxfleZwYa+hnbp8`T z5_(Dc>EI_dUR&FN=CKz(uZC+ZSgVtUt1MYdiCcyJb%f8XSdgmkx-LdedWEnEOK z*Vj*W)V_LKl{l0V2>$c|%voE%5&HoOXWK%wa$J7e#J@WYeX--NwlyVs{Qf?NM+;jM z26F9&ddfY3p+G|X`ob74s=N8kac&H^_|K%Ezj^mX+&#d_wa7R6d}wf$Z_>6pLU~mb zmSyCkgC(V;S51i-S&)xA9E(3=tBW_cH;%xR`P`=8x;N5TOCeZ~<=2@{+OJoFp`zP& z$ZS=tCDH|d30bn_(uJpnh6Wv~z@kVHtG|e{SOsgmHWQkgrfVZgCzIZO8%#g{m z%&In&3OP*nCi&rg!xGZ9Wmg%{eDGI_P4*7025Gor)28LEb(F z+=swKU1b4bnn{HK&}#P+v0>sl0u-VBcLf@qlV4pPO+2^PJ#pNw288I7`zN%zncj1e zVl0_#Y6~DMe9qvHDCndglsI$PGcNOc7V{@l1h@H_5x`EL#E#M4Dq%da6pkc5*K8-) zbrHP5s)Qw^8~;kjG`ga(87r*A$dic}hcmpoM(_kEtxB=9j30STi#iX-tgcVuE;Gpa z=E%oX6Q_b+Dl2jpg|i$NtXzAR)Z+*#zKB6m5-lReR*oe6(VdPZA%>Xw!CN|Bmw#j+ zO$VdRO5=$Ug)6az1kGg1+;RHcmCqZC;yzrtTRxd<+1%neRb z9KH?eF%6)BO_5tgaw-l_U&*L2-c`?u=&F~72hW$(2?9SV5!r!uyPA2Vu2X0@$38Lb zAf4B>AySS_6Iw;Dp}o3nB1?=W2SxQFW{H{LV@$FI?+M!X!)ad7U<9Kwa8FwCuV>Ev zKda9Qzkor*^jYMO)qC!^-?W8bZ@l*~AN+4qCykSpOryMSxcA&6L%)XwM=cqc{;n=U z8xC~(28YJX(nEyX+gSv+U?QMJ{c0-g+X*T8)zpYD5I;zW$=s-dXSv9WMNqZ6@9f63 zlxxqFpd$IP5$D4|N3_R}co=@yU_b6XY7Jxf9Jg|lL@u0OLgU3~GfvJ8Xy= zWb_Cgs6kryMoNCAqO-EJ^$bo`K5Pr&;HdneiDe0QDEGoMbQHG*NHpEWXBs*jo2Ggg zBY6F@0_gbDtq6BpE|h}6K@hsJf8)#n`gZ-zs|M(4v}du`Ky#zb`_5OjC;;a(@wl00 zVz%M7;5OFgoku!gkKViHQ`dEtVVDgm?e5$AZNr^k$q*d7AV5=i5gdp1^EtSVYDXFj z8)!BR!v{}12DCen6+H=wH}=4|k2tx+9IPq%5Oj|pSfjEweP{r~)2^ZnxZtGgMq`$u zMPcxvEGNOK>rPS4%1#)(pbSQm1Lkyy1Oul89ssu%m|HGfIK?LCTV!0s!WPiiIP?KR zZDhak7;OexV^S?@g5}i@yu&KU4@?$gaG@1{s-w@mseC6$j)QAqDi7w*dX!ZBp9qNy z{T(boFdB2pN(3A4G`*!=IBqRyPY0V-@eYP?9bJ6jaQ(eM^smmYQ*>GXub#eAFZM#s zC#&m4-Ze$8%kWXp!BPTuM82JVh10#<%OhUn3`us+HGWL&F$cecossps{M(|&$Q_Q8 z-NHX&5!ZC%R~}gB2J15l&sAb=v;03p^Tk~SI^g)O)jJZXH6pVvC)~Z+Tq!NQ+VnE@ zy1R)H80|w%`DkbqV4uUedhCd-2(!AD!((?>FBmdqEflq7B|4i86(Vd68CG}tXO8#W zb;iUw$|gk`+KE}|kC-{i9HfpOfqY$~C1cJ4AoK$J1_p;gwhhm+$@jN(;YvPd@9skR&1Mfvh^t;s8FvZ;%&J=##k3uk@x{JBhE2*SLvcQ_tYgkt6KwlfIOr zJ`XP%1i{4CVvWLC2G?idtjqBc(k^%zvdHZ@nH$29b>#Zm6dsPqvS|gXEN<;QdLkYc zZ1c+3Zr?6Gr8~(#LcacKE7SzwiaDm@FS`i8M+cX;DBZ|;`xQ={YP~jcHf>e#t(CE( zwr{2I45a4>j%T69p1k%lOy?V)D4*!&n)8k^$N4V;#fDL>%WS~L zI*(&Rq|zYInl~uA#jEhxBOtN0+AX|zhA^bFew|)gGQ}_S&`!lfdRzSO)G59T^a$lJ zZVr;8mPk)=fic+*5_O`b6Vy91&E(wmJ+e&{^=^}>Q`^;F=R_Iw5ia;VlNwH2OuXg^ zl1b8*V2*6XCo}i`rEoEqi?Ba_W>;B^^QNc*bm@;~zAe01kO^m&JX&C*YZO}}3HM#_ zo6e)y8=xrW-mcaUe4*Qt(%Wm0to)&TKX;0OqHcV$@v1svi3kzG?zp;kZKg{GCT+RZ zzv|F`rseAC4TVP_dSI57U=Z7sL{Cttgj!2AVPaZH2``7iCib;?|KGl)A=Xt}ruuzw=&6+X})i-zf z3dC<;ucZp*nGs#HAs@!L_BeYL-26Zfl@_-P)x#|e!NbSy?1BTpd}O@4jJ=QD6Qf=Z z)X5#lUcI7II{<;*xCA>F2BvbV=JKh;mHm^BjgrG-jPQnqtKLWwdua-VoDWDf&U`_URNVxfoJ7^8TZR}@zhqvC#LW!o#}H-_03$O4Uji1Y>9#57IBRW39#tczIo3_>t8F&t zbJFm6!MUS-^V7*3f!X?qfi5Iw*sJxl$iOA?ZUkv_cgiFR>kRQOGIpLbf>$%$v1Sgl zgmAv$2s|rd#dhLt3K!1Hkv=H>v_;k*_$xxQM1YWkMD5m1l$@*Sx!JkPkY8JH_%O5=QLMu$^t-)Rq6h;@vq;#$ zJ(hVGoFVr5Yrn%heF(RP2c2#8k#}XUF8tTkm2J-9eZ%@|kx)6#B~z%QE=4w`Z_s-O zY6PidZG5*P@ldpVUDxuj0iutAg*ru*!@`D(lAri!ZQP5HqE;p_u$nAm?SvYN!VST7 z3NX@$!DS;>0=&pM1?CYIr78Q{KDtaX`}(z8fx1dZ{YDnKi36t`91JsfFu3lEIrzNO z+YL7CDBC1IQU4;i-mrM??P#moGqKXRQWUP@&*(8?6 zFaPbyf*#f0rcoRTaJqF2lQkRAmg#WGb|GiunZPUL7Km(R^0XVb8RBzM)zA_!7l_6_ z^b1R6o-TkDU1!6Z>|qL)k96SxQ!ry?uFQk@gG6XDDhfv%DQ}06H+j+$Z%43`gl>Y8 z%XrcZVS}t&HDi`;gUYMSpQv8Fsoma{O0Pm_6uV5hrt&1NQ5rDuo$_Q#yX3egBj2Q+ zQ&_*k!BY54rCZ8Bqu`=oPRTpDPCaAeQR9mLz$;p};tg5rS}=Z*f7%-Kpwlt)Mx=YQ zHOj%YYa)DVX;=C%d+BznNq0111m9qXF~iq~!&+!8pUv3H%Cc*bJoP;wM%b_O$kHKQ zILcueZ?4-?(Zt$n(cIdi(cIcPa9L6(@qAcE;i0f%;o-27xc;WiyS7rNe<`x}_?wn> z5`(-bIKlT-yg1ITW}NdcMseT+jriF?)rZo-wAk@bx6}hrN}Ux(JMNeG64k{Ne&i1Ta46c@-iC?>-dYa=#FryRYnk*pQp;J0|o>{{NFbG^X(rE~G{NR=X_AJz&$Yl}O3q5FS&05Gh^ojK^n zx8Pw}3?Xm~;YERnm_8^hJ<*<7080{HlQU#i$ps!rX$;ayCg|-6<(b*;mKt#lcPdl* zYuz8(`qR4dU_S{xhr?%8x@2Kh@jgfveQAed<7Uuy8_4{+57f;9lcyssu}Bq~6O|AL z)Dme#`5P%HcwLpB%!!psH5yt0o@Exbsj$KMPLZ>v?akN*dB!=EheT+T-3+Cn_QV

0ht~`;PssANp-0FMH^p0)A71L5^>cfLBR7tgkHZSIWAuF9bvf(!e1;AQKG{ zy4kV9q+3CWis?M7N@}`@Zv#r?tp9Q(QEUU{%bl9&pRn%g03Xogj%Mv%ad9v9tMHiv z1)z+F+W>>1GOq5nL_~8<6nbZC`&l4Ox!^601wV2kOXhRTM3}kRr5ff3*uu0Rv}xV+ z(SX8ff+YunV3>7n4ew*U%}EL{b>Eg zdR0woR(q%lRs*;l;((wBF%CTjmpux<9g3}6R%Ynu=x;n%!~mQL)`tQhl5t4tRVOaw zD~dE*6ffNOrsKJwFr*XquljRYb9cvUZB9wpyHu9!864QBGE86{-67b{bRR?0XIyjf zk=Eiw#paY0H5-@(X%K-{7LFh<>~`u=H)rUFoig+Kkxk2*$7^o?)fyS+-!r2l-553z zzP31y*M(XcT_reImN=dRlp4~Assyh5izs(BNxqDWs8pK`ZU10xhQoRX$>g4=9t-^) zho!XPOXj#7d{rYg<$Uw7vJ$@a*G;n`-mC*WrmOwM04_6VS?m3#UeCWyvw`Yy8gk_q?p4$Pl`bT22?N~Z2qt@*z zxClh+TsqXI%qPVlbjeRzi?sR%Oo%thXx85jz&+pv98uKma}_^5*_vtPUIx^2A$4}VEf(^#!9D>HSm z+ji((nq;L}S8W2`c5A{|BG&=M+>i$rgF}RgGu#wLr${>xekBHfj>w~=VJpGM1+$0z zI#_YY|F#@R*o%(HBF-1rbq~!e($*5aRzULHbSR!zk6~|DZ)K@FWa@R@sB_9N)2Ev) z(wWe9s&8Q=S!p8KC5bt)JxN>}HTe6mE=Yw}67g@6?L2PGzQ>~s^Rwv;kT^dCNv8ED zQuU8TDUgsOkur>;*7LB*>Z+CbPy+8Ij@MBGtKDHxmcxlP7^~?M{+$s8HcAv8lyq+5UDj(|~>?`crzPC9RZoTCrfEAhvfvtMOzhj5GpMF{Pt~h6F;DnuRL~hcq=nmMtBfmXDj0QAt6v(I z=Ui!EG=oi;JlC{@{gcq1_^hEjELw6(Y{}E4rhWTH|;__-rJ@vzD~=~ z|JJQO?7mxXr~^~@a&%)I*onAf-`)wJDKkD;q#~{U$327J6sDv&u)N81DZ$RbCS$KT zy8q3gIsLm-@aOQhe%-ZR*}Tc8A%2i^+#Ux`VXJ}&m=@0@72Qu&jvbSZkyTCiCtkpK(z zf+?7iWD0#OJ;9w+od+jLeLBcI0pZl+LJUC5_DkfxA#*j*vHU2Ct2hiNl^ovJtWAN# z5#o+-G^lg47Zs+W08E`C|Nu4c+|ku$Np+giL7fH9^aHI^yysx3E&rYmnHu zHzw1{3m`X5Sn@2R$=U!0G%31%m#BXcW@zEl1^?MtvYCX8rG+d($7g9A^&SL%TAAi!W=E}AJ(-ZVg zg)LwNBh%h`Kk*ARu7xz+C}(`3{ED{V3yp|A(_Ij|%6+vnTnKS1wq_FsPPvYWHOly= zD5w*QVr|T0)3-0l%n@tVpIfj_s4PwXdSx%nG-&XwlTBV&W+P2jGJray<*B@H1;sN(ut+^Q}Iz`7>h^{yJ`MX;`Sv^lIK0jYpcR{w|T~P`$ zZ$rW`$uBc+Tf#KSdg%B0VQyq!^#(9eW@5Y~hmesa#qm(@8lnzK6r#AO57MF##r&ua zNRhvTki>@0U8aV8C=kbiF!d}n^Fw9X(r~GxW@^UtsUFzMZOqkasvRLU(xeIACf)!?IruC_^(D72aj`1kt?zlIrnm+oqF*DRuNi?rBQE9ulT1P)TsJNo?VGTqZLqylg_uL<;xZ>JEIB$U=oZinEau9_q z2eOwMx9D^bhKJG+yF`_H07x{y*t(D#uV?-OrCTsO2tnp&OEd5-mCic#i}(KYYcwk3{f)~KIZSskoJHr(=rY-+AvjWwGI#T1b*9UI!SSa(uX0^xH> z`Qy3~Im~+7%&rAjRk{8On_QuNwhagztl~a80|`zV`Ayx+ZDprwtFf~(%3vSb8ZJ8vd2qy4zI8HQu)XP44uWSt79 z-P7-sh?Xw%O>n##I8fsa8779G-QAn_JC?fx_}?*JCMtL5hC$q^XVM17 z0jou(H}ikT&?hO5VXJ6jEa4TeuI*m<=X5l4@>!dkIvseTnNsl~E4GTEa}Mqe!O*4R zPuYCMx>pp+v#|3{9;~mOy|*e-$@n>yM3<@S^Xw>}x7m>Szatp<0ckmIlT$f6JRr#x z2V_e&mH-gA4HR8eeEaBx6*osq6H`xLS4&4%O-;*|@^0E>5~~){<>jsU93&^#?re6R zdr|+0#bHl_z75>EJeLX1blSL}a8;RxgIDOP9p-g7ta-QXF>k){<2XA0^kf0a$%{}> znoyWu-VLW#78zk={z-FTXh%HfX3Gp*s3D&6py0%GgbC_8AW0t;%l!6*epfagj9ArGZ{h|5O&mFhv^sX ziiT!wJ%Nzzk+9I^HFgZ88Cu1Md%jbqP8)B5V!7gKco7lE1VMlG2BDTy>r<-=f2~;-DF+IJyRyVB|1|V#(&Y}o9U>O zn1z5lhhC1qXXDh3EDryo)8CRkXGuKi$JYJGX*QD=m9Vw)+Uxz%Cf_qRV7VNjMQ~-a`rH|^{|A+9)wT01jk*ju%993hAd%-f9oFInVA54 zxW|nS1$EZcorj>8T}?=ouBRpF-DLkS=yiNJ958Y`M^<7f30o+ktFf|5+r^1jkB&Sv zBlVMpPj<_w5(j5JLc91z2Ei%pC?L!r?H$T^x5QM04-f8N^aW}bKHR_H3()JBNX?~2 zyy3h@Z@TWFa)>SMz6q@}Or>;2BlKB6LAol&7Kd?Yr{Ltm3g22buTY8 z^>byll$y=bM|b4>#*Uz2^HXO@9s=XOlu?u9Sm$IW5>u0d;Z`cr$*!?u_D`=q&oALX zfC5^j13H(23#C$U>QZGviNm943V-O;Qs32p^`lB;XCh9|zolJ#qEF)k!g7a(nB1Qw zm?xNnS{UfE2lx(TFTle)4yvFJS&fO``zLPNZG7nn;^k^Qc<3tUQf?YF^b*HvIx!k$ z3+c&LVO3b_Q8M!^CCBqh7kCc1!+Q5a+ow%Mk27wj_&!k&ZGrMH|Ne*D&@{SN(&O7C z^@#Z2iJbEPFQe4|hoR!Xyi+b@;$Cj%4rbnGS2P3#S{#Lg{0G+%!|p#yX`* zb3{|kRG?9CizwpgIN{vL5Tbg*h&*BEDp5VE{X}XIqFkj$SUDlc$@yZ1A+=VTOyLd? zi)ZO`d1v|hE1K@pP3@Zr#y?;12ams#p|+;8y3&`gfoW4JK$>}8!?swIgJ_p}LkgGp z=umf(oiweFT<3`n`37CFaVp`rxiDkwYqDR8o7N1(9JX2?8J#IdGy442DrI*`%DEGF zD%~z52F-^4QYm75AWzrmlD&bs8c%IkkEk zq7ng7BV$~4h_SXbj3xQ8O(obWjd$qRtO+u*F%M!u(~Hq1I>{xw%^L2|x^7#u=b0rT z(~v+3#_9YyCq6M_vLn&R8Jx|}=&>#>$6}9PQ#ztAxR~eb&5^VhB+& zb44Xdup4;N03;(rWQ*OZBNky&?Wk1fqB0S68ctyZX;0VVpXjQUGE! z(Yy4Va+_vQ#EWQPvBHOkpH(4ZJj@C`9IKkD@)29-_XhEnVAn{DG7-hK7>&Av@CDFH*CV~xh>S{uctCQ&m*Cbe2jQyn zAp{Ky!XegZij?~}K;&sm))H;wX+aXX2ePhg?+$Xb4qj zk|M*KCeH}ZHC&M%=%2X%)j%pLLX*OnT#`}x$EsI3Iw+&6S3G=_GL*mRGGj0e zb{R~Zz?S2vmD+H?bX@p`o-U_puF6aT8bOK@p54p@HG+lKiyPs-v1jUDBecNy44`Sb z1<;UeYNdAfr!nGpWhCoZYW?4*H~8(nBh-^Jg?`F86VHf#%iK;I_g5EzYX1fmLQA$*OF&2&?YqOXzFB^ z?6hbr?XIt-4jjUZ63v*X;@A{=!|= zvopz()&Hdihkdba3uOnBBL&%@Bt49twPrM>>Daav}>xK|%@nBna--*!k^we#%ZYH`5$n#(*r zJsl-x1VcSN-TY4}yKQaFU@}{IR}0Uts&TzK^xRR*c9Y~va+JrqCE0lg7>LxaYb6@v zGh&SMsf*)|`q&IwGk11x1a+!62o~sUhZl|u~&oQpddlmF}Rhs;VAvnfjHhWtR z0KsA$ByR>rO^1>DEok@w=}{b=X%aRQDe zl#8U&_!-ALEw%xkSW-kdm{Rz6uY4ru(ZXG2W*Sjx34QG06tG{;!%c)}^7bRdL{GUg z4oXv{A1dkVN4S1~H}v=4jQ>aLtw3L119944SE%N=8hwr^7MYAz1%?c;0_3EW-hl?W z60x$NN-Y9mcChJx-r%UiFuw9NT>B=dAdF)R+oQWtpB8ut@#1b={I@Kr2xB7Dybz>o!Cbu20UofL>(*Ol|A2!td$t}Jq?dREEf25B`hAL*Bmu^mx47ML zf@|m>;|($a9`ZU94YI~K?jE8n>+~pn*gYH-rAX&vqNe-6A8Bb}6*Z?E=w6YqA1QSf zGG!~f|B;1NQp;Xmp< z!o#@gJ<`Lu>OJDay6Qdh!<*_`4F?G5H%fO3=r<~N4(K<^cLwM;s&}h20d*wKuZ-w| z%6GrjKk8!&YhDFu0xAw})ILIE0@dzZ(SIx5Dbjpu5C5)u)kOcQh$*hYs)$jp!77bu zuEDB|fvKU=7#35PQyHdHpVAoCQ?FAIFQr+j9cYZfPzP5-RuiwK;iw`hi)o@EsR1gW zuht$ws&7@rT-A8#q2vF@_PJQ&C4oL$bHJ_MQx@Z?b|)*|Mr~3`!a;4)O431ffG3_; z8%0QUz$c#97zKysTuj2CGBg~sRug3(0~ABfq&Z*_*K3HPMRP7EX;2wL5Z@|};zf7P zB)QfY;*SApB7aAk2Z-|(MM0t8C6N$lh;@?$R7DX|A0UhK)kKY=-Sv~oLwO)AS-5h8Glf}0 z*#RpoR>pG^L{mfqMe+R-kf$tb7eVA+*`)n`N;Dg*fhz`@G94uY?nhs^6Y zVh%+IaIkVkf>;v_%mXNxvigm{LXkoru=V=(L?AJlzgsHgfChIVELS|Fep|e6*elj6 zUK5Nvl$~*RUMMB+o?g!Yh+toj8W-wlM@FXW2xL&QX%vPX}oZsKH*cdL_TMXet!u3_ZnqWb#rR z41_M;7ZlHkFofxj_!=P1nX95)2&;?ke{G}417`NC0t(D;Y2^X^iEPYzRXJF}gf=E* zpIWFt+mf-SfqWklsBcr|e5_8)RD01PAURYA{1vQs06#TU%7;riKMB-IrUo>_#UZ}r z>S&frBPSu$h}I^=0#(s6J%*^2_d_IpdZ_AKwIlgCRr@<8AH@_I zMwJu_gQINCXoT}(-=9c-P$% zs7J-$B0Y znB0B`cx3f=Prd2;iJ}TxO79Lvvfl}G*dBm=V27@<-@z5UqJOHO61F7$9yvw&0pf~$ zr1uw*nB3ES4An}H*B4EsW0?^Sav(Wr6f=rRW85A9ne&r#?yTKQN7p}?y+3|OX>9fA zI67~O!m+h-nlL)o(oe=gwyAphhJ78Jk6uEN)n{Krv6Q*(6jBt@9nzBz#BdGu4gxGr zde+`v_v(dx1nGGz$&=KRLnfa|*H~?ACO5b=hW@ZL7<+?b>DAwr!)!w$)X;Y+Ji*TRnYF zoSD1%FJ>+>Bjbz6i;T#Om6_{V&&x#gYyTO18J$tTXLkI2h8t0~ou+S1=G*1cNSG=Nq z@!HTiAw{>MGjTPt-V2ZbQBXYrb09dj;FM z+)Rep=LqPzUPCY!6Wv3zg9|Z58Y3kPHqKx|dkXi+#YMCwM$bT*!^8}nyQk11_Y-e~ zQ{AmcX2Bh@O`<8u2AL0<@4LpwG-!El*y#pq{%XqN!T0<17}<+-v#1bC#h=Ls7az^k0juP+&zrQXL9yXrg*sjo zB`9RrJ2Ug|B`s$n?wx`>nXAK&k`T-Eu0_LVq(uTaNHTM4*|`|i7B*bwtD)slHbwun zS8wj@Xc_Vk&m-BcFEkia^2AM2e;pXJ1BX#bu>rN&sOINmcO~Q_GB#aYSj@)U52BA%sRxOhI`TQqCSwu4$!ofArQVDQ_gI;Fo3l zkIedmtynzXj-eh$?DRXIzuDehUe7uX_qtM!KT&;Ul1)DNZu<^W+Q`Wcd zztg2G*naN4eIQ_@-oU+wz~-459@Vpb#D>Zyd}TAJm-{3I-b+i#pK-m-J*O$ zubpR?jL3+Dtf7|PH1#`)yZ+Y)UeB0j&MQe75#kxlb6u!m&_s}*0KULG&XV8fMEB&E zUE*9{cQoS_N#5{d4oki!1P4fT|D+gA-dOVPZ0K;Jl15!kRZUf6adGZ@Rg3GP&z{e6 zOMk0kQ97$-NTNg^2u)MNNt{NiG5P2UQNg%~^XGInmm^7{X0Q|*<&m6y9*gVz!EmX5 z(ij@$Gh*o=Qt92^i6Jf$nF3l(v$ysuq$B0zu@L^fF%-VqP94EN%nGkp1Z#AzeHrbJ zO*}>jcdgpd_tdJgmKuKU(fR=jxJVP*pN7fEnDQ!248Kc(S$8CsY0BEU{=wh4`sOEm zmFWNzNf(OZ5=lBo?#thCGaOQ)mAVccImt}2u6M9yB#CbLY?iqpOq^7c1cZupJy(&_ zs-tS=x_Xyml-gW|*3M5@G|6R*ly#bw#Sx28W}$VRI3JG;;^`meSYSfLReR|7_08OucOpGK0W@JI?O7d*|(ZN&zKu zaU2PtzbI9#9yBT~T0^^ip_Q$5Y67*kn)&Nw==N{h41Re>xe(`UOm4+%U@4{Y-(0r0 zr%329dzf64uQgadav$2Gb>uQjz!@CfgIAVE#U32?3=YniIhS&(dr`U`<apUtkOE=?{5t1&i83bQ^SQJ5Ng z?-XOj9QsD91kn{b4*Sh=uAmoh^`C#QuzqHjuZn!_$bw)2doYSR$}v1~{_*l!nHc zWVs5Atx4OT>;(Rn7O^O6dl4+}y_PQcNylVL6*YAuks_U>B5`?nqU5r&^1rj6jb3Re zC)BB$$wkmQw*OYK;hXzlqG?{+YM;Q=m6!)_3NZH-;^ic4k&R02cf%rH!l;Vp%$y{D zC+6B`wqYeDdzl6_Rh+L6Bn}a>! z42ze45p*2N{l@J=61eXbZ%PD>nQ5qM(6jZjDEr9h=c}+Um$AjeC6n%zaSf_@a;g+O zkEs51y|~UqXAXv0i$}L9(y?Z(7EEtnIxmqVX@S(ivE&DF*AxC*a^QvkW zX{lIkW{dEU@f$c*B@#>5bSsIEb}q{BCbpQOj9-L>cGIWir;~SI-HY^NvU0`otT(3y z9m`o|x!n~nsn!;Uza8Kp3WEdre~CpgWfuD3=%|rp=!CG=?LidoQtZFpy~gIqz~c317q~L1=cS-tO<8*` z3m(g5u^4N!hHA6_goDYPy9&{Kq6nuRUGNOm`Q)#~rXmX`QCBO({ZUjq%F+e16k)b& z^eATcQO}ZK;mUf3Bu_>iKhdN-MP0?;peb>W=lo0#Qwu=SS*s)6Z2cuW5woWl3tnUy zK#$6n=UG(p%twxjP+H7Br%2io`f(7A$IZ&oxG0y0%tOgZ5igfBkpSWd*N}+PfC@&T zH5(9x?#C2dg%<%p|BZzH){<4PQgxq?j(sWrv2K9`Gh~6JeaYi7Xp!Hv?mJ-C+!l#m z_VP0yBX?Qg{PezirIYqvj$8vo853~uAub+g+! z3)pD?^n}{%zvRKM&)wkU*)_A5nJA!>L^o{_PI< zNn2D>xR`LYz4sA z{T<+eHXYh#NZXB|(a+<6)(z+WX9o1$1BoE?8Gz;U?HxoqiYfri&%l(`fSgbsM{K8oyeEY%?aja40>y?_?Gc z#QNy3;l`JT;}(c2{t>xXX%u3qr0NHsN+7m0sEDJa>It9f`EACu zX}p9ed3*4;S!=_yvF$A!ew`d$gd5x}twQPK;_gt0G!bwOS#&XnM&X}rp{jXQ7CLbL z0vA<^_EySuSj}BVyutJmA zWSzKMK5sm+J}UGGF6E=w4OwXA;@tZxY_WVCzj3g0Ej$79kxrX(I7BpJA2yKstkd}y z%*hJ)t<$%cC^3fs{I==(vt>G>mbE0qp9)>gI{DVSZhwe;OP6F8wlKF8M$>fGJQx<- z#J{aWZTtddni~c$!Vq?>(lqL0!oj%>=$N63oJ|JXqeL9XzB9$uEZ-p#xG9T^?fj#%IC%`!396TW8G@u8m$} zB~I&7Qt5>U0BqTVmB$2>iqnwrD@ZXrsLYx~DjyA5{kF=koJxRlyeYHBqLEDpZ@>7d#z1gaKE11STM+|1@(XiY&T44ZNx*is>iQlQ{uP$Jj2VM9dO zKY{aPKgrbg6+PJylcOElb%RI`R2+(RBRlStd8PCti%b~f(tZh#8sWRHJt{wibWL_5)t<2G z*T23muL=N%p8EDs-%)tw#{QIy6IH8;8|5m;PNXa}=yRiKP;E4bcRf4H^t?LEb%3pchAJ>R-d+9O7Oz#nz9@!`^;@eW`* z8++#i(@$^Oo+v8r1C-r1!Q6QBr(eOaqp{|34B(p_XeUQ(6(iGy>GcuK_iAP%+JxY? z5doXuH*VxTQcm3Vw@i{(ZcQgB;QpAc9^462n&)83VMt@)VCd57iEquOF>;*I75e8ZLV5b@JR1Xji;C`Th z-=MsF=Z8)I$Ag2p1AE$`-1hWeQq~3b{8-McnI$$v#Y@H!Bsw1w>E12?O3xith8e_G zvHCXrbLL>!4}2j%4we)G;}n3%yprLL z(53rLOj!BBk1xi*0sG@PcNkpN=b8!JEqiCn9z(uyECsZBUS+ zUAeRJ($^=xeR^JYbVllfwVqP|2jnYp5}Hb8O78`sHPHvmbPhD;UQX%N4x*-(;3-S@ zjoemXeN@r{gU476^64XA!Xeu))e6`C9r%OGEh@2Sn57EsAyNM47$c+HMEZ|KDgI|P zSh^ha?u%0Vi%W@ZmNo%)ig+Uu%pv5;z||jEigPFO#7a3FWqY!qx5Op3aR~_?6D<~Q ze6z<4>HUrNH;f=M&8y|&q~;v9iM7EY2yV-x;KbVeGg;1$OR?o*cd=leA}G6v4w*2e zW%9QWK&$HaJ$=~1CxX#frHv$B+`+H$%lh%zDP7g`Ne5-ue>pIkQ}Ogi+EW9RT#Qd- z(#bIg-+dEHrcXrutPIBKy+XyfIm9cpYz!`CbW747N!>ID>mJAnDvosq5#(zq<&ldP zm*4cq7=Keje1!PsNXmT}Ty^ax@b~7&jlpjtwkZ-=D=%l;9iF+~!;=O~Iyv%5_VX=Z zV-8`jC#z2RHm;mTt_gbyV>^~oGETRZ0Nz)0V(=_}1*Y4w1y0S^7J*<|cC6~R2EW4O zUkrKBv6oNHcfTm2#po9cb9T)OK-WZ2{z#-ZNQzcmwT29#J8f5*iX!zi(K$>rrSF>0sC~dt;Lr zSVICggY`?Fq48Fn`P(%b_e(u?#7wl%OPa%8x>5NkDsV_Ok@jqm3WQQLBu1j3OFkTt zYgB!Lx<3d-V&FqE96b{v;}jumSx=Q@7DcTbnnB|^#70deT?{wqa>~Li(y5+9w?(r# zug|RFB|e#$hvi(-7fp9+t;A3k@#@s6Nri-wdN*d2*e$6#aXy3^HQiS_GU<~iJqGn^ zeU7am(&D%Z)fQajK)AVxm>KF=_Lo*V@aohT{%12h8P9u=JA5n@YB9`9mq`)Jc_&Ecl5& z6xB!OY8gUg`&BSty`9eYvQs5*(`*WAwkY)ZbT%(mr6QOFY24H#@q5W4mih7=w#tBp zd7lg#`fqK`$kHX!ROy)gU=_HHgPAE^8A*qUCKv6MRfelK@-K1*8UhIDmee)aC_1+! zbIxOpYag>;v$$iQxzW+2fGhBW?n!N{{RO*s{AW-wQpRa`cpR&2rn+)oVc_BB&MaZweHO->dndGLp`+T&Ek*X~LGht}{`O9^C;M<0N_W@!f}-sc<+@~648aRCWX*R~XaO}FV_eg`(;FMZmASAffr-s4B~AbZc-nUAtE?I~XA z6p2G{bE3Q%E+H@b5Oh`FA4v;BFnT zjV4I&iFk#gc3~<9?DdlRaI_|bIrMeGEq7k4L~R2dcaZI(i2aXF*mmihK)pN9n*<;J zH6yDFUi0MVyE;et^{Y}o2VybDE7yT7GiMFLrg5x5k=~5f9oXPQ+d(VFC(Y`!>N1j< zl6XTsIP8-kK;azzM8cC~d4>25c_Jfe2}8K*E9{;RIebc85a4c!NxL`9?Ij`##yOlg zKztSxBHZZwlCIZ>Z>}Rg_{!{ew)0PyHngpSjKmz?iN9{o#9hgwr9ZPT2uQpSS52PL z)}$I<9C0$U{WbJv1^hIprQ9c>#p74H9ss-}ekLB}{If2tRVNKs=8mNB<48(N4@|{M zhs6`|3AB9D%YN{~6Y)S3Ps+&z_kLYZ&dK!WBt`NDmjF|&F_UYuM$UM-WjT|&VFr@q zv9ZxSuvDkwHu^@;$KQMvYgMx@R4EVv$y6NQIcc@|))h^GC@aq8H=mW?_)^>B%~4gm z4~Xp~IF5)I=VK9!QsTVGJJKXKC1a9MpqY2L90y5e?GN8cM>m{duy$UR)AwK_{=Ws9 zLjqLGH%L)%R(LFz!!uC|SE12r$(eWx*GDA{HPg6^4JkrqO71Mn31G#7&M6u7`8A;Vn~Rt zB&KT4MV<{L1<(1U4Z}7D@?oh{a>O;Sw&4?fELU#Sx&lx(0~|>0oHFYdKTDOIuYS)WPIVDDIkgD z7-5&OggKfbNoa^J6LjM?{wtJYjrlO?dntuVVVDO2G(<2*=Cle}x= zJtZie30<|_WEar9Oj4|eUsBSTl9@@bEI1T2<3QR^Da#Z1w3ZSc3ws_jjj|`LKwDd< z$wf$1rLDs)n3ejyIQQtzP#qV_t!iuDRMbHI-T;DTg!GL(H)A^()|eW$UyKh467Em_A9V*q*1W8aCnw2Grjra)KBG3P zRGes=Zq2?vq6%48etZP%$97ENpyb-+G0NBoyXjo-OuKE7h+VGC1#&kvP7a$2#5<3{ zC}lf0HKd0&`)#EjbI#yCZX;YrV989-(OUe6I_)?PSS~))M%-h|WAj7ngZnc}iVLoz z9}_pn0FYw=Aua! zwtoaKcAO9v7DKZBjp4|PszbOt(@ML+N)$IXzh@aN)3y9X@`V{0;jDJ!>IGl9Kr%O0sXo#pu+s#Y5e zGP!3k5T*uAp*+Hv@^{!J6C2}}?+gVlFz}dpXO`m>RmjtY0cGac)-Cr^JFG^g?>MNO zpyG(YEo``?CaIPBtlqHc3?{qSIzt=F{U@Jz8RO9}{Ij#o>mG4<+WgxjlCI5*ZaY-pK;15KE;iEUWaVEQ-sJS!K^po=L?#I~31 z7VxjC1~plI+^XnFBB8{8;iG`8iUXCpdFjIwmWv@G(%dqDv9jss<8s1YRu9Gj#&J{fU~}EqtrBXeJMa0uAx;W0h29ASlAxo~8DV z2zTlssgH;a&A@k$fN<3S36B^7%YQ!i$cOX>SoKKZ*4Rsvsp1**snSGbF0rN)OPYBc z;e7Hb?&=Xwi}rxa_6Uxwo2v!Q=YXiU1d1*Ji+XQS(kp+J#Z^{LRB(**`=x9q*adCW4Mz@*NC4HUhekS9?ca%N3UJ&?Ly1KZyx|qDUn7k*2x+VY%$lT+_(J`?}ZIBaJ z8+OCtF}X-#rrM{#;hexuYLFSOk3qrdGTkIGRvpfWam4X9*`(R;V3{ZBVaa3MhhWjC z+Q%~KmK1(I?sGz+p_irDf5l*gs^kOTVpR@sk^-HR)C!t}^M?9=0~= zRv$h$`RI({G5H9KS()$%j^P|{R~go0*(TfPH1!c5HaGQ=A3iqyh>c+!ebK=&NVz4) z@u%FMVCkjVw`Td$AMP6+pp7nR)TD@ZHbF@hizhRa`mK%HM9wUMN-6zY9F#%z(2q%D2iBa+kl4+wh)4KiuEI>MthsSR z<9gObR-d>)vz$N(c`h_^%I3XW_+*??rb&S2fZ}lohnf1VD5cFn1-u%(b1t$`FLK^q z2ou3X{VEI!opHHY2C#k>W^40o7?;upK-|W3k6CIvXL+BCJfuuHc!M35oLig|315Y? z%v5b^VWc@yp9lB>6c00_v>6IOn;ycD=R(4u)D0j(njR>DkM+Fab~)%jo*Ppfb)W$r z-|yP|yNVR38WrEydD@&jV$cp1t=upj9b2I6lR5cUiITJ46gU7|n!<`~v5rj6zWX1iopz_Bjq z@4z+fi~UDQ&|lE9P`6mW$cwGmx9w?%aFPTaFhZ3Oh-=jl3_`^a1pjJi9%i`6f1f_V zI`U@I#)=kvi8G3S6i181UPQ#ujr&M+U+~3Jb^}O;Zv?5oL{W@A*O^~P#ePnlcX%59 z{8B~P?_mGlR}vA)>+q9RVNZ;2laHXUDq?KX2}}eBh#pqaJ;G*}YMf6}x<7*sGA=hP zH_7N%9AZN2D~k|&X(qwkw-ciStUKuw1w1ov)}w=@fGSwA=ASKM}!)Kn{zLzi$n_3)tflM)V%L zmlYMVtsa5^MdpQw!o&cz?Dd+kjYS9RYp@WKTvY2=$gcfX1@d$4aKcji@|CMSX!qvZf}rk z!4gl$Jca~+lj6Zdg3COXI>}wb&r6Atj(I}O!it9s0>qNeEUe=|yMqJqH{Aqf6Oof|$U%(=*KdCB!`Wm)Bl8!Y5pSjLOGrXG9#BGt2UF94o|l29M= z@aYA5SeM~0h03qV-PNx>d`e<@1z!}e<=|zOF>f&Kh>(IbnFuy3&v2QZH+~dmRNA6& zlFdBdY^E7fVP7wC3e?e}$GuAREutk$jXgC=D(J>0wN8|+Btjo9oO437LRJPPRvz{z z{EFw0N)Rqyt|TuMq!V9cwp>i5&t6>i!1T1`m3AX)q!qe~gc4Dc8G{u4nJr#TN)2`^ znF1NL0Er8>rPbU*W8a!#Q4Fm7rWYjNLet)ozR6vwe!4Edy(`&l6}cufu}qmN9IYDD zEm>b_TBk>Mip3kU~80#LcZ{VnKYEC|hr)Q?k?&Db$!6H&OeKB0B6wGPH zvzzT=Gam*WlNCv;8UJQixQ~0p%@fk-@OjJZh~dpXEozbwE@892Ix#QNN-WL$^;RL1 ze9PDHIDbG?(|lv0F0E5&_CQOwKnUAPC|CW-hStL5()_+0I`ys|?`PEWWjIG>?TP;` zn}&*AU>W&J*= zvVtEmeBxT1Q*LFfLFMeezxdd&EJ&r;(oMNJJ~K^sDLJKOZG|N@gZ6`-Y`f7U88y3h z6Ltz8emLcLvyno`E;dBCl0$I`9YOICd0+1=ID2WcF^*BdI3$6skyB8Rf~ddBHv_XCFFq%J(h`hr&MMFB(*1__=6WS!2Lt24q{h zQmJ8UvC)^YAI%BM3cM<-)JS6u%brf@-e)w=sUFqBb+Tz0D~DD*${E0nXU@vNxYe_( z9UkgW$XBBaj|x3N9{YAyi-$+6h~$b!LQdjlCCJGh-G%wJfPT@8x*q0>hhEUon2Z&b zkAU4UpAV?gD;8VHWO9~9w+de_QQ}q<0)SpcV9P@s;|RX=e4S=Atc=_P-f3jsW@?h6 zrnj#B$xw4G4f7A<!I41yk$-{0gO$eD8yXaF=vZPB{$Z(#cl|R5u#Y$BV2NHOZRQ zY$q@sax7ic&U7H5pg%)k%(mQDtPB7T&J7r@v9-}*E^WGm(2wPvU@Pah(gGTl79|zA zftp3*SIo4lC~*!Mj}IX{KR~Ss>qj`J+HVyH?vTbZF$!d4N(pB$mpo|ncroCi1VII%PAkjf~b(OEV<&7I0OR?N3|VbM66_CXX?sKim@9^3>qB;eKDofrO&9K z_Qn1ARW#U|sM!o$_8P5}SJ9Hh2dKya;j0xUV&!R~d`(GjG%T^Ae_zeMYx#^P>t%hp zCOyseIFU`{Z>AGm`elxm{TrQ;@)Er$%0PTa#8tiQjiB|<1p`(qvR*PRU>0LLi*izD zP4h*JLb|XLhCg}>*FezB6Fwy1%RVA7vgLb5hFXXi7 zt2%uu`&BBQMB?stoNa70|1a{9Z@)cODTjZ6ZCn5K1ZBX|o9^RRP zipW-nJgbwb*tH@VJygBxeY8+r_mhlP2{kGGoJFQo#B^rf4}xs@eTzce*OQl+!iNtI zXEwe${GwR1L#!zABbRFG*+k^g6lsUnf;nLq!5$hanN$o@98;0CFvIR-eL$nS7ETXm z9_bP>dT;z-QmYw}EbawNI?j;j8r{FU8P25~AWfiC^&o|i=ukkvD*Y-RGcUR?)ifM* ze9wb<9&4GFvY$AJW?N$Ld)4bV@g*S^nM}0$>FTs8CwK_Ks-OokG}Vzm99Y2a-;a6w z?=AZM8{ck`7J2g86Z~wqdv03z76pBBXQj}-9|d82>>?lU1^?n9_<_o0aX?yE*HG9< z*vkg*`LTdxuw9R|%zhnA@ust&+kwU4mfMTPum-pBq29LC(oEQT9ei}uOQt6-rW!M+yZtWom^d-AP z#dXtp2hH>OX}!BnJSW!DfaE+VE2z3kdtP@B{w|(LHxiJvbey>7yCZK4S=eE(+ivYU z$Cf?P!BMEgZ;|j;VakhVjeVE$-c^z}5V;?FRx+2NqoXWjsprK#>C&i_2 z%_~aGZC0U2X~i(!DLj^ei_|qtz+&!57)@6hsgn|kmHOd882!lmn%|dT2hp_LAd#Hi z0ryXKOyCjt#n_Fh_EiB`hb5cc?*H|h^uY)TZYF8iE84D>_B>4Cu$Ol(X zu*eseaI5EXgKqViaP0hkBm1*e=!Lb88sT!dik{u6;vy}~uHV+KYhr9~V`0;PCRHl3 z&wOfU9Z6u~DTK^hjpN;xw4N}(|0@~ALt%XP^Ax)K5T{#U`e;eQz?hs1(l9-n?T!Pi z6m^6?tz5QI$Q`Zn4~UE)Oy(D~J=2OTeKP`uY?2l;16u zwwhJgI=1S5<5Oq<4br#PI3mFdrc#JlmHMhpr79RqhcYIZNH$2034%i>XV#KBh~a#O z$fgI4WVs6f2LVhIWg6t6#+H8QiV6bdYZAJb?D>g=5`bE6fq-y8v zAmtvgLJ@VNZM$zgiLBZ`cic(28D!-v23)z%9Y}yrt!B6?8f2wgOg>7OrR1R*6i$khQ70CrwTtbQ~HT3!eRVABl=FBH_N)dn@VY+1RUSYT_G zktW|zd+H6YRy{r;Gu;cS)~tNrEo^7K&I1;FFn>Dv0^~z^tc%KmY{GE1ubl5uI{i?# zuY~UrZI;nwZvp@Pr!2GCj=ZgGS9S{2#a*f1ly<%&t8EEKTWaktdj5vCm5i<|o}O&! zFQ2!b=D8l9{xZD@w)FW{u8Z8o%NdmWT?A^{x^QKy_5Lu_gCcnHlYr3jKW20x_?qC) zM$FxT3?4`_vq2>%e;T+Tx0Ynr~vQ^L^AHXMPVh3Y;=5q z^p_lTdKb=G&e{i$bN{oU(A|yv^^fjmhK=q{ZSYn5K9O0HRj8%KUjZ zf$`921MdboN9Lq?aHLOanR(ce)fOY+J>cmcR!U{go}!0~cq6Jo`tWd*hnLZZzAD|_A~!)nw)1rf(0Nu6+2#qrWI z-`XZCu=^k@KqDYTBPd8C5UwuF(E~wtz#0In9}ah8r5nX_l@#=t3_b?N4mX=fbLaE5CD1mxM2j@F6`Cr%r7A2Wa?nyLO}wO@A0u)dZ1 z#tD&9Tg{@roP-0l_T>_i0MjuDAuL0H=_${gX5p=HX0mhK@N&Y4ovc0?T*Ct)no1u? zg!QhA_%&TLcN#~rQZq4?-c;B$v3pxjgbv1luPvMrk8!mt3^9AqcFxHL6iw-lQ z0VaWum~6ks5X%Q2UyDET)=T$lR9$rR4MGF{4*JS&{@~+_c>~@KD%6ebEmJS)_VDxI zzr0rFUVFzqs5Fv~*}dUDt)qPby7L>-iOiRVL!Xa9Cz!8^Mtmgg-uQ63Hn@6Tl8{3w zKvOr>McPV;$*bUKT6`$b3Prw3Y~; zO!Lady9$YZ>W7%p-_d_LB>K?#llP!dw1;rk6QqEvVy@B&R2lK6y$WJ{BM9mCt5ut84cj++Cv!%C1HjnA zl+n`7lhMJ()!fp~nb85@SI`$hJ=zj76A5ik%mVsRIUFn5Y zIv%#cIOjhL+*TeF(=0MGL@SFACi-@WT^np*NO+4oUqd)E*f5Fct zK}&Jtan0D-;frQW@Qr^2-aC4&yf1lvo}JwtD)ekGkqe*jxxyq5iZDyu1{DB8^j7x?AX9pi=BySf5K5>L6<~mjii$ab7eWzY z@wxwgdNResRwh*Wr-%gu98H}~{|^z(TvW9G+)o$4Op%q-PonYNV8!g;V!5S-AWmZ1DOg$XicO?19^* z67_&tlDHzQ&o8T%&i?g1JcbXyE0e!`qVBw&D?h4O6XWTKl!e=S$y5bX%<)1k9L7};7kOm5)l~zU7>oKYFGw^4P#kdNZxW0=+8;> zppo8_w>Pcbn}Ga+OiF~8%((2dn=;?{CRTRvdU@IOCE(wae^Mw26ifyo3Vx(bI-(B# zIF{i1LG@ zCh6BJ*upEnu1rG8`BsNtTr(RbV__RqPk4ia=%t7qJV^R}k{Td+qO7yuNL3s|-S9_< z*z~zJ8@e7`TPI@WwJT&wwMx1oM7qv5+C_2v$1jk=uLXc8tpA%%`1tm7)n;8QOq8^@ zV)&*^5Uhr*ljfjH+*W%}G475&?6{bBAv$V}NRY9H@dni@JGXL6SAFS(9HWdf^Cz*9 z^;(i}hcJD0T6KnlQc>qiX;ieBV_VUFUM((KcK*peHHFdCoi z(LZFoyAC0r5CNP|iJo%D<8E>RC;St31db3Opw-mcmNcUQgCIQ(9+0aiOb?c)WkOKA zGN{s(8={IZllF;`+kowmSc7R(pxI+l9`iEvbv8>w2}h6w-P(0rR$Mc(EK|n!18Ix% zb-P>rMCa6^g_nssyB~91@29o&=iPw3{xFxkRqHP1q1ZK-LKL};qGx@zj`fC`PHc`b zhROTO`kf1r|3H$zgvLa!;Z~Tt_eGP&;kaytovCTtla=I`nl>!fPfYtKz2*v!5%#gRq?-}y$B4fge{taVkj=xb%A+{8~d^#?GxgBANk#;Ao- z$T?(*LKd6hVA<`hpRR#4Zl&U7!U zLF=uMu04vaCW@@#Sz}JsW4_j}Z^?{2uj{N#JdS8fa(7eNiSjxnroDJcTREqsutpU* zsciY0!Mvy_yI9WDuhQ)084XjzFCcZ|V7XYcisj-~${KiUW1VNQ`CfTknUX{ibLO35 z=9$Pt>B~X!IW@Gu(gA!y67ddvw7PQmJHnxbr#co)kG^miVao<3pyn%oKZ4`;Q3>3? z*lb_Ko*N#ipAH?_0u*l4xh@%Gqm3dvf;yaLQymUfh>1EmN%Tzj?(xmh$tQaiGu((|mW9o?QfoEZ zzC%94KYHx{#osmLmepSH!CAS6ZgR>u_5e;^Rwq(Q(5fOj@^RQRXxkwK6c~9Qai_!R zh7tnuK*NXM$#MaI{pH@6>)#<^RgX0wPbZXPZ~56*!mRWux0>91x)PhNY4vQ;6aId! z`JTxgkhVM%Eb&}3vNrDMw7&F9mhgF}>FSkoob1H+zyJLw4}pT#Ne81p|EF-<*XZ1ER=&Ux*Tm&b! zo;SKy$JoGkwgRPB;!^EgejK!TUb4&=ACPa2hx=e;5D}t*^BC5{Ac~=&&&LS~{Ru1x zAf)_DKfWh1qv8g})vZWP^0W?y-9h z2Xu9J2YAVFT058s4NTfs|B0xMg<)!q+LuC=b&rH`7|h#)KvrSa*lsL7sUV6l9bL@` ze0T$+7Z#v=SRK=aTj9@imIoHy2?u;20=zGL7q*@8>b~$d4HloN(4qG2Q1bS*P}oh- zfO5{YFu9$1doqXz{uzCb>Vkf7?gY>t0>TTu*&h(51IE&P1?4k!|G8lS(!&1}q;P&{ z5AEEN0PV5;>h^5Ai}&!mYxnZI-}!<5w^>n0H_jM@d(0Sw%pXWlYCbU%YCb`PRu#%{Wr8~s?0l9-Ljm>$Pi%DVEK$cA7NJEf`o9vcuhWQDk`h$YHpX< z>}<;It1B&WJFwu7q}e#D>ujy6cmANZoLk{pS>aWlTV;#OjJhA*=ti5Bb zZc(%(y3e+4+qP}nwr$(CZQHhuv(L6|`|W$XU+1OMxi>HAZ+)4`TEAB2k6Lq#s!{b6 z_CQwuN>8GFNULWB*CDlM@PNU~=drkk8|TtUMiVCcNV2cF3{Ez<Jq@sTlgp<;T?2C2;#p}h7u=bZMlqt-poIx`E~3G`mK@#>sB|aX ztZj)ABFBcTW-+^<&;`xuq!ktviz6@|?c3m16{0RiGCL}>uWcKZZ$&`VW)ICTAVWM< z3$kG@IT`MCX7*AeUPp+1U_p=_*rt|uAAyTSq zaH+BdWeekVCgoZ<`xJY>xWA#hjayAkbJ%M~S0mflkZJR{Z%BfYdYHeskDpdDTbIr| zglCnTo$?L$x6cgbN_2?9v*0`y`qcqJ+h{0M9yy9Zo|uucXbMibNQe?_IWCxczDs(1 z4g%`>eNzs3C4F|7gjT4%!5xKOW}O^aGo@XGMf$STG1Xad&-FG$jX*yz!~Psw;^r`C zCKY{VjY-!gJS{31su)FJnX8v&WXedab@DVn6;I7*p560#kW9_goD#E+BNu~KdPOm5C=Tfs))49) zHye@gN(NqSX~cqKxl?HFgcpk#htVvv#bApu#@65ucG1+1E5&J{{jCfxw5@yqAdnGg zmX6N_RjnCoh9QNLYDdm8jAwPt6RTHEw~9G3ec21F{W@FCMgL)!$g%&!Eo#GMtu)Hf zLEsVCp=in2OdJxmF;fd{bW42mA8g>)F}iwY?U{n~ak!S6BGh;aLnQW-m^{8>K=jU* z7=mkw+=+dXj&v$_j+l6xL`50(Hiq?e{e%Bw>O-yIPUP84)DY50h7|1pB2yQh!+Q)z;rQx2#)!6xW6eHH{@Uhd|C1Jxg5w+4RhCzI#yj{5a$B zsuyhZO1iU#{{?{lkTv^wcnXRy?`xi*@Y$h;K4wkkR=BfH!Y&g2MW2sOp;Hh@!d7UK>x>9t~Rm;?J}dg_MC2 z-)b}P6;S!Zzn25-fGv0bajZaCt&*h!X{48Z*Sf8P%{W?!xWaA{^0N6>s5B1?>#Yzd z^^7%!zj6Qa4Xdbck>?HRP`z++10(aRrY|IayW>yOwO3-=f_RbLk!vk@3=mjeP{X}) zxjg}og=%p4UVsc zh7DQ-zb`X%aQhGJ1rl#-Kp|o9j7H!jNj=kK{$GV78XVm$dcTB_>vO@H)JvKBa{*_a z@xrYkQ{9%4CFF66n#bPZdN2bqQ**v`l`?mv8Yx~KLxk>n(fn)%%|IaXV-Wd_dVK?B zX~TBJnFx6i!(vtZ3#ccK;m#Ococ$)yNDCZ?W?=08{U2}JLj$;eF^B6h$IL9u*<+oy zifo&T>?_HZrD&2cno{XmaC(hX~Kc!DVPutHz?Hj4Vudrlc4>OJWVt zjP%dzGu(K4QyWu@zSgj)vFw@WG|~kFoH-nJnEiLD98MnR`p=#^v+v0W7ms(+@Bge@ zN4c$6IcCUn-YVSIg%PkeVcIrW{AJsc0?byqj$hc5?_6bp!^SL*w~F^hG%#Fpj+;D3 zr{0m>2oS{mphO~8m8id9K|TJ4Unca(GB)B*g)b2$ZNEzox;#%7zlhX1iQmZ2AYsi* zKR{AijOkORDp*t(L%LI=8gP7Z!B)W!YgYG@Ru8LU$Ju6wM=6BhsLO~{fx0cc<7Nl1 zh2O|9@7rV(0K? zzAX^b46ZA}=PW{mOInr7hLpdGIK(!`oUe({7vqSFb0s9d&(mBDxSrWQioYCapBZG6 z;S9emI<@nABIqUS4umXXwTpYArHSM1DrN=iX2GRxpzcC$AYRtGIJZiuzd($+6s4?jIev=8(Kj#6%#^Fgq!Uo=elLfW?44bwa?IaKKk*?Mu=Hs!_e zJlkHj=*b!3UDxk~a$ULU<7t2PwVdHLV!@6x+frinSEhy{2Qoe9mv6EXg`mKE)O&-b z`0JOKEMA8zbDG-2*rR@wgTi~1!~{yTkPcG=l`|;KOHllkw=}V`;5e{{tMZq0_owC+ zzB)CK9w15=*)rmFVdldX0*F&O%;o2SErgIO1d&TP)FS@PUHSP>XXOS@Z#$-68K^n> ze|0Qp`frcr|Hm`AjogeR5CiN+%A^K z?H_M)28o11XIiEm`iC!XA3txv_EC+P1I#2$GU%ms?*@dR-QV=My1+G27U-L%kf%r} zvO^{Zo^A&V9tXME_x`ly4cmMZfthV74SkMi0WGJbyC8bOgq@$gw&K3SnkhRUCO=8i zHc1<3+ED&)N-op%YU{_uYiBX&mcm5}anR5yKUQtD z6xJ`=BRqxB$Ph8#smsn-jNJh?hSATVf~O<0d7U&L`gkl$CHt7$fNpMncpbP=Kg&9Jg|p z9lV$)@g6s!Gm;!+``pzsN36}C#wW4qv0S%(Nj7m~qJOu$sQstfP34X+6z`k%iYob!~It zsAE$tD|699(X){tH-vMf{%KV`jzw7&$Bceg5S(^*ut45jz2sB2@b3&ok#dttTBWnP zTlGCNN(>&%Jz+?hrx>_j$vjp}J8^1}ne|0C=Y$mYq{)1v+o`*_5luUzt`pi+zfnhX zVMM7{;cPr6gS1bz zYR2+a5jTyqfMBLDL}o>*70HmZt*Aqz;xiGu0aooA**-#*sTC(3w9v_5M;=b#VLO7O zSdFSnlQ>jN)+GJqJ(@Ay!Nyd4kMxl4Xj_^_Z5%;q3QbtF2ur6;v3y=Z)-gRe^VH}v zWR7#Pvzvqp#wv(LuV(1}(+;qjObzSB77GMMQ;+*mEdMV8d_&n$5ixduTNZ(@+^S0H5Vg5Z42vX%2xuL#1&lWY&M}7E-ti=r* zv0qHIz|X{Qs(KdmL>s|O(7qofCN$gI{K0-v*8tQNod9dZ315hUm;R?JMvE778CerE zv4F+5TeofF=xk%1AA99oJigG|oi2Bb;?^q@`x8j35ufC!g#(mroht?wVy~CEvM_}5s53?z%Gr393L0kpR7~V8B-!Nh zIBPJ*jR>M4IcD0xu(SPF_>GjLZOvf~#rh2b>zkhvCr`+4&&;|#lHFmkfwOh|${H4K zyduS3Ay&#^a|FHX9ZpO7hI0}&zG#{vxy2VXXi$@Z-=eI|mS_b?7bAeV!*i}B!J}Cz z93#@`;zqZ(sJO^R0t6PmX5J2J+V4N1+dR_LL+y7Kf*Jb1f^MDv_5=98pj-YwnI2*# z1NZ;^(f@Y;KYh^u$E012286rPTFQ4E@db|*T(k^yIB5VpSoFX+f4G40z^E%d13LYm z%j&tsrZ4_fWS)t?%mi5L|A=K4P0nSTq&JeCJ2^+RvO}B$NNt#wNUgQcusP?t~*Y$y{=O}?=R-K;qpVbtM<((XH+XBOHwFRs+uKB z@+Ga5HLDg!C~K6?AXCnjE7i_0Q)*StI8$upDQXo3OR|(DE0a|#NCya9x=~PXMSbmEkbna~Jw$ejTa0L5j?QkSsd!+Tr`bigg?we9Eb+ihS>R&Ul&%y-1E(%9jG;}|1^JdM ziYjhX*2nl3Dv}Hoow~~urMi{FtSNSkQJv~1dDV-tyyFV+TQleJtyuv56fJ_cz_b~j zW9dZxDO@mt{ZuHL5*Z-H_$wX7Yzg0}kZL{1XSp1!XR%CBv^Igcwj>y1JA8q8rd+H* z=3Ahk4NCnPCra@tP{cg5QXY3~c|`N^#LI)eVzf1|I{PKc-g2kxszqrr+L_UVKP6i>ZEo+xjcMY*gnLln zp%j{_B&sMklEf{~X_7w!|74YpCg%$F6pVY#5NNu?Pme8!Ik4?cjKdO$Fa$|913U+} zN}^<%X1k%xGgMrh5Kg8v5c)cx&Ri?@PtlXo#-^FAeQXp~(BtUd3i7I{yRl>UTke-N z>L2>#S+q_mZC*@ZI2a&hvb09tUYtC8H&rk6$It5Q1^AgMpZ?qJO=X}#;EP+uiyMd8 zDSSsK!8)cW+D2fY^NvPLrqg;K4Scbrqtm(_nK5z&Y&ow48l|Hke`&o~l;89`-G9<< zP2SIk6nolzBQu*5VdQ0)m78ubm0@Ho8n|(}OutgjXG?%k^*7_ykA;iM-}A+oxwi$JH(@ zw>jEaBx~Cb@|PJhXP|MA*x0C6eHJtK=4wWjZfjaoqKjlzoprXB(=*Ara;M}Hj-VQz z1Gy|5rSwGTV-w;&R5Wl^CpPLlX=F~lPk%SdQ(WAP&6$!Hd1&vUPF}H z8Z(tAesPBU#e3Q&8nZ_T2#3TPE44Kt9FA0i}6-Lx~h>K zA&b6KT`W#`LkVHF2Xa&2eO^p=mW3R{U}Coz+D_S)XEW`34yqXIb68BUI-`l^^MM z9!@v#ZrQ6{>NA7R7OnE$0PS2)Ls!$397QViP!Ski${_2019Zr@g25%T?GQPrjqOMw_XG}FsXEmtWgWGMDh5N{! zQMggUA`jhzYt(t=p8>6%J|1M265sN12r}prR9Hh2BbZQU3`>*6rV^X75$D=_6ON8& zTswQG3Sz`|#pDF4P8y9`W!&Wv?IxTY=&Deol@lnjMI_Gnb51YU5aqZK4${J)LPMcdl7IqpVDr zEF<8@ZR4y=Y2-;Y8CNF_U(>c(%)$th+{9t)lgwb#!KadU0eRhZ-mDf^EG69THPHkk zs+~)d%L;atW6hMWs87Yx6`+!owsv`KIgw^jHDoOD@l<3uT=w%F5Xrn9x|&_;0b5P- zK``ye{-9}}^20o7pNYn8R7^I0r7}y}T~|eKu3l_3jT;vh1zWbW^I_!rZ9vWb8;ox{ z;aYG`lH+iN9MU1X$9(z*E0G4;DlCy3RiyGuypb1S3)&(>e;J31giq*aTS?T4&|%oY zWXMarZbOU4yfB=k)Ze5`N>+&e@`OKB0p>P#9K8BN4Vl9iQVT7^rGQvtc*d?(9=6mT znFtRiCHzgM^DTiFVPy!tBKpf3&3QcC#!*mM$-f|>t-6An=>t7;;2uv8KH`Fm27*26 z1%BY}-&eqSID{QLo_V0o7?_ib6*q`SjyXh28oDs9siJiOfvn~c+Taq-F`~qwQf&e# z^{>MQdj!_DHTr;n?OD~KEr9RobA33E%A?i~+AW%ZF?wdY-4YsT(;9bdix!nHv{PW( z?KRBtv#4DV{F^z30Q+7xzm;bfM})hWz40b_&z~{&W2+U_bSpb+Hj1_$#um;)F2DHP zLL2usb~n?2S*2s-y9cQ~-vlZ2eY#0pgQVet2naW|)o@aLSY%!$%^^Q_Qs82KbBhpc|XW zgbCFYy)AkdKZzuxoEjWiMqlBB*bJG5NSukoF0;Qon;$Rf2qU0$;v3x-TbQCwl{s)2 z{Sjf4nYd}Kc>_C@i_B{HFk4|0-_XRC>`czIhoR$0e!-YBtb3DZ=Ztdmi9^Q)&}yTJ z0nQyrl7FUnmYh>)1WA1jmmD`_0ry^_wh`#HpwZEciN~X0YoGYN>+mu9Ko{bM)60rvNqfzttCnoeMTeS=YtDs)W960NJjwsz$6grPTJIp?*C$vwuXJW3uNS`R*7cS7F7BeUsFXff|La z3(de)E#~H4QqU)DEiWj6;;IufoPQBr> zO}&w_t({@B&ANT3>z%!aYrehoGIXL`{M;w&ey3be?i-5{s9ad=7j*Xpq#-o!w%w>) zuUnG2q1VO}h3s4IkfdCrX+Kapuiimj$`Z^^zhg;GSTto@JE{lH3IdZPsTU%}i5ucz zZp2$6#Z$ZZZJl8y)q5Fzwfvz!y;Ro;isM&4#1w}!;&q9+G(ELE#nJg67TR~jlQ|x3 zpy^8~6oYo%j*l^7PG~!nuNZv;34SPO`&##L)Y`E(_T7C^Rvf)suYW*qi8iD2c2z&4 z%TnU$nzBT^Ejrs(DplLX9JFLB(2RFVg-_Po9^d>x|3fwa(9*Gv{F^@#fb(CKXQKZX z%CrAjhpE`OA&DUSlG-%dbjtioRwXA?m$F8@N)@EQ6gJc|C;4Md+58%~WGtT6v>wxy zwdngI_lozLfAY7`F*Z!j_ZH2AJ92y7TB3732zPI2yW?fnDKDeL&Gg1@&i5OnAG-Cu zA7xY7UTT00auqolrJCwhdB6rFkK$E(0PPFC--2RR{H_!^NijQxkVu3z7B@Sx2o1?W zxCAw+5KhVAk%(qw!=c>W2mo?hks;24g0@m!RCk1hLbX_Zq&hOrL3RX3TeX3}C(B)J zM5nuGUuwF%oyuDfrWZ^=dsKZ?`qeo75Yr3 zEoT@smP-+cEs0&@v>Q;Zw|K&2U3%*<)^O(|rFw;A!&GNQmm0b!T6LwqII4@%=I`)C zaL;TT^gVHGYMw#)NAyJP*T|IWQY|>fpgnc{eX}r9s2b&<1L@q%baTwQ?16WgL%~w| z1@kdijMSpIW8Y+r2~Oy@5Zsw%i;Ip_t1gL?xcAm!Xp<(>Stqr9nCqqz5-ksBVhH{M zgd^yUqE#X+4|=m62Lw)rj44{&I>?}veW$b}DV4hgBIZIMGiGOVifz_Cd%LwVwS*Z0P+ zPd>DY&g%fpxzSqNCzg_qr+c0WDL%L72(K=bUE$`@GUPK$0`Wo1_6=N-# z1Wcl*79VI6rw{bE+}CnR;-%$rf4=g$uUO*mVfODq4elbPPZ-Z#byMgy4d9{W%F-_! zo#Ie&CG-yjGFbZDuRla``L>JSvbekMSzH7U{e|HT=%N{d*lP(79I^}K9EHd{uUilC z>CTPF0#XI@d-%@L`D}s(75L8C`6Agb1s0N((pG`_LLGUxp1^VlaHG=lV?Xd{-$SzG zMn3JBkTRhHQ0Y(supt6`kR~v>b3wHFt6|5V4>87UQGJ97QVl@SzES2kMY&=G0Z|23 zz*GkUX@Uh2w&X)#+cmxS{~4VuVE#UI3IqU<1My#B%YV;({BM1o|7o41Vxxv6hU^O( zFO#Z5+>5NGwYC~c+)YhBx=KDC)oSsFqJVI*jUXeqBx@^s3qsIO@<;fBg9Ura;>ye~ zxy-FGN+0@Je=hf7dXvj3&*OMDCkKu0=M$@+yc?Z2jsR~+5PHY*RdgsZ6p!VzIKWDM zO@3K^5uZ7kMVZ;!y#9ba=7N(MngV?lA*h5XHW?O20*8md7#W)KRXQx7InavpojzY* zc%VFx0YnWY`em265z-d@6|xOQnj&4revSXn5~L|Kwb5Wd&>?j7ZfbzaidQflbT!3; z7YN^OZh*|lJ<~A^d%Hz`BppoqIKv}MI*A-`>4k)n^+o`;!Krk~1GJgaS%*yT2u)ZQ zN)^7(MvXkb;f_vPtI>>ta4p7NencF4ZONjQQ?6cvdkh*|u!-H3y2u=w>I&5gPNUSS zcT*d2>t+A*rgsE-qnOBB;qlvI=6#9Iu?WWnk1I}GT56_kOMo)@*mJd4E0Lq*xcqti z{l`UCA-R*Ct!bw&Efn7xBhSB6toF2e#{9klPpe}F^IRk@i; zb$@N^>COyRAk_2@eepl3Kzg_K3tMDETQy3RSisO91Jz!l1aTNq%%(mONKy?haVYIR zx6W&`)+`5U6v{ko07JhS$!tiJJWXXH7=<8H8n+|kl`aBKmP^zR9fN8U?u9oA$_(*O zs^yypX|A!$iwSZo>CM_QXR#eBk# zmoI(@LK6;fcvq$xdk zQ=C1QP8%!nc%M2el8atZH>IlN@w%%SV&M-m^QZ44&egOaD ze4>%~S$*(}Av@szmoVi2Yo1f13E`c*f{^#tn00e?jTRe%KXrtNAecyk9oP zvx7S!NYs~z_jA+Z@o+qo#l~c(yL(1=y#WWXl#&7X1$Y8cjlizA3m8BKsZDO5&_4p0 z2Chw6ue!?^Kn9`xYiPU+89)Z5O;N9_AJi`zPz9VzXy4LL8ej!ni_$K;>luIr%nG$h zeybdyhv1&sUkABKeCr(WCx8Yp2v7*nC7=dy1K1W79kpF#S2X}3;0mw^&BX(Z|AKjXfEjzTqZWH#e zLpx>LGr~FyO1g&t#JLIbe2zfCxCbI(W~_qOF@5Izjv4vUe^K0;)QfLsc`ZRsE z0dr880rgPBuOE#P1|_;UKbH6SEyHi**Zfs5y|<`;Km7oN)-DhzVjsvDeYDe{YW4tq zk^$<+3eC$#-e28C%KFhmyJt!H2wA+AY_B4O9%Pr{;7qI$O;M1uw`zQ#O`=c^4 z?JK0WfWIr~xvxrK(7?#9=`EwQg8;CYr8}8!zdgec3G|Lpd+Hu1%*}uoOE+BKls&?b z3A7shr~D8vx7N_dog)mMKE=nKCk)=O6pJ@_7qd5dU%?(xyV8)@ofalpex5CiLoi>bL2nCFu zeyZL<3TE$k5#8Pz%O^~mZ`20MCyuY!(2h}AsGt5nJdE$*d)?6{W^a(WwObX`TeIy! zEADr+Zh@gb#|t~|E0@jEy&aDNwp(tPpMmM0L!a&m>YqUz%-)eAJEs?%P&bV2gOQyv zJFC|}Lt(EzFns%~W6*iKDwy9{T6v7#qqiKn6%<4DT1ufww2@iBG8MyFEG3Zx zWArdD=8`N!&YI!7sxlb?ZTJ9#!eMu1q^NLhAw|97)ac@&MhbT92F*kdQ}Ce@sptIM z7`8CLNl@^TBol@1MFFlnLzzd?@E7ykvZc`sf|U35oXL&jGRErj-1#ucl^xl8C-*V;G=~#o6h5=gTzUwo$JeOWCwrp7?dWNua@*0 zkL}ehC1Nwr-?+TGPEPLiw(Wx({EsNcY#iawvbtOehSc_Uf72=EM z#6WxD1r8Bx$Q!6J!+-=l#^qNfXASIZVO}*%;THkH7Qj)a*vhR!STUr76ZU1c)mB%V zM_brdYg~Xfi_5xG9mTGz2<<(kH%Fb?q3K*^(Gt+Y#75isQyU0IIL32Bwg*H&#NsLT z%x6;#{rwHJizL{G4Te)&+S|dn;b3f2J(b(S?qLNrLs{aWi50=cq(hEyBiqWG^H^s$ z^Ne(F2HJANggKoCQZ3OohT$|1uEm|!3Sk4M1wCR@Fh=|T= zq(?mnM@G7U+7?SI^FA#L#_}Yc1##7;-0K!d7)@P(icah+TUGsbv+q&DFj^MKEmSme zAy!6}8;MegS$XDp?eb%%c?hR^cyUY38OuwKfQZ~JaRv_s9|Z?=pZL%}Dssuz7Hi`b zsz6856E@`IzwB{U*Vb3yq0U;oJL0Wm8Znc703Y5lNXMc<&JMAaAQzxIr|t){gXUc# z9BxQSC+H$7$?++}8cm(JTWRJ6HL_V=lF?-QxLGT@{l013$)TpeQZ&emuo+j{U}c46 z9XX3_qNSb2p_{sH1&^6!5%yY@H`~`{+}X-Z2aQp#I4wpcf1u3h$4t~&-+ok#>Zd^C zoYw(qwb-Y|_5Z5R%Z;KIni)O?-z$L~M+=mOP#l3EQ;%0DdqA{zQ&boLy&i5?@O2yy zUtJ;4k+^F>&!w~GK$e-)qZVd7Os&>x9A{~{5;DxPp!-A^s|m|luRaxPu#r?uR&M8p zCws7sxlsBvEtWB)l%Xo^VIPl-E*pzIq#X`Ul^>NGXbFOenQSbISXr{sSZV9@nZuyC zQw=Hf5Ugw$j;+}v6GbOsv8>ubwz}ny-#T(dSk~ z)OeD6QLTlL&g8B*cKd4QhnyQ>>REG3l=GLgN=JD3=HLeziq8O@`P=QjF5{HKLJJly8~OAUzDAEjwAIWQ;!K z(YQ%8ZfYkWjq&<)POLp;>{>kaCV%oIGyvaSy3re&d|Op9yypJV9M4Jb%18hyhw$)* zd4wf4J4PMR6y&rrnkCYhwPnupSWCncc~Y7lCQ}~Wtx+tAn<(0ySJ&VOi+wzkj)!!KgU+UfTO4n@?`$jDdJ4^Fa zUy!;P-=0xMfv#=j`TT}jt^;T4b$*Ma{|K*S3$6(A9ah3|9ftdDse&qmFMb%`#$^g+ zp3&3cdjk4Jpy94KiM`_%vwf%in2hl6%zRG{)H{@3HzMl)h=ATvK?WCn$Rys`9oDr- z6lJ38g&PTdO&)m`2azNXD<&5-S)z?aC(E!ETdVWN)2nABaxleRYPD8IO?uA58{O<4 z%DP3~#lVXY8>!Z+aJ|7=S%>V^qo@M@DExWb@dUpq-FDBPOz(Y;#Qocwh8fOBLd_c9 zq~ER>z!zUFvZ4jih|UJrMk`Ju1TG`0$a z!xMkILA7MBL-!egsN692E2)9WdDI~1i#i-L@_x^6=m48_PAfGiD`SU2y9z2RKjV$X zhv~X^7tQ3};ennbPdeET;1=$=1(2S&hh_1gqaOdr(R~FGr0eF-lIO9c;|{6^Q?>H2794Hi~Oxvp#9VBz++S)aU`9_NWNUD};+0Ts!G;O&j@{)|K(Mrr_Vs@x5cx-3t$ zJjjLF3M?CXO@Wvj@adMBxnHXuayv|E{!di{@~?zzb)Ql_3_?9lZ4h)54(rZ=9VRbK z=9M!$_C|=mA~3!Y*trBjGbLueG#fN41wJ^1^>3lDK~-X4cq7n1fjz(8qyeM83_(kF z|0|MW^7@iJGO+yN*U~)#zXl8Kdm6%$mwwu~^IQQ|?HdMix5q&|gkqbBLtb6qv*3^X ziVTkD=l+iSVh?XySIP;$;@^*E4iA^z70kxHdL)N@l-Hgc zIM}$8@=JqgHD>U4G>P;N4d8`9BWAyVtyVF0u<3_1%ss2PHqGpP0ididIcqpT%AqIGw+cgvAvATllW_E9 zdi}jld;OMi zu*-lh8_LHXms@gKZ4b~Lh<2F3o*_0+${tiZ5^9fe6I{&@yS9+<*s>eDb^vUT(;K*U z>}6M!8=Q^8ZQyFM$3CnP?^-!Guqzo&zwRDmvj{ubt0J>`MFgUG(`io(7B z8T+!W2OKUi=_e>e16@aki@emXzH(491Z(Cv$KvnKrage_>tT=S!{6y_#x$+t9@DyJ zXsT%MM6~nui}v3nT#W-6P_VrsN?7k#KRox?z-PYqvxC!V%#KYaGeBw0j0KT1 zg4L(r8ubOIt&A7cBf5GO=cG1Ib2UXnu!G^<2)bbNX z85ujg%u7u2i!JNtYR{#FB>TC+@lc#(<)jc8gjH3>n6k3(;GuU^~S2WFT`sn78UM!R^e{@J{ro z7%zHhFoI_H=n#Dw$erxoq+084mVd9Ap6X4!`v^PE%su*-lxINoz*ukGQLTrVx=aPj zuVSK$$$AFDREGV53jz+}R^kP5!fn-kB%N{WI!Do#QG=*Nj6AJT{P_mZPF>>tf*AZP8Sx}061nuLOVo?bi z)U6JoSbo`a+?Pa33+(v;`Ok*f)X7ObA^^ZX)qm9xqyBFRyI+?Kn*TBKOKarh)S|Xw zyTK0cBMa~|gg**svlq2bydmuCXlQ0-&GyIAEsA#$l>{M0m_}sG0qyMtC$UpLjr6as zB6E%SYzC7VZQ-b$=h!XiF3-jyaS^UKEMqaf)CB5}hJ)U9M85r$TZ9<#0YrCZjjbLK zui_u>f_{);ZTX<4qX=Z{rlfXE3~Bs;eCE=KX;AtUr5dTQo6R;Ws!-U_eBj%qx;;X|;?tTPEQX8L*1JtN+S8W8! znZ~!6ML#p`91W;DoH#=7+6gQAk`j&i_b8Uh-hU;nDY57KNW;e4Lv%%J?mjw5DMdK{ zPJ4?+6KiQk)jbAvde0iXiXt}SbK>sy`ap*(myT+G1uK?byLa#0hzTR%IUqzwv5(y- z>~*aerYEtv;w6QfVnDvk#X=x^I{_pTcG0ZkpM+nk2)ax$vrZlP(l_@CO0onIq4*;2?(Xcdd@0#n(%HI`b*Rk{R#2s zlTRq0l&A>TC|%4ZkeCAyp*B@qQ(uj>RUYRU)$2Kcd4avPyLlZSJI8w`HeDFt(SYh) zH;^U%Y~jSvC-5L92;wsr#2eQqMUMczF@NMIk>nc}PlREjOy{GqXM9fS)Q~PTkL1nu z9P2TP<9Q^)wLg+R+KFRNpgnC=3tQH#8Hutb2CKr)z8g!K- za2xD7&J{yyLETfPUxYg%7zvzrXjPa17!S85@yc(>KZ$zVKUN*H zS+V}PKfixBRQL^w#lrQlmozeeBuJkP@*F}r+9=Sei2K(y-qJ;{$)cnhXA(_RIO^ca zlclH2|6|I)*O$x2dJ;AA_B)V9$wJVnTp!s1v#5Yg$JeQ%$F_*QU@jf1s%007W_cCwn zx$G>}d~iM4piW1NrnVNvdci2=Y46tJ&q7Q_2QzUlf}|98weRS9?k4#1+T41MwEV9; z^mG`va8DyBUh%3L^L?lA*W^g>$3gSbz)=_XTi5o4F2cm&djisv80w78B6rnX*Bfod zTOsZ);sN{PK7?fu3w7)x!eV+f3Lnq^pRe4?7f_Osu9)g?z+6f7SYvoeuazbm7>R=L zVsz(-A97;C?&RByXBfg~b2+Ze1_q;CYfwC5-NK#-=JaDr=X-Nzj7g-W{?XSpN285c z+xv-$7|V8zm+>@G?AhN%H2=v0l}dvH&u1E$3@>%o<|SM@-t+sV(Rm4?pP%O?;Zy9b zt4n5|Ly={!ReGYh(serW0?f}jaw4qjG&IGSkGQnOVyKr7sft|w5}MS2nB!0RNk@Jv z3zT!E2YDitt*~I8_sw^m{H6XXtmHkm==o_Z3DTLiV9a6C)1|1@E_uI=j<@csS7vMI zy*AmYkPD^&wd-)hRAIN5R!LdNZrrlhio9{aYrg+Lee}vxm+W5^J>@U*%S!01ZbS6m3Gk55S_7yNKq5K`bgexVlg&Z?z|j?xdRFF^lSYmR~c1i;3&^|GO-Y#KB#lZyiv_EgE0u#N>K` zlNoTvB_93l%H4+T`}tnE*voi1I9y+fvI`wDlS2^1D}*QG$Sooi8G$UCl7S!mDLpIv z3xXkfvt+{v2Z?>hY7x3h;U8k5XuM#p)gP3eAUwc0PpJQ2_2s#(0lLZ23oR^}xysMaw`T^-f3tN2v77?kS4ST+h?8 zsk0|%f8Va&kGJPFuE&j|vA8rgw>32-=MRLQo*oZ3A2%N_pq=fasjMA}bGj@nr6J)35Ra6#k2X;RFPyje!SP!e zY6%2VPgo%HZ8@Al+mKJ(g9KJ2&<9p6^uweQS2ptybT#|bk#2pMlK*6w)ON#f=vnd^njLnp}5uV6< zt&%$-PRPq=(PQu#_q*{w60F-NbMGLy*gu>r-zx4Pyo+kTVMwIs-!Hj>z#bVGlIMJw zMdyw*Aqkr&L=p!2IQ-m`2+!D_v1h{+Ugbo;g%pOJM(@{j`iHdvAP?~829WPa(TI$p zp}lLpXAaFc+262^pjDe0L3tfjX@rz4*;TW-V+fQrL$~)t>SLg$z~g#Zf7xg~*41V) zWui$mqyLApcMOws+m?m9s=I94w!3WGR+nwtwr$(CZQHh8b!+W)_SN^C_3is7pUnK1 zGv^y4M$9oII_m5@m1ja6adBbj%sraqxs4~!9c_69Uz^5GkKugsQ=OmK6zA9Jdh)W*_Y1FbvDYPp!T{MJU6D3JK!;HeOea#`UM2RW;B`<~u zt4?lrB*G8ar{rKha5;G3cRP|Ibn8IUREm1_ZWtX28^j+a4K702f+$kW_H5AM{T~#$ zg^d_wJTlh=Fd(W1&O4Hw2XxGvyLjpQgfM}nfzY%m!Sw10b_#}7*`Zid=2qUW!|R!n z^5ofd9t@ca&-;OUyA)0P?^EO^C_m^ve%^QAum%4FvZ?oHAnb1gY!nP5lVISA)o>8T z7Oh_I-XLWo$St-hj&lNMB1s)nWYP=!jn!O1$wi=8H|O5QMfD-|a1Kxj8k@|OD*SYO z{t|XItR$%Ic}nWp)SVlhP7?(-`8%W`NiD_WB*ZEfIaH%xr{Rj#N7R8XIQIQsU~mIK zt`?(ZfyxB>{K`|i$U`BLqzi2XJLO0^_KJ)Te)l;sG%uhyQXG8ZnV=9A`Ww)e#1S<` zb6qgc(vg%jWjxf1x)ma%&@qbC;5}?(tbBjXpzkfOMdLfyV$Q-py^Kxt!I0-Ss&1Os zB|~w|=$l7W!VCDW=#_5_%#@tx+>FG|%i8xVtrFE(xxa z=rZ&9)#ok><+f}XzD+0PAC|Nm2Mz`&Oxzb^R3wp3@u})T$MSW+1z$dS)d>=T2f;RU; z?i6gDI&XHHgpvE;wb#~Cr9dKEFE`#U*ZRsJw1wMM2$E=vlMU`?EG_oLnnA75YTLsJ zYhuZypuze^yJTJy19DcXWgvAB+!6Dt2s|}U#v~sFx?UaKn%2+c4xY_?f<3XIEDs-e z=sykNNy*Ly_TlyNmu4ldX7vddJ)UF_xG1v3so&lEZ$nN3*Bc+g#)5sAUg!S2*G9F+ zV&|MU^QH~yG!&Rpr@ku3c>a$55hN;|Bs&I(aps^9j5nUXzeRlyKRd&Q-!{C%Xa?HcLFJrzng1Olf3lj! zy|x=irWw2z6Gg9puGXBtn9|gOQ~yqxIzVMC_Qwv>&&8RzI>!Rr_|!17wv%tu)I)cdCi%PY zT*c#TR)an44mAE#bI^>%+|fjVP;&8-C=Dh9KHgaAdUe4aaK*5a`Wtqc*TH`3QIV+! z$T3Nvb!bW^+$H)1E~!?T0^nB=1r>@hqK`%M5uS*L&MRMy13-;rgBdJkvQ^C!c)^I; zzSsC!a_w}|>))idK@8#XxyE8@Gk2BibD2tm>>tY^4zdP_;d8aVaTeAi;dbCbv$X4m zXbq*FNt$kCm!&W(4Y#>z)GcF$ifRT>W|#O@oA}~n)l)$gSO?_;)g3?L&Ax6MyDnq9 zcqzd-1sS13H46%0m)tTm??=uke@&B1c5F>*dCNKZE#`yQT!eBd^`>L*I%|JZy|3WS z1q!8T?rNWT8y#I+YF>*LkppCYKIL1u4knovqAJn5^?3m|+%NeOw5LJ`1x)5AO%dxa z!|NNNszPv&`FY@pHuVxdO`}Tc8T6ltS3U~n&lRXPsWvarBLph9@!&xkPl{Bk8o#(I zsnnONY&n)5)>WG0crq+u)H1*PPZqT6SUK`HEFajD(9c^fjybGfQ{N>_Rv+*iH%L}t z@K#H>)(Y11L;2%%Y}lYZNTCl{A{^#Z#5akE5Qm# zW~((d+w4FQ)>i#ZqdrUf!G}LoNr|W8gD=m&DE(f>77xZQAI5~iH68sbiw>v<*|i96 zc;O4~Q!!H$k;^;FOy9COvZWHW*D@BI;cTj{H?)`W;~J(I2E%sW&l47pRNK0uuI?@$ zp%Z0k4g+&b+e)}h8Y?PinKq-%knG?V5niW$M0f&N5ACc&qB#^g=_8cuuN(Y+;4A@e z?TNHB7K+ySfr|mei&S|KxQZwC(rWmIFZ}YUkW@NKvxfFE65^ZD&OY9_GWjgY(^OmM zzE`jE#ItZ?fq1i|Qd~OQ(6#+}ELh$eGV=BUqSye*JwX1W zT~nL=bIDA?CE`02Qw1>G)V=-n(8r%^cr3||iNz(qeQ|Sj1~qbHQN#jiCV{e+-Ctkp zOJjUl?Hny+HNzYVq|(xWI_tMljF_Wz^NnI-Dl|iNU}F)QXfmof0k$wt?jOHJUO*jU zhDfRebW6pI)fs;-)egWbaxZaEz^^i~ZRYf3B@=k!KbL6>`${8w;i-u~S^(}GAz4pb zb)VUIk(e#~+Qz%OThseXsaKn*rk)J|UZ+iq&Nb=6B9sXGLm@6=oy1$SmxxEphY~P2 zc@MX|vW14?NHPLi*?dQAH&V7ZTA0CAJC~G6c93;Cf2wWh7ZNF9!p~|Jsyb9#T28o` z9SAH4Iap;c5=%H;FB63Xo<$%7fej@Hh3N_)RqPaz^B|52nkv!?X~|Cfso%=gDb#)8 zM89>4!}rm@yFr#@4zOgFMxc-eX?aKY@@(2_4y?o;-wK5zxN?3xMs0jI9N%lRAG4Cc zI@>JpJ3r(!zSDDJ6j3m(9%r6W$AX6`?7Mx_Z_C^{pU>>Uxx{K~8#*m4L9>@Krh-V) zG>>ip6QXwCj4(T^EsPq?ly!c#`m67idpXt4g#Pj4r@}w=y=>nd@PFeA{FlUOXJT(< z>q5h;YiDn(qi6q@1IXWB{XI#Xpro#hpp5tl%t!1G2MAnm+AUs<*AwnvmaBk~6AD+6 z(^y;qJtB&lroTJ>YoN08f#kkKvn=XiK?}06(%@~e;dNeHzwL?eFceAak<}>c!{hCI zEQP_V>+=K2d+vGEZwz~1Ki>cp4Gm2d4TIhu%>@kzEihb{x%GxC3>G3`z=Vbk?lBjF zL?!*UcY*7L`sMp`sD$3;O6Pd)geGY zQOPf0iG=ongNa1X{CXiEtA&M!^?XW_tA$#_lk90tm@ewQe~izi{3aeAK=2LXCS*vl zx8TmfHFfGK>c{&^Wc(yKic}Zf1!Zd=(p#I!z#aLT%w=H@Io6^AS6PQvNeR>4gETK# z{MgZA^KOJbP6JIiW*6;pL-HyUvIF4*QxqF=6IWqNJ6j5n?F!1ZsukoBp~Rb@?8H0B zDD(6S{r9%-GhwAB3mt!|GBx-nz!k=^56zux{gGTrQ~UZ>3dcrGkHTDjx5aQbcu#k5W8?t5EC_wxZ^W(r2zY`XzEr zoPkJv;+`u-m62GTrEk`aXc8KQ8c(a75?%IZKrvbasI1c0LAl2QVRlhss3V8DYnJk=S;zG8@-WqQ$_b9hfeEBMS`vshKB)6GB2v4UiFneoP+j*yrL<4snt1 zIlHgW&a9d%ntLew_U@bp?dmJC_56@{o9qM<5z4%Ny~~KB4y8=koA92fwPuBblpJla zO|)E6*G2qH3rv^DP1>4A%ROtXE%DFiD+3&2+F(?h^$G($UsM*1k8s-ry4&B+j2|iE zGUP^B&wNOpDUZ%i#!ymwA}zJldn)W}uvn?`+dUjL8GQihL!dV=HbsM}gcybqw5ys1 zRC>?s=SQjrNT@T*Q|g!+gdjM&-nI^y;-K=EAJ6s34^*X8!dM3O$`icVZ)-TkB%3UE zodm%Kw>C(9`Dg{5?1G%fLq_aIN}pNdp1h1tq$ubB$5s{KQ*)VNKX-`9_O@3Wlr>KR zx}Sn0Ky=xEPedK#z%*YDefq<{Kn<~9OCFsSeSOM1IR#rnMiV>vzS8kCG#>mM7xwBBcRuTa(roBi@>j;8H0fmU@-K~YoFV8_;9H{v|3{|}p8pK1ZS74A|Ap0dRt~m$266^= zR_2cX4n1{<-vA=NBJNJggz=6Be;1Hr~s>9KrfJI}uvG7Xj}dy{zc} zqhLyX~LP zGJ`ARsi4I7v|skOQ_ORnUhSO#EPcG*8$a9t)Br1_o(%H~#fV4yZIjt8b~C$ejA~lv zo$tGBV$TFNk137T2ISHlY7|PqvqVl$Bnn3e7m4E{<i{1n$>KO;DG~|zC!Kx+a|g}=QpjMut^lcLm*gH>Wp%4bv~)uc^)lrv zunzq4@fL3~9glFBJe>M7?!I^Sd0;1EQdyzXeKqz9LjazlD&-v9JwooqC=Ik4Ip`=B z7PeTYks{I}(j^n{Bm_24lIX$$=HEfpN^GA`Tbc)> z2W|^~3l7c(Ol$cESsqvdNKgW!dVW-{3?3U*Wgy>2k`lf6k5A4NWoLv9_aH%_r=4~{ zyDF#6B<7kE;8JCv&znSz^n@v?#+nK1gPx_E5mwtllO|2Dd5^lz{)mkIxpIRDco8+X zmb^{HgR+IXYi9>aBO8vuF*vqh)s!>t-giQ**7gXBz7dVNbzn>>%C*zQ96Cyn)C7s) zeIa+?MO%uViil%}W2qLQOZPx&YX~nl=vwaNmh zMk+G&MBd)?*L9Odxzn&B7M^!XL(n7RCSLc@-QWkH)&c71MlqD>$q?OSlJOWsT2U51 z$inCW#=&aIGI1YGbWS0g!PoG6r+rdm(-pzOu=AExM>=k?Y8 zg%I8a@B0airV=Y8-NZd9>W_b2l`*TLGFJTF#MQt4X(QAAhrseT?*7e?Bcdn1VFfWj z8m{m&hZI#?XE4BFYS7;}7mX+;qm4}?C^jcU0>VexZs#vh83S##)ny0H(!eQp9qZAT(u+ec!0^m#*5}w&Y-Gwbvpo#W{xwuBuz(c zJ{>fF!3H&c)i2m@*eLr)82`^YlfSClUsX=P%E{8)N=N_SP!SQ?BleRQR-jC8e9#$= z;~C;G_Ho-n+83M;H#_=Gzp+j#*&L>>Zst8T$eo9aJQ)g%PwFwpq!e4RW8=g82gf!W zKX;#EAb^ew_o7NI*q|qP+d^uE&`tqz1PXP2<4iz)OqQ;#ou0hsB*<2LWhtxJ^yGfLR#tmh&kSGIP9;lWo?%zYai&<9BMrNrjF^gD`@J!?ZEUv!bzsSw;;nmE*sN- zv`H0gbu8@+4QvG+4J_^dtCv&Ml*dp+^@M&|mQ>+a1+4d%%SP-?w9iTK1uBy%2@Lf1 zZm3^vB_z=qs|`tmBA9gEVc9$8WcCc-yqBX+CzGkjd-?>U`M~VBQS+FB7v*R0+}~w& z+q_GE+H;v|e|e0e=>nz=cR+)nM>c4u+++G~x|oHH8^j1TLk;94cfOadvo^s8v7eGDm#^O@k~} zsBYbOFQ`Qxp8=V5Ee0l)+IXT>m|lfgo9+{%5-w&YmhxLViCL+o&b(aBp8np;oxwk; zy{fFRxr~}UP&1d_=y%ISCc=n3YHZsM@_<~px#MU7z0w$UjGs`=@C7BmKfb|Y2gR7W zAe+f#JeKhJI24zWTB53Mbe+MF^@Q&P?5<)d&6MUX4~rh!0Q^)aJE9nBDP>5b2|p(2 zp=}RSIbze@B|3kYi4}Tc%Ee>4V_W}~A&X%mwhmPYYR7^SAywEBl-cM=67%uwI0l*) zhsee9j3-_Ff){ARdPVFQGwC4Q2?J2jnU=1rt=$+!Fz?&dhtBj@hQ=I2?Bq^WlFNu> zbEbW*-LW{?LNGcRR?Mw)g^$7+Mhh)h>e!58(kh=1{aGe)t@^WiW-x^%Vd;j2JRZB@SXv(rqh%+HWF;}^gVNVnrzcempirP)$# zi|mUd`>Y*u9TdXeCOCg_y=$ssXApkpq*`|v1ZSKHmBF}#VGAxcl%e zJaS;ofY(B0Y+5WgZ+TzrH^e;Z$~kT!!F^5Vu?Ft ziv@)@QP6F-=bN8k&jbaJ2yE933D*$t&qzl%7$NuUD^+^;vd@`e}jw!qTL z#U1%&(&;8ejx%ad7{~$DsDR~h8e84YSWNQ-rqy!-6T0Wq54^~{5o&qZ4xBudd-sA~ zSV*0cOreb$h^G$Nq^J=mbGj$Sv z>r{u&(Nk5xzikXXC?ij{u6_qs{RbM< zzai`Yq=XUDZQ$SiO!@;7PDI2r#HusCAc$5SqLEUoY-j+LJmX_iU<1=Z>u~-Y0 za2Wr~NhJ8YQGE$i_lJfXAF46Jg#zMxZE=HDVpBT_R%rj7G;-gOdG_-&-0{z<8-|Cz zpw&=u{|+m-(`-}Ig`)Z5(A=BW>GJBpMTEcVpHS3KwCeBQ{Occqoc%vq6+;tq0~(?4 zm;c3||Lc;!(#BtIAzx4%`g0d@xYJ_Q13!Rb84Y|qe~{rHcz zTO#=Zy;lZ189>}?5Pgv5wnk}XPEJpC(Oc6f^pUYDfV^_Y;kzts!di=Vs{>Y+4)5fKatHy1mu)xa=6ctM;!_vX`d={8 zlN-_&_x9$4t5c5ew1>JtmNbWyhpMB>-+*n(-;h<7JwpdO=c`2H=a5kFu)l060P{o!R(HIuFOrClD zeq}h|^LY<$KARqS*<7w$>0b&Qv)+EmZ%`Dm3~t6$7`i+0loY@Uc5J1#(1a5=Y%x}; zJiLy*K+n7wH``lp7?X;ulI}Y6Ep4uvd|y2WDm!W^_o7-0SqApS9B$3bJ8Hi|Q8@SE1Tih7;)EFI~UXi0hXW8426%=w@u{6%jIPE$=qF!Snsa0HOlq@AcepM|Dp<{Aa>7+@v61sa zj!b$Zkl^0)_sm^qi08fk3p%GMp#klUD}1M99wO&_xW1V#z)U~>JS3SLwvHR0tN-;+ zH67<9S_zJA+#PJ!fTuhlI&%bRqXp^o%~#;qHRPFeQF}w7UZ^NI?QWm9F>2pWp`ej- zU~1(FPl7t3SZxyc45ag_Y(H~(ye*a#@cjIl%An=W;6KFpir2G%j*8XH_=14CYOdja zTX>zpJce*Nqizp^SxZ7t~VxeFi( zrcds0x5blAiS{kPZPA_FTjC@6kbqoC-?hX{un`vGJ4s;ZM>vS!?-54A6{`2JiQ|y! zH~TTf7 zdA-(%HM(0nXJ7Q|GmbbF6jk&c;9HtV(F`@{`So%;Ihk^_ZfJD4bcMqW0QrY5(gg|u zotTMZiMC*;AW#*miji_2FH&9X@+5e>Fm_FhEq;9gZomPnp}CO14=bbD5zC=j2RFi! zh*L}sI1|_84=4;2WZqUcgFPCm@*K)BBW;B~G%-J8ikhymSRwt|Vx_@qZ+a6JSI@MT zftr1oK?}9``f5%><7lW}3sBs6qk8f+LC4d>Pky^wbCw4B434XW(O|ir`cNgyu(Zd^ zPF4F+ptXl_cpC7j#N2wTN*n2TaaP5-Yal#oaCB3HZ|f4qwEkCUv!ce3JIQ`xi={+) zNTxdcsgjv)g5qlJ9mtUIi`#@-sl=pBHPxWvL2E%hWy)Y)?I(FMcK)9&M_iNooICME z6J~ai)+XUMoNIB|iY+I0l0ei(0TeA14C_?GRKO7uywSfr3@D|k!`eGc?%b;zbuX^B z&ms=)8k68y$8=~*-E!>YI4jslAuEc8j2tX!hQel%sTXdayebTvT%_a#jwD`_?d0;P zL)R+=#I-;3X~Q*4+|U9)$Eso_X%qv370YeNe1D2^Ub(267zRWiw&`xhrdJxJThh(g zml2$X&i28|GRF#KoZAP9x|#eIcqEMoxra-g>i{X1e!}AU5+e>A%BL#EyCCe4rs?^~ zmrMV2>sS`uVH#>R*0C5+G6yE#CqN z&UXyje?KJqpFJNAw&pa7auWX^LZbYptf7qb2@R=%YAK2vz$Pkg9Gf_DolHnb$deWb zt1Al3=i@J>9+9*I$c%L;b=eWeGtV`v($Kl~n!Q;1;GU8SIdAQ+EQ%f!F zth#_lzRUg~e#Oxiihv8paHtToFWKEDKF+Xs>5#e<*xwbkC_dfA8rzG}6ghb_j3YoDu=ecZEXkvyeY_OXc7%2 z+;~wCS)PZAwf?FY(O5tx(Uoaqbd>V{eZDTybY3>W~2z z)G163*PU-T@ftiMSrRH{6GIC}b`!VkmTdMVhvOD1)f}`c%GHA?(ojfPQq*NBsEkHA z*i`tobY)UyUIZh=hQU-bIkYe?g^uDt2#dj@4OyHG>4I64eUEUwq2*8_{RNiQtePRp zT~>ExW>0DM-ltGV>y4UnXxFnBa14wN*K zIW4bt@la*1fPFwU5sS1W9mPy*$^<{7&bh+fBh16<3`crw5M*ZAHlj2tlo?4-UrZxp zY3!t-IU6wet#V#$ez?4qI+f*+@=7M_@)5^a!*z*tZoP(Aru0 zQI|>!GHck2-T?0r$d9nvyz`hEw-30NIV4=GFj+e{+`O(}*fNAF!gs4rpXf$zkv?V3 ztL+l@l6b(?ChZ_eJYH|KLKbklaD^e?Npw7w&tcZ9k<;OFVW3nz`SrNJPSXgWeP=t}_|Dshb@VWH(BRH*XH)s6 zD${JSFklc8*V##&PmrQ-KBD+KAo|rSOLu@X^7&N|^RoZhwUGn6^30Gu{hpOjP@&Wi z(YyJjt2{ZE@uH@5xotuZ?tpP_ym@dhH)>vBA6P)`l$hR}kt^n7=IYmIL}-_RWL+U1 z526_rQ8zz-kh#Gm>PJT4vhGN1yi|u$MgC|zUAk+m8VragGmJFsyel6`oew~oQV=|K zbC&-FH7|H*ja%z|%94_3R*a_&m(}-awk+j{vGP{6Q(2~<_zYhYi_sE+`u&K+s74~- z3YQS|!Q=(=S`S9Fhg>||u_$>r*$B5EahhxMkal|CI)7k3j^}g?Eg2^ZE8_*C8zylo z%mGdhgVlv><}UZRw1B-G9~f{n>>tp~&xlM3;XkO>8_b-}xnL?gpa03eA5g9iI5b0+ zfvw4V2&Ng8B7z;uV86{CGQ+z6rYLIvnReMlr`qo#OceH zM)BhsS4B16ImH9s9DQksC@>M`yd-hVK^SV;qiq41jv%O?_d$)!jiUI7SF+xGpVzfL zol2h*7g0Z>2&)B^*26`4b-;{#Y7ZlTSVlD3VM8>UJJhljMqCBV>E?(em&g-pmKx|y zYC>WPG}hnZz$WdU9tZg{PS5D!KBM^Axzk0(gVG0evq_mdM`|{nTVe!El^>BW*#R-_ zY-rF9-1gnON0SSat70yOzBvx80V;GDks>y?c<(y*VVVxiZ_Fjt6P?FmZ1M9>_GLEi_99n^?HwxD5GW9a=IqL3i>)ySb&Q2* z&diU4c)=h|bo#IxEl@(RhLhN!HBn(r5zWIQaAd+YhKz6%=C$|faTMDiZ}sxC4DGpN z8qxM-nAjezkV8MY)Z2w5Qg5Sa?0fl(wg-~6Pxg)8PImc`Tyrsfh88K%Whx;xm*eNV zSqn1lsX9c1lwP-Q1=74@{G|w7AY`SXerqJ3-*%$^efeenkLcxjFNkncd_-<*bMJ$-I72(KHc@W}r8yrO~ciGZ5#tr+rwNI?785|2?)eTp# zK1qv?EE9kDAam`vv`ujyxv)NsY{1}ry<&9#0?$y9xse!14aHF`FLSQk4G7IrY?r$U z0~a3 z6>i5sho4PmyWA~gWSDiY+QXA$!`|tPWb~Y_@h8I?dzKmsbsyc1j4apJKe&`%pS_9u z)zxo-#kxKnlO(k5`<`o(P_b^NvzoSM$W)D0cghZbGi`epd0o^OKgIW^(c}E?e7^^F zr3?`e?fN9(EWcPQPqk_}a4)!0w(N6cm4+sJQ|c*2+rKS}r!r;yZp{BhG(1kXej=%T zxaX^WEpla5$lx#(ezDqv1!PgDr9~>^jPGHxhBUS?8+ff;0Tr~lY1q?XzC71X<)n9i z;p8F-h!*(jsB%zieFrXwp|m;7nn?k6?nqQj>n8iFPWXwj6+75h(E-r0FR5M;EU~>^vl{43 z_C(bX53-l!u#5pehcqDGpSCUb(KazJd#zIcmtbX>O{EPBg`GO9EVTKQBCV};Ekqk_<|*aMlg*x|Zf{+|0jqh94Bk+@Xho3pq) zT$@KaMOqg&7WA5*lfx%|&h4RJDT@p~47tN_*G;ThKFm!>pj!>;F60H-4A-Zow^LH^ z#4d9UKAxMtRJV^iRBQNG#nuN#)2Y?>0$_mRC;I7VlN0FU)yv`$8~|squjL8qeReNF_saZ! zlNd=aLEG6sq$0glC9|>wi=)@YbXr;`|E?h+AHJ6Jhne6J{Yl?r72XsEVuFs!j>{CvWDcUJLL_0+DOd{$GC0%K|5C}?}7^1=v zsXZmXiuC?VkF|bJiARLmI(_b*CO9`1-KMe2)G5=5>Gu@rN1&Oqh}uAWd-1b1;&qB$ zt=st1=9%J`%)nkDh*bU~?2aw|*eE|9Y30(f3Ao-tTfHFFN zg&SMi@#)wZ=>MPdB^xR)UJ46HUz_^$R*4Y)Jih#ZgT=HG@W61SQp>g80&zb=vMI@F z7-&ZNMkL&?Byk%TS2G%w)YYk$)yymGYb?wS{^YB*$(lVamTF#TwLQ&oXFsMhGo+03 ziR5QcC-0HPs#Zt&eAeXj7`B75%e! zk^bDMIfPda_5Kf>nU+5%WdFGyRm1W{01BJmPC!3&pa-OEAgbR$^tCSZ-Gol3*kPGK z7Rp_YYt0X4HygOXc9n2-LaFTx_^f_S_^ci`G%m^;QtZTM$~mr}HzKa6Hlo3ltFY(d z{yi3A8*msA!W-<6xn}uY$W79iUMzU&ZLW|y=d}Jf=dylzj#v6Bu8C`;cER1U=j{F^ z=U0bz&0P%Vpng`&glFPSsvAfy!W+s>!W#?6J)m~gT|o}~8~JsD8wSjzXI9IkXM|0n z8|8N1T`Omq-K+8|9Oo>ambD9Hm1D=Uzyg?Joia~>2Fi)x6g(dn%O);*$AnOyNSZlOGpDPImBjc*Aqe7Mvm^zC<$o=WDUZ3{uyAi<` zyA!Ei!*&n~c6ne-achv>nBSvQHlR_!+B!VWC{lYA|kq_x5qW9?&dZ^$^M?v&a`k(+Q7Wkv=fV_zVG2EFeflYM4tq_%5FkW^A}TEnIx+Taof}UQT5Ng&I7K zSso3}lQ^68T%|5w=q6`VtHJR0A%5d0%WD;GBTr7XLX=N>STML zf!ht^W3G;_1lLHmM|pA`6zjliN~N^_B+cYyiaM%lqlC^7?H;j_2C=TYODjUQZRLIH zpk=YkDs;)$PpQP?_2W&v-9T*mmlpBj$_xFPLi?ft{lz>h^D*7VssK&lqURjFF}suD zrHVyq%Ax*Cf_b#u3U-vSJ;vFro|?37vo=Ge{K>6gQ_XHta?w;{jpUI7FV&w9^Ek03 z2qyTPH58r3V|p#{aXj6ll|ea*yJ(9+9{pYUnBnIY(8Pc(oSjlhQ0Dz zFNwo4FWH7OoZi(9q~Of5YmiKXp5%;&;39X9#y?H*O&<);#QwxcPITVbA_+v!iBVhS zFN*u0vy)a-R-exvy3XrR1E62f0xu+6&H`~r?aad)DDy>}{%l2FarHMXS;+NU&Z=gd zr8aSh)i$RrW8oYla^ae|txC~%7GZL$Cx}M~h%Llg;Z1QoGaOQpz*_A4&I@vser4uS zuQ0mY&ezPsG&Io3$~H4QS>zQRMsg_ADb~|iv7Z>1UE7->ZGgOu|CpRR|Zi}dx4XCT9AYuqmQFv%i9bx;L6$wEQsE`E5YecV}Zb!Rf!L>6GioO()V zs8&%(nB)6|;9z_c0MM++?W0zlkpw06qrt+IM*LpV=U}e=7|gs~rYf~o8=O&FB=h;f zI?`)F;;Ra&$ldkg{=M=$p-Y&g&6``(E7hdbFJF?b{@cR96(zY~bwQ&pduE(~?^DMi z`PHI)By?>kT&fCb3VU)?ApCcHDtMRivpownFBCEWsW6(CS+*>i*K2ha%`Tj)65lll zd~ei@IlB$G>mrEMUTL>#^JC`VK~Xg!b2&tOR`2Xku<-VtKT|O-WRDM1a*%^c_(L-) zRsAcFJ8oVB40y%Qo9S$bzA5Q!YOWSHKBI_1X3K{!)f(5OkAkwoJkrL-$^h z1gg%EdGkRH^B%zZnNhuFX1zZo>A!^$(3%V5eOd>3T**Rs;LlRUjJs0ntw=61@;}tZ zk6?JwE`S4$^Np<3EQIeiWdz|Rk7&1srbhq3qa7$FAmAzJwaFQ#q4& zrw@T5vrpV%5sb;Or0IL$Dv~a&4iH+X+>^m1*}-KX(RLX=lF0EByVcQ<;y4+JyDq41 z2tQZw=K#nWI;v)1FK$!A)X zRKYY!fi>26`&D{(mXMm=FGBI)SIV)RG0Ji7zWJW3x{zPlh1l7o7({F#|57wE=gTOB zq5)+_|7uFuA)w085T6E65U5UwF|XgU_k%7$!WZ#6?r*)WWnaHsq(bAqi(IUyQ0R4% z##tc`2w{dE>XH0TUq82c+^aRPFW)G{gPPw}K!tb9f-dKQADpaxU1_htWm*{3ZGuGH z%{10^#h~FuMeCfWCd(pSNcZY!BFrJai@1-vIm9ngqrN%_2&RuGX+l>Z=CeqS z1KyCJ#{D}j&Dv%~(usNI9c}VC_bRfJO^@Hq$o;;@N!ZAJ+UC9o#N!&qn!`yH5Gp5n zvKWT-Y5)A?Ib)0SY4YRos;U#1*0+InGU&z>)dk3kR7J^J{ku7_B6C*nVF%w-{6jiS zN{ghAWDV3JWv{0YYDdlnt5^eLgM?QSK>Kaj5t<)L8>nEPs~r$}q+q{O9$G|OuFDOq zfXTx*FgINw;=tP%9B9@|IuMUICpT`KJd_^@#6dGa1QjM?rqXKw#Y|hc}X^miD;jsI1%fk%v+)C?0VP(2LfCgi?FV2%$J847*b`u*HmnFiUR?Rvp< z-^4n`oI=b6ljKAO?E?&>qb?iul1s>1r)P&kW{lXjOJ&PZY8;iC#+2=py`T^J5{ahv zKBOJNs1<8zNmu^D!dHWO`lhiY%d4W!P4;<)g#k13uw6NPkuTbm6WNmP zLiA>1k$N(yaWaEB0Zw&_i|;-AJ(Bx^859+w`_8h~G{5H){#<+a1!dF3?cUgVttieX zNkZytTF+MMlrKfGx%bK47mR#a-{Z`sOju|2J^85lhwifz8l)zC<2B9PidFi6zqt=} z3mM56TB}jO#L>6ZcezPRkE|M~o~jtAe^%tfs|hX=O*xCx$JlZtC@d@l+AJH5F_vvm zR&d;Ss>dQVI4BtsX*d`+q7Z89|Gbf_z3^Y!M8xerSAaB98XBK?N?wnRT5?hljY>S`pw>=Dq-Wkh1 zPztIRFA)fxwydP-aa$Scg7Q+W}>M())2+LJ;;gWFJYez$)2D54e-jmHML%M5uAMt1=+i2yFgX!-w=kd!Bh8N%sOn7Aq)u?M5`3(v94GO1v3+&d+ zxdz-Nv^XJpYToW?(ay<|7B}wc_s%xV?zVKgp;wGUWVwa&3C>^boC%xy6T$aPjnQ{B z_J6MxN&Y{ybN>^F{{80foY3#fJN71)Ms|OrQ$~Wi{2UMB(3g6?5Bz>1=_b!FN-B72 zUwo%gK|@mVU8QoNkx{=iibhMJDMbFsyAy=PwtmD{C(w zM@KY2iN&=~07!OzjLN*~7_jn^l!Rw!0@nAaTH&YC7l3j}5@Pr1* zL5k544(K7xiHOk^>;lb_Bh%*WA`Rv{7wW=ED^ThP4A2uRmT#AYDu$OX4>pHlR}wE4(mi^ql0Rx3lP)HA~k7vi9?uCg-*r z4xqhg7giCsegD~KZF?m_bCO!H<{~yw&@*sTJ7QmNF&7Irn44iL38@Z+MG(t?`PNSl zm1E#5Y50LEo~ft_ga%x>74vSI)#BzSqDh`e_C-k)HVA2GXwYYX8I0ZU&(8 zNjDCqIb*NUM%J$Xkk5y2{k4WfV*_?6e!c7EYx_4Lsy4laeEGCTIZjr5*q}AR-aZTe zQl`n>$L{D38aTX-gef&12nAg;vZo&1v%5f zS5XUV9W>x~AEBn$oIHA^QsV^no@uIv)iaU8Q!rOM_d%#0TDY4xppA!jJ<_z(eEEq= zN2KA_)h651@|B)Y}e!VFElkVTt} z_1{;6B8q8OGXBa{SWkfxsA+uQRi{u?U-PX} zQaAPhObAFWop1cpu+TZrW!|8u-cYgVW|j4{xM_D-6ScCU&phpk!+OU(dB*+4dc=J< zb$bpH#SQl3?#2V|6R-Cx^x6ymUEUu?;5jK67eDkE-o`(e2tP4Yg*X>CmsGbd*i2A^ zWOo3+F?g9=S42RtUkWY{e^8)ZtltkG8~)}DpNn`G2WXvS+Y_LRe0%AaJ5M(V5C`QS zxo!eLY@}-K?jayd*lP?xF1+nHKrW(fI)E;+?KwbC@op=KXDomy2xq?U!_N0g?o-6ynr#TcPo8cZ@Mi)5FUoD{U+*d2UbcOZ&fQR9+h&$3;4rN_?#})_d62P? zo_*U6z*e-*aXnUO?@1try?59wKRb7QRId$yrJc7WSk8S`)-#}B=$w;#yxjgkV_x$R zaSUv$!YI&*A+?_Ya5ueQ|Mqi2egD2`i~1hZlZD!T<@4jc|L5I~4KN8u^^eT|!P+}^ zXBu>E!s*zyb;Y)A+qP}nwr!_l+wR!5opcA2XYQE~?}vHTJ8RAP3(mEwj;dXIA9(5) z!o2G@GHbxxdFI)l=4XDR4({{ZBj(V6Uv@|t&^~6s=+%XP+)F=JfDxbE=XsOnqzONrD#Xt-yQ2l) zHNGPb&QB8t`H&i9#?I0LT8Z`UM98!bU4tXJ~M!6vbVr?O-h2n$p zdlFFmQ7C{9%rt2aUb&<@a9rB`*t66-GF<9?x^9KR_(KmU{zx?BooSSOA<4n*m~Na? zi8o6#SL)zRhuhoDgW(TAkn!J5AC(lX&CK^Mx^m+BMVN8E^)C@XnemOc`W``1dyfd=RLS zfPQU5+O=_0K(tY`@PWo>V-MKwTVS}(y10)CMMfM%Q}-CM;zs`!GGwI~6P6k$ukO_# zEg8;$4-C2itt3k)JK^`SkM8A482cixV?&i#T(F z8R~vlIaY5`Ll8SPQ!5Z8s=87RzSS(K(aH_8T(yLuXmKEG5LrXR$(0OfbW;<)Ht>S% z@AURx?C@n}WO5_LjG*P!#*4C=hKotCdL}&0KrM~5IF#cqt&9%o?o$-fS7o<~+1 z(xTs1pZX=^rcP*yoXiQAJy9vr&i6CK>SRN#ucZh`C2^|VIwtDSke5cZaz~M;dXb}Ks#}3v-+L3SDTxT zp?`|Q@LqpyXeiTA!VskJ(i!34M`tKcD)_G=N!v223Jm)hq634s>u*kKlSGkIThX5e zWQ$g<*)*Jx0$REs+o%LP-+YLLdhxhwDV_#7|7Ing4hySrXdfH8#P1{i=`o=2}$fw zW#K2Xa$W+nWcaMPTd4$XE$zZI^3JZ7R5SKM4b$ z3P+uq>fFNOW*^fM?=PS_&OyN|kN6}6g`A{BHY&4%nPRe~8lk7sgV*$sUr-sEE*War zS`NKPlPkk2w#I@s~JHs`tEEZg0Iw2sbVzy!P3SA)Z3RMCdlcq zGC4tc7kWMjdO#G+agGxthfa4EAHN#z<3(dj8glWdmkc#u9=t`yZa@E#SZB2` zNlK9R`AcW`ct&%`uy25}2ToQ!$J>rVq9aEds7zZWvpf!q#Sgl0ayvhpnmzBzdx&9h z$xSZVIMkqt&@KVB}m*G>{7Y~WEopw-?B!r=5 zrM*a2e=4lFcM0I+T1$`ph~?v3g{9>4jT1*zsHHD-{7?On3v(YBj~h)19UpGgx1>D2 z-uc)@xF3k~fshJl;j$2B!bz;h#FBek|>TW6yon9%uT z=jaM}$U^LJ9p}P!;qc~B9AsgPbGx+A9oKP9=uk%q?b}yCjP6#dd80cc>xiz(`jO`ps1=j zZ3ch7*%5x7p`~+>wgp5)n zma&#IF*4EX-c`UJG?I~}#JPkE5hNu$daTQQp~E5zcOxXA)4QfLDQ;2~ic<(qa$-$X z3zE}j9GN?fdlajctBM<(WY~DSwu~hHTB-TUI`4{+#8lzqPlmD(sj&ugB~qgJTQbCY zXm;l773mJ<0RkE%-wo-3qgKFUq_i0k>MR#za);>}#bz5U-${xyAtv>Bw@7Q1T1evG znA8Bv#SVLFTi9xov4F;xFq+GzIBMmWnF<)oe9Nej)cdCN6gOp(7R5?8FU6QO%u6O2 z1CKnD)@hrOR+-FPO%wfE9O4?<8iq}|pF;mM#^Sfv3b+IP%)%6}uOpV1OvylN#!yyf zYi+$aW3xG26a5O3+^I-bkJGjvWguEiLQ*O(d-iG0YDz2zQw`l5kIR;!YbcuS)=e=g zD*VMYdTBE9Gg8@Y8d_Zr_jk&JwNO`J#y9p_+O zG0PPs6|WdaOr!p`IUH7HKJ1l`xV)5@jojf3J_D>idL)=@t#pysZm#12ZQW~Z2! zYOrv!2Gu~w$7tT($MJD}7UCuN@1!svLyuR?;OcV+WdV9c(cLxWT>hH)4oFq7^wH3L zt@Xan2w*%zxxm{I$ge?uQsq>Y4{e&|5IGSw6NqV2^OSNoQ*g@X%0cQf;_11~Lj39N z&%rulr(nBrjCq}Gcu6MfUxQUeku_nT^LgB;<$r0NII`5-brs*5a&n2iNy3{2Zcg8T zX>99Pol)fABftK5Cf01|lIO?f^zANcm%-_uZi;~MmugZW#6ap>=9sx`jXstfQdVtM zul?3EZz>9DufUv3PIFxxU=xR%M9MUZfes)Pw=Li5^+4fo)1ER`s5s`ZGSMkz;IAlcgQXH0FY) zQ>MjQnN=mZT$7b@zlby4O?}ZW6wp2gYm6K-btEN(M0R`DB2R?BQy_Q}@HseO2ND z@$L#~`oA2Y{e2|bD&Z$!J4M(t`+P^%tK%O!c~kp5hd3gjMcyiS)d_fddW-J|`0>Eb zKvagW$vL1(LTG>h92nbmr44BOe#tYRm)+B!pJ`tB!*0NrNK8HnrYi6UE6Xfl4Tp(u zgE$CsN3Pq7@5=S;9$2CHl3D(U_57a=Eu^gv7sTrWmDVJ$>_y{)$uKrG^~HPG8~#K- zkGQR@qFmw#e5;m4@@t<8K8hPEBeRk3OL5QmHZ#61jz?2w0m=l+Vf(LMNE|ZVwUFt| zm(LPIsN)80-SGbv0KZ)F0Aum!h~9eyc<@M&*khQbl~uNH zbgSv%gxU@_nBk)s0dY=7l)Pk4Gy&;`KHO)`fVCH*?uNU*gYVe2%Q3lN-_@D9<5 zLLTkq@16Y0rriPiwG-^*kq>--lX9M4)WY~o2VZ>H5R88rH-vqwVnY=b&Rpp) zJp}9(#Gk)B!Ut@z3xZ+Q3`z8>JZJ1@>r0myA$q`;mDat}l_XsJn7uGW8)^Lme zWQ~0zW0#GXS3+crO=n`{Lqon1eIh6pr}BipU`gfr&l&Jf`+1&*s2(oOa>PH~?{ZZ9 zSf)?v$lQC*DYoojAa#V!d^uJ{@=c4RxxA41d{Fs}%PW?_j*8o71C27}9R2*F<~AtP zE>T-5RV)=^n4qfu?bBhbS+1N>YbN*u+}9|xTq)lv>YNKKh@$Y?DN=2up6*aSmswM4 z_{_$3Dvasb8VQO5P7`(|uiRA^v0oIP^^-#`%dZPHg^#6=|f*Ol2CFOESMZ| zv8~@SLo7rkk=@7&-4SvwQ)GHTWm01GCS=5R8Yu||LC^q}a1*W-KV1SN#1jL^ss@w` zM;M#2+WBy#sL*_F0^8V^wM!z^dg2fDocrvnH*Uzj_`BQw1^C7ltKp5At>&=}IFoW5 z$`v=on(geU7Oy7I3_I}k9$b@J@ajabznrw(pB|j2hcl=TZS&yQVT<=}N~fwdZc2UY zb$sOypCR8+Qn(%EhE3ZL_=HRD%AIr!isBd2M=3&B+jZ+C3PJHf5fBx691_@i(XM3g zc3G#7Rqj9^&EJB^YZ)Pkyje*F^M2Nn?OybdfPaM{xW5;1*FkQC8?r{vSgxy1m@U1N zW_h-!hQQ_Ykz3z7)=lyPj-0uZ9F`e1d_e6agvJ=}Hq&1)&s$OG8Cz=! z2>(^X32`qoZglH<7*4JLxKho5(ps1&Xx%u(pfO2np0DBn`uP^g<4E2QV;~nFXP^=s-z(5x2)(= zuRyn$mdf{cT$6wo)b$8)0O#A;(l2Zhw^{R|nM$@d4l3{8gma4vgqjZz`kv5TNlGK< z0uoE>&%=8ly@~qotzDSTCUeef&i`DUPZQdQIjUrP7fh?dCt0I<=oTCPwua3)#fEXE zqu05>hEVSu9Wq-5^_XWx%5#$qGS^Ao*Hj1VzR80AcE}Ce%L=1gb*XW9w>bTY-J8y} zF8n+>YHK+8iH|3~V=B2m245yC)(gvJn!71yeHuS5);eXjn zk$umcAbAyjgQj?rYfe4mK+Wd2CZFZD6G-*wPWvtXbQkCYC|if9kt~=fB#Pgeraf@%p(#I5bw_j>Hpl;{cqLX|EsO^!^*KV{@<0bd#ixH_#$_LHJ)>8rphIGHTJbRzB&dFT-{&o1M4rJV3W~lKq?@@%)8OjKA zgi~kHS!O6SWC{dT;=+`>IO})BV|=>c2vX*9$@wwDC1rWSkvO#R$YD+xu>|4FVPpvX z1+vItZvb8vsHtPXVPbZIDK;as@I1BkK;y0Hps9bDr%XL%)$hCC5vWqvH%buIU;ExV zF5Wx_%jit^7*n`OQ>Wn?EuFrb8aE!^r*~@_ULJ+Zzfk47_+{GP#mTXogcaxPQr~0K? zx~=q){-iyE+e=riv~6(KVF~tU$7_`U4zIGOZ7a#(jApW`UB8+J>N~OJ2$5-74+`@s zR6_oZZ-f>`wHa7+=OGjo5JpP0nS~W|;$nG}JyVuxbhe$@{LT%h?IG2v8deACT4R+t zM_&u6&G;n#^JX0>PRZ&X^HVqEW_qpGd@4oe8xr9bH5mV)%~f!GNRDuxDSoy6+E^Ta z;utc~CK11edTAps+!FG=R%d_eJTB>>mAqeO;MUTI(vo0NeR>Yz4pY;D@U?_Rc8atM zHVKc&oADK3>WVbJG-1>!V>SuoJS!VvX+G|Y;izRcAbW-T>p=B_lzX}UD&Z$MwwNf3 zJM&XTq2YXmY?&yjl zltlsq(iE5sWIS}&g*231GdHb&^!JPXK~OAXNqe*2&JdVp?O+y^5`_+g&1W+&*_q3^ zcm4g|p$(wxc{1+ydLu$2Jdxx@Xd*nQ4l*M|k)(;!HVFsqlNqyPycdFg(d(!>ZITVK zbsUOPi*-8~FTo&$bz2M-V6Y=5P{_`cF=h_~c*MrZ456IcsfnO82cgXXH5;p^9u)gv zW~ALBA%y^rQ}Z*Z(AAgJBH~k2T55aEuqsoO64=mPreUGJb2hgZ&kd;@c73X=8z&M) zb3GW>Ku>!T=SeN-+H)~AwFfD$_$jMK(o^TqnySp-1CO1RTAhkX>>ZUDF)Z>dQt#^S z%5!KPL>M%zW zM`{>z`>CpI3p%Eczl-aiJn~>`%{hm#0aekN`t2QLIrFOyu{?N23ZLy@%NOoQDW5D{x zet*M$#tj~Xr+sYL%}g3*ozh;=3U^als7Ncn1X_lzQ~MEPn$t86Od z>Z7e$4r%AAiyk(-s4z2(>@_}FWuI%PI0eGWW-rYqmK}SQV`9_7B&Na5#8u4cK{Yq} zy!bhA*uowJHI0x>SY^pEc6LR)iWH3{5H}jTh#ZVr#H-A|Cmz8fM07?9V~VlY=2^@< z^4ijGI+Hv={ejwOEpCLiTPO|oyEc8YtM?bC=H?0vSGkA&8@D3Aw^3IxMV_)Bowjgo z>=Kc_AY2oZoxd;_hZ{f=q41^{f~N<@C^RB?XDom@)W8s@Xb6l%6j6aFBn6qgOj6$M z-S-0`2*`AGpvQ+FP*wl8qWb^)n}Yd2pej|h{n0TZ`nA&XTF3_$QBd`wQ~K@)M0KO8 z6d+6f7pnHkOWRTaW4g}nwEOy%NLDrn2zT;t@-bcFzXF43ecIQ1wtdaa%=6aYpHIi+ zfrK;E8OjWV21VDY6M$iRZ`kbFD|MeLybH(~QZ+)6wt z2a1q_TQG%ayi`gaWTGlje3alN5)~eSNNDu;DkB_u)fu9FQJkczCW&hI=tH0+je(+5 zbQaALkJ>Va$Go)m>oS{-Pe}tTxtyNa7`js{W;W^ef4Z}Y?#96RkV}kbhyhlAl~7k( za@e)!nWHzGWmto=z-*UWIyCR@AY%8`H8Q!km_qa9;W{%-Ma?dzr*H8%b9i^AS+GlK zHcJ3QyeJOc*4W43C|HtJJaU1f%fzRRU{6ZdwV?(hmMQebvm_@nzm*LkgjxYZ`7 znmrb}uhRys#=4-T)*HQhRe5;V<2&avp6+^m!*(<0niqFIvZ1=W z3V2MC$9>=RA0Z$ZTJ6v$z`t{hCrn`{7}GD7O2>|B7|ljDXKSW#z>Rhga}LPd^^rv6 zz^oO@ByqEvT*T&k$7NepHr5SZAUX{rQp2c|2^T=qp_<8(rS`P7$RKOIp0ZB=bkqSm z&xHjP9ENyK;cUv#iORHUR}X?MI#y<4oSj>%)>G`9?BE6kh8+qsI3CwTzjnO~p}bpA z;VrNXtfA#22Q7pph@U#Izb?6DM0iLm%0Y7Bo#er3Xw=j)S;6EIFA2Iwb1x79m&(MR z!LN!osfs`VEKs(YiT~wT^dqf54WFUtLpwHBM{4%Hy+$|k=BU{^ zN6<-wnqlT<2-bj85z}Zf+|@yUYklA>nSkLPac`toS!hb*?U{k)S|O|C>KHT9npIu(q%HG^v1=8|3JjjebrPFKo z&p{GzTdn15w#a2pG8DBLjFyvo=Aa+O!LVI z(WLRDSQMyklw|R*kG+WOZ$@}T#$5Uo9d$&nRGe&)N%kOSor{V3i8F-0&N>%JT48BZ zGbbc7a@-|1JmPJZUuj;gAP}hv0;slK6HxT2l9=u=s(f>Dx2iI75F{%w*|;uA_)UcA zR`x~O$zbIvM-6mktiEe%cCIc?+!s;mCP>e_CDmEn>pEFp+ei=|yb zBt+TNLfg0FW#iaR=J|MZ1CdX3yVvN zeNcVdO$3Ixp~KK(C@_^1lpO>IL_xNt{sa?&a4|vC;Yd)h2fCw`pmp2#4_5>lbJP=V zMi+b3V}R43FlQ(*m>R*i$xI&_gC&#Q>_999e)Bt3%+vDS9Xb#IFAttp3!pYPa z?SiN!!PI3Xwpbg>cIpnXgVV6P(u_Fr@Mr7qx!OwXx0C8J3^>+{bZ$~>Ln#vUJdNW2 zpd_}ajH4{4;`i}4*^^v&~74xs6OYep{0=saVNBlBh*X~CIRHMc`E8V z95uZP(IT=4q>}iWp^2Xk`_XM5bbTrHAuewjQoc-m&$?WZG?aN#+zeNXXFuJ0IUJU0 zP`}J7YB9LvM3!C6t$89az3L?7#aQwue6wn(!o%D)+(Cu?^}LMO>d-|S#-r&N)xs&U zVnWS|RlLXdTWJR!DAK1E&^>C&c*0@D+lh--p~k*Poqg*#8j+mYRYuETNtik(gI>1f zIW$Yo97#_+RSeHmV@_q%LwaTllG&%KQLivHA9)bXWbnNf$-~7^ zC5c71kK4Oy*E_SLI|Dj>|DL54SQhFleTpx}x`g;KzlCJTPjE^18pTYH%b(5J>@S-i zF&;rJOIgr?EA+?eeiFERyDUD2H-aa{Cl-wLKd?2Y+LEfZ-f_K?V|$Qzh7|nhe;j;t zaC`%uVM*v6()Ky^tPSoM!ysC~<-Wk1-fi}loKD1__{FWnYu4KGisTcECq4fS6o@s- z`^P`Ab!x9C>86PF@Fkm=B<{&DpHugJDT}66g`Wtn`ebZ?i!}qSwjj2(K)yC}No|n* zghYFk_(qd&xHnt1u9tHEhs}|gLr|angZ#n&wwO}<|G}pE|314_?VSGOyJ{k0>R@VT zVrpmXDe7Tt>fmB&Z}*>*AvK{(4*16~;yX|m9sQ{6peG49{cOXUrkfQR#vW|@w~PVo zv|U(q!kYve88$KuL4P*0IH3<@H!{x!O&}UfKH8y04 z%@k873gY_G1Yxi#SObX_w;N(xCYn|3*PS(I@4Tz_Rbhb1sX$jM|6f_#68KQmA(F6% zMs~z6Gy>2bMs;tmtN{+Fjhw=H#3cQ=zqECB-D>e%FRlzqeEb-l2F+_nQeaU7$iYpw zHom=gR;1xYb8}>T2;syDuw8){?RsbLx;IhxOK{%BaS0&mBmcJb`wZLPR(+n;M#cW6 zgzL0+JeBiro)3fLspQiY#!F-hdLnWDVNz8kr3CpSmkBU$$SYnKRpLJAY#<~iCP3K~ zu^4l@7$p!0Kpljn7#7I=D;cF(&L=#%?i0!x{*d?#go5H_5QVJPfEcWe`n)T13U;!H zkQ9{_Um}f)LgD-CzbfXO1?5|_pCexG=ZgP7AMwopw~PFrWt{Pa@=+0e@OOPBy)``n z5P@*wpF&a)LF^|C2_|S0lM)hQ@{muE#>kodc%Vmg9(Z->MY9X)-d03IL!uBnQDAr9 zI`^zDKWo{qTBrIibDzjb1a*Duf4upAc6L=&|JV1=buhV)w;csYOgm&6A@$uOQ$FEc zBvU@=-H^$B(&zUyLi#&urhMu<5>r0e<2h44(PJi4KHzbkDWBw##FS6{g3Uc zPyYms$zSM1e)5Adlb`m{+~lifIzRdSk;z~1gnse^kl9D`sBZFAHLaige#zu7eBz({ zp3UShePTcLftT4w{rGC~)jR!7_lRKfg^=k_@fdINC7*xoj%={ z@Se`}Eq1ayDKt*3i|%WZNCMME^LMf+pT^0QQ3PG|w+0{@#?-Nq<|bi;_xD>h4cqvE zo~93BG=k}UBFz^`w7x0KI67S4|7R2OBnT#+9eFfCbdfm|)lhbX?YVn)pm|~pv(V(f7oqIP+Lxip7ptSUBm1Lk6l5!{UhF~XkZ@%j zAO|3W7Sw(;R#0Cu4QYE)phdN&_9)%K{o+~!GqVjsc#ZYsQC@CoXlf1CH4e}OR-ww8 z%mWx|cS@ECwq$FPcMJpkh#vewJiw_1`~wEmA#3Z9IogJ3bEKFO3=swqXmaM8kT1yH z5_ix8IH0#kK0*CWkiTmkz|GY-l6j@>GSI&ITebITkk4q`I?!g#tReHr-mn4@2sy@~ z{Y~by_HkQ-4G;#T)i~0xiPvQBum>(dZ&5xhLLb{32H&rQf_Opw$=}Ea?A6{-ymR(w zsdc1c^+#u029HxZgFXkK{mYiL+;|7Lg3yDWLD`W$QVl=`dO%;1w&WWU_kIO22yzbu zgUTay3*Au-K!D~GIZy}27SIba3PK4|4p0O!QR_^6S%g-vy{oln=^BJuuX(Dqr|oei zJ#k?_Uan<=&LMqRg`&R{qOCMJL*|jbWdMaAkh$Rv@KJw+@*0O`uhW9og7OpIF@hS~ z?iqeJ3WEHD4}5J5jMo)&Ao+#<)-IDE$PiO75AIiWWxh*-7@B{&QIJT1y^aT(jp)H8 z$R`*GCub~==wliT)V5X-sVnV(Gf)o6GX_q*J_V}3VZgd6bq6`HYcoIHir9?ABQh^| z$1^YmYLDD4eTP3F4w{eh5z-%y$YJB~>)#4E#=|~adq$5zu=JWfK5(R~-xS5*Y8Um0=D2cPI#h$s@YzN7=!00CO!guj{y z`nM|5`iG+E(S0~t;+#54Gi6c0ZZRYp0PXW#AC)lx6Sibk5H$FmGs@vE8qE>WO%V1t zHVSjJUVYf%E1MYpWQ^(n(aHxj6MkX~JcvhRl$_(mH=amY?(#L_2- zWb7GDJbG^y#0qLCiaQAsg*g`-l_7ze{00`q5ks#Ol5DRuY|^WV6ux+g>X$`a_MnaC z2wJZ+%%WEwaqbgIY<7Q$#+aR762bWaDvB$a`V+z=n)29$=7_3a7UB7UDmr~CPF)m4 zuOI^W`8VKR1tgfthXv4RNc6`;NHz*1D1JqxF$Emy$VVnB-n4oJ5wt#Gq^S;Vq%p;v zdlfI>S411t;m8zsr`iDpp^F0oP*#HY} ziG`(H#)TOX6LW|;j)heiFs2)gwGtzyPIv~-QMM=TCZ;@pSnbXN)*Z*?59HhqLNv8c zLF?KMOeCLMO;&P@PAn~MI8u|SqNQ=|x}0v(V~|%Tesm3?!GiVc1vT{5H>$p>qNJywsUEuK3m1L6&zbxI~k& zAWyO1@=le@9*=L$RU1<&NGSvk`71|I?bhW5Vhkk4BFzl;!xUpK^{ai*$1p6O+k#g$ zo01pUg@-1CgNeu3_@QtZaXJ1`)}__chxFOO#4={{bJGueF33A#Zza*cg9mrM^e`~u#z3q zx+OaZGtLoI&^xmx9ZV@~o3efdbk+EnKFajbAL+@a!e5st#lCcoZ=WG}HLRQ2-cjlB zsrO`?U~WQb2KS+}>Zo19uYmHzN@%ccGtiJ`ZP*rR-iWo;uUuDK=Ali9fX~`7SIj#d zL_~Lzy)0pi9JllDb2}0QPu?1pQ!DofN9DSDb zz5{zpaM~?hHH(roV~I2H#w0n$t}rdOWYJ5qb6huGnZJR+5hPKvin4;TATnDhT@hF| zwsCQK-iEM6T!NUTy0{psaH_ZwXOR<2t;CT^yNaJ$hp0wSwy;aLYTpUGBa|dw`dE5Q z(Jx+iCULtwii*hSrO}S0t%0XBo&a$|{R(u5xkMFe}r^Em?mU{|d@W?@fy?4mTsm2N6NhQecgRAyY$9~qBGb*VNI zAz4&XyIsJQS#K-NJu1c)FPW6*q&6$BTR-$nN)M9fjLeWqK_%%-`4CNKJw(Hct)RO> z*L-y*9#Ei1zfsw^qQt=NQ`yjw(5iHD%HS22v)WB$~mr%F+4N~~nqmZ0c?N z@oAInco`JgrYX%Rj1HJ%i3R(Gu+g$od8oX4J6O{hMS+5Cpzz7vc>8C1?72I!G)wa- zf$G`C{sxerF|c(TRkO@`SX=>Pq%^m=(-+NrnRP8v6pr66&fF>^h-OT;(K=LP1s

Gzd0ZL(_;kZ0)e+6vPdkqUvBDA6_Z7h5!u@e;O6tG(O@7|VUHF9nEfJ(T z9c9Co_pcr%gTw0EHXG8VOT{7*0)b>D_P2TMv(ul2x8;vIAvd<2bOHxz-D@dtB;Igun)zF&mp`8pm9xu7aHV(q zC`vFpt|b;%GoAi5QlQmcq-%2>7lFAGc}tp$opTp@cUz*(+viq`pJ;qxUF*3NuSxh0 z`ch-ZJ=~#yjSV@=E>`7E?=9|9@tI+QnEBUoFbIMCzXp_^5S{*U+dQvPu32bJO2br( z2+g**?H$|ug8DG50Fgc|02PMjvg^R)*%Yk$yfVuAqTA_oJ>_1hb8bf1If{}zj?j7C zCFR1m8FNq)?KdwVI+`f)z3ZQO-huYWt)O#&ps%|QOD+)8lz(}6rEw|<@5A|XnY{Z` zy&NAvU0TnUKwrCm?RPxO58WAB1G=U7IT$8h`d*iK ze+pNPUPhB%6hBxR+WP}=><&4A95f~2<1X@>1!}idmU7b9^-D$fyWJU{7x&xF?$*0a z{}V5+D=>B#(y?SwaU0diLx>~@KD}6N7oX#U&ponuU|Hwdt;UJlN{3H8cT7DRt!kUd zMP4QD3Gnt!0ghEX4WsyWW76M}EhIMv2@aC8v0Q%fBgt(V6m$nOq6c7=dYLCnL0pus zai`CrEY-OTqhp+n{1Rqv6!r8<^b03#CB%U#!f2M5mh8FpD_abts^>&ItWzvjfD4_4 z^aYs^MG@&cHuYH)kJ}jY0gqbT*W0d0T`aBklk4mWjd{K(flRW^SWMVHhHKYtnTZ>p zVQgAdsm|Ag!C}1#0CU;CSVsaY1ZT0A6e-Symj~h6EGWmO6!H`A;WmjAcA2z-OKL=% zNFhxkb7wfSu&H|a*|rOUx6!I3ZHL%Ag4gi5bn^9ofUq(Djp)aDQ%6 zwT>WVmh%HEG3=-%iUMqEon9^bX#wtL>gaV9)DCJ3*j_*Rg=QWtAV>6rE4h4`T%oXs zeOA5rIh3p@&v2Z`X3DqVCmU=KZygG*s4W;eOy+xrgcZ0xG- zK)@15Qfc>e`jtXc^35*V>s~idTX*hz+Y);`Jb!|`6D-*i9h)ITZ#p-DUI)-}YrEBb z(mEQss$6|WkH`K~W)qeMhPCay8m#U&8! zV1B^Jd1nuiycB5gka#?PEc}LQbXs;WrYb}%&|hZ^U7+W89IMFI#6dK0wj3Ke7eBbz zLPG4yY7{hN(F8Gk*$5v7uA&ey!kuS02@RkFX$Uepp!*T9Mi>DfMKhOQ^XNr7w&>*G zn;b|iu}o+6yu|=!#+L$7;2{=V{Mv@CaD1-$CCnl$`^Dc+(N1QeSsGc**!7!e1!3ow zRUTdAgPRw9t< zRN~4c?|B=@;)u+~S_>f~WZimIs@0K+s=mvwekWBWC(-rGde3D$7V z;@nlS8hj^RzJ&shM(kBY+rC=KQ_{J@qKR&c+trPV)8kI{@R?-6D8RABa@L{?t33~1 zE~jr+i~23^F?qr0+ZB#0g{NgXoOuEasL8xO(`)z3%dRM6M$9UDy}vd2*93xaCmG!4 z_-xY)TF#?G1lKY<1NsadnJ>zIk&E&^li)aZu3EPS7FBr3Js{DftyfH|b1So#(0E0s zYDP}J}~B9U@E2y^a0zQJQe1wUDjfrkeY$PjjYE8Wzegx$@3=HF)d1{kM%%>`8Bs1*XC=)#n`Y-*c?)p86q>nIa=nJZn}B1;z){`h)mC4;iNGQzYu z<1>}zWXSlc&Oe&f!-MH?6awSN>}_#5`s6Wk9FOt{c@MXF-k9n3J?aW(N#}9dLd`8W z8Qu0=2($0%A3j_t7sbDGtrEZ{R=I{fem-})QO~y()wzZO@|@3#G9Wm;yd$Y@vs1)b zEeW4iP)SN_UBv=hWt1-+Tbl?+%`bc$*O~|79%km9TLw~^T`WTBjt2wykLvHeU32MT zz?nOFVf&|x9gD29r2lY0VHuMxMRhb(h;hBAb@SO9v2~gqFST+OEHYhW&}v~v^qx2A z`ZK@J!)jd#k=?V`K3Sdk>h>U%9Y05wkQ609q0Cg2JXJ)7 z@qSs+2Ma%0Abd{6oSh7#GI99$JR2U8$KknSvNaE))hwOV!zsg99zQ0_6!Y{(_{*b> zjt&38oI9|hD1>mS*9+L05RwL$6Om^qv%P90x0v?8RglU4+M%!-E!|KS_6cfo|Zj4f^m?2zBAh)PO<;1>+%lcJj?DsG90t+Klay)gWcp5U| zgsoRxILXBahIQ{x(J;Po@qKpvF^pM#8HWVUd*OmX<<_O^YS;YcDm!bzk4eh2B+qTc z)SqA)mZGILBU!;1>ln;hi~1w~spbi!MTKvR1*0t!PfO@_#prfRM(%=P`5% z&ywS!a8J^%he#v|D(lT;V+5oHwqndDok7$L8sZQMAJpuQDV+?SmA7t(2_qw4j=lr0 z`%RQC5!Iz*NadZe0fUz>%gO3W-HtYkJr*KabFY6dQH-dc!cr`IpBUW>7nq~`_}b3d z4$ukmy;BXJ7SM*mG8fpR<+FW7+mU4ROh@hG4b^${Z0*Z$#lsDckA$NFaAb#N2XghM z4i*O)*6TGvWfKM}*(8q>^~H@8*0!UI+qZAKt<1kXROaBHF=aRl5awZXdieQ9Y_)HD zRa~}bP|NLMCp!rxo~5N>N1xuBxNgY03eG>$b1-ABUJNZ?FS6+VHt6{;3|#vm#Cbb* zx*W1sO`~j|1tFn00OX$Rd^D~{-?CJTxx*uS zx}pCplb&Bx%gLV^^$EG!MOOL~Ac35P{Pps&1U+@$-Vv)}VNeBZ@Dc<=ZH<;v#a$KQ z`O$hgdZxd%%#(_ar~M3LcJ7>6gz zlX!wFO@O}-n&LrCG6&5b&$ZMhq&!+>Bcjk7=l5nEJ?F4#_7`Lg`#2GcOd93P25Y)2 zWWze$Xxz<_=xZ03UFEkSbY>Z_;)iQ8O{M-Z(W?IV_4k4ej#XOH_IOcV;&;}4c~McT z+s#&3n8CagVofMNNC0fk7bIvI$L9phb>@WsN*+jvsP{(9sYEYAzT01Kstsq3Z@7Fa zhQHSI5xa+*0E1s)b6}#jG_@AoOX@8H#@dD;WIY)^R@BYtnyrJ|pzcN|$Evn6Hlck+ z4_Q!Av9RuzZxNHA}$*bFX z55gwU>2aah1pr%+GSf%wXF;Z05xTBG5(IX^7*x`OcebgEe|||vN2x|LuddInt6#6q zrWrQpmcPI;x8xRbR(W3A0K%c(t?V%Ure^E|{Zjsx<`$|p2bfrbMRROAt#FMC(OGWp zy_;HXj2fA1DLQ66JC>zaSsHp&qH240R7|eB!`N7uD<|0QvYCy1X)d$$NJ`4CqBtc| zo&OHm_x(4`U{jlwF!c@W+#Iu|Gc{pop`N(VW4QJQ*mQHhv+7zQIBMJO)NoC*=yZRP zFH|oAJl}9_c@)~+vWA5Bviy~%SVEQ>s_BZVyai;XTCK_XvLHv#kG2z}Y=lk0i0 zqm<#Q!CWz_s+GA}fC{Igts_>}CbXk6f;@BBix#y|qJ=A*1EN~^C1_ZSNER-6bC2LM=||;901e7pxJV5Y8=(tpG%wj$ib2rPPN!h zW7+_x=S(G9rSxZ%DOYot&f5OeXO#SY0=h|YOn!Q{-3#xiYI3N-3eTi!(oQ2a9WQLP zD`mt_L?pWd>vH}JH3j-Tb_blGnVB5|m3lgigSl;JUDe8rn>$bt$4BdUdL6bcd|%%x z4gjlf6aH8o6}2S8D&9OWCr8butWaW>mj89 z#+DykCA;(b#Io^yMrA&r*S@0i*GS2w&N4cDbox`rHmoKX`oN7i^W~qnVT7c+&a{oN zlDQ!DG{aOH`Q7l*Kb8H0I_mjD-5__!9Tk|R58FF;^VY1=qRj3>{;w5ikn6{`MoGmH zkyKN7=R`ixD?;PuMhzv~vOJ3$b!GuxgC;x%0owTw5f8klg`}l6+sgtX_F}NBDzcS- z7Pd>lRIK*&8wUI{(;Mr^E}9wK$}_?}ZUrbM!xPD z(P{($L-=j~b#t5xFYW`TB|ob`vri2q-QUguP5!CMv9Q}fE&kjvP%aCL$vh=i(+xxC zrF~^k#hE&jSP6T~iq3#pA6Zx5(Id} zGzt(!T@_C7KR)+dkh}uaC9lESarqhr_XA$)yCSnKCS~o%I5-=#=;cFS zydZURhH4=zE-0{SjG8sIzcX55W9>yHpN~YeCP}307wZ65U4$SYA;*7o9q`)Mz`cG$ z42M3Urc1Y3ysLbnhas!P$=BlMYjN_Gc9p0%JJDVlkPkA+*b7xbB)p=J2h}#9LU?m# zL(zF4S{Y*GKzHr4cwnjx;k7%4Rwqwg4ih1Drg^+8a^9()gak2sBnH%qcodl<^ zYrWxY_M82N*t2Z~*)b#-*j*>TLwJGn8Sf33Uz@wbGQZ=b52Al3ean&$w|}E~OG_Ve z=KWCjn*Js8ocgt4$LOm89I}4 zV-zmw&RpIcDw~FH6fW((u5!-Frt1afnj!2@C5y1Uri@u-y7yRuTwWS6>gk{wichf5 zaXZ9XO(Szd?Mn~K7VC$!(V=GG5g}<~TxHr4Qzoo69GjsUB9@zxEtk4jtj9G&i6*Ke zlhw$jQ(lIKV<$sZiuUgxRr%I0^J*u|-t&Ovr;kn$GHZE~X6#g{X2nD_e-oJx7E0CS z$ou=`71Y1pT4ya!VywEUQ6$a^D{B7^kX1*QpYw{8oD+93BP-K|@yYr?0?Eux`2%u4 zA%o|i!hY);F#SW2`H^={*bo@R?=#JwBXkX3^Wfc_B8POqdbgwLowAX6uyKc+HZ*R{ zlHL2bkE(i6mX3MpBV~vniP)+VEq4&=+sAAeA4N1SPa1)tmR_FtYo)S~>)Ya&I#r<9FeHo0 zsb=wQrjjHBeVy2!ik{`!6D7n}pid(%Ta}z6?YyET6ESMVRAnpkRMc#qf{RLIFEs^2 zS8nb+lHD2F1qF||6SZb&QiEO4Yb>5K)kSgSdI|a*r>a>35uT$XKP%4{OUG^NZFaKN zq7l{`2_0oDl6dUHF~-sDCTcG}2Tm`nJ!gSC62X0ELrA(*S06r|Ich^*eaP4Q;(hPl zkk~zGO(g!bnS-fyzz1k+aQ?8HGlqAhdQ+Px?Z2$9chcUV{Ge~vJ^xb&0{;Z|hqiai z4_^Eu{>KD~L)oH`dSvUj^8+=%Fpr3QO1Jx_VG^I%wW&FD{d?!YNX8GxAzt0En~As^ zOWO0{O!P)$bFRgew58)s1x;T-Pt& z(vx|EnlWBfq{_?<8_#YyuEFV*S$L8|Oe74Fvcw%}%2X`PBdynMb*luwz3D*<#Jp-J zGxvVV=GV7Ekg8=HTC_0#13FYc9SE3YPxR4873S&0S(cOJA_O;X?wlc}oXB#==ll;FZQ4WLbgYV!^;b`TQOR z@{S3{e~VsbI@Eo9WiJctFYUkTolYe0KC8RW-yuqZH>p> zaPMxdzpln3YcDR3$lJZy``oLlUYG zzUei`(>Eb6jwk-v#{@T+&#Ldjas9f0f|qdvJYQw0gBmz;Um>YO+px5E>$=cx`sjUv zP{n7_J(dqyd$vzCtr32dl)LF(LEqxF&_-CsseCa>!jeFnk?A~+a&+n>U{3FtZ?D@q zn@^R%V&E=b7Q*zZ~X#*hDlxk<4#cRWk{X(r@o=)YO${ zL6-6Xk%VuyR8-X)ptuqVL6p*jQuvTnKVJQX?gMs-FvErUlwi?aX+9ZbJ6AatWVx2S zmyOuI&LhWkDROjk@*Yu!uN_jl`1~cZ)Wj0TENp^OwDMTtl6OrUA7UmVF601n#n!B~BgA}7L3^Ew4H=0&vqq9t!k;Up%o z*IWh2K0QY0p)ahWI=o*&5Lj|(Ko50rp9CZP6l(u2kv+o(8KPg60XPp0?5;B-U<+;V zCYoXv%`J*CoVN6*67#~Psvo0CFfmuzuXAbRve}frIWbild+oXbr^6&O{@^_o)L+<- zhAfGpj0_r)LFNW{6R07AO;|`aj3}x%`W^jPCqnkAvuwMxYl_Nq*dHz&aGEbG<|wFh z4)fuzTd?ht5svdi`K7MrZ&mRO=j&6^GX2Uv%o`%OV=-5OL~pZ+uMC`X(N`SgP5z$D z8#Z$A6523Nq4O7#_MzRsRL!50PTbF2(+-OVeVw^7PTxh{`?B3hVI6^5C| z5*F`kWrBbWN@3BP6eJ)04B`Ns33GxE$lH~rLM=zU)T%4>p&AhK&_@TS)PCR^Ft$Ys z#3B6(nbE~1Psz`Ch=XSvY{#yHU+2PT@}ce?%jHD14mf)D@+QYM30PYtLR3ay-&w>E zqh3{0n&_L5NtRvL^gh)UEdk3bH{dPs^UJ=OdwHhDT^$0qZNRA?JdGuv`V9(%!XldT zXR3pY1{~*-@B@W|Ndn+cDP6`TS0cijR%YejF$&HGPf}-Eb3qXvInwRY{Yo(!V@C(e zA*2hlXoo52w54JlNLjz%L7|tO{+U|=a2?FIC#bXmd0;{gy{+vY2gItpp-?@4V#EmO z&iY-M)3+55?=k?vrBG!1P@eg4$B)8HXLV(I9emCJ{9@$>UY;2g9 z%aL)XR~tkuwHf>=+l&M&q$veWMYiH#Z2>vPDKa_I6ot?qATv0nB+R_$^#&kL3PZ`t ze_Oh&L@YC?*8aSa%O|s?p8=l{yNv z_dp>50y1fJf*l8`?ULNkZHA7s_MDY`_-vVvPN5qy6+WuxtgWoz)cZ!`q6R8U3v2_V z+ry*Y4JO$}w&9~y%uGvSb!dWxvrHC>6xy}7r><%-)4C4j-umXO?f69=IV$Gl+J_B% z<@_1zj5v2!`pItVa8uOTIOZiO@`|tfo<+tLi?j{E)@E>8v9J6S^ho*G=X-iFF3YHm zsf-E~_=JnRAcY1oISzFDK!NDb1Is6Y-9fB-o~=LMag*-BHb{K}JqKj%x}UL2PHSqx zj+OXd*~#IHq**i9*|X@1D4rtHHz^v2bOWilgCA2Y?|&^EN)%flGI`a!l+~o_;P=w~4&cWp)SX5%Unto~y~(^(R9H&;a0DMi_?TK|<6k<&5hD zqDWD3Xw~h4ijW%BF6?TG2&@_p)M@VNeSRbbeW)WH#1Zp_1yYt2m{4|U80o51I&&3> zSp{*kuf{Oyo(4P@t8rJ&CAH4${EiKFERk4Ap{=f8iBa!Bc*HS%HR}P!g&SiSshJ2h z8zZ8rMF;jKJ2;^*ZD$|!H23JD;$h)|F+nC4m+*<9VtwQNnf3RX&CN+s@S$OrRRMjf z`_13m#G(M5EE9(HLO>6A-(~qAUCAex*(#W**&-LU zu~$^)q_LMic6lTW23B~mn#pc;IsXg+0TRWgb}=@b1*YjbC9Q3ZD5PBUyP%_zMlXTf zh1#FgV;^Uj6Fx){yG)~;w-ETVf;=MFvbAcJy`e2Uk#I`y)2ZKkaC2{gp&ZQb_mR-c z^4vrmzNj(?J3bsDD?i+m8UrH!nX&3I@tWBAa*THU$56n3;~4#4H1efM`_1EOJ;<1H?AK8Ejt2#QRO!q^?T$6fEIVYgk%Rn^iR} z-ec5A8zdG`EosVCEG-o@QCV1OdJ+shnV34z5!U@};`izP^twCR-1fYFOXhjp_4Wjr zz(B|M8V}IqeQt)@2JWyS**xVg1|0LaZ3X1G-}pn>dJd4paOm_|#@k8sk;1f)kchjH z>?Q@u0$$={N#!ZRh{Vw(4H8L2B=dg(3`y)WNrof@;|4gyBU1*z#ii~IhyXfLB5NUn z8y7+-I9MzM;dPr|baK;`v0P>>cwLj$I6o4J)FSW)esB-m>G=VGM&x}aUqU`Z^S z8+}k_*`KKWL3~nznY(cCx(3f}NJVBw&piEu@F|R1qc_3?@0EH?>DW_-t^WE@{d5Gy z1wp$Q1kuAq@T!OV92Z8<5`05Mqjr4#i}1@#UP-%T!L|p6RLzW;<8~(ef<=Wzn0;NLTeeUc5x0CJ)iRt62(=&xd6Hsa14{V1{fzrd4tX%1v!3=f)7* zcL*Qbx7P>TcknyYNAUIdGxt?85J~4Rs_WP@=D1Jjn~@VCF&qQ*`!5Vcox@$IdWs%b zV@6P&0f=CwOd;|+Q%Pss!GH2G6)X1WJ3x;AGd2LG~h~6L%iHeBSWFaAp zayClIHkje&(m|R$Ws3$Av(aD5k%3e(M1>X7;=Jfu@mAsk32`fDN+d~ap`Q!AFBOYQ zBpBkt7pYb+%15U0Ij&0X#Mrhf0TuPrYPzc8c8@=Mi{09EB(WN$DHp{cQZkU%=scEe zr>(~^>LyNmNi7o6L(9TWT*jG9TXaiP9&3318bh9=%e&LWzbP~&pX;&!+2HP{TQAJ**7FI5|9W+p#n8wlp*;Zi-LyIq9eHRau zs;LsWs<^Z=L;DArOj-S@-FKJ5!RGe<}pF_O}1kAp74(;sEnLG8V ztfhM0C_G}!2Z+JXvN!HcV(U0oP@SKYu%;T;amQijQ zXM}B~M!H)CqHOYR{vOdhEt5KbFpwThzaA}D`Ipe2{+g&Et&f~I;1iZKD}!(v)SOTi z^VBr82R$Lt&5SG?XtdH_ye*Oh7ol4QSV*_ksVpwvP>QO~`Ek+`EIE(2tl{~Lvn;Wh zu22?BN|9ioQV=zUt&+s8$C1&g>R?^qZkA1YGP!?0j8rB)39cx~P@n>~8dzAj>HG{Y zbB=S$Oyy-9R|Vlrwz1NvDpp7|3b-%RJQb0NKylU}Qd#WC>eMY-io=x}S@KXeU=$TV zT&gIf&6#nL(^INp=RWo>6Jdj8+9N;PYpkvaQ@Y|Lllw`-+iBjtzDm^c+$noxP9N5+ zw4C74WPugu;jkc_NEN;$Iej5DD)E>_pGcIy_zP!|rX=gq%h5W7=*fe*9x-nBB$?HN zM8aP|XrNTSB`&oPTM{=zp>U}J+Y(h1a6qOUWKtP1z={Jpa4}n{OO%+0$1dS#&%2;} z`b)x6$cM?b#=@f0{%!&f^eWNy?&n`U z9bTx5t`BO0M0dKrs^Cc$%YmV+bx?bsckr|*>7a>EYN=);%MjNVXhuvH zZgkTqaqYIBC{r{D^BI>lWDv~?cA-NmR+_UiO6Mvo zI2`(&!HLgz=TJMIR|Zms48`W**~`2iWw7 zJrUWJP~O1_5{QG6_k|f#^qsrS#ls_Kb&nW7{gB)8b4IMZrdO6;vJg81InHT8ukM3q!*76Pj`ycz~~SF@}dG8DlRPA=5mfB$_f%2cWLm zo2a`7F$WwDxFfUv<+y`EQ-#dAI#SWvm767w{Y5RoEsvuT&QxlkHdY(C>tu?X9H@mS z3)Kfo*s^!kgX5>|7Ncqj@n~qlK@yjA7K?{?7$QWBq_;+gMn(K&r~!4N^kvWw6oP=V zH>j&bGm@*rH6HsW!a9zHlXpPvz^y_-JfRx12kTCaW(ygCUe=KhmNPCiQsjmxs7x{_ z6j`-`b*s^ffvNUFk<6hLrmCakzFwD}z1l;KZtLQxHd$!HS|G$n_Dtiyx4s-b-ROc& zmH6~V&3&Wpx7nJ3fm?QB(>t2v{$1@l7aT2E=n9Nt4WedmD< zfn;L5f{7G<9s!aJ6kR&VSM-xi80MmoW_81Lc{3SZQvVCiAF9N8*YPLdj=13YntQ z`|ZlAs%qtuRBgnkKNkerwy$d5u;;u(od|}|cqH4O@r+NQtE4s!=*%Bcmi4pQ4lh%e zgJowJp9i>aq*lj*j)C*4TE`{r(Ll z^7(H>QP`P+RHet5jV|BwJ1?e$5RAXMfxMH413CgxDQ*>kOiOe3+rL$F_mr{R5HROy zw#BcPcM-H?O@*kp3Zr}@`aCjy)ZvFTVp(XhAjW^ZNKwn&kW8PmJp|i)))J-o45}C2P66jw-JcAp-KWIYoQ$ z1F1FJ5ucRC(cH{9apr|#juhE*D5DFi@goA|>`0v-Wf<}OKUEBZPz&rby$}&A$L6+_ zvF(3o%||D=^ed146&8LR^`RvL#S&&V^rz@oO2;+M2Sqo)@F#*M_&gn~=a}ZwjToOu zD9DXOL}ZdX{wa$#c#*&nCDRl%f*Da7lRT56dKv*n@BKfuCfppn@X#)Lk zSP=hTIPl+N=)Xn;rAZr9Mr8c?cv|UCiu;(Hw?}_!Gg;2OUK99;onk_E9aR$|MWtutt<=uP#L1+8D4fHp!+xK1*a7~m81~>d` z{G2caJKFlhrw(QHHG^7Nb3;Ws2f}V*)x%gx!?KOO3Ym1fwKF zLjNPjYWk|>EVj0aA?yl<&c^P6im?O0<&bM{5It@XtzmWR1#O z@oOM9&7=Q3aZy~aAd`hrZ6aDghs%IoQBD1UA!p@T+lpyzW`0gAQz!)=IvCMv-r@z^ zfS$F3E!XyxoCoAO;URG2PPUKh-HwjY&1hyu1XecD_R|G?{!wf;*nWi6f^E(U1V?;+?Ma4)WK-Cl(r1UIP( zc$L$?&?-J)|9H5d7)v9X{TeC@zS{f$_c6r(Mm8emR>q3%j!wqb|Ep}WRLwnc7twue zNj!&cIb0H}!V?($ejBgG0eVO|C1*=n{+IJTc8+ z8WWp;=UD;_R>^3cASBjHykEgy?^7CiT^rYu;Vx4guQNT9pRy0nPX79KzF_?R>n1zo zj@=vyW#lD3#BSuJIMiaqL%$D!eaEDe0BnogmVB-Y%n^Go3(OIJ{u8Jx)=IKlA4m<@ zinUYj^A9wJv6W~g+f4y%SqqV00S*$ZPliH&*bB(O^g)b5d>sy;Buw}}G5Jt>$fTg- zp?`$#`$xjWBj`muze7*KI@o?A{NjI?fUOWBwgD*x$RPfR#y0+#2f08i;kFlyl^2Ky zx$wvDvB|U=Hp#RcR*{0a#THPwTnV1{H(IOvp6(x_2jd{nOZ;5!Pkxi?Pd<1>+3Tqh zTVtrw%lw?~Z))G|Z@Yg7+ZDhGF}>>uOJ^X`>k5Q`*zPZY)d?)vc^jd2gDLhWfW8$1 zzA$M))IfvP`+)L0{H~?9 zQe-04q}Nui*hn+)qP)BW8=dUp+<>f%?X|mw5=^_hHG8)fiNV|`NXM4VH9ddzrPx5r z^B}ys7Ufl8Rdxenx)#>u#?v3IdwNQ)fhf0jYXwgX7l@+7riO}&$Hh}?ULvm00}Fq) zYKOxgADFBVVpxRh!*v53BwPKsd8$@ne0<|D=icRr0U5{7R#sAu^Ya^zl~oUfJKwGP zY>8Ik^?coEJDNgc&>GZ{db=J5aglQtv|w%YS@+7{Po1Bg5;QYzmLEzp>h{}uN2OUd ze(SHWz+a({!I~jh1%vI%&rc1qNEXZt-KYIYHAZvv=*`1bD=tvfo){#v&_ucemZtAe z!^$%&wUt$sgwv^ND_+iqSFaYNH?Jiu!iRXSSMpV~&1*GTF&p1_CFYS@m^Y9AAdtcJ z5OBTLGvn1-eGZv~Cy(0~DhM-#Umt`(b&MXrL)~DE!nP3|Ov2(4eE^NYyrZ-sXJwkG z{X-}xDmA~8-fCW}{^HCwU@X-Tiuy_~4o@n<(>KZq*SKN<&o?j#S1A2md^ z_s`1Y5Vi}@AY(Ra_ekDToHaf9aWpcab=j`mK%+Cl;AD<2zgV7)P-!yD-1GgKkb+CA ziY=#{v%J{EDFYyJO87~1yRb_+xjX9NZ^4(O#QCFywLmQ8B)78U8du`0p~@LvVQoTl z>%mCyD06nxyic4~T)oi9ZIpyuUA8a_qaJ7ZHWRVMG$Ocei8SlkAv1nMyzwk_YUIkK zR~?%qeoJh_BQcZ^Mde8XhE@bP>)>TyGU^!=EkNWRk8gITuOvqF=g zh&5nS!%8@c>mUzbg;nxU6M?h;L&TDRj#k1jhsjDqF&BP$(Vci9mvd6AHGMp0viQ%T zhfli-Cd1*M7cgOZ-v*x=uE!Nu^C^}jp2f4Z9MG}8u^f={MD!@ZQ=MaH8wUkyK1$uB zvI~m#5C4=lKO~=^U$gGO3Vu;|_L>S+9})x0?I@%?r6)PnGo2zOLTMQ zipjg9$P?<2_9%wG(LR1Fy<;(MLK$+4Z6>T)->J8h6dbK*i`K(~!ilCU<&vP)$Ixh3 zS$3g|7B#ny=j~nNn@DOwLgGg`tF<^xw0B z5?ub~jT7F$hmnA5WpqH$B*R3}?m=kW6j;}kXuV{DVe|?*VRL|t?2iEfw@WmsO^~x^ zpZv`)*k#b1p`yQpY1OWzX!W2bjstJr`h^zujjdzT(2W`hxb3QFtqU9x7VC0nYqE8i z@kRICHYOMRpl-DR@f7+1KtgH`IGpQQmKtM=Vn>;#$wk;=_v4xkt+7yO_h!c`^-U%O zIKaH+h>DI7C}G&!Wxoi1#7ec*L1+;W&9es@KwOXK8-(=SM{^iE{VrIUGsTXddQhwMXAK5>DhR6aHDSgnkJ4x@g27VUW zqJ)r(rU31O>pqM(M$OJPg)<1rs>v?yb$;P8Ml(!&zU6qQ@I-mg*#gx2Ju83nGhtpg zh~AJtl6*{L59+SKJyUz4_!`phUMo4X9H@lvfhI-9EV)6r6b8n?&P0moVqdG(5-ZU9 zZ!XWUs-s+DI&{@f5cSzU(eE0@Imj=-IYKF8sgbv-V|uKR`OEkUtf)lRT`FbDF56v@ zD}jot>N3;@at{$1JroP$m20k1Os!hSt_V2QuZzExnXJ(wtOZmw#l$``m=gp=LRtQ8 z9zF-jrES%Rf!gZH9^I|Ho-d7=0HG{nhGl`+v?<{#T)*xS+X#mAUP| zkJ^7%Dn==4e-&k+@d`<5E(HlfR6a+czd{H-2p|mof`&xR4^7WAoj?Gv9$hSpuBr>>uMV(`Yg(6&^P&brVQ{(_KI!RlRK4 zaLq4v5WP2vjfCpFC#iUFfT- zihFhuh18`UxgC`|8a%zADQ0&8~y#O{V+=5 zUqx+IczEQA3TWtFz4^=3wP)-BjPm|*g|mSGLZ9_VHs{eT?aph5QmNP^f?wjx6h&YIBCUSicm=H>p>JF4zjm2ks zct=(0slq76NYrYNwZW43Hhc9U=?_cxv;DM|Y*nj$&cWrnw)hc#I~oJpTTEr)?d$Nu z-5>t?qo2f29o{tDChZ^byIr5Qlb=7DkM~-eZ4-~AlhW|AJaP?;@tG1`;^pEiJBQ1w zlxti{PDPGSmU0Z8`kRL8}Jf=x!3 zz4Mp_+vhGw>f(e1zr!Lr)bNyL=wDqE!Z(TGpuiGygpDf!P%kGYr|^Q?#n%q=$+>Ds zR0G&MO5U`E1D!fBa?~1JVtR8*M@yoOU?vD@StuL9HDZzMAIBVz#-THC%@S`ys$JXIf3FH z^ye*S^o@B3!L*qG}HH_8;amO!5|v z@-CQ&(wbV#`NwBWAfgmw?`voP`;RjEZ>q`uf1j~`4w{P^aBfPAO&_Xa4l-mR-#|c# z{R4gkr;!N#{3-eqUp_4UcVNhFUbc};--wY(CMQ@}zV~aX?E+t}YL!Nj(xq?>nLwpd z)w7kbmzJ%qr7c18V)K$^RbxxDOO@l)gb^94SgX$y5NLeau&iTw-L$0hru&w;8`RDJ z2>u(}DI=U+`8^|?pzPTdonHlrif&uRqEW<16{TDzSR_s*tzsT7p{UFyUSv*{RXN9$ zgrj7kWd28jTFFAeJV_#@;$A&SM#-#6&=O6fV&(;wmAX+T?*Z0Y&a6NXlfo%aFar&i z(y2}`109xnAy4jCUa25s5S=_lwVWWKVqT%3CfZuLT#2AIthR=KS!)qEGcYms8qQsM zgD04CW!Bw-88OMPq&^eXj1ES#c|6P1qBu2EV! zfFJo*h>*U0XAqT6X_pYQ94L)xaq~MiB+zRag8vHrwl>0gmGA(Oa2wenCAK|igZO}t za2woVRiWqzAK^4K=Z^z*!f6~Ijzb)S}%>#ouT+HG*Guk$Vo&#BUsJ(;R)_gtQB*+DsN#M8p!YDC zsF@LJ96p>fd31D3CE7MtrRq3cNRypuoJEB+5oTbvNQ3yI8R8$7bV*^Htne0AE`;tm zir>0sPF&~~EHPpQFK(g*5YJ5_rlBK(?^*@I&ru=~#(FRa^QDO0iziZEg|q5R9&B6a zx7<-@z&w#-=xZ(YTOLAcpkx$XpDgQ~__{#kYVC$OY7Iz$qaFOj zx+XGg=>+kSKWln$>Iev(jQtIxvi??)No!;{C{- zMPQLAz#YuZAR%E-5G}YAw23RT8Ppdv7Z;aemnO!Caf7xH7w5rYJ+zk+7iZFEInj(1 z3(!9k*jxzGwx~0ljSjpIPH7ZQh|49nv8sDbBHfkigBM;4LX`;2j|(iJ#lO698sS;| z?oGy(N0M0rwCWW7%8zYhkxBygl^tQgXP96z-yvpgmU0wTkk zBzT&%I;JH$x-DtiWZ3e%_^z9?6w_RvtW5?O{#K~cac^pLHV|LPo@9MUi*T3>pnh-_ zNXrIK7I~0hvF(+NYbYoEMJ@g;k4Uh#4kqbc@Ed$99$fJV$qaWWmF zy*}1AE{G=>XPUI`$&{+O`yLu%Mb%5|9lWzGZSnDU+s1tCu`Gs67PHz;mc%&9p;CsW z-23rrord%6nQ1%b&gxl}CT;6jx6X=(gsIb$>2wG2`15{laYP&k5fM6qytR_xus~Nc zEcCWfHb=fQ?w(q{ImhZgLM%*?ZM7152)_5ibzAh0x<&tq1nF*ipSmFf zEnKy0<{!Y)%!*?U*F?TA@){Tt3BK!gS@0@u1megi1T) zre6rJmbG)Lx}vKAedQPIb~MYhgf?p!CN1xX&^_wFlDck_T6V}(-ydY@5Zy!~4b<7~ z`q@N#-2qY~1=2>F^XC|4$oPSIW!P^I#eBSQb1q4JZZHy(IBsU$JxSq|W0~1=QVZ?y znTdTk5T0UPv5+Y4Y<0|qzJYm8a}k{x=K)dFAqpR9)zk+3X~`x$ILawBN!lYnt|5iN zJi&Yj)CUgP!m@IrA}+v~ck-a2CQzd?VY0?353!l1m$%MnSRB9fNdGqh(#%QlGW%2VI%BfMe|(4~)3%w({@k-Z3hsZ=VS^^&eeC6f?CL8c4lZaAN#mz=Y@YN1xUo|5sx z&_6r%Ti1{YPX`4-*e#FGg4l63PA;x{4EDG+hvT>Qd!>)Xp%jvmNX zy^nEAd&HMglx8EFHK3YU46LeRNThg)UUY{#(PyI<#E&J=>Ks44p@mcxFJxBQU-t9TCvpF4Gpeu8+c-yjhJQdi@Fs~>bo!$?l}Zuy zE-HJwmd06O=;^P|M&rplA9=?FsDGKGriJMwM^$rDgfI?g#J))TEcQO^a68u0;VT9v z8fBw^VK;%L} zUmzlLU=P?CD>h~DD>_###szL02j@patpk!V23@i4TYaMT!LblqxBG4O zbuAr*xETZn0{JjfvVmibVJQ`&FjK(!({(x>p$a4;O(>Lh!eORA& zNDpjy7gTCk_!+~AClupC09WT1G`&9aurWq2%;!2q^WJT4N(!S#t>3;KdfqHzg2rXw zkp#>n6*)`BW$(^66b+t{Z)g$wOBS1I^9b4-)BA72d^I&ON=Y2qE}l5He8~Got>DDS zq7Z=8-y|mb)YYQo%>{5R`nbniT2%GgbkDKcM|!ht_J-u)W-%#_HC0gGxhZ}jM6YK4 zxHEPXT?u~ks1x)DRBlCB!uF6|4W>YI+ZAQxR3Eysi_*&_dQep7d#5ha=sf;dix}S& z-sE^fth}*LW`#=B-?PgaXrx<-Y5)x{pt$ge(;c7({3$eBxOq;EN`!y$)3UJ7{KUAg;<~S-?!?eeubrLg21fX zdDrSa^oEeFg&1zwN&P9FJWKkfoJ^OC!Gz}%+93Em*Ckrs$P#T)xYnf*7MwD%krEe&ysjaE0YYDm0wAtRD<=^>M=9b3NAxI3w@$?EE zvg1aawr}Q>ZQ^0e7?v^I;=#|Dz7*^bi?{()?bpG%RR+03#x&XE8t5dt!}x6VbyRKW zgtNcQ=#Jj4hxmXQ+JLkqZNc!KWY&Tukgn$lhqZV$e!C$~HFX{X5A{SI4ca6qfac$& z=nRUxP=)Xrl7Htc-bv}e?cQrpku;u?nAXGk1dHzi6B{9d+Nm#LzaaJQa(L3wnkFyz zQykD79KRL}ikV(;hPWw)oENEkezpR-6Y9N585>hYa|HU8}=yhoGj9yCY#-KP9zeX}TKZFQ+Y$hin^9n?VRaypq5p zN*%9ar1!0Gq8qay?4KAd%sEXE5@5)oV<@t|*(tW2R6Bd!tc&q&RRFH87SKp+ZMxa-S;r)IMCWXIRgZ7Ax90FIrX;s`q*nxYpKddRa>54Mj&?O- z`i?tlPvvos8FZqU$2<91%>&fJIZ5+!@a3vcCliwb_(u{C&Gq~5;Y)|yjy~-UB*%;* zwx?W6dOp?PP3?T{6gv{MI=gznxmyKyhX;^Z>vq0B!PGd*4fM#weH)$61|am_@A#DQ zSAFAJv*kQQX_UM~9g`?* z>Kw@^U!|nFaikp^D0~?~DfgH<*`!|i#9nxCs&*C(l8$Ox!HIttJ#&HBvP`kE2V&Mo z5q-ZlqTB6j=C>z}k0Bthgk(MuAY1z*VRBZy0cPvLHR_i(JcdoSJ7DIc!o;OKo&DQ> zG44pfE1n7VXiu_MBN@Hyp2^x<#y>mH_;6Fi4&dH0f3S9q3)RROjw{Xtmylh&RPMRDl8n(jYgt8We#^=WFrH_PJsE4-wgu$& zzkRDut2i=fQc$@o@q3~Zy^QCx2vY)Jt{5BnP=(M-Bgm#u~2$FAHTJ4d3HZ}E?={^%XU7_6nsmL5N5JT0_tP88*Py7i(~611JSYHj4o1u z%GkOIz{JSzvF8k$;&Fpw8k5m{nZ9k(ccDIh(s#)(_Sw5=pD*dVEZ{J{H1^A`oD?5R zLMYis4x|rsoY* zEf7B#zMm9+8On}?&HxK~$4smD=IUGW9xqi_^cz;!$W3VY^`1QX^o}|Dc7F*LU&)@j zm%`u^5CWaQCpVmcY1q+CYw&Kv6C_6kNKE#^h_3)7!dC>M^5pH2^5pKBK9`2)?9-ul z_wZ8rjCSho%Aj`-^;78#byE36OjkUAr+>#MPz6G5)a=FZyfZ(2_{)~MLFC8-sqmG6 zI5wuOLWAdx8Q&;Lz^&=V7z)F2F60N{x%^YZY|9MTY9Q-W(H-kF(VIw=Ci+E{7ztb} z_Mp8~(BT*;e!PP!`*atTNOSL|F8!>O={bJlCTsHX%`!wFiPSkuq1Rrz*gFNCPo}e6 zEq^ed4aDYUjayKKo*0oPWi4QDo2~o8ilY?eanClaWr)f)%_xNyOI5I!7d2hcda%s0 zBVV%7@|HiH2BT^ZuaskJ)YzZ^&t?Zsjf&azS7sYcR@>06S@+Wlw+h!cRfwcbx%#tg zmOW9~zK8sx;V%rO85h~KKf!x^rh!-&Ucb{e;Pm0aSeJU7<-61=F)J}doDvCi>0`;y zsJE^-LWWom+^8QNoCJ+45{@=IUn_UzrWScKkuEM7HqPS2csYu9pU%MpksL~=PuDO^ z7uRMMyPXZmoq!TI5)5>YQjHPkLpfeC89G~%M9M-Hz~FI+d;ufQDvRc)Si~ZyC=@gE zYL2E&{-b?<``BthvsdRpatP_kg7VbYrF|`FXU8%;ToiNMK)hzg8HR5?K|xp*qsg^u z&FPUT?kL?;5&|H+8Ox}>G^%nL!JFm?Gva&J$fz$GTwl?)oWt|(_)6^x6}ZVWY;&K@ z>|-{7Ra_eZOz&FhD4Ar;Aza4S4$jEQd+!ns+|RSRFEaaGG`$Q$?oN=drwXi9WGph` z4MbhN_w{d5k{1)mhzk1W;4`Sa;5O&l1lpL7VfO9tFJO(kW`p$d&0m(5W7w;ZwOX8% zSY(>r{@ibg%#t~_$FVfIX|Cn~Sq?e-mC?coZO4i`pQ=DH!%7j{ zQ7$vAwp?HIQKaM`(4O(PUg-ZY{Sy`E7#tu)frx*od+Xlu!yMQ@o@p38E8I!p<7#ETa>3I~v! zV;VOWuAh=iml8jaM7kL=i3cC*%>T|S0&H@&+jS|Rw(iO>kbnip{)=Cwxk+u+}V+yyL!^~i6S6?2_;)zQM ztzhopU~Hkuqd#laGt+|E(dkwRSlVFU5uWzoO*65^AyYkskm2btU4Q0|l2<%kKe5ZE z1SRi|qkj#!)=THxe%hA@_N8=WHl;yw<5i=g1Q2v%<*oJ-8OiNM|LytwL2gs@%Z%p%Q zH@7A}TU63}y`&?TaE(idnZ@ri5XYn$kXM7xg*SfB$Xxe?lvS({yorek+|HE`J%VVG zFvB_*+!F{S$c^h^hpCX0SqX)$3?!(Ki`w2X#NYP%qP=0RMUCiX7tLj9g;+2`5#yf{ zsrWAo?n0qZhN^C?nmpTa%}x*Higu3`)F1?f)MX#?M+ddb4R#LKGVlZz`^VzUA)1jA z;xeFHr$#U{g7FX9si>{`D44^dbmQO%T2{u_T7%yMLd69rdHk!HF8Vs;OrWP0GdrMgg*NyT9$?3aIa=Ve7(bz> zFi9M33yVFV$&)stJS~eCVs@Hbp&qj8dXRCn^hpjt{q*#} zj}zzT(sO;GEyW)Y1c*3Yps52sZc&_GM+UCZ8q4Zq8dZXCn%9vh$~5}RRZNtlD4+Ca zdokXfmKGTg^=VGwUjxP%)}Y{u({Nx_0*N04wi#pyCd83BVOSfFVGUvypWVZMB(p|h z(Ht4RO`hGiHbnXv3{@t%uN|+Z41yo(XVyMN-3S)9!bW7J5)v6ityiN5ucsS8wOqY2 z9JQTq+;X*a$W&n(spdx4CJ34{llK1M06CVI$c#bM`pJx$4~3&}B8X#u@jF_x7UJoL zfOvC~RwF{p{8>ZeD=1yxc|2(g_MA1`&fMw}z4ZXa$6Q9WU5)(-sQ zY`ZS?E8mS;xb4BNiqU+&Be|PuiV+hu`5+Q-O{y0S8V7DX0=f?3bY)q?D-u;INiu>K zjkNo1=nj=GM&gn@$w#PUIw=m3*@ZZctk`st>Gz6%OwxDr*9>VXfV+W9-cShl>afJ= zYTSSi4kr8p;<BEi}d2MS2%W*J8DjK2-HykGssxe~SB}u(J z#&Zx@po~)2hJxZMEqW`l{8ch5QT(wO?ymOZh*` zx5fYH-DUeP(xd;rH!B)DIhos-I{x$ioTMl%i_3`YT}(+ySmc!m`Zw0298lJ$w3m!e zRZ2xj+DN2;*qJst!0NCRcNvw~y(HxO(-+NP2|-TJ@hA1=UF3G8Y4)%$z3(T8-EUXt zj_Q5s@3s(#Um)Z=jqPb$@PFVUSH{gnZXwF3bS zbjn|=(lasH^3xnfzV>3(Luf0~xAz`{AUPONQln7&4xG;+yy$82OAmRAopOqU!Ur0+ zyJX8*k3uOu@}$5xT5or$_%pJnBg$~IN*< zM>H)>GmSrDLzt@9@ePh}>vtc(vb{Z`W zUHqIRQeF$J#sa;zZ7DAi(F>q_>QBFDpB2)Jnso2nz~g3Hoy`dn+y7LnxWJ>ZU2s?T zOnbcg0bRL$2K@W(&x-!sx}!rj26cNY>Ten4)`vOsS4-Mmcgfg~9QmCVNNl+Pu44e9 zw=pmus98$7Aws;1n2dV0r{3qxBikC{(Wh!gw$Vdnissh)Nm{A>KX{OFS*|9<5+qA+L3F2O6$ZX!bawxWyohF4x_O+T z5hs(83AEQ5Cqsc=f}I$^|HIikM%Te_;e&13*tTukb{bnJwr!p`X>6N~ZQDj;TTRkv z^1koAv(~J+xBoly;jFWAzGeHj_kNyjE2h;NYhv%f^F9&bdq_G~j$H(aptl~z*@+t%D)&7u8Ak`An? zH^bj}QTw&WC?ZvmPAD&FN238yYE<$gtio?(UI{SRRZvFkxbTH|p}F6p=irri@k_wY>AJp*5*@Fqg*)9hh);E%i{Y;Y&> zH*z+nRWmzi{De$!V;5^((*IuFqP0=3WBO3Y%g5)xFG}Y*i-L7|Zc)s#{ZT|sSu~Cn!ZiC!g;(7e+ zlfV&XRyrH#n8H5Z&VGCFx_b{R5FLS{5$@Q#;G#C~@V$3bEP{z+?#NSU@_f;pkXz$C zk)e-*Yt3@bjSOr*kita&b3UGiL)&Y%!Glb*Pr_TXH*+eXk?~ra6fZ_OgsEmQJ=*q7 z@tZ0FTL^fTu7wKY`9%byi+qZMUr&^ zp-7r#8b5Mkf)Pb#q_oWH@$VUVF%L)5;ji>iQ3NA=1o)|1nT;c8AlB9$yCyr5$K1U= zeLh!*EbvIV(@X{S)PEP81O{_rQ>z(aPAe|e@| zh~b$*tAQ!F!G@WGiEbnc+7b zcW|#=;)(5q`G#40SVoT#E_)nNZnlj2d-d2OfivZt(y}r_(%RvdjRR}of@HpQs*Iki z=;JL6a958%?w2KA@a*oKCkEVmf5ri(M@h}Mli}~$NYcFo^unFNBCOCT3jhLXv!%w< zZ^Mf&c!2fdSGE-p6@S>2<$@2sbakP(cnvb6pqAAyk!|$hsbo#VXmiHFu77E&m7jly z|3gdFf4oM-{l9AJKkF$`$=CAZil=vaLCXtAI$Ps{h2dQN;uxz*!$2325b9S_Ds{x) zhombJT9a*dO^|oOkWg%*p!|OLQW+g&qLU&N;I=mSwm-+=>biV<-yyd?!#Om^_7(Z_ zeu4e+0Lub<#RF4ercO%`RS3d_UR{w}0jLb6sY)-|X$Y!^vZag*U0to8FP zVk)Xc5Lz$jF;%GF{|aMvXg+-O_9+lLe$axE%?qra%JiRmC1}RUyzys0ljh&a#rPVl zEa*168BT!#)R_t2_{q#t(KY_lNlrwq;qBg+%Si{Z5$Q(q{Cya{viy#vFOAE{8F*%R zEPWqDcuR8x2~^bjy>owMC?0f~5iKs`z9@sGmOfne_8^wWzMr-lpKFJboh7Y9aS8+0 zd#M2Lox3VmpbN@-EF~xtn__-)#}XF{;{`0-OTb(-iHYjTnrk?uK&14+pgQK#x$kui zeA>ZOGjd~5!_ni}6DH-&!ufIz1!;i8xTkD4L)U25GN-RD5YD%%?g2~LQICQi`eO%f z8!Zzq^qUgR$|1$Fr|IxY6!Tse!hk$6&79*WHVEorrBx!f8W`+TD@7s1I}F_#^V<|C7o66%u4c};L4*tgH(&jqT_4D? zPmVAkcOYMbOpq|sYbZoKzE_+8p>>R@T&;`&8B|f$Zq%8jR#eF|3iAOC!_0D0OxY$U8(kEG!rdkL>E=pevr+K zP|TM9G~81V9~W(|=O*3@_Fujj8{n#1{o!due~g+R&DKx04yN<~JAk2?F}=BqJH4Hi zv#Gg_BfXP@xv8nK1HJM;-~UKLCr0o?5-_5^^^KaM?Lm3iyFVKO2%4v%BT1u)3lop) zr^{QeKH*7al$9e%`1HQ>+<5S3eNx2`I}*nb35A?{GRW4NIH`&&uP(Kq%anXFP$W-k z+N>e#A8M};d(?r*k7w14(D|uc2w_NTqa=CYW7AvE;#kHs5ppYwbjV375vK((7iup* z>7dLol66|V*_cyIuh4%WzRAWx`k=VNeMbZhg80By-gX3D{S8f;FI%jx`uH!;K0g0F zu+}=9#q0c7g z0}bzHr|-wHZm^*RLLq1^4?ox%@FCU{33UD!988 zM=9N`xTG+rB|BBo?(2paR7q?u6Vc|)MN15gqFZiJq(DxlXTd%Ot)lxlY&p}->X#30 z=uMlpJ8SH8skI$kawjj_4Kl@h7{%dWf~MS>pDUXeCttszNpr)*4=7z{FS@pnCi!^B ziol`oo_h{N@ZGz*rH80))d%4_xyMAqN*AKTEb>R8g(NHddiPA|(`*dh#VYryGX^Uf zwaWxh@wr%`8^6xM<4hrr!2~CIP=L%Kar1mvV3QTZ3oyqFBWyTX%gD9uyLPMGsJPuq z)o>kQ!eIB?kC0KcS+fO4({n6nt2ftZC&h5e)AVH3n_sOO%h8~c6*7aRvoMLT2Qk;; zx!Zti%11RAo;z1S0lpojC`x7BFzsCv{~oDkpn2O!n9(B=vT8X8?!G46jZ!C$^kNQA z0=`5(0_7v7wneX>uAE4jw!^4i`jQm9bwiv1^WJ(GQ(BFbZWalu3IRTp1q+f|A-~jx z=;R8M zG|dQ*(f=e0ek{mJhC0LlBe-TzJV@*j;}b<7pn80B3y-euMV zOKML=8A8Q89sG-^fYMiKXe=|uTt75?hE$ok+!2|KROOS?;oifU;3;D4CaYz)=tk8^ zh4p5;?a55;clhfE(s;JQQk>}8;Y{v_w#|% z=Opj;C!JM)7DH?Gx6nWUg#<(mf20i#R6cm1P=By|Fb2|&UqHN@RH(uSB(52O93^8= z0o3%W9$^S2qc1ez)=jJj1&+Gej~Gg~HzgpMu_DJW14P&=eJ~!%kM7A9J6 z@qneFKtt4-$aiJBW1D*-KFd&3G_8qcTxE);6m!asWy+=b`(D5~ib`ePF*DF&a-zKT z9HY$A4X>%y!o2wFv4SfM6>XP5tG)*N{kEF?<*=YA8Bnnl-a~Lx zuhRgv!e~=&* zb(^HhcJCAkQN*q48;`*B8sit#nWz;M&i^PJWcLr-i$pI8a7Vj5}l*nN<aSmyQKMoqsq!Q zgL2@WfqWia(=P>c=dfV0KYxiZH3W$V5+w555V1HTXYx$R&in$klBChU1!<()@y|}? zyg1;;tX{^Y5PSiI_6qS^{SplpzovrkF)rkB*w;hKK2=c_!_jI?J!Klx9z856!a|wr z0n*N5TYWz-8pYy6LeTcLrP9LJLP;SnT#`J7{nejt)!tTyTCEgDe_s^^nuIi zCu!z`#G&`>lb;hGNoI`1q2cUPfokUN=?vrxIG6Y7^izTpA93cC z%L~Co-3=fYML390)JH48EYoUC%_ukiYtSnpT8a3 zMfNk6X)@^VX%2)0kB2MLX6bPT2-C+Cs55eQ{x7uCSM z!h%sJ(=);Bb*|$k(xPYL2zaOBFP0;# z%!Nhw20G}-zNOx0tDVW>o;XG1x|35;1GhUY3Fq>{VGraB3Q*}&OEFrZxRk{;3JZ3a z6`&x_vcxoK7CPCYXbY4V5}=$D!w3Lo3Stqt8*0ZxbuiXkD;lJs zpMHn>OUD3W{V<=8%K7S#FImq2UOoRid;Zg&`H$&SWlarP9OE4pa;2JvO2Mk8ve6A1 zLA`XLQK%z$HIGnN9_7q>Inv&$HhuEK0q*tNYE$R!Y_iH3CokW#`2F28+h=P5YF=mj z&b61$wQ=^ksdtaxUR2#s%(+eS^dcABk3ahkT3HJ4HIAW$OGDH27Dl5s$T!;IUv{O*AmWdoR_5^G( zrf}i#9xL77wN>LzX{HMLB^(b9d8gDd2}6u+XIh{zs7?9e$!%x2EDs8+Bb^gYoIY0s zv=&1Q5kIQw9Z!FpYA>xnDYq7?I+uL$txEEEZMOWev~5Xs7ilBvfBZOMgE?#U^F8fc zg}1Wfo`xP)m(#4VAl5v@X}AUR{Pd_@pb=+2q1#SWj4Bc>5wPTrsQ|@1aslE=y{fj* z+>$qL-7-s1-+|CSsHDb1&3WuzhFUXiLD@y7?{3bdtHeK0i#U6_rJ4_s5E-wweWVO6ZOAAToyW|CTNgK?h#YhKGZy{M=OGss zo-pgu)BvM`B~~=2koN)GVnCh(^!*eMk8(s^776tN)@Cdh)uSIbSHgjun#K?E_)ARJm2e>9Bm0`QZa8+1u$-`$6`qDjv*Gk9K z#KgnFnIhv5WaV@Rli2x)+Pd)G5=r{|m~~^k&J31@BAs`OXf8XE15%NgcJ@q&zd-&{ zrF<3%2<(@_cGL6Q+iV4Cay5baQXdxhh@|HPf_M292Lqd)6ypuFBlX@|*0C+wo!jo( z_Al2Z_)hw+{L$}j`{Uj7Ka4^C?Yc_N2LI7-mH#2hTx+&JNoj<%_6J(}FPg`3$s@pv zN|cl+7Sv0ipnfbPcSto!oek@Xj}~9SdvB9M;5~lggJL`_(oT~ok$42Oo4QP{&)U*j=Qx7VyJUBjNV{kilr z>}p}MK>U=R`-6p=ci}w?ux2i9KeX?Cfe)<)2du&$=sgz|TqVV6eqK3=!Wb4215gjE zzZGh{#~D9v)@9t2sMMZbh;{JAw1n?e#8eD^_oWjq)n&kFQy{=VTV+B4H=jbj*1B*W zF~k)!Ka1ck?HL7};D)J7Upk74hc?yo9NTT8dk^S;9B&Ax4P}t{Dd{{ee+J>~LjvSX zZN~z(lY0XPBJ?7CvdzJZU~~{&>X#SyvAVDBKyo)_xtV0vdo&0yr0EqH9~QEM8Cl+Y z(~m$|sTFus4^_xZmoUJU7J}j`M6fX852X0PFeVn9Yb}E=41ubCEgD%*LDcx>Bg2sD z^`9${N&}GC>@8)~iMxK*SP(B8vk7hAk&|ajZWKK_*P32WQRE6vqW$buwxzL-SKiB-N~-g3UorWb30I=L#>f}HGm z3%G&7`~F=l(7t9mQ+*_n%s=+0{y?|+Z)^OIrWm^^)yIG`*u`vmdeRl<8G0^6?A9V3 z!mx~j6MBRpZ19#<6HnTF2SjVC2jm&+lKz=HlH*ZZ{3eQ0$5>jr8Igq$#1=ngV) z+pKYq)s%7yxv^Soeas--ek?yE{tpvm*!Y2 z>yjuJ*(WVudONek{ha$ns4akQ8$qYrJgEEhT8#M2V?+Gt{u6gJ!C6a3;7{(qf`CGU zcQV>X{!aa4{^t5W=kI@sT@heoYW%O3&%aUfnaWzWNP;MDU}%k2YSxxoYnqp4E%drb z$O_29tc3n)e&zvuv||FrbZt)bB9XrR{9DNMWUFFPgC98MYZ_(P#Ddun&Zeiy4Cce} zRD6DZ-!I=o$aaB|u$YjTl9-g3A6JMmS1^odsPs9Izq^AfWT-F%wrGe{=NRCQeb!L1 zhdos8x7QtTrYRlXMiS0_NawbYrjg&tdw+1RZrP7J9y;Ot*}1rg?m zT{lseN^rf;lA|cE4{w9!wVM87*#%2{_iQ9IZUclZkDqnV!WfyFn zm=;WMvEke9bGWuE=Qr0V-Y&K6Ci(zz0K=s6He+j9yR&d^H z?~LVW68aN8=?afYLbPq9l-n3^P1`lMHObroh})l(r0dO@x7uWUME(FxFnx}PMl$^l z+cvj=3nKcI^&_3Gr7>s$=RRInD&2Fp!#IU|l>#`>WlRAFKhL`c7?xe)0?xmy#Sk)2 z#2YoUdJ~qu%yAFcGNF9iQY}ql8=}X!cL~Ctn$sR*sfQudgzgcUtq3uzG>oD_muQvp z&{t4AjtMP<%V{T%+9sNzu^&amOCE-9~_Tfhq6VvoAOgF~xax?C~JyZ4x(_7VXSwxB0l-Vz1o>R9hsu!ou^|J(bBB~LfD zkTzbC_dfbx)@boV;hgNl5-a~01SJ3eS);6?q{#o9$~jR{M;hdV!ohI=z{1?vIQwoo z(0W>e7fleuKNl3OBc^SatEi1JT8N*D;^>RV5%^txUYiNN2=60<>%mXY$(9GEhx_v< zP=U|oXbvAmB#nl8V|83NMjYG6>))VSX;_~r*j=We+7A;L`!dKV%+{q~6P~q2i}?65 zZif2U_PPh!C)bH%ksU$3%|@cuVR-@6ym2{l*4p!IpY>HS{^qQVko_T~R!h>Yj$W_j}~e@S+@pP8Q=C zMv~kUDGwOQa4#uFqs|cCO4dTll8Hvupf#*2ay_Hza|M9F;^P<+u)c7;^c&4S?sOF-}nCZhj`cGk(+Nq zAx3qva3-67H{!0=bbNsyp=k1tq3C~jF8+s5^iNa1L2bP^>QXAiHFuDqU;`(1pmINb&r2zEa{NegHSWc&Gi z<_3j*6$^_eGgui}0oZd9)8uJ}bY+^|>p`h7E|>{q zVqus)>rt+_)yEP}jlPduACOr7g8u>IKzAnES?(WJ6+=Pnm=?7%H3BCFBZkk-jFm%d z%L!YKf4|XuxC5VoOLVuu#l4NGYq51Xc{7Qc`No=nj16khxZ4vq^s5lMmnv zTBQ+Q*lEBCV5!8u{+_i|Kx8{(Z7Td}b3%E6vCfrZhuM(AGC5_i0!^R|)>P6UQ7hVWB@AsbCU?L|5-~3?u z7y3M|tF9kRdB0EYjiicdXJFeJZE@>ds%A4XiR(GXv{u04EZI~wddtj&7L+YvLv~aq za|)_z?6oX)ZZlzVwC_D+wh5}>*+bab_DqUdY?f!ITWQz5_V4<|RFu-@Mr;O_DO|}V zrqXQokU-B z5Sh0w*RdD_S6n@~h!gv%+sdepwMg7>{U^^v&_f>HEQfViR6^YdPsOPyFF>hLOb|rV zvJ5z}R20#$3lziA^6x}Lr~|)bdn;tbW-9U(-3=dqJ%%XIw^69ng^S+}cQz)q!}#Xd zLO0sE|K^ReZZG_;n4>~5o^pdlg-UZ|mejNbOOnl%qJ~Vku&bKVyp?oB(Mljb-a?rP zXBg`@d~(S_>lw>RVOo?&m%(zr>>g~?!Rs98wun&JSQr;^mK;^uLUsI;wU#QMSjhwt zx|oCv)`rZy^pmf?^7WHPWM6&xi~FsYy+@3&xq-4shngJAm}F9n5!?u-g4>X#l&?B7 zZR*=^oQqkfFs@`cmK98>5<=r#lDVGNp1>(6+DwVanZocK*rT-Y2&gqj&PUrjf`RR% z)3YAujq%x%eJifMkA=6#pA!K0ZK>O2yOMWw3)#wTXJ=sHukYu8O7Y@@g0EOfbB+Op zhh;O#i!Np!U2J5`s5oRC(#mU;JxE+G-}1-4#$cW~$>w98`B^*mZFjIe)C#j*+n*g- zIdbtLqj%TTaVaGJRP7Tb?nb|o!|09cc7I&`BHX-BGx|k$o94hr0-uw_N&tQ4l>#Wa z+hH{Pxs?m2`xQ?4dlCf^Uny0fsH`aibBdrdm`ZU7A`i)DuCBJ@M?Oq=q7$<>$rPUJ ze3`l@7?W5{XV0jJ1n`>K!#u&9!E?tJOnNxwr}Z9-%Vrof7{C)j(X0rg>=ouXes8T{ zvrch@Tz1b{ECsUzMRX&vjDcen=>d(9Tc2D-?i$J!6enXSB!|#Lett@np}GKjtfw80 zHHW`kyX0hML1eqAO{Z)RSFo^K$Oro;!gS%Fy%#QMMatt#*%oJz*+PuMg`=8p0IDd3 zBWwUY7wA%RyhaR=mZG8Wg)jK*OKXj{L?}44uv6x6L<(U)p%9Pw$Vya7;3I3H4nb!K zY4M=w=i8Rrum)n;mD6DfDMjmslot~lIAl7%~lCR%W+5$k}y~Lt*3Ff+T#!ix*ggZhVyn>;$dli{l z(UKKnY^Nh_N1`sLV=hN3d6$oxoo6Ut`cMduk$!iJ>RxDyXq*g+f<{n{ew$-a{>gk8F7|{{p>G11PqErBkcp3&5!cWv%Ri2~ zZNFc08^6Mj%1qhOvoFJacI*&6M0kOyYad|CK6hU8ie56lP0jag4b&$}396i*t4PPM zZi2r3`}J-EKS4bFF|uk#_)~M0{{LMn{k;$M?-{MEv7;lv)L6#$uL<@)NitZ)+7U|y z#V6?#w9$yxT1rurb%nLNq;SJvGFeoizXc6pY(Z}6Vrw^SE)rUDu#Ab*3n-V*dDk?i zbmipZmluWTDV<>(DYoDKj*~w5qeb^2oD@m zK~b@(@F>?*=3xvnR7u(Egg2u&B7VI#`D*b^JBbKWP{K#-d+78&EoGI&btB4BK`1P7 zq_pC=g&RIQYr%Ci(9=z=FZ^X(;u_M%{83pGzPbPaeQEASQb}KMPkLeD761yX2ha_S zCI!v4Ym`@~U+=8gvZBfIEONb5*0EJqN>9?TR7OXY!J%!c)Z|SpZ=j)r8`xQcnjDD! z{6PsWvZ5KuugZYVjd^5r=6SxM1E7No;&d>5l#IDoX|bnGVjj1UvOU@7{Wwhx=h;#j zrYij-JVT+9smck4ZUKwcQlrPIb-q@n!>Z$enZA$u>IVjDuE3ZPJJd?zohlpgS}s=< z=EMRP(dJKjNJnY3%eX?Yb|0h2S~vDV5*%B&gz>n!Ur0DCWL0+~7e^_brwvB*vxpZYE{-Xt)0^`GyhF_6i znqp^EJ-aq|7HP+hSpBx58KlJ`Mt;LD^RwxcCHW?~vO{wNy*>(a zJ>52yiDc|RC0`FJ(qqYjh~l}yDyaC`{^>ZkSho_r!1Nb}Zvx?d&v5#=@ElVjDr8)Y z+owA+ab0wizj=jd^uuv6hFZ~0yT90lWQ={~;RCe;AbckzknUcB6S(_qPA=J%UQoe& zjV%`42y(GVt$mj}JW*pb}uDnG+a}_3dcT`IrSFkaO5O45Xj^Kwa-QwQQ##q zimTc(=zm9HTC#5rH9XzLQJvHAm{9dCBM|+v4T#)j$TTy6O*aOC4DK?bvJc%Lu0H6$ zao$HK1yoB>AnauLc z`Hpi+0{sxY{5cx_>Y=@XAED(W@(c`|OdRc;P#7}W$O7RvaW4OL+56eUm>PP{0kW0n zMh*RM@b|wL218!xh7UhFfD90SiXjyLC58x_85>$UI$KK^TiN|Pctojcc_J^N>g2CQ z4t>EJTPA?m;(&||Mm7907%FW5(+|}s$>&+On&u86w zs3fTEc^^!ox++&`0ZpjmRL%KY#GuenZ#26_k$S^!;)88WK9svjk!wsoq`ON@KE%7c zm|cn2+?bbRDY4g{nD~nXeuoy-kP1LUDi)q{1K8*?^)dMnK2)w~=Jt|2JRlVR8kU{< z02d5nhS4KVpE;18DZ`inQw(DVMo5&{*0LWF=mY!=3<@X9(B~Mq0tG6dVlTTRgJEWw zVyIq2@aaWFx%m?g+KKh40T_)iaDb3NX{=7*5~~;b=Byp==HeA^0Y+!R_VZ&c5dWqT zpYX>vtc=AY0yLa)*G1BtU&lFU~Jf&|8>mYW)E?IbE zx5HN;Z94!cnz3Yuug3C%Xo&UJe}%4{n8P+Sx0~uEM*=6Ij&60%Zz}h(^*ATj2)|#O zl(|mcgKNc0+JkJx>u6?vlvqu<{9JvR`*Kt=d|_Dpb8(JGL@OT&%e(|FIhoQ5E#L%yoi+3qqf{BoJMir6-9jM_%sXhfKqx;?e2XnGSU z^?FQ92J-cKnUypdSz$P_{i4pmn=jA`T2=CXE@J-r_#!&=uuj#c%v*pXIu#|PvcL{+ zv^Aiv?AIK%)1IShSqDEZdkft{_#tc-XTPTTPEA^%4D?K%Zb4|!szhDN_;f==tIDq- zr6lWpIH{|1dfE04Z(ed>iA;wfkv=G>;5jgB%h3K=3bXinzg z2QW%bwPDci%%UO9@d2t9kok@A%;y}|OUITfn3RpimERQH{><|@ix4sKGt1ZoyZpB_o zV3@a&mZG^@tGSDWS9;J-8afk1gxGTj77BnWzqae2)X<0}@^-K51t7L$GWw7Q(byp= zv8gF)MaARWOPwto%xWj)Wkl$`9!C(ZS=I(5YPZVNhKQ*O|KyuOvefm#DB#xPi)U~I z!M#gg6_^Nyrg|5ewIY5unWe}F;}?KK>ijH$zC{Wz@e|Z+5M&W+?QohYx)BvttH-CQ4QM`wGT?Q z=VKbt#_v77&w8IdA5ru;b|4!tW7FJmGKT#3t%!g6kqeLmi)LE|nOIC=7G*xV(SK6Yph*{HBx+a+aG#BGiokZTC|iN2pI4>^3IGk$)I5M@+uR zeqD1oU%M2Z?H1>H!IL%2mN$$6+SIOaM*mRgb~Xp8yIJvQ<_OYjGZ;Jy)-zi5AjHJ> zjQ{zq+bRYPf#Q#?ETrpt`?7B|_8RDLZsArJZTBNxeV|U_UQYQ#UpiAmwKIkvC~aU{ zDaUSPHN&r`;#lRp^#U~86)a%lm%*Sj6MWeR&CgAnwMR>WNr-1kCJii)=sdLDY&nyD zG{+TU%kL`RbV;$>&2Uuj`3{Tn&T!?wUoO{vdS`lS5OFAxxr=`|j`K7H^7r`%G$|Oa z``;Jlv(IjKf4%>KUaFE=u#jwakEFBBR46Q~N116ZH6*-SI=z__gti%=mbT3-6p=N; zd4^?&zm7u`b`EF#QzPbGyM<1#wQ5PHrL)T#O^XfHSd1E8y{bjh4IjD3(OFqm3RtpaRZQnd&zj0BvcRb6>CYvD-c(n}gI^dsa;G>GNZqOw zq1`F0=9YZoV!iU}a;;d93~EKt`3u{%df~?-uj|a|AcMlC+Uw6kc%808*7e`fDcdvL z20*pTNK8I&)A8^3I*h_Iu0t>z#jc2>zwv}bFUcxq9+1jfCDqGg5G*`0?4q&=w9BD; zxx86iz_nCs@6qc#BAL7cG+(U?GS$rXxd(i?_1^y`4&6eqTNeHp%l(A;Q(~t1Z;9E~ z#>Cvz`2*ojj6Il0UJz0g19C=Mo-`C9xr88A zb?1nKtode@`qD<^4K;&GAg_b)Iv-(*Ft6SV!v4BqiTU>D^5xhPeD84hvQq!x|to$3*o{GZ#nFN|aHpauq;3vbn}^=faM?`I$fn zTAx*Z%UJcw(ZxtCifU`F+u|m%$7%N719sO3mFuy7f?Jsm{ctRtJBo7wl@~^nLCaIf zV5&g`pz0hv4RGHy--c`oEUdEkXp?#7y4ztCzUfUCJdO@a4@wTF1a zgX~+wPkRcR$qe)M9<@c{zDZ{1%m0mdw$FzK7oE%%mr0z_llu2tkENyNpy8u4iuFfy z62t$|9Bds(1f4$Cl?|Mo{)OcT{yy;mfI$hKS}v$z)ON(OxEZ8ZV7cJCE zt)Jr{m1W69@uCKe*y;tp%I!~AHZzAtN=fy+U^kV!c$hjw_hZJvSZ1m=)MpPwQQajq z4Ctw@j%I}h-J`K09+Iz6#`fmhX&mxivMH7SeE}+5rfAKaKo!xU{!LFxuZ$Y363+=# zxC-x)_gIWjP0$TX4=~JDK%X|#jkwM0%9D39hR38Oh~m;;83|cV?ZI=SobHL3SVTQP zA(DHf%<|;7H&Hr~IfssGVc|2&2_>|2@U8up!iv+QhM1JslMP$=oPY08Q-O_K(cW*{ z%5R(G)1-M?18*iEpY5y6A|d6|UPxWnUxN&_ z1+zKj4~<#c z%#=ihdFdvfo?+ar8=P>tS*RlXZhkqaERI&V< zlzR@XF=1Z;L0*}HF9$xS6IJjlRHz~pNPtQg%y~V{$_3k4RjhAeSdu>SJ$4xXsGBKt zk~ng?-sI#uuj5pQtI5U7%l$pJpO)L#TzCFmHGDryC#y4sag zj~KG@`gepR1jz0P|4&{JpwsgHr%1prW(i87LeOBOFp{BQ)8YOmkM@{ecXt(B4@)@vXdl3Xa~kyU%DhC zQZ>=R`dU+J`?DY^PUSw`bWW@)B}di0bx`=X&K#GprBiivjl@5@AMa1yIq?~zrs|iXa&zZl!>7@<$63)uy=76 ze#I2>EyoqEe_Io*%<1gV51sEYcN%)%cbIXfI`NA6Rem(=p;5&OfV7F|Z!ALYu}x1@ zoa1$htQbC8M_NJPmm>~Zn-BAzAMUss{wYBmW0iH`I`I0tyQGbeC?jVjVC<{VjJK26 znEdQ>^CPJB0>G2y7iiD7en1MzSy2+4#ROkkL#z$+R9Wl|DMrIMP_&J*q6J<6b=}XKkO5+bT zq2?r#%(A=TMPlJ`jH+UmT#Yf*(tE|cE`EIQKHb@JxOv5qVzZZ$Exuw@P{ajJ3xN~c zF}?-Ak19cujyW|2;S;Z!RPAvsLw8XNn+0prF}UNRzL(JGPs^$R=`xe;T288BJ9CP| z6viF!vtGdd@=^o)DHWNI%wzq>J)!^BX%zmyi;;hOsfe-3Kfx?h)!YeL1?82K2%gx`z6QU=)B)G6|Qo? zqoGeYv+{j;j_fyGpNj80jamz$8uzEXA2wVs*)QoXC%vDu)uKUCVh+#E8uJIDEzjH} z`fU)K!kTsZafvQa?2|)zi7wFWV?sSeKL%R>#Agwnj~N(ZOxQC0&NU9l*4W2XZOe$1 z+&M%0%nucvQ@>9U{EV_TBVHQU?#7xg4TqweNlw*%c6bB~97gt5j)ltU!4}ODTl?eawLMcD7_p_%>gMjT`e5&nJgDNJSfv^ zZDOQhnPi(wn{*q@D$$Pk1<9t@1d`p+0h)g!n3{Awe2ip0W{i~QlTNR$2r19-H~1Xk zUgTx6ZOVE{H!L-2H!woTDF?s9YEi$wLW?Rn(j0MZkQ*I~@xnS>dlgebAA+rn+1lT5 zh0@nRRNB3b>D3;P!P?jz6JfOmgs=$ketTh9&vN2zhSGAxsOa$7=o6SCYB7=Gz~1^m z57xnJ=-D7xDW`;rRQryz+1d%oRuuush58tZ~O zWw#u+EKTB_4#ByW? zC+zfdquf4z4Rw#ca~l?%BKcN@`Cvd`Dv?@(S%P`Nwmeu+)pj~YPc_4-QJUZxiDKnq zssRHsJ9j_KXQUZGN=sjp(SCuqqMj)hV9titQfUWJi()=k;uNO(hkW{-cWaq_zmHP|Ls1}~{Tf&y2m zh0NL9;%1lH>9g^!QUO#xey@lwd@w7PYHMjxzJ05_e^x?~N*juF`Rz+e?vYrUm=DcQ zEN11v$#fK*U;4H#&%lA4(_V?nfRoY1iyVhM+HHZ?42GgQJW|6sEe zmxkdYl*-Md0H^RHatrEXv9uE-|T zm+}`xXbEMaxkkJp2D13HzXvZVxY;4}iBa3dS-}agT1O+(J?O(X9rp!;{N6&{svAz? z8%M9kC!C~3;7FL#q|-vCV9gzfSD$*gmCL(B{gY^-5s<0@$-P-rZd~dPKCaOT3NL zN@TMtkY#u4t}toGoj&YmbdzzC2k(BzC7Yz0EVKjb+DXVfIF^}A)r(!n(HY)s8u7>SRkWzr4L&uG~|?*EsvQ!Xyg@c;045d%$U6a z<#|L}c!a{f`e0UIPXOAE9B`ivJ}AHNPq)7i>vn(R2+F1$?2>2lE$n|4EAHBbc|)kZ zD%Ky+3qUpv3Vzg5_xBB2I4Wi;RM?=I!>qsAX}F<|DKQfyS>RZeU2wRy-Y3`ltdd5q zVk~Ss(ld4J=pG=F^o>YgQ`I*NKiKH;ANW@5|-O-z!1* zG-GjuA9F7MKi+U*`d=#PziUKAV?$d9qkj@mqVk&D2O3dl4rFdjAlE^mJvZ+T>^__# zR0z3GxJM-P>!#dV4NMXylZ@meYv!ue z^6E1Ylg;crm&-s66OK;wDlRe7boNEk##4Da)e1&584`TTBS{uVa(rrx+puSX*C1K) zG-uM=B%R4{Rt4?Fy1~JNs_z-be2k+iS{zBd9Ms{B_ww|bF6Qg`mN7vd=w3TMt zN1OGqAgg3Bb9Q8?^?Ma@lZ$M%(ZeX9HSK^C$AyIe8~RY8;F4y&{2ia$Q>!f93UilO z_i_vL`)A-DT)1Kw;r91DdFMN4(A=F`1yy(#!_lsYs|!I6k1xI38ctOk0iDDm?uC-(_V44=99< z^%l$aqE;@PyFkynZThW2)V}Ta2DD**9~)8(#u2LktMVGE~7+9HgKtt-UcSi+lNmf0HoE zKWtbg^Lw3}TAPG^^Y~(JLob_;*U`ks`_~bpUvTq<*m#sKJCaR?%dRClpTo8x`ed`+ zVz47!hU+#mI$iTsDYRaT-AM3fcpmjL!OuzV44~3q+iCP${J=i{fHFgwCdpZWCO`RF z@z5JA3Ze=D;jujk_BkEY>bf^V1w<8EeOI*8j;>pg-c;jMlu+}A5RmB?^^_8n2yy~y zH9BO}P9)d^2dPkk6iq-xNGA|TBZ%I}my0*R2RT%&){AYHc!lO7?S{QZn$b-MNgjNT zq8oq3=OXp^$&b1nS2PS zwlI0m#SbeInR_)h+=S=7)leMRrA2Wg*>Yl(L-i&XJmuCq#sfCqDzQE*I@wg@R;*I% z`NZamp`CVxl)9OsP#l@Ixo$Y(O+4sd8Z8IS4g|d~4HMx%kxvg^k$m%waTnegN2s!s zis9y}VOjjn5sKpQT0D$*w%0qsBmK93-&-*^mZ38;WGF!6!XhEfm0z!11Mj$QYmuBX$0N+Q8i4fB#XX7szj6QK|uei&T|AAyf=h|jBANi+^E3v-s*#7 zq^4XNOk(kmNJM*LQ1Vn~dhRzaFn0X${a%%GlF7Gy?b#D`;%=qtD9lY6V;pH!G4&}x z0~LFs_UA%hh|G!(Z|!0pj$s^SS)=aUyq)qoK$yoR5)=0|;7rY}j#?;ny#Lz&qW9`OL;bMi)yJ_S_IXO+{Xz?V^JDBKsWM!Awq;(}m zBQduI3h`1F!l8MAL~SZ7vYu?({?8U^R8_Agkj^NAJ0goLbdQs_<f;O8{{e-K9)0g*QmW;yL_OLB` zi7f~77K3}rK|g2Js0wTjN-vwzT$Pet74a`ioE8Py&EemNm-)@{-$nn}M44CHaJt;r zj=9xMWm`|~rMOQ)Zwxi9pEBk2OO&oY$~u$kQojrU_Fb4dQq_txnYfNvMsQlADKXiP zXht5ihf(6No%(i8C59(uhuteU%#VOoVqJU86*PX%)YVJFF<~a3D%(-{p)IpV) z5xPe{plzEvTtdLRd(1C>C9Hq|rZ5A1uTNq|aRw1%u=)n7$|wFx(E8ych1m`Dq5K_7 zh)?$@pe;}cOAJ(ZKYFD;(bj-?bCB#cb;56^LAcw2`vG^PGwKozo!6)7{5d)-#;)ha z@2}&9c2QFD_3LOM`o}(t?Ej6-hQ|2WsH#p}@QlU%ApNwkx(4y92}^dnPEfChZ;*ydveE6TBkr9u&+j^)L+LWz$IO z1p}mK`djvxlWRJR{2GADJ8P`=LSdL0{%u*#v=ac1e_0T>*52Z=-JTjMT9k{>;R{Ks&>z%ESEC*V$<2tvjfJhtBf zt`kBR4kykC=VaI$T+Sap-TSIHLgS#9@`fB-&ZM5{Wi{Y1`kvvI>6WqFKxwQfa%@PZ zgfV0Kk1r^n;|@#{mp$9e(M_tanhWLIuK=fAA2NLod%T&WTWh|~{a#Rx$pGlzjaXYQ z2C!>5Z5Ufl2DC|}liK5rCO6{XbY?ohQgAw;pXa?xH{jr8J2v2C4pqR)Fmix=VD4KW z@U6i-ec=w^p6Nzj>tXgCdbl?-Y!BwQAP6Y%t^P*sxxL;l?>8@v?f1?@U$s}-7wjbY zbZ;>b+yhm&EG##0ZS8?MSN48}OXhr%q}SG=YO!X*^7(jmwQd}MyQatH@XoS94yws9S#(&32StHGA(_rU>OBA2qSeY zwGHB0U4@>cYgsi3Z?Sb9D<|v&?Ds`ky=2R;IRprP26XoojW^fE1rOuN0o6!K z`_;hAv&kUsZ=7wW5Ub9%23(rmMY`=sByQD{9PCWoXn(EM*so&z+Hosi(fPUJcQAa57;1Cb*wB_gzI-<-}<+t)%Egd2qXjG{p zSsGKLBIent`~quyBT|i_=%b)~k@~e$lUvz6nk>SU3sp~wb8v5`VFXzdgeQ7mSoiXb)lQD%$^F~7G)x-{h?3nA*VTQ(oRwox#c#Q_VE zS15=Ezx8u;umXOVP8QlSd)SzvDZ(8&%#|guOHp=67|IC|Ds!1_MX$N$XH9!c!t^YP zhOYETp>$^nF^)y?KBy5RX@xA3HI$GqgQUemqEzC;XsSV_SL5>1NsGLwb~ATg6)b3> zBI_xcq??Ya9>Ic87S^J%FclJjFRvY?uU!5`rWnwy(rVijF@{IQ>Il@w zYF`}s04^*^DyIgkwSSGX79AD31q-bHVmOvB(Sc8)2!7ivRPY>a5uhvoh%HKm z3Rga7%z#6x9I2JUHXeKyO-)K#V69eA)D_>UhO8baITSAVo$(d>yNArLL?|(du)--6 zbtQosI|y4cRgX(3Y{{m~J@cZ%`HF2ml2ZByzwrkPjX6_|}G)xXIaaqO-V;bqZ zrxxZ&%t&>sgN#ac>BDf@BovR3)K`Uv> zv7t@(?4bKP5t#SG*_W7WcFC@yDkJ-S(;wc8niGK$aI{HKjr7FCOY`cApAO9Eh^Vq6 z5iL&IquY}{QJJ`>bm3I%4;mq5mL+2421fO5SW(Ic$$hO7uVDuoLW!XUREY=umu|ry zPL47v^lLb23IP=%0}PVpn#_y9V13L7$P z7P=qqZ)xbX6J743XF%tierA-QcuLDCvfUy;Dt%5yl{0=wkaE2c51}Hj$gs4x2pnUm z8${fuxX=!}KtIrVBbt5hVcnpG8q#+N!R-O6a`@F{PX#m>{K*J9O zi?@q`dKS36u#S$|rr;DO)0P)%Doo;=1IwJE0qGeyG+ro>H;5R3U8Z+mA@C1E4DGl@ z6c1LI2VQ`S-nN4~0sOL=%BtY(gQWcqy~Ks>%PfMnh*03nH7_J+mx7=dwNFt&J9djl z=PK|DECaB7JSMs1%mM>W-gk69>DQ7{c`d2qgQW4$*Z?#hc%*Ity=`beRuU>nYV0fH z2t5sqn^(`BRNGa~(O3NiIr@c9uU%zbwptu(gb~{hzr75r|ELXNN9av3 zV`4#)|M-ik)q;sNGvgN==j9QUUmnfiWg$i;TN1hLPE0dgkW8V^Ae^iIALcu zdI^{(04sovTX*vZ6UAB4hKdFLQiga@0XA3oFW7p#wuTw{EYX8TAm9&g^lH84t(0xE zR17EAC4zyZ8zQoDIK4plrd#3cNn*La^rxwFaD#@HY2w<2vg-TRT+7;C8L4LVvUvGE zh&R~i5%qi&ePH_BP>&ZbnxSnW;O_5MH*WTr-H@~|Xu-G72z_n@0oVPu-te~l%yr?REB8`Y^pckdg-`^=-}n=C z+J}|>SgR=7+8OqK-#95&o+v7unRxHDTY~banvhD->EqtMNaIoA&E8c?OPhA%URs2t zaMP3%>XxUc%bMV-`$_eXXb|wVa(Kklk7%g%T~#run6bc)JVkp-b(SxTudk!9%gBS$ zzpF7Khel}bprrL`Fk+L1b1G2LfySjns~X}>BHZ`UqxDNE!deXxnWIGkg33au2a(JX zr2(ZHU|xezZL*64yafR`t(izia!pu1PtcA!d@)JjB$J}Ruhzi7RjsmzaI~*-oCmSt zRLw6yEuMf({q_|=Lw2wH^ER4Q#3#4F2V4d(9$-zM##Wt7vCd8lU0BiEyLDVH?VYX(bL@%aPEhkq^Q)re@I=sXR zrh@`aS{VfF{uBrzn~g`*NJN8Z5Wu`wtM7%-HAo?=qavl!RWji#Dcq-I%!f;}u$F@M0ED|p!$s&Bv&ETq5<^YQ4_vUT`X7ET6w=O`9& zcIMbM?9H;-4}|@e$cY$wdMWOrHNO%&nu9HeO?=YOA;d0d#mvN9HZ|&_;t9eEwN`=p zkbhAP_2*G5nkzcD*v%wa>Dp7n8a0POvy);A#je;(on1wtT~Dy{#Og1yUBA2H4hq&;XF4vmV3^7Wr3;-64>NL%_F4Y9pRS_(y- z=Qrzs)1q>m;V=4e4cUG5b%OZOR`w?LxOix5M&t)!g~G9i+#QlM$e!YT+#=;18doW8 z5#=M+hn)7{8lLDHp1Url6DT$QRZKyR^no`4mGi5;x0m+ z%}*4%A00kthfbJfxAw@Rcs%m$o`zy~FvS$*uDY?_K$jPMhKTs=fuOv$_1hgr*ZrXY zX;mCsAhwF0IFk-bV6_Z5kWAY)g)14@Q1Jmy9CU@bVkC_i^boyGemxQF0QZH+VDIO` zkaT#X9IB7nb5GxM58iu$kU_I{bKSX6fBG^mJI*(|RpIjm0B;Cbb*PIvVATMGwRd`|j!@ z?4>3TMj(&q!RN*en0M;sQDI*CTFrCMr+RCDKP{KOQIZgN&?=r@PB_6mXOLd9=M^2R zhiC&2>Y(}Isd_)mp51%1(#?UGUWVRNcioFYE1U57J)o)DKMvMLSpwaPdeV~!p64oz z#0eb@gNaXgc-I9DC&-Ie?=^(gzE!@Dj4uf2mh*j_TKbx8Gv3*=Y{+HlF)&VvRAjjI z$pWrv)~I~3llh9s!|e590kZFNcnvQo3fN=%Hs*WO!X`!?7ri+T*cfr$n?lRss0T$G=#TW)FKx`d{)VOp+uZZExui!5N(MPC| zU6msu>H(FzCPiDMx`?7kucaugSSOF{#B2@Sx+ym+hF=7I_iIJOVp+uPue@D&#hzbk zA_7J-Np{oOoZDZH1&s^z+kD0ES4~K_P z*U?Vy*M(Sd7!n%okcrXK5SZ+;#X^Z(>^2F6@n(C_i-Wek5bX)reX>4w&ZqGbh9Ko)&1-{4EqpoWQg1QI~g!Z3-nVMPNl{ zZpBhoy{MJL=QHymjTaS%c??90siqcQ*`P1??*3LDbtJ6_2XJ#o`u2=x>dTc z;JDDw2ITy1`c$|&Mt2see9bAp!3VU_bkUU6+m zG*We@!SpPOOs%dEPxv9Gt?_~qyY!oUl)b_xRDUR!e_s4o3}aK`YJjwiS|R^&M*hr- zT~>LqMGxmI-X9RXA6dh`0W;c5=~+IuKC+L<|iDGfzk|1Xg-Mqxr~Kmmbwp(!S2uBDr;fQYI|P)1b?VJJj2 zUqMA>W1D0pT`4JX3*}Xw4h5wz1b;_7B1wtrM2s$*IgWW?;j`13=AfQc+4QxN;WY!sRpWAORgw?T23IJ_15|f%*m-v)~3qXJEMvDQ{O*M1wzl_ zC7t+q8pstO} zjPn$vz%OeceE#&GN~j~Ye7Kr&$)EqomHhk7?tk8bjD?f4iS2(kyO5Z!uPPt} zn<2A)5Vky_&Gjn_qR@y|`Ue{L4YP^~Aze-T*P8AEzbF_ILAlD+<4?jge(|Z`48lu- zDuS^3T)nu3wjYsDAu^;s2Z<(Y_6++aXc8ZODoXy)I5!!t&ptmZ zHs71M1uxr9Mr}qK81VgatD0nIHap6$-QM>3W{Oo z0Pz6%0DU1(5IGH!IZGOkc7nQ^CLqIRxJp?kZD1FO*8%zy^y!2nuqsv=1EtOxX8%X* z0>=*{kdjztpksp*jDh%A$ppGcXUUz03pvOGhN5qvDQXaAV04sCW?+u+mm@8z4HKYN z!_OOzfVOTAWq`Wr+xBMtS`c`wcvkErzL)wj9@qWHO)qYU^^yMY=PZnkK{qDQPdS@$ z7v+a4HBBhl(#?kRxVz&jA>+gEv6kq+CL6!YXY zik?B4;-e|1zt@coAel49LW09>KIxS&t zKbfz{EX7pp>(--7l%rl+I?#?u)GIxn&aWo)s8~f%TCcZZ$`l?XL;ETMxe2tGYt{KIV7u-1;$5{6vxY0`b>eI z54~a?JLBbd^)r6RD$)CL$EaE}PMWY_|1uej=`%SIGc(4FSc&)GBV_B87RY4u9G#Cs zdjL?s&P8;Y4^T|{H%cGnSuMH;4OY55m;7XsUTDP5Jc}t2#|&9vVCb5Z5!Qj-XVy|4 z*AQHX(#j>a?DmXXy;LnKvy>NwAnQSOemk-fOy>wDks}hFW^1qievP3?nt7XL(t!{H zk(e?HNKmgYOmL%}fn>_oJnk6bk1&;d)JIV5nV*O-1RYD*q>pF$qI|hLX`YNs;BQa% zNOb(>37CqkPjKj_vavQmFY%b=#AqA${7ZeX^N(l1-|aqMj8A}zwe$Z&Y;L18VY?uJ!n?rp)+;;D zM%h#}t1h({-WG0xh_pwj3RR|r_(zhdtwyKAG5d!vyfzyF1QIl#f1FUZgU|qRmY3Od zj?;8k_s8x1y4$xf^DGVaFB!s$z(8+62+lYfltxQ^a3mZx+nv3&At^X}{gDBD7TlAf zlRdl(xU33plLG1}tM-VdH?e}1>&&5sXMVKzHH&QEDF6J+E6|$;x`_CX7mDz}hkAhz z3myd^?3qPvPN;=6OOE93v*GUA#Mq;I-s( zx@m9l`JnhBkCGlk-=mNPqKmu>kU9FXT?oN~-lfaJI<&7SRVlq0<)3iKuqQ5k!2N0% zMm_FS87RFU=}vVw8N;}>cS-2br{mhw?+UGNV7zPeBu#gK-jHKc!h3O>1hSG)wH$=Bqc^3m&aK zDZTmv`qvUFLcQKT{yMyv{!zs8cVc5P3u}}A#9TQ3*H!MYIx;&MUI|2mTj75zAy#J(ri7N!TjdQ*D1-%${vT zrOotUyDLNJ{Wz#!Kp`_CJK~R!VfddD`>ulTxX$aig->_j+k30oIvOAUVrm0iw7g-z zN?i4iJ;L8%xc|Gv|Hg1b6lA5p)Ir%RgCw#AHt&O3WnoWyRPI)YBA1jDt5ohAgQT_* z5ml1Cd8BwCpnm@RQX4j)pzuC7c#Q*{9uwZaT|VDdM}9_`pi|SSWtsOr0{S2kd!g&Y z+4|0&Y9PZl&HJsUnQ}|5Rj4vb-#=yS#Kr<)O~_tB?Bb1?(hd>(FyFrYDQZInd48~V z_ePC@#3k-Vi4#?WZZ^Y#sFpa|>--rPf_2Za@OoLMK)Dak(~c~&EMG-sGW=0<=Jiol zra}ZGZT|UjO_Up6L}ER4KkIfR=?V`KkjV>k9X}o8oEj{PoFN9ARTo+!hUdK*tgw$; z&HNQt{?lWqANu0>R7+ak%(6b;Yy`vL^j~2SA%{QI;J&Jx|Bn?W`kSWXuL}RA$NMir z4(*-1l=7K(kma2;1|BRBpAbGEoj4Z!O=!`d0%TSIX&xk+I8ug5KQUlzl7o47w^HRc zptiy%@KJHaW>E>WnGhqz=5pRrTYDu%OWVq3<&tkhymjW%wfK`6o(WPpzk}cRzUMjD z>#F_u?YjNg_h1P=&kwTKyS)bih~qXEncrbgm)!Sk&`a)ZG9r)rP^MJINh(Bn+Hu^u zH2V@IPC1psq+B_bvv`aWv2^1snOClTJ=s-xIjf;Bc|zut;m}d}twaYfE~Z?=a!94j zmf_e;nI#LEK5rqFMk%Olt^8B=l(BdvIiAu%8L5QiOZ#S$MY$feJN%5QM?VbYJsnd1 zlH5N4we8}t#}_g~!Bcia8d6Jvqrg#ilijb53Wlnu;I6n!8q%Zq^0S{EPrH& zQ=R^9JoNDnVF*1nj+(pPt`$@*WoNm)%q}?Hq$;&jy*ePT9~eqsVFxe8T?U}B8xE~n zp%Zi6u29&8#gl(S8)8S@x!cd;g@DRGOAil_QQ9r8V-6je{{r34!K zb_L3=qYh#wxi@NtnnwvJ@9Ah?GDIEqqZ{OS&q3OBZyV(!jUWU{6pAhcDI@|ai6(kBwAs?G`-~Prk%3l|LQZ;ms7I?o;*wVFkc+bM zfG5Lx?DtCyDf%di^qI@AkiC`zBGq;cS=y1X)T-VdjrkvSNcb8JP(5|V&2aR6{92CX zPs=-bw3?L@nYUIU+uHL5V9VuIs=G!|Hc;9(-K5&L@K6^aLjXzqkA=jt>EXpKR;Ip0 z+!u0bSE+i^l?uO=2$Qi#$52xCit&+K{ZwzEaoToCD%y5UQ_TkKPoptf%inC9Z?11@ zyFIk+$e(~Q-JchDZ_|nLREWSM#4trSw*9)lE z>SC|8G`8?#LajWF3^Ze}WzxZl1Om9bNe*93C_4iaRMl*(5oC;+*`h5{V9D)z3VJ%Y zO^l^XgmX_xE$izm2sc?Q&13w?4u(4`yQpT+MTJ>f8d>o!Bi17ZD{NdvVmPRP=hT&q7QaX$by&e$!RgbY3 zxxQ#Bd4{4UGM>#_^E`6IpUY1o9lV0v2@lT9B7DA}tea5Y1SWXg_b6w-!@1I2Zxdih z(;y>LO}Pr-1%z45p{rDN|Oo z;Hc0V%D&jA+N@+a6ix7v^&^<{AzQn3XUTo{H-I6jv#zSj6&lzAxZO&wgo4WrweN

*kBFF^S@yu7zC*AV`afUipc*Z)u5*ZIIh}%lQedn}IiKXqI z98Hqyqxcm8OpELFr=c@fB~bj-)NZ;cVZMb25B_5DH8sX)vT+hBS6-%PIIbp6<^fUA zi)pr*RoX$UfUg)S-b6Z%8%Kg9tFX$PP&TTd4)=79hi!nRg$Kv9#DYF)8WiBEwAjju z5$U9YQ}0Hi+|q;}^9Y1~jOE!wnp;J}y%LGlL3U8sG@OvsN0Lt?EFeTA0KZTIbpD|Etu zTbVw-ofko>X4wFD{SnUS`{xu80f+Ku-pm#V%85Q00aI2XW;S9A+%=&!Pgx{W+KJHj zdN43cMH)AV`7_WM`yc5+e=5Ko=JX~3PGfvB_+lY`7*6kD2vk`t;e3$;LE$(;=ZLP@ zu>~Sof>oCo{Yz7tQ%shg;0}yg4gC)lDQoPII=sQ!KGWz9u|0LxU%(hec-yxa>!&piVRTSf-$krX7sY&utr={#VZthLYFKSYeKyZi+^eu2VWJk*=OC z4;1B%lJ&71t_{<2D+FyWu-KkFY;lXV4tq^SvL6kQ7j09D`MOQ;jgIKDlV@0`7)+Mv z_pROt0bOOrcADZ_q_GL4;8_zT3~zwRACWyn>@aaXM+kKIgCiJ+sKRYS+O)$6wig@{ zVueBBxCe^gRUKKW!n2gBW1kI7Evsx^&Fu~|fFBv^LH6rWR5e0>Yed{TK#HdoU|ZRv zXWmja8PG59GB)j5Gyx#aZ<*M3*4to4cF=*Ga9Uq=H+ALcy;H3BOuW21g*)bB)Dp@#8OFB*!2brHS zJms}fx?qTJ$Z@&FfBEv=bIp4IqZy&A?VurfQb%;g6m7=FtOk8yJ7D(mRBvM$H`E^P zA=cG|8%NboxWl|)!&^gt6#I$iR838nNOcYkxUD8p@8OIxdtMQif1knnMP*quSPg1u zQprEfZGvk+;u)*)fTVq((%U4Ho7ls->u7m5aYkz)x%x{7>?VZq%!AodyB{6-!GXzhYqw=eghuqDOa;}-?lsXF_H$` zdqPlcb742DHnN5pQI`2ubCM7)eq+R9AJX!A#F!_fyonQcrT}XOfY?30?hvj5 z51`E*L3M}BvJd9&p5%>lT(KjFR!7*cmiQ86e}eJ4=iqfp;%^n8kJM#d+I+pj>xDS> zldI~q*%#1bf8=KXethl10J3#bNI?mA11(bP@tODg~ zTbBTMg#X8v$P;?sL`WEIAeqb7o=-}Xcf1(SRx=Q@0Y$GJf$s@@9BP_<&wV)H%ZRLa zpv>^#_UNoIJ-?atJs9K(kXZZ5a7E!?A|<@gd;8ouX^_WGYh)ww0+)SbSh9?55vr{9 z!c;RLV{tfblTLnFmIPatHqqvlu1eSS2j{AK#Xg(8(dQAt7LQTAXhSMqosuSYGBJxO ztaVKsUL9vR7bv&6@YSDPbziFDboJ7iMs{DOxW2e zzzFhWm>tX@*q`3d#I0M9PRtl;4~r&=GqGBe>>FAq4aM|PHLa47F<@`yqj6?dtlcbM zTfKf=Y#_q;mDYKFS_XEh{0-1`s)CxowA5#p=hByg1PjTkx?z z`B*B*h^(GOEo^#jqk1LZIOD<7gW(NRc|_pwhrK<5DQ`p_UobKcq*n>z?%TZ*`QsVW*pvrz}&r^7tsvJ(znlvU#>h@mA5vzt`S_!Q=`6HNuPty zo$5a!xaOEx*D!a696HyQW=PIfZqt(2ur;^NUYoWSZfLMvBDRgg8r^^6s9fNh5qvwot33|wZC}aI zFj@?`fUJADSf);^qzDCH|6p0XU^qk@`O9OX6v(pwXqxvpm(%GwXQsB*?He@(8AGL^ zmZ+Yn;81obN2rU)kMnRt39tbHFbiO039NLH@m`o=jD}*z$NrxUGAl^1_Y^m{a~fvp zCv$<|M|dGqo}0&)Y$*lBca*{fh8?l?fv{l3^RkB8dO6H6*PfR;(8;kzhj9}g<|;VscO$W&bw zK4C)me2C~m+-Qr&o?@o71NI@$w%)I+LSA^hIj(=hE?8%zx!=_=i%GbleRnz#QP`dlNiEU1CO% zQ@G7eGeslqT}u`ch{g^6v{eocAdQ?DtW_E!X+?c~VkKdiw=0=-iIXnfHcgIC9dh6z z9`*fKc4ZXC!&BJTNXF<(ANTiI5aHj9W&Sgl{*MYSMO%460b%4bpyiYlROA+{z+XTs zIeagMD3TyBbr0qYX8J^C9?!5&)-n_21?I~Ky!*7V+cN`k7V|4Qcicf9TM#SyVD}4> z{&>pXOp^8eczpuy<)jf|B;6|sw?J$anGA2EB|;a*&~;$h)xeqJ7%C=@Mesv$;0b`> zYR}%8pE@K-)1&jzk#jmdX6iMGkD>v9n33oVYeK3s(Cku)kw!aE513?6uojW(jKl|H zOGu_4bZf>=Ih@nAHf&^$)cLz|rsyhGR_t+!DUM;Gk`w<(%TBGPw$z;BH;Gi#RliG} zN^U=UO*Y}OjsbuC6J*JGq*+{WMq_o#)ks;RDaVV#K(UfWSy56$V@^$TZTxXZ!_@*f z!^HbVczuZ<|6Gp)A02xf6>7nO3`De_i@A;ou-znih!VRvY&T- z-bP9qqR}QQ>}2xxmHI~vUWr@jPK_>ip~yjjh>BDMQV5LGuY$H=Ic4HVyjfiD8lZ!& z*Ik<;!0ojEGUUl`JEPOG@pV}Q8<(YZ-ywC6hP~8{JGp$rlwX;u`Km&d6*fqXYI8WE zjWpsO3Umrd77G0=-rt&2 zu`l%_!MudlOy9UqMpXJ)`j3F>;P{6jw8^Y#zF9pQfdQz4r@#%w-!&o_lIm{1UHIdB z$)(F;_&oAlG`S5c;jRmH_zySw3WE3`^9tFLz(IGr=n(~|i%flFaUGQEAwPau9SSJg zd=zXcDPE6a{nD!nyTXAkWk;}r@ssyUhYz3EKb3dapOUw`cKZTX2`)Xo&S^17yF{MJ zbl(1jEhk@O()|_Swe+R4{`=2|{GUIc|Lq}c6E`Cl^fj3`)7DxeiZnm(Nosvk>U^$> zkc=Udedz3Mn^#DMnH^TY-}%CohrhG2kzE1&8kx6!neQLpZy~?2+nWU04U~WL3p$dKTbk7@V6)mMlxwLVtl>}u>SI7>NcSnO1ytZ%pu2AhG}8cca?sd|?63ki zpD-Kjm;`ME0r3hh?!@<23Z8^R;6M%qcewCe1e0AhjnW;Uqv~?NN*~smYA0s zA{I7EqB_H)V~uN*BSJ0pm@sl$mL(y6uW_VVac!GxGBqMdNDy{iFz3DvPiZMYrb&U`dfb>92Yq|;{bdgQ z9;Y%?e4pT8fH<2I_v(7@o)TM0UmZ}%=cE594Olv~CY?l)nGbHuM5z9)AdhqA2hYR-6 z>U>f!StWy1daX)?;dGPIu2RYaGEC|fw;2}|;Bj%#6!{=O1J}@HRwRhk@;S0FJNr5q zR+#@D!PgGndoeG?C@DN>1+FW0@yh$!O;IYRRt4Yv5+cD?wTEks_dHA+H^8?5Fw?Y^NsfyFf z_cQdHMcxzXRlYoJR(>J-ZhK?iMj%Nr^~cUBune!?UkIzZ;IE-*H&;3%J5YC#0jP>} zO=aaFAU4;#R)5(E^YUH_c5H4ZOU7d|Q|M{SAW@6sBEE3RtHB8{3vFx>ZE^|S3<71C zL_>ru!THWHlF&Or*CS{p+KatID;}fJnzUsX8w!`4d@D!Ym0_h7YUeIhb6@eTX12~? zn_bY0HvRbktIj?bU!!A*C^U0*%3=;5>un}ii_vEL^KaBQO`%?^B8Ql;*5$45oBV!~ zYZK;=I_G%QF}X_7zlPyC)AjtfJ4}7AThd0t z4}lY3)QRMw>H+gi3&fWiVrBCqS_l3bL{>Gk<1KH{48hIOa8o0i0A*joW7FRfIUf=| zjPt||wrO39c%Q|-rEfv0&z%-7xB7oI@3Zft7NcLuK;r+HC;E2?@)rs1%ck1d!o=w> z81CQAo{!2er7xx@??58tJ^~veLfWV)8pC(g5DF@YFEeF9#O=@`WUv7lOQ!E{DDU6% zs^<+PQ|#_`#5Hr}Bp?u~%ej1>ubjMIKR)jF$^9wVI@50qhoefVax2~QiE0cLhY}-A zP^qcZRI6A8M}7qRhl0UKd|AL1BYBgPa5 zBK32TA*L{t>_dJDNf#JtZ5lC_7{kK2atRBJ@LHW|45UV+H|B1oJhyjhVqH#sqW(Z> z0r$PM)LV(ja5I^iipca0HBcI9oQ_tUr>kWiy$amOB5qTfLubn+=6*1@+_M_1JWDPH zrc8A>LmHKyK;^Zi-d{!3y5{c+r1*F^X_IWM{&tc-R;Jt?oDNQ#c;oJDRO4`~&U&D} z;bEX%$WfYQFd`2LOl`A>D1MtRv0sj!f}!nhY)m;3B(zw;my~kQV@0DV#>(`>Rn3wqUd3eED zqRl-*{4|xFMLeQ%OQ5CI7DbfCJg<^jUWE}`x3RBm>tMG_FhnWRz47$w;e5K2O<3hhvov|W*^E6+^xc>x4z|nN3sV$Qp0R`KOSm1f z(PDRDF7tPHY%T#iiZ|G){9U`75NPw?;hH9=2r<~Zz)kcaVo+_bm;qyHvz#+R>lrP`J#_YrmaUM+5{ zoH%-b?2TG%^Nx@UTaLG5$b(XHK)thO27*2@cs5puqQA$cBTFk|X}qafpcV$1cx#Bf z?yjHD7Hhf((#7~N%OudC$Wvevx@M#|f^0fTG*tO(*vWL7I0${xSEuubxZl5`nk0%+ z@L9fi91s6U2l_kio${9jsjb;x7wP{P)5fGidaEpnTxb&r z2*nE^r5gJaS|{z2!pNBR(+FrRt#!FJL$_#IX~mqn~MkG5_0#uvaHn) z#D?Q4avQo1Uane<jZb696Nl zDx*4~+@pv_g2Hh$HzFsYBz-+Qp?0>in^Krky03vZ=4gbGMoJA~k+T^EA@fzKsBkM* zs=sR-4x;fkD4dD#qH))&BK9hgx9kFuyBZ}Rd)045_)xeR@~2W#Wpns5k;54_BcmJE z!`2aUj-XG}6ON^5n+o%*+x}+TmeUuUMX#WzfP5{O6T*IVv#O3110(9N zI;@1%3aJjNJ{qmr$TY-uEVFI>^R`||%-5re)r3{05dGe$W5PQfKj{j`(yO|B|JYa4 zvMkVM;K8$u5qBGTXhpG@1?Hls-FH>bnjkP4*`zn_;(}S_+`4TU4cedIk5@W+z2@d| z$u@z1o^RyPSrZ#{F6DXBJss*$a}WNxeeYvC(s=KpEp?3cz~L+obE`q1p@8@Fhp*Ns zva&?uo@HIt>BI74#-(WL5ga&QplUR9vHcQ!jC7Ya71n10n6Q#*OTkLCf>jHfX$F20 zOk9-%?^ZhscujZ>?5R%Nxxlq0Kdm&<=nocl^(@-(1$=1XMY`o>_@6*UwnfnWxoViA zt;l9HCh{YI1qo8DipE@7ZEhE?HK&*1v1D;=fOwb3;ul*f?s%y?d}~XQmd)5LS=~W- zQVS#ECG+PpL0&Q5cpPK)y}Qf9xn7jcO?yNuK*DUH!kq%myzb`%{;Pg&oo!(=X(GJp z4C@kYZX>9Pu%4OKU!|zrt(109Sn`h@=)}~d8_bV>k^Mz_voj&Og=ACS7-9^zFFBf`ExhY|8nh8jG z%c5E|8}*K6JO|ILV20fW266v~w6~0^q)WPl3kv7r?(XhR;qLD4?pz9YcXue<-6`DN zp>QqSokE7EyWjbGzBNzJdS`yzmFwoejC1ltWb7S#2ah6lP2E$V{k*#$cQ=BY7Mt@D zBp%#|>g5p7Pd=4|HlpL!c0lopCo5c9p12dMkH4cu0qAIY&dyXb< z0Xyb^KYT%E4e8PGj%(nTop>(TP0BUvn}RiKb$_k0o&7WP-MFzr==OxnuHXaFz8hF_ zPaGHobIgPuwhCyQq%tz*D!PV;p1H*yCKzKPP0pbGF0M~2Oni77pbe@mE!IP$Csu{Y z6TJ2Z-@-Y7Ej?!y-wHN?%hJpPR;klt@<@MYO5OnR1EP_ze|+SH%~*qMx`8~4#}OQ| ze1CxEX;Lp-P-C!D#gV6+$v-_kYVoWI)_RQszEPgH!#f>KQuNn_u{sIOR`%pU^lug8 zC7lElp4a2k&znLwT!$L6vngOh2kvVLJ6WhQzS!>CPosEZvAdA&yy_fqa5j%_NwFJ^ zauEyjLm>rJn}wvY99+NlWF@bG%F29t9UFVH(zie|RUo_a@iiT<(lO(?6;h|DPIAIWd^-FFlXq>;FUUq}3z<8Vm>M|zplaEXOS@|lxx z$@@oaM#)3R9h`!q2%SlVVv;uYDbwX*QV#Ypf62roLpWv85)Dhoxr`bytE0!+QBm9$ z22G@5atWp+pqL_Q5?YT)B9l{$vpPl4l9Y_mJ7vg_myA<7g;Wxok4ZVD*btkK3pvHG zCIr`KWYO@9dLR==njmoLgYJWll3|S5)N7QFAdiw@jEdB&oQw?a5;Gt&w22JGn<~lE zCqwTurwgS-W>&`iG$~q$8#iTb6M9IbF(q%4GK*7Hl2IGypCf@BSyYm!jt6zBs-)n; zX_Nb560MSOYD(88*qn%Ciqj@(7N2PfI8G$5SKBSrZ%)u2Ag22zv?NKT$zZy#cAW^4 zE^n6@z3=u*0K$~_7oqWJ+yqHKbGS_s*FO9&K{3+Ms4;G4@@EqrtI=%c)O9lD(GBJV z8)Sku@raS1N`e=OR!-fOga~1wycSnc!`2S@z0N0*0DM(_o0%W)0ljP|(f~ar-`Q>U z`&2-Vn(wde(ia+t)begC2dy`@!0`orj#lY6xIpcN)cT5-ObCL~Zc_)DH`~C(lS@pU ziudZ?$&*X0^@^@2;qq?#tMV5}h}a5WirTVbaMK7H=uPkwfa?)`0dxHwR}?S9+Nu z1t3S2g9ZFVJI&~x0C1tOW#y3D9;{u)KUSr*HAJUEV0^OZ%LA(j#KzJONu-guq*evcOE9tBtGjZ|9b z4OO|)CxfigM-09^$5d=_`dU)q+cb<~chATILp|{V!{`6*iEQ=mzOcobdgjFsMxgc^ zHjUz2d$0J3FLr0XPhjW?hbGoZOPb+*H(*n-dy;73gZ)Y54W6NF>ma7|!}u!yh3_Kk z#apfQAkHv4^w7*(f+-|XDHv94Y z>Am$IvNx8OvU8RWp%F36+uqBG&f5vk=?c8=kF6HFFBW|S*_;O=ec^>D?I`-LyVi2A zGvTmuoks()SxIlTIpQogp%;R8yUJN7*55=IU{v z_RzHHdK?qg9>$_ZRR5wzI6cPiy*OCdy!Ki|NF9x!*mehk7qIZQZFF9aRq$iPLglzb z`t{gWSm~=U!a#aj(QZO<@bLCpTyZcD6~sw!1PCxL@{u^gph}Sz!eQ{|R@jOwCJeQ` zJi>nZO2Jsd*tTs^c{sJa76^jgaO_G^kzVH!XeX^Wd8nJD0DB{uIC-ik^sfAE4-gL! zeYGn*6ErW3T;)&y98@p;I06HaUU@^xCoFvRIB_jRsPQqOE zD;lD^<)_8~D&B%^J}V=;t=-Ci$H5^;qi3Ed>ZhLycaVjoDobUf3-HjYu2y3s z4!fn&X-hBHl|vQ^WwfeBnE-oZ2R*y-0!pWHlrppp+%u~#T zGLK9}v%6ZGA)ChW6aBZzC83Sjj8eD_YqhzMWwa?WMlYPJH5N%`jK#d_`TS%n7x@D+ ziO^tnR_Sf>ZL3T)_-`;;asx>&&bM`q5dA(bR-jU38oHAu)_&bwp%%{1y2f~ANR$j$ zvq1arccFbshG^d(y`-jP#mbUd%wubRMbZ_oHw+03FN1F%MD0Zt5 z?#5Z5h-^}OlR0{@b5vBndC^)MOG4PGqccXw&Q#yPtQEi#8QpEbwTdPacVX`54+ssn zK^c6r-gT+g@udxPj69&Pv%9+I@_Q{uGCPp=cyh3HXVRj0l=>FmB;hBC6)}IImh}&d z0OhQ*trf`*tOr2v$qC3SJQ(2TMOO5qr)~@Q>)E+IP;^8Dl9Ia2MSLNNad5#}M~&EZ2Yy{c$VR zt*r{-$Bi|cS3^nxHaMkJQ|CWiX(S~w2tCgY~}gM z<3_17(D@Y(D=SM~*+wkO)!(def8;oqZgy(pD@Y^8o~eU&BppSwbR8A9z}ngKJLlH0;@UlCtD1g3 z2Q;{gFA0evx?M=15tBcJQitRCX`;Qk^xLMG1Eg zbB$`S8PYTh?#$OhBPcqhxivc2uIUXT9TiOJ?}i7!`K&7RI_4{^2zbBy%{YuoMR4x- z@y?OvrnZ~DKKJ#0T%{!%OTqBGDr*R=y)~c1cMmi9hxR8&D+64MKEo^}Y};B+3a3g8 zM@l!y3Z%wK^RvYtMpcUh(}W*0D$|f2m-e_phSQ7YUbwaV6$ye{;EBdLVDJ@bz@hxe zjv%*5Af!Wc1kj9A;1!hf`5>@fmue%e4TnDa%t_i+#A)82ERZZMaAwF~8|xK_5p_kS;;E{y!} z^dH2>WhkzSw)fozP5wF3*)4LYKg8VGm-ll_$_tMV7zpwmCDtH|?K_sDj$!l-8QtcL z7iTWf1MOyU!zHswbA$YxX1XyWqkpz;G~C1^e>hOD7J^1l8ltaLuPSG3)FF8rSWeCt z<9G_+v6FAd1hdSiVISRG<+?-%Ozlu!|M(*P{Pb{T+7*Cs%jT83oN#!ky-xNHu26fR zD23+QVF#KC$&Fb=clY5h$Nt6KubcyXh^N|k*6)LwT~5oiJBv6K4RuK?`HwloA%n`B z`qINz>C@%<2c+Uh;jVApNv?2po(-KiQ9%Mmq22Grr#Pc`S$}=XK6OolKD;y9mhFf8 zHK`Z0nA@xSkAGnBgQnVxYoE0eBl3T$mAL<=R{EU8H?=b{G_o;e_}8?U=>NPU>0)a8 zuYr_!dFg&e%%R-IhQ&(m3Jx?E8He}Y?^nbqg0Elu?wId$sx_kWbn&^%J$Z)et|!%V37r$DgZ7v_7y-u|OYz7PdnK==tFKL1A`k>&rq4*$Ca zFpcf~E+hgZGh%qK+#4$c|Ahmgf!Vv{Ytf4B^sJASU zf`Q<%DR@R&O*MVAz*XqzaJNJca&BzA4WWH5Ni(SLvYx(!uM}r$Q9pVWJSP(3j7NSri!gr53y{ePblMNwrguiNh z?~L-y>Cb2g{Kt5G`#;A^)y~%5#L~?2uTA{Fi(Ry;wzBr8-Gv-7Co2~svPBUgxRrDw zBGgY!`ebw@t^K78bg8N_Xs#e(;eb&twD(|^m)*oz??o)>=qXPAMXcO6=y&*g?NH;3 zsIJPUhn8;FU)JZp{ycX3yny#`)>E^q@G91o?UqHvs`4sdheyb(+LX;q_rww*U4~(z zG12dqhn@ORV%~j{c*BAS^mK0fDbfgxK^nIs!Zf~AwNmaTgdrhae^JpvXTlZ0`GNhL zG2ajg7Mp*74o=#LZm&5cv^D?J0v?H+jdv(1#C3h)D&2spv&0am0ef!Oa62XhP|S8V zHiudl+JT*8EGXzs+TK~xJ%ggjmnvCV^bloht<~G+Yq-YT5H31Sd8F3W6Rj{g6yY=D zM+OJI9WAnq_pIX2P>-pr;*e5Ktv*Mc)&?l+j~{}`Kxdl`ditho?bMo{iSAqkT^CL> z&5l`SEn;lk2Z#Csw-_kPS=BO`WjR^v>ipc8PuObnJ4)5_n<*;(aam>H0>v%Gt()PR z2BWFxwxiv*Ty}RE!)XF7F?u?5$hL3fF@>k-(+Tx-1*LhDBbvb^axwnV!WDVbYPR25 z0<4ovRRlfn6ABTRZ`pEqeZGnr(e#q-NeB`|hc@S!#*4N4FAZ(D@>#P)wO3%MYogJy zq_V?*g2T4$>Pmv`pOiT>9&s;g|N(`J$^w_If%0b9$F9X4dAcAd|>4ziyk!i$v* zlhR+UHRL$Fxk|2Py@(QL5bn-(NesIPW393Jq?)J(b9(?s3bmXnjd!aE85Ku3Ls0^a-)Q zWt-{}B1HcK;!iO927`yDf}-L)V*qZ8(YPVVBfjY%IUSF@`9Sy%!E%i}MMG&Q(0gJ5 zZBO+Cp@_`~;}gt%5Sjg$Ji`DRtAT-gig~E_{xAbcolfDxG%u?+B!iW zMG{~+0IX(uhlBR{s(uZ&TpN4JIRNJsp7{jRg4hRRoRtMkBS;8s@2_fDCc)~aIkYN* zC_ZXXd(g}aO?~;1)p*v>c7x+N6q6L63?!3FTa-G$YZd-O5An-6_f^PFCJ`x^>3F~- z^QChI^EiuJ4Cp*`OJ{xhXp2{1{cCjlf`_fct%wifNfY#ukiZ0Ec&q=?5wRld777LE zXLpt~FP5$rGq=f`v~?_td{Y#HxvRTewbf=X0K36`lG!A95?H)oz;};TM!;6&wxrxb zd8z2>@J~NKU=KOo<0oG7S1;l3`v`wWuraf=aWQpb_=GJ#FaJ!0Joom?{16ZjUm@h& zAl%#_;Kd*m;c_bn>P6nN20C55KME@s_K~e8Da9b{S|+j*7Kr)cNJ+k`JNQ^CI#Dao zCdd2JF>!G<%7Kcu_ z3C>(ux56HFo`DH)-zU3OzgW(|+waaNC6e6N0MBoZsRFMY2$Xw{)ONGYL{2$P0{xx% zvCq1^rV<@uXa~J$bsqezo+6vQcbblGEepJboF zY!9O)5DYn@b%D+vnNRLP?;1z(p_{N?aNusp3Wqt6e{BdWgkvPpIc3MLGA*)c7Nb{N z0(FeB^UuAp*@t|l2{>4<-@*o=4mR;3l>jJKaPEp=3;@_RSyL2E)xAHb<4LS+g zx3DQ^^r7sDKmIZDpUP^|DF0bj(f&tT?%(-Vx;oh~2pWA#QyUt)e8LF-t3;RDsXnSH zst-g;wjnkWJF9x)j0!qpD(mC`t%yW~DrhII75f={E%!uaKdry+nNRl$UsY{~Zl!|t za-Y7>k+!d-uV{CU`--l+E0wQz+999wrt_3%&TWqS@3-5_Eg#642z@jFRTxzm4I?UV z(Ka|Z0}W8Tw*zKV*+FtBlv!?+lWva$i(vdoWT-KUg_%G3Iz4LI)Qf0O8cTolNo$A; z>)Xgv9rE}L-F6-f91qwT5+a&|#2_f3v{WcGA~+0+Gw+91r`jF}NP7gfOBvL*56B-d zTs1`+m!=gmgBCwM5CrHWl&i2WFZ6wR5raU59z8T^5ePvf{;+J&?_hVZK#f5qrVi9> z>iLbD9cWT8Tr>k12e3jg8A~Acv?UNp5{_XfEkr%&x}uh~GXx3F9tx=3BMD*;4I~(5 z^YVj+Gj|l}=fIr5q7!5D3KeI2B{efQfOTNY8fL+!56i>2P7Bc&@oV%0h><4M=(&al znN}>@1Mn8fLV3Jb02l32hi@AZKLCt?by*q&pp392wghl{eijWHBF* zPh4G#Bq;Y6Utq@^))T432@jWE+~Eu>g`lylV#BWE8V{ra#9cVprM^K&!u)hYz+*D9 z)85MqV#`0pN+xZ42m!Q|goOUQ*ZR%DFjAxLE;-)L-krtw%Z?_WL3c7i3p(=4~$^=>~;ql4JRpu@s4-MD4B zXlyAhy%za`9s0dWb0_5?-*I`?nBLp{L~Wfl%as_v-(|&7SSYVwF5R)Bw<`A6QqWH8 zJsfg5_aB~BPacHFW(0uC!+8RsV}39n^!Q3LHV$(Zh@gbfmHw*Wd5f~##E}$Ke|_NO zgrqz<9L-de@q@hYuAjdG71xd(Sj+3}8#g3e$#>6t7*hEZ&2>4|0 zm#poK0%SYn1mh0333g4GF1{pY+?N-s2dsYH{hXde_6spU$nWo6NceK_1=ZvELI52d zIgMYH)S_rScMF^crfnBlJVauhl<}FjKRY^{naxNyf@~c}t>tavt?b4}tM9N(X!hOt ze`NMIvzQIEMT1{A^~I{u51v_nBpsY2-eNx1viyEtNJ<;MC=Ws#)NjITuW)Cg+o01h z^QfDA3*7iAnyjVsG7i+eS{I3Lu$OUoOd2}(w;(9(MStr|0^$>Jn_i2+biavYXh4wnEF4$K^ zxyO+E&3Q9G<$C#mQh7S}+cnr$igOaD8mnpT3okhIH1b=H6PZ25w>^m^7w}` zyAG@J?_+OQL=jS*(L0sTReDf8?jSAM9pqPnz{h~f1A>XNti5mnQ)TSac@1amCJ;6T znk4qA31UPvYW^y(i|q#g>1?b@*#fd?R}kMk-sq4K@C8L@cLAxr>hv7V1rc*ayh%dT zQi+0i`bc2ra*!fim?E(2q(RCWoRMU+zX8o6Q@h80yd_y_!Q@Hzv$~7dtZFL53^dwa zbb@nwZDj!m?cV5u=tXqxg9LZU3`Uk0a4*(lAMKTeb+%iyDPr2ptqjO_C63dYtW%aj zu?&G#SdE8f=yvgZRFwLbF_qawfr1)oJ#?GE1T6V0cn{`jM+W;U0}_ZU-jgd(Y+#Me zL6Z|oo87^r6C^_lk4Mp&DGl7NuG*XMP`Vl36yqG#(jViY%;I}2vFssBFR97Y%E`Uw zV!Mgv%>aR4WHw!4?Gea$9TD_!3%v4e0QvWSbgFnsq$jyPsqVCZ|C^%qA8Elhmvo{<5>Q%QWeC3&y5I1fe@fE@sX5gR-Pb0#&jA86OWIT-zsyEAB9C zpi)5}iw4vDI^Sp5H z;(N?-c7MG1nb`s|(ZfcOAm_qa*B_oO=fZtmkJ6E~az3CfhwHX$44o^%NxV}UWQ^QS ziks>n)F%m@EA=GPcZl2-=O8B})#m|?LF&Z@kT(bifV(n>80SYB(Z*eKszF(x)~fYz zgW!PMAYTRe(J|`;ob@$;ra-UI8k3?!mtxdv^iYFTfb&x1X%0~$v17}EE{$LYLV%o_ zMh)IFuWQgNtI;tJ8Ji+W)iy%y)e81jSfI_26H*JH)av?`ZOeA%8xR=5Yz|3-uA-fZ z>-2*3dP24bxP#-0DD-+NpyBqZfnUNtePN(}QqUmqr{N5a3z+Y7M_)trf)?&w4x?-h z3kr}HI;h{)71QWNLvlmY#)9EHfCCip9z?G#z%LRgH--5|TZ%MEu=T4ror(?A3@sYq zJc{#hze%!S-%>=U`RY3FF{{E3wkq0|Rkk>g1WwKql1f{QlyI%i$N3tyQ|v{byo-)H zq~Y)`Oe`WE9~Q-)mvC5I7fPb&*w&zXM4jc1QN_8ZnET0iXIfS6tR?Ee-eh$(u?C7i z$@wN)%>_h*Xlg1}XtiADrBmd}^eq}KdK}?|j+iW8`I81=nPd&I&qqO0A~BJVU}k?T%rSLtp;e@u0d88c7TL6X za`dop4OM*wM6PF0*hZ2q6Lqi=aLFM|4FV4?49|VC4NJ3sSOEGV-O2mYLV1yICQQb~ zanO}f%mUGOdDMd~1C4YdHF&T|2HP>u^A~bZ;$6#)%&q}@w|vH;2g~t-=c)GX;S|VL zK_}ejh6qPQwxbFUN8((8kF$kl_(Y-wiOnfD2z0X(rM<>3Tua-Hx1E8JJWAl?)0O2) zs!dktRUC%pA?fE+hT$gG#M+_xi}a7D|(Z)TqlN= z9DkU1yjk1LltHX{=AQOtGH+`_uQg1cp3+h!EG&Yx*uB5y2Zoe1LrBYA-jL_lCh(&n zQC%ZW`U3!&B0^h}VfW6xnt3b(mRjTFfi-BDy{?5OR$T4*(bKFT>!m(nTg3=wqc?OsKn zj45ppb8GrjmYKEqWnTtf1dEF@6xE^*1QL4RtYPQT{2XNT+(Fth$#vroQNp%}5@yDazb=jP!P-i=PK(lI#!s!&9!g~HBH6RVnjTpp*>lCRA9;$#=)T>3)ayJR zB7S-Kx+dMgWrst#R1TDgv&v+wYPF+C{uAMPLT?l_d}M2yupL8-_r|1-2G! z1vCM)i*{QF36%bJ$6}|K_U_36F!ivbQHg|z;o$FA z0j6`_Z?hx)MHdtR0u6Esp2e_DOz_(v5W#u2dfU)TwkI3385Ste>wX;+VD6adw+6nR zzs=<(-1``GbGuT${bzRocBODzl*kilOXW(~M)``Q?%LQ(wO8JuARu?w4*XmHB#a*N zRsG5nmz~lTEYqJNKlPwZ8l#8uv^S{mUwBK836z>uI;J*EIF#LUQFz?Zy zS9I#!ZG7-Qj;Vb~tCbSSa1whC6~qS)>Q2xOdxFrMFaN2e2KfmNGu|x2i3yKwg zuh_UbB!j6N$Y}=$bqd^cyWO%~UZvwu-H2AYm@H3GPtMrs<(GS4|<)RR?$6 zC0NMYOU}g%Nl{M%FJZ>d8b`z^GiqG}-GZG({S8}|T5@H##`*i>J7*WvqZt!D7JRMA zD>ivUvVNyV5fdaw)cxS_l=(`c%SrAaz&qT?>Fx6KOsQ`-5LFBopsa94zZ@kY(_-eR zW$Tj0tH-zd^ntv;t7sd(=(H9?kk%A%LGso3pt1HhbY*!N5{i3Xl~7U)i_`gmhAy31 z;_osVqxO^;Rq2*0y`@|iP+#<$tKCwhrc{NZtRk$I$cm_1ETU&*r}6B^(nT+X(<&ip zk$&NY2^4(EG!3ls>T+wP3T*5XoB(R+g1M+8

?0yiJHX3f>NR%-yqM9SCIn|` zTi~TO+;UoL+{xd9?(uv-5dYC%Gdn-{;6RR7{r?+>-2XuA|8%?pg*RX?B7J1jSmAoa zuS{7S+amS*b zEG+u;O8iK!~Pt*JVqZ@IzMS&R{q z9PWg3cER&0EPr9ID%bQSoP|a@2|(YXPY?)RR-Qb4q9H zOKuJX6FqjedMfcXa$7c+&#m+5|J~Pj1Rnqu*$-qh92n~OC5D*h=d9P|!JY{>JT#&$ zb)?)Ag0a;Z>dcNuG+&ny>J>F(&G^eLwbmfsC9R+wyf(*U01hENIHXq4xL~q?0gFLO zS8h1od6PzGJDC|FeH29s4oJr^o^u%qk+MWOJ2a7!3J1kbiTAoc8;CS6+^xD7oIe}N z9PSC#PchY(Hdi5Cwlv8vNhU+2NgYRRwJyvKVCJAJmz-rRErwaK+x-e5cK;>0pAf6S z^M+Zj_7P;sPsAUcQynv=d0rg%BrPl!&M0Svb}{$9E^K()c1{(YJvsa$0Rrs+t$2xj zb9^&_l+Nyu%szGM6)5*Uz^T8BNvFL?*J)8*%H4*og)J#87nz|d!rEb=+%Lw*zvx*-KHSHK~)^7vj6I~=GEu<<5l0N3<-NVsB^+* zxHPws8XHUglrBmUB&p&nw|eiHU`u;vj6b9JjNUsDvK$VhRs<-SKD9UW{a zhYPxEiib5TR(=E`r$>G;QPfb4b%U52b6Q@`l>qjkp|K*gP>;%WLNmwvq5`K2wd_)@6!zE7g=XGQfYm~PsGZMpGVKZ8(;(bBUvzBC?y_p`3`7z%l zm6XFr_w?AZ*O{E5!@XbWqs^`R0JA$?#THITpwaFhG>Bu<*MNLJJZ97*M4*xB5w0gb zSDrkbM0M8g69k(>LvH8#H1@`Bk(y5(`2+#?W?In>R+JnuIH`@cb1;V}eM+cWZkTQa zkr?WcW|luYBTdqj}V5%W!B zZEohqOfU1SiFh!tayaI*-uwC1j~ItLrZFg})}s}B1g`PaW)fJP7v(VGKMtIhq2)2+ zF|wb@a|3Xgg}6yjqXAP^6O`0)GYh!kT~0|_wdoJc$DC=Dek9=onx>?{w&N2 zo~R8O=qtrd4LR#CrLeL%#~?{G4Zhru7Ss_SprHQYlTSc*=Npk~I#$#p@E{cs9$dke zs6cgcLb7UUKR6-fG^;Ya$FRPdhZ7D%PA19q)RkzH>5zvCU>dDsHVX7LvwwlABiRJa%by4d?RWfBEt_mGyi3;*SzR zMI+4eyK`szC3wSnW}3J>g+|r%G?%i1fC=?6@njBWUZH&U3$JwaXh( z9aRIG%?uYs*=9pi1??$8*VkpPNuw)G4H!TMs?kLxk(xnT*-Gjt=Mg+*|tF=H*A$j)34D0lR{nyO$YM+MdhNi3^1;7-M0g+FHi>n1 zBiAwA)sP{Y20MJI?wCRGOq4rcr}LY$5u9ypLbMHDi;5diT`!%5on7N0Q&Pake-8>* z+k*@30I;ndpf;teWJy>jmYD6P@NbeRB%?bR)bkIm-&&-1#O;;sSY;(iWRODB^*}-3 zvu;w#|FX}N(J@t`I!`QUr4q|1;~-a{vhU=#iO=(dYhGVqafQ*&N{Y>mXDSeEl1@W!o8*#6!bY_`T4dal_>y<`W`5k567Uu=$=;X& zos{$Z$fgQyIvK<6O;VSra(h9q2HVpQ7LgI3(!P%%Jt@D&@8{vF`@=9n$#Q)}G($Ac zmBbxNckWzUbjEwup>6QjfRc8SGnfo&YrIOEcy=jMImwSK3a3IHoA88fv0H58<}Av; z^-WnzO^|2q=Om|B8?JFwS!HiwEium7BP@mc2Nm%{R+`*y`lng3xlQRg98tOX4+7?i z!!v7tzu+tz>Yl&b%imB%=f+YeVoH%{sNx8@Sgq@*```jK@^hOl3zlc)6Jt5@dt+x$ z&F$3%8{rWYElcVO(pJvBwVL&1uQ*X@2&#W%YI>e>`6 zujZIEW@^|JS6&+1OfI>ut;sGeS=*%X$o!(PLdPQ|A8I}{yeJA9dCv(oEO>B4Uy?6v z!s~_#r-pE?xcs(+RwWlCW?xKGmo?`1YsX7&JX8J5q7xZD3t3%t1s3&VY8=D&mF^#m z2dB!Pwg+K65@^Kpf2)-L|16*Wt(2dS-i_NgWn_|v60Ng0U98kHiDw+@lgUGs%+K9X z^Q?kZf+*gcf=0)gP7`ZQQB!kHaSe#H$M5$aV!4Q$ADa2Jho_?TskkZ}&C9>U@{j8c zr)i!i-k{*@u4A7sG-caC-Bx4yrJNw8e1TA=azVMXXvuP{xD--`#j;F7>DatXL+PA( zF|Slr=7nXksMJ-aa#>l7#dP;&F(tqNhy<*`Ms+X8R>3e1Y1tKyAh-g;25+YLNo}S? zg4+@zwlbhkgVsH14O>!nTj7%FG2p%ee+G>HFl@;n7>m(oANePYtpF2&LA%#Ja90wD z&d`yvO9@P-*Pgsl1ST_R_un`I)#y9oc2(e(89RWxE^yUm?%@5{m|Fukrod+Uj!H-@ zUHJg4ju?XyNKR2Jdfe*DUTJOB0L>1~VVCFBsPkKJT(Mht5WA}a+cAfn-{5^0Z!vrd z_Z@J|TH<%dIkS2!aOLini*F!+*=Ns)KBZJU20AJ`bZuqE_8?tc6u5mNnUC}>do~dk ze_n-!w~C6btE3zYq-AsVGkfW&&5u&CfFsh&+tJ|7E>dsgmajfHR2}U&wWWpIom=8{ z6_pP@7gZ(QEEhg=fuyRVqKy20AcEvmk0 zs;%Fs=-3QLgLLxk*+tqnb+yjtj;hBUtu{_X-x(GBuHv0X$o}Atf3y5_unl4Dr_|z~hBbP&vcjdw|s?Cb== z{0baG3RGyV`oCZvg+c1Ecwge|N%H+Qeu#i6)iAW{=Fk+g!Mpa$d>Zju4JK3k?w9GX z^(=R0g8{~RJ&N1@*%R|Lk@VL-im1LY5X_gT$_V}o=B-IJmYQbmiOn6glM?ex1m!U8 zV}qbja2`4{5S&V5?|Rkvrq-AYi6g(z2)Gd>HZDbwUE@X~pqf-wpaKU$ZPjDV;7{@!jH!6weO(}4a%pBn}n)B8DrDJ zuBe{T-3Y=XGPOC$9CcUDQnWooW)o=g(HxB4)F8fhueX1y%iF|+aO`Mg7TKad9t*fT zw=K=|vhV;*6NmWhA|VrzN3QX{lsJxgtQ#{r3h45!))+!q%#DVfa-SNHO9`FsdZ8uq z?y-gE&kg!(yQ8_v$o%{zK*5l5ghDy{S#k9M9?xffwk|zZ{HCS68mexoUu7u;Q3%SX zws9KQfQyT_P{V{d#3f6XmMOgeCu2o@H;dC>PWbHk)M3aX&b|;vyR)#;0GH~HxJXG8 z?G4fcN$AccuFgrlRE~!@8FMptRSxabvh53-)Qlvh6nYT9#yqq~{ika?A5u%zr;O#Y zOJ%F1J%ar1GG)>TOqrjABb3Xsw+w_#%SK?66Tm#Kd!6ks*z!X|Vrjh_#8cDASf;iW z<3K5njsjf1EY&oFFP~K^QtzwP108?C_b=D?ZbK|?G)~qq{)O|V5XYh*oF@a}yy>4H z5YA8j;^WGrmx|Nn9jylF?wP;8HtDT2_m|5{paog>IaL_IT=_klGwOpBbh#2%|K15N z)Up#7PDf#bv_;Q+zb#!mN&#A^h?S^-e9^!Yzl&*d>!{&5QT8Oo66gVBvB%*`HV4n} zF|T!2ESOXYyYZOr`br-51iNecwSaS>57jgudd%Ws>HVSgEwrCTIguL-M5Yacwpy1< z(>BbpMZ?m~ns#F33v+V}4Ox2@b@2yN6BAG$gG9(IVKzg;vy++`M+TIy1Fpadq72gH zml`(NPR)?H*6p3u;JsM=Y|K4^T=%&U;TASoIxs&Z$G$CO*QS|z8}LtdgW-M_!vax8 zsS6B-$e%P{*;3DuLW7jJ7@5*gWh2*>w1O#Gm^rQW0#Gbr0g?tyH?W8vv7bs)-<1AQ z(v2)PHjFzDQ(wwzMenjt}CC7r)pS7(NM3z@MA}mIy3C)F#JPlnWD`|Fi z*Kp61u6!f|l2m@3W_MdwdIgPPZF*UAMEAnNt|&u$`ey(l-IkCL_YzaDRkNJYksv(EqyA9^%LZ=SKiIU zzO{sQ=l0Bop$$2I%e1Uk?OW2hXl?_0u)bV|}xPbxRzEyB(d=OG~gdz1r%S|UnA$q@dj0h!Y@XN#-a9{Kk(L@W1( z>@O>P>gU3xs#3V`KDg^Lp(Sc#6c_y`yV8xBV6ll*j_TzQ4QqWA-q;dM_P{xHd^05C zT1Vl54@qNMb*=+kI?0`**9a+H(`Ln8q(QpyWp6OVnQdrWLkMj8H^!~d>9qqA=4zO$ zdzJ`;NMde8d?c@vHklmP{Y_SvT$##@LgGmtjy?BDpHR&pY=|?CxeC1DOjeg5T(IV9 zgPKni+LGS~SE62$M|@&ORvhop3RX!i^$r0$uEG3+$bV|QR)dVxsYY#kWgzN`8DwBK z;JAN!tkaIRnOt#(azR}KOqVEHYcTN>O`DYV$}%nhzsfnhqZpVaVBb+>S2j%SXYhLt zt<)Rb;kwo#4pp8Dm|HLB4m=&CQ?#=W2D%+r<;7^mRoRJ0L9%*vvf<9ZW%UPl@s0Cx zl35E8m;gP(OQj?YW=St>UQ3Kyu+m3R1 z_gM_Ag2@)t?&Lr>3B2n*s37qSG6lDtPOyKPhW^f=!TeeaI4zS$p1AJgG3`_3b;7|X zhBt)O{Nk_=bul@k_+U!iPU6i(E?LLX{fX(2fPz=!Ci2)^Fxooaxt(@c*hhp%woR}} z2RNHZM1gW{;Ls4%E_>K%Wbtm!C>;-wvrgHRI?-6e5w2>esJ-?Xa$^E3tE#M*Hn^CQ zsD5;R&y9~i$#ErY;@A1lj%c7Q&#)~ zv(~i5IfsT^{ewC4DXvK3e$&{Sc>%d4tRq)zJX%`ic*aw%WlfwABmK#a20WZulGt8iS_4^>rK&Oi@>wn@dBzXTVmUW&RDh?3!R?f+sce@UlG^tXCx2&MiwOf95qY8fubT zkguajAK%Bn7CYtpF#fiJ`T=IpaLoU*0b7N>HVNo$N>1dMI0buZdzDa6ITI3LeG<|X#i+?3DH_xQ!XRhi{ zG%?4a#TQUYviTfB6df^$B zdtk)}jBEM?0MMA$H-?y?tVy`V9U{LN68qAKelSMrfa6$p{=FAsUEw}CK_Vn#1_{G3 z;RN9f;W*(i;j{q)fLa{8;2tG_j^I`iKu2`z2&n#f3v!~3cSj9Xi*ErAG!p9y?QsIC zNp3v>c0X@P0d}OfXaGBsTUNkwygOGY_T|rB6W5vmNkV-n2!wS5f{s}Pl^?Z2@Ex*# z={x8!I6=`A_ONf_er~I+pTE1QZcyGt{fc*a0mb58Dt=}=@nK)yD8G4y z4osseAatt+LUfgUr+Sk?0zhMjN&*-+B=_9H}W=<{@LPJN|q-TOQFOlvn{SxN1i7ccjSsH>*AUcui%g zz&B$G$HDIrvKhi$q~2i384Lg8RNk?$tG6 zOnm1t?HO$2{@F7}FML{qf=!FB4bRlztz;?qun+iiAQWmfFMVUq$*Az7wbpOe=`meh zf4AeD0zNx*m;{;yZyAgW8TvsSQN+*94Kp7<4}=|)`NYEzlk9| z%1|c*AiT^kMz6Z?C7qeT2(Mws>r?8Z?oemDYyc*%AXtgk{nQ)$9cO{ z9;Ww4N+QX!kxy5qJrrEI0g)LeZ z`eH}5zcc&bW+(P*?vP9>(#S`mz&#(Kl{$5lrzgY=-08%taE*F^Otd;Ov=HZ)i( z%FLjq$`5;TEJ+)5{Y^`snm>bJX}?84BR8hpb=PEK-xll_&X3$PL23mv}&2J@s z4$E)KdE9dNr2Kf%g~&eeh_nszrI27lUJ%MVPVUbkzac!&Bc6kS8KdlN7i#(7Q1g}; zyB`y;=!(O66f7t*Fg346w?wZ-e~GwxMKHHL=X-OK>{;d9xwO{7kX!#EG}6rA{b}0y z10ITzrokUsU%419=}Xm?AJiybzuJ^ixorLLANtKQhJ{()oze*B@k<*s;{pXF;bXc zol-ZP%O7HZNh5PU2_eIil`~Xy-?JnvCE<=mlj}xPp5I;AQZLX8w{mwBdlwTW9kmq< zCt95=z?y}$S7V$@HmSIvNbpt0t?vm;T_Igl(Kpl^$kjbY4b3 z)UKoNg`3<8M4Rz%VX=(3-fxk%WOq)iJxoF;4+WX7a!Z(HQ2Go4cF3Q(fzi|=*^A#6 z&^mtZYhz94UBQ-QzedLWUK|{c_|4+9KtM}gkMw*_107U;MXO#7<)C=bsLPz`Lphbw zEpeIi1>!JbOkV<6H78zv%uFgs?~D=8k(woLSz7xOtR^Su0Z7mP=|W?*en>`6jsBaJh)}xQAc`Dc4OdtcVTNiBER_Fh zA`i`??}zml8}Un1-0IpylcL=(q$l(i>~Gwm+i1UUx!7H{yMf3G$IwEQX$b=7?` ze%`1hsOq@ED~BVs%nL8&Im{)svTrm29BiKxLbbpTpTU2&Vr|ab^FnQXcT3$-DN0AJ zvN$=;Y2WRK`vZn>M!nOA0QpbEn9Pl3h8pHWD^{@qlbALXzJb?@CptI}{+5;CxwN@Q zmYC-yJ+# z({>JPktEl~*2N0iab|b+ir@33t~9FfuuZIL`(rBeJ6*v7?K|@u2av6!k$u4_qmhoQ zkJhZ%mbD~njId&@5t~<;8y&1>M)D0CJv28QuGMVz1uPISLfBAM{=u#49vs#z(Kd%Q zjW@ic-+6Evk|(Qma2m3LZ1;1{HK3ZDW_WIVAUP)lo_dM2zdt&zB(8F5TFu?P)P+_y zgKHnQK;s!FyCR$UI()U~f}#!Jvq$ENUe^OFx(`*86+(u>3*>Vc5XwOAzY!n>5aIj* zYl-x`+Ree<*?$@A-(-gO=Kf1CwXKUco@vR|iu;E0hQuf)2k~(t7TrGThZ^0fe=&kb zwt5ij$h4<9AW%de+IdnDVPlqm&a2LFhHcRXPCU+F?!pJD_`+b{j{kLBLH)!BgPQ%N zaX1H!(jhc_u{2_%SX*fct;QVXF0doZx>zE{as1I;O?%$-Q9z`y9H_7-L-+n3GVI=!2&^oyF&=>?k<7A?Iru%JN|#1 zb@o0F%}X~=HA}u(RkM)tnEEU^p6vRyt*^n?7YVO4GyQl8p5ny9_G+(J4?l&^7^c)9 z39vfLw9ERS(fjRTZ9C_M6>^?e<&VTZSMSa}a#m-7eyvh&of>2`TvP2(D}MeNkOis88VT?40O1Ri<(~;?QAqR*&Gy>W&D0#vsWN@*c&uyR2v+iS?lQR6e z_LE8A6ra##5M;L;KLT27BFo}7YcS!l-?UngVp?)?we@thINKD9ztvldwGD4RrL^m} zKNfy{r}3;5%pq>^!?Q?#BM`@hz!QLZ){|5({Jk2%3xb68N#6-p1;5svD>n$6`2pUk zHB)I*JA4mnzD=p8@?}n;`+Egi-}k>>9(EfL@DyO}R3QC7y*&QMU1qnx);MDm+7KW_ zkw9(ofovGsdj_>0x^bf+ymd? z74{6XUB^Ha4b!rX#vl}pmZL9sz8;^-Foj`S)e!B#bU6uKnC@yZd{MQKlUd??R0+MM zZp|j*@@yu@dbp+!OMO+U{_0yr&DmATlc={RH&2qhGwHG-3 z6rx8eRVbBE?o4ST7LoUZO;#odtr6$zMgdeooSP1rEdVKl=Z_9eFJINsAb8*Tk7d{Y z{zWDP{)H>R#Lk2T;Am%O4sf$}bYQXmm#_|tx|@r+i9IEyiG!nqg}VbKrIVez>tBEV z-fxIi*9Y%HW4)F#t$DN$OJXGmFfJoDaA!WgV^cL%EPk&T3Gb+sh7N0KnRUjf8kj=) z;SDLFLil+zS}QTkq$8_Uw0T!9;CS{^YyUpP-ZwvG=4eQG4o2i# zXM7T550^HSyA_@G1I$A${QdR>)#_NJ(1+qBZuAH6KpJuO^E_@}Nuh9t=W46waw21_ zka*?I+~VARaUvZ}juZ(;df`vLZ^(0nl%xVmyN3*|>8C zg*C3|^LYAG?sW+zlM6Uso}^6*dWs9oo1~E@rfpNmoJdO$^Dod zE&whE&_bR!KnD!{QbeZFi`z_u#f7?B*v9C*Cczgi?W1Z(StGf*uvRWtaL7<}Hunc= z{6zE*Z*J4lU+cD<7s{G}H+AF1ni`FT->;PN<3^BFv8K)P?qR;+!+tS8_vcmS&qb>g zd?=nU(OJNu{o-_l_KsxbZ7(x8BDg4*YtpJaXZ%r?BBF3-U5k1Mxwf0mcPfAjKZK zs%2%sHOT8XapVZbYpRcA{qH}i9F%t&;-=uBAq}?Te;+vb{xb`jI60X+m{I;KXs8+5 z&VR%bnp?B?Xgqq*tNIbJSiWRv%?SNGO&+mWrjo)WMrt{yS~{tedRkomeN@a=Rn#Bz zm(dI2c8=LhIQBBsxx#tp!D`Z2+vD#&!XRW$;%a^!*G+-elg-?aXD$s^lfEWIkSwmb zLL1(Y88OwkSNZ@GDx#H_&L9`6XZ;~rH&I_}o|#lJQfxjx@fCoCzIG#Ys$H6#dj+W^ z|0!xq6%E<6)~hw^XmMCVWYqzM!?`ed*tOvFj=is4E|fS4VV&Yp%PFiy)qy~FEfTCm zBS;-9&?o<7?Lv!JN?tMWY2hTwZR47CJVu4J+E6xsMYe$xr+s$TOVUROcGh710?~h1 zr6Ag8s{Ynd)Hg$o1*5{+b83B{(D4aAAu9Na2hSpUi8w*7b~jvXZ!{wRcQR?ToAFqJ zpFC4lnvKO`N7K+lo6_xx!t(&Bj^r3L{X|oO5W4UYioz)1#uLzF*jGaCt2xYr1bC9~ zA{&d3D zj^_TxN?+&DFjdcN&*3tYoPD|i%Tq{h=Od%+&Q=Z0m(48&j?F0j!|hlQ{|DJsjMC-UvZJ05&<@J0}i@G&0 z&OqwpWXoba3A=D5KFZN}UTMgXA*S4KPkm)LN44cyfQz*dbFRe1N4RfVP*9LgRa;A? zLbc4R(x^V0Zayr<3nEW9p>CeqtA@uy1U7drk*`7E4sLgFGL>@eTs& zpJJZC{}#D_)s{Kx_O^2>SOE?j4W$`bu2~ChHLgpkEseU^&`T;3j7Y~C{GkCIiWj$& zE5*elDuF4GhBLvx6fdPCX{@D1G<4YFUC)JBro7t@US4m`1cK<#t!b@kZ5J1uq{Sse zYD}cXxuTJ3kB!sk5(fPwaNMj~O92S3e3W0(#SwlYuXcY6hDarM8SBPzAgHgm%a7;? zZXf0_Q(!>~onm#wJ{fumU5RRUJjJa6Y(cdn5mKJ>891+8C6i~|8t8lQn{?~TEyi}^ zc8Gh(Ry}5#^bib1?4nka$uPMAB970^>a!N0EL1})grsG)VLq1^cW#m0&D$b^GQPPD zh%Q>gzp=Uibn<9h)-T02(7Ev2@F1P3Jj`Tn9JKxO>+P2iDSt?qd@ha>$natC$}%yZ zj|@?%NXvDT{e9X=@l932r%MzAc}u=_cSzJ`43Ke_q+MJ!=2^@VbA;+2K>o5mY<4!g zjdyNj1IS#xTP>RAVv*ylLTY}yCLGyrk^EQ%`9Z`DatZxDNsHyO4bxzJ2&I{AQ*DOC z%&HO}<4PG)u{}Wor%H!>gkY_d?Qzjs!qUhD^QqvNtwuEQQL94tE#_2voc%@O8&RGm zUT&Vat-}MGKgTl_a7w9E5Qf;NT zE!$b9;MhNpeydCw$wGm_Hv5lZ@_!J){Qp+w7S?ud<}T)D8YW;(x&KPDijH$1(fkQ% z+!MbQE6R^~eqUX1R#oH;QemLfsGva%#n9Ytb#Ad*x#Ms6nX;*^S^?ej-g#e|Y|xn( zn(j`H<0R){UGQx3LPR70;;{GY7i+-SU<>lk!MIpMLSN+hY5HDMOq_F06MzW`HY1-- zUqgs{HSc0KBrFLD|6%?_{kD6rYrqKPz}?X1+NOcTetge0e;E=>2Wxcec8_c=1qJ4s z(5Zq_p#&Y9FrpdJ0SGl2&W9RXkouSW1x~%Sy^XF&RPy|6hef(Trd{RGhze_ySjE7K zAW1}%(IG3NqAaW3#;9>)+O9RFk{Q`f@(?V5Ts-!()G_9rNYZFTGBaS;1An>1LAf}Q z;i=87p6y3<-nD#!g(P1xF4R)?QMidM^`N;I7vZi;&S9fLdx<;SscQwoO?zY8P~;@u zg1m92Ed>C*1K*5#FX73vZ|X3t0pl&3WH4SZ77$AvJ5xDbI8%sTgXJoe>m!NsbZ#=rpqcTHH?&VhSHviNEIQ9W%L65EYP8J|i40rLakz4Er3;IOLrRyOw@E$P0XX#O%gP<=yA&+J4L|8ovtK zA&hh6>)LtD-GPlG<%5zA>)X_cp$JwrAKaGNb|dn~QwrKLgJAQ^{78gvE`LnJ+^>I2mPCPthmuJ~xPmAjvn-pxnY|h(;3SozG**2##(5H^48b~9A ztt64?U7kXM=8n!QgC&@uA=*dnac;b1!y!;%pqahke;>=hmmbIJQ#Cb91 zltSQ4bghk5F=Z4W&7W zPAsUiT6QVbtdg<(gl)WtD`Asg^+CLpXG)`M!6~m4uKGeIK3I7Qd>MSa3=gBkXN=9P zM?->BWC))nK0R7Wd^s8k{TD!p>ibn(Y7?2YtRA^qN-?OabW-)Wl4{j@=EH+H9c_Bn zLr{Rzn7628hv7!!f&g&a7k zPmOiEBh3B|vd=l|8O_LI#7AO^=$|LnO?lU=ejlo@YPPc>-3zjUFgsCGw!|D&NxIMij{vHa0U_jB!*6LX=fP(#LLIl?bW3Ee(>B9q~;uYOpsh;lxy zNmg>7?VkL|s}{c#zwTeWDR;f>|FnNE;Yb{w%2poE6aN87?}tp<+1+DqwGOZ&=I;qq z=Ql~TOV+ASGdm(brS;Cm($XU#35eZ=(eXxx{bV7fHIs*LIPlQ|nN$Q-hINm4BJJ(Z z65E$=?y3|V32tEjsZinm&!U0-{}Tx`^wsA+V!gI}{HhqTx#lI#W=d!L5vvJI?D#O` z11IOdQdJWuCMjjI9!_0O7;RSZvPO)1dAkgf?s>H^9WU~ipju{uh2t((@1L)3<}BY) zc#X65IcM#l3>n7xf>o1s<$Y%DN8|ZToBX~n*e!4?Ob0!cp~G_FWfsvooDUSBrfs){ zFm`X4<`#FMJ5@{rIn+`b=1JkQy63l5$Nww?QLTv@?eYT4^*S;rg6CXcYho9j)~1|FL&YR@rFe+)h8sJ+{f5a;rwJ)Dfd zS885{05!H}^7kF3#sUt(SI0nG3sR+*eizoe8S@xF#=91n=eo2_WG+0g@v0X}n~Ppw z@L=Aa#cpch9HU2Xj#D#R80eGbl_vxvI;QDlZEY$rV0uvkMKm(qX^GMfIb~O+Z)mQg z>7pp2V(Acy8e`7~jc3lxe;re*X^6Cxnms_8b`<>jL1;QcnN!Cx(98d=#VLD>;jx?l zn}}&3cf#I)-Kxy^Br=LUSWW~-ss9rBm%akQ6%&!USM)CIseib02t#avU0i>Ctl3Ni zp&uq}1M$+my_ICq+TmhkNd{xg?zGXV>j z|BKdAQIzikmuV|?^XUOPCp1yC=z6GwxlBMfA#6$dbsp!Gnk+?G9^fmzubQCO&Gu&u z8pTUWH(2l#Xot7J_`}o?O%O>7Q?;(n*Lo9^x!%-R@>IKIRd!_2=o^?0&15k#pazw_L9SO6~pQq+e1T`(hDoBqUh1)QP=h=gA1kTsWiLGKxGKFRI2aJRyQg`vA*wzbA+$F(dVJ3qb zM?f@67-};wOPW>E`hzJM zT4f^nt#bphdFIkmeFoEUUBk4VhDsjRi|TfcVmEA<2h90ehVNt!N8`cIiz%b`Zqsh{ z=bN=5im+WFz9=n~w15SWH!hMZvs~-GDU2%@)|X8abOsWFa^J;nd(;)3YqU-NzL-CG zs=NFpfqfc54TJCSd8l!cW+ZG_A^QxVn*5d1Sc+ONJSsU=S!(`d=WKjo74p+8jmz}S zl^$$k!WSdyGesHgF30^U-KoAsF|9uCdHog{r*;sp);!W+7C{557m zoP8TjIMhSo6{|J}=LACtvm)GBK`s5R^ZZ}v8HXNsgX^+}Va<|1; zpIN?SmGM`PT&yf2n+9fAI#CO(!Rq;fdGuC|gb3$iKkVjv1wVMw9$w3KG`ncd({Mfk z%Rr8syrm@9`-5lHcd~-&@oM$O#=1RGKuu^{gj{-=aP;mR)^AQcKolgYVJgi2GdO-W ztYGprG??}tC*{m%RRt4`Og3xkBAE}=ih@xzUT6dSQg|TKtJU`++nFOZ{`R&H9}gWe z$&^;Y`FYw(S4w4lLH5?qy%=!?w)hU6<#-lsCyP|3+OI=hm37eJr0@sv7ny}0-q8$w20!B71k^DZv3Mg}Vs4pXd#TM?$ggw*c$qL<{F4gHpHPOLGUk^YM4 zYPOS)GIG>)F9&Xv4F;a};q=M|8|;0{ck!c4E6Y_n$L3o$i71Y~;lX4J=a=|RCo{iX z*s3+!(K7;Hm4};8R@^>AA0`4j!gg!K9mzrqOp`rIM+l2Gh2@Zg=?iIcF$V=f{>I~o zW%UGN0-CphSBXN@D^%i;B5XmDg5M+>`D@VDe#*QRt`bQ3{VFgLe8f(Wy5>ocRe0Pb zH>?rW3&$+OGcG4N&M^pph;HkK+4Tk-wjTkh}JUoA2lw*MhheZ90E%l6=!TQ7N%kO*QZi>~PUy1=F=x`!K3fC9aed~ zImXJy;d`pmJ`FQRz9hB|8;-4=>%8&yQ`Swu0t(+Df$hT(GM=GB?SO!3^&hPaD!b(apHI4f<;Oyhg-=(K=#vG*9 z^n)}fy39{ZRxCmHbr+r{EO?zL==FqKF9yzBLE9L7_t6eIeD{uin&Z$&evH--t_JR> zmT{E3h#R_M)n1=R>m=HUgu@E*O=f_BW)qP&P6HVW?zQQ?W&h-UxwqOGgDgWaLGkf- z^f@KmCPcf<2_(XAL{wtcE4VTNVm$$CFI@7C++5p6CU7qjZV+o-8ikh4W2C2CywZ1a zd|(ku>;hXHq?*t|Ezm%RM77PrsJshqS=&d`fJ~-A#R(k3v? zkNA-iO~6B0jfRUCJ651nT%40Q`^zxXN`~_a^5w%CzGEM3{rgQU zUjl9}JL|=T`lN6KL)gAV6liZqNj$o0g2e&;5Z&`Dle$w-04>dMzD6Yvl4Xiko`}oJ z!F5m9@qHMfz>kILJ>jpBhw7(cv-iLck={RXJ!s;fP>Dc&e4`ch8N**r`(^c$>0RgA zT$O#UphK47E}+E=_GD+2fVB~|b4r|dq{JcX@JYkiiY!!kNhWbaoS^MfL8NCyo&rf6 zm(f@i6#U~iighVJRy1^R>iZ)V`W)!ng<19ZwBShdYL&s_a=ABp-A0Sf4=Y`M;VzyU zq@uo=Q9%ZJAD^fh3t3Z$RWgut*D5>7ZDZqobGHJ~QwPtu1O9wBh6zPi3xYEr=RamX z=6}mLe=1-9EqfiCxS<5Dj|RV+oH(>Is>!3jPhm<`@!1-*jo?wKqerfY-&}JO=#F0! zS(+s9rg4bCRivEpC6&M=?e(&-PFBHKGn>lbyIFCZ{K~Nr;QtD-iOoIgr8kHOJ4Ia0 zcS+1iUw<_n7RHB5g-O!wTCwk4USuOS(TB;%Hm|S0=K2~88#PoW>gdEQ);VF^+!7b{ zm}~OIJ^JuOVX$n)qPYW;9$)@J~Vjx<(JQ%$BET?p;61t z;IKzR;!C!1S4nAHmM=xB`Og%u<$Uez>~km^*}P8@L>wTr=IkImH!qOz&4CxDgr$BO ziDVf+cr7=U5syO<0!@5hBONRfR7_HcHetcW=ka$F29a|?!hHYW%A{GJcYAYLT_izi z>W^ssG6y4m5?GT6J5fT8O0DRd&b24xEGFVA18YCW27xrTe`sY^1ccV8Xpm`y9^H4Y zHTlfutEoV;h2P23jbI)!p3IDxF%$-+Sxwr{S;lw5{LttVi*3Lt(xR=@nIhra0_nA8ad2J%08 zt@OVIjK3-)F?VY_Gjo^!+pTjkcLiT(`qyhGC1Fesk_`*==C-V)U#55Q@Jw3+6#t1W z3cZFEt+84LNS2|d$cV{THrFmhbUw3vBrz>jUyb=d?ZiCd`RXbo-1TYW9p@KQz~EqY zEFa+wH;x_SvQA~UlsOb%3~zbCu`xbMk5=bQoF9GpLAV&J(T$3aAe^b9FVNet1`k;e zN~_aZw%*WjW7hW=p5C!T>0;rsM(REyzv%*zxlQ;3?Kx`5i?i1N+~Bf7LnH4Yn|s3T zhjO<*ou|`Kn#iPHMby(%;S<1=+qfWxO+!)8p@IHd2Iq*T#Usu-2BnTWW(wbE$v^xK$ljw4VcNTB zN^b5?p#PkpN5V~eDp)lC2A#vHi=w{qN4=}Tdu?t7Hg4QMdOg>FZrs0ua*F1P7MLtr zgKQF?i-`DH>4mHpPjJP2hoI&Nt$|h}jBH*x!+9o$Y4}L<3zBhTzJz?+<)|nCM7mYv zLtTQvtuo%nmyzLrf9mSOesgvZuKezo)8`!m=~O^EAQX_i!#{)^mqi31+9CSNOU>)x zx*A!4?F+E#6Gm3WBjCNaiYSl{0T~i(j~*`8jf*1N&etQi?-1hIEI zOH7v(k%p)6m5$R9A2?|Zy^A}+>sKYlb(R`JQ#u^(o_B_qb~BL5i({}!SKntwp>s_% z5pNK9RuRN0Wy*fuKin11c$h^pGt!w-L5%}@5zr%@#A6PNF`sWY7KdhNr%4I;8u!9eru|nL~cDsE{SWmg&fVU zC>Rf1Xw!r*hu6ia)h>Y6PBk=jG9UOV1nP9(W|p;7U6alcJloI>|F$jrbpZ{jo#0LZ zQpP>4RhX*TtHk$58u<~x0Mp3Ubd2+=1wwqoztT3jv!x5J z8m>A{O|AA^YNQk&F~UiK6-6ZU*4HSPkJyd;qWOZ!qd!I`C`4Z(QV^qpn+$g|D z1S5s~_ow1Jq$M_;t;?NUJGsa8Bm<9g)_tQ55Ibfo}k^4zYa z9f~B3<}y*YF`&2f-*!?W?#h*bDJULOTyORz7@Akr$& zWp{UIuq>oHvDHwRj=@OKmMIKVYpN)NPlHSWPc*kd)W9C&Y^%*SzY8_v?Z9?wB!ojI zUJo*6(IF38Tsw6s;D!taZ#Fef44w6M7j(e$Nc|Q2!R9`C6y(&40d8wQnv-kepYIY- zWCDG%plb*D_D@i>S97BUX_A)-83yWaRH=dpNyK+ziV zAOO-kv0#Ah6A4E{vq@W+pm`2MY$U^*S9-02hI&^^L*I@?= zZhVF8+svpk#uMQvEil%T87?tBjOmD4*IQ6-)&lzWPk9s)MfftYg(qUNG0m*en#PxV zXH`=*s29sWc^=Y!CfXG2Ls4sYJ2ut%;8^MvG>Ov@-WFoFHLX){;s9b{9d!&0q~ZM% zYnI8msGNucio;M)!FML}f7&Kt533svfb#A9e0tz#f=c`(=1h9vp)7P2vg@Q~`>Q@Fnu7rNl&Cl$U3xklji+$r=BU#=zSHEwbpf_QLxb?VE?1+|mU3xZK zSyfz3dMk?FYiIF?%p3``t-1r@19x(_uc{tcb7c~K^%yt}beiBqb1H1xe%F_hC`@KF z{e&l}J31uiQ{+bNJx@Y;wQ9+`QBDFc;9teX4doY9#iX@&|B zkZZ_@?^6l+mA9>_n$oj9SK(v1sQbg5*KmcKt7FLu3HKxt7-7#JEUW+X^Fo~cAt5LH zS(>UZJV$sZ?Z?<&eX6Y(42In`s+fay^SEPJvdkDeO`0sGhS)bC&91v_Ra&23wF;q` zC%k6;!=K}lLBHv~(Sw5t;Xl?`H2;nf{)!*XU^z>D?Jv0rR7U#WoK00}oIA|aF{B$9 zE}XMDt+y8=30GvZIBV6v8OUq5U>wQKB$k}o2e+u)m(WoeRa6$b>kwF+cP82&$)v7*p4#AfusRoUwb&oe;FDgssS~=Qe8;(EDZklH zbs2(jO4~C*)mj48PKhLgmvC7eT1MF++RG#RXaT=LzNJs)mlf)mc1et}PP?UuvCg<9 zK;@SfdXajG1Miq}$qm0jvE@tEVc7d1`xpQ(@?i^>>LosuH1(1KBS53~v+U!Qnhwm@ zv|a))Bf9w>A&>Y?DS_nfuNi(rDDqn&3X`mDStF1N!c)753SFbSwjQ%CEJ?6u^+fA{ za5=9!Vy$i_k8+^rQUQTOqd4-aO}L)ccZ_xWa80L1Vy(WFjE*@J=79FNo|7$mh4r4@ z9ffmam&}nI1>v4uX6igM7t*BX48N!myrl4S!SP*m>d)*?oJmvc_?FkaNmFbY#^3`^ zL`gL1ehDLK>4GD>V+vipySmgyme8$_@6fk%>+kH$sRe9zGL*nvV(Mk2Qdh0)7YY-%>)N4w{_?M7-?f( zL}|lbu*^wUoG)U7?T=@MI!rq>x~E8l0X!XyJDrCKSHcD~JCHieJ7rF>F<_;H9r0GJ z!SETX9g5SHqRYe>%dOAs0R#cz@8CSrM7&n_qC?3E#M*2%B0m%4sR_0-5XcR>1B2Y* zB;5&p(U~h(7Cj~SYQ4mxxVP7M{3PzETFe?yDNRM~Ll@;Eh#%<42HIIN;iNcCYZMx- z3ejXSS5~T?ek4`C1^L9FafwNY#nSbWaa}x1P)c3cN?tqzGb{UJ;8UaD4Kk!1{^&_} zF^C=T{?)aq+Wc-&cw(p?2Hj5L$Gpz;!J55bJNZ_5A^sO#hF;jdEi+(BBC2!TUFHjI zUjmuyFMRuBeP_ExbXyqECfizQ#%#~c3=3GN#*Iq^11EaN21Z5 zGAaZ5&4-3=_8G+Ais74z!9BP&iAVa%YI8!i^Ap(6!jOvw-B=d*m|870V#YneT?!vQ z!AV;on#*fpA;>jGLjl8~PO1Hqsxj*&*$kL^&E0~T+Sq_=}ck9WmJ@H?g^_WjCmS2BnTxts<{APQ82_RF5{ zNI2RNGMF9v`98oR{KQa)jJfm(OZMu>V;*_WmAhl}0l{@p(sz&&_JQe!X zTZ->N0ej$+nSB-XGW*<3s$aTpT5H&OHSi+h$n{?EUew)i-D_J4O8z%}=yr`LQ~IJ* z_8HtlZG@4e&UjgVu$h-mEU{S#T6Q6w0rW0Lpt{<-Y5^j}Y}5Mr?~XrH{MVSFo`YrB z_tlRErEhQvd7+l@463UnXfE{I_`~XD0>AZkt$)%Mj8fTFZ#oCa zQc1XU&8OLq$63Ksl@Fj3b;;b$_YBABD3CoN?95G?eaIg)Q0!wOuCagJQh22zK^yHcsnIZ zc%x0%wVTzl;YGx=FX2_ahRDSWet`j=He%(oFVxki13nfEJ# zGq+&kOp4xo39I{Zx}d5_rs)xYXUUz2G}w-Olil+eQ_}F``r~%K#a5&5&*#g#etssybhV{WzO*Jrsr5>`QTG6vX+-zo6G)PWp%dC2PGm=-AnhjYbe2M-sBvUZY@e$!$K?HXHe?^Y?9sN*;3ZF z`+1~0t?THw<#f(U-PM{UR)5T|;#fIe^H^T%LlS2RKx7ecJlPX>;WXETE&NgwGBKR< z?N0HZb86cjp*7ctqr1SWf3dr&aKrSO-;0 zf2nC(IZ9kkB6E=(6)qW+_ZzdaEd%Z|V_gvxwTDg_707(8bRSeKXQoh@N%OQNRIzfs z&{Fq!zp4C;sG=~yQ>W^nog|zkD8mrj6RNFRs>Ai(roKfv+JtR}qUQ_98QCjjWFv_2 zEI}@yv*a+y@{9V`256047AJTAK&xDw8DI;D4s`WR;*Z9)TAFXrEr0}OM~rJ2CPXM0 zJas#i*IFFHx0xFGK=NXGgflsssFn%%a2Pi zSv0V>oG-nhwI9GFvIF|L(xb_;2Eb#)ABFZtvUG532sr(TWwssnl@{U5oV@vYI+wWi z7abm2aM24U$Zp`LIRL#*S%7D5U7Y+d#nks9QelxoDeMz^3+n67{#poz-|v%E=;sxB z)`y@D*PE~z#-UquB=!0(>mY_|(V6<)#ST-*55L?_j^uG@lSP@jx4{V@iC`#*q$5kd zmPEfh#@A>i6N->VNwPqP z3F@n~j_EInP2Vqu#X8vdS7@H)g=>o6At%*SxB@q3%Z$IxpHh+34UHDooWJ_0x78e%HI{o^`dB)j}%i7RH&cb_-hJxAAPsfqpd^H6`W<{y zcT^eK4!~*g!D1pWuKAX?bx3IN^k6eROrd;6agmyJ>sSFZE(xB-m?}4vm~%anr<(-) zPe14|>w!*oy|u>!It(-qeNRh@`i1>Vu9cn zBa)gBG=T9xgx~XQdAFkGg~GqE3yQZ|UNKxM`lQ}W5l(0#s1a{+c2cGkdy0jX1uPHJ zS@9yvM_yTQS2g7xOAhf4*;!zSqJUcS3syQ z_ZK`ID*wz`0p&U48RJnhf3$#o%D}#Xd5W4jo%oJ`vd{M!l0tW;d-3fy7|dr)?s`vDXU;3tnU0h{f99`VBY81RiHXEA zy@Z&$zMk!m*zb)$*`FUlwd1p>X#Hjf9Cf`nPdzaHvghq$5%_K2Gb`hpLks!ZH2u1W z{`2#qobqo*LiZF&;!WCRBxW!IX=cf4*;ix|8hI(=B`s4=^VJs_j+)ql?wj^>g}g@g z+-4p#HX*LGY?#F^MQ~99-n|+Fe`acSZbkb`re` zJ?ADv`byn0^+7d7*XQ++W45n%sHe0Eb)*>n(pslV#K&6GrOwxNLq7`DZtP`)zgwcQ z?a#Sw+N}NlXX7-&e3K{&_1!x%aO3*F=L<#uo#(wQctOL&&GG-~j8R%USi4dFt1}j> zx+VvvY+oB3PP7Dg4qn>PM5FC6tJ~MnB=F$G6Cxk1Cnyt3i{zpG$ytbLfw;m!s8rNv z%+P7we0&Gb!Pz%|@Gkk`#Dq+QgxuWrcA6#y+xc)SlE`t<_MC8fXOoA)!fHk;-WNvD zsEiu^5LIh!RJ?YdEy%FvQ2}I}-GHMTS~+ZFsGA{r6V78oQ*$%~<~DvRT2!3R+unXd zaK{wPOXdX7jBRY}zz`}0_-yBFYBE9>3g?@9XKG5?nIjEBbWYbIHhEs`7rw6-FDC&7 zI%1mHXt!zmJ|rn$vWsVL#@__%r0dhFk~HW^*xne^D?&jw>7(N(5~zT|{{E9AO&y<0 z%64cDuTBy~%~mHjOHFOA*evmVYVO)EwOd5X6J{SPohUOosnSX21=KpUy2Xn9g>S@NRu7G=* zXhKH=&F(+6PJY*FE+f?XC*z2Q0#&do9&>9|3zw>kbydHQ0=M@>{qBqUi%+t|6pg7l zTqhM0Z=UY%QSzhA!Fz@l#N&MFp7oo?Y$2SW2E30RaPY0n*LcosvA}35Bup&1y?v_A zg53P;UI7~;cB7wgi6z%`JyI1*R~7lWqzEPmg2^PPK$*D~<4^JoQX!epC=32QR6pCH zAn)?TZA1hCP(_N&OD0%N8T?;3fz5%ORKqR3GEdb#OarlV`a`IY;}-}Uk2}n44AI!tl*B|0&ehr7gT4`!S6JtOwjPASR#46bt|XS z29G%YVd3lN`89X2g`@t_!s7os?{stgd(`h=>r`MXgXeUGR`i$CKULpFk;{Y*!UFPY zfN;#d7|GGo~*Q3#xf-0CH!k9_*r{ENHtuY0~D3^L9SK|NCoAy?!F;31m02&M{u zE|i%2$QoFTU6ine-M-Ew+Dr=ePhqO^+nanhk_|TCe!RR#hQxby9&3hRp(sr3hWErgp zp{H_^9*2cGj0*4mmqz1CY~;HS39ii)H~3UH=lurx5d_Ch3agyu4)jN4jq@$FUmyfn z><^Gjcr-q#A?OWi%^FQXklZ8~a&@trS`2VP=$H9xZKohj|8 zULc>{tC(`iMbme#3n-H7b3W45wgo%3=i*8imK1jqhN*dt%l|y~4ZU0L+pUoSkt6gD zGP!fyV{W^Mdk)vObrE)cFyjV(P5h`I+EaQmcD86cavAF&LHpY?P31_|(u5~MT-xE= zx7eq$j${MM+ejs8VoVn%o@g*^}0#_lvASG>=#rXW*F5Kkz^HcIHim_jXP{J-%ZA`uUARl6S z#nZ`xHeLp5UcKoVPu(lEI!>O;hA{?n!V$S6m65QAiOx?P_k{zrGyZhRG(B$#wuO#cfGU2v;7UJ=EP$@snWXK z4T6K)rq(9b=1cvvlMNZ>h+|2N%3s{-1t+fUP^eABQS7WNtO4HSuo+(2=2^fqFYTUB zU|HJ!>o35%s}kVv$B#YlJBF|0fESltzz{r*OCdo7lr2r@@T){1g_mxR>b*}AXfm-Ub~3S@Ol;ej z*tYF-Y}>ZYj%_=WWMUih<-gzkUYt|)?R_rJMfY`A*Hh10>&KFrd+9pP6gKI!6OE(O zImjl%+@mXM+q3hzEj1l$r%rUncb!FTfhob(qlJjE@K^=lJtq4b?<@V*Ld_K^v7Us6 zr$FN*$U|kp=M<=o01|k!T<-Rf+gUIjhJl!!UGI=W{RH$aEhbs%x(gsB3hWoC&Ka3) zKv?Hl1g1FA2F|h#$`N-)Vr^6391kMX1~TERDS<&u8Of$H-JkHN`koy^g8VAEw7KG# z=U`Z;E=7=%d1B|5S!9;Z`3`fL_1EW&H1u$6hfPyh_?C!FD#f7~(&^yDoZQ7g?PYSX zVaeC=qLu@!R5BJh6_y9d=wVrTC(Wb%7mBG6#;?eZ)#0bNXI%IzZ`E@9f?IQmF~*ml z?LT=5>%Fv$JZ&#`mXu(zAX>;F==%Z7@hKLTxEp}*7r7R#mwCtJ$D`GxaeK+ck$T>T zovvVtg;l#}9?u^aL&bk|g}hD^5&r0sM`U_yGOiZ9Ak@kseTdAo+~i*4RkBg}hr6aG z)P;t8L#oGKON@H)JJ2~zb7!Sk!Sw}d(h(Jo-Ty%;>45LuUfK1iHMPa7PV(txRL0gqt8=JXuGq=4bB)mc>&{(For-A0<`Kd6 zC+;^;8-;G7yGTp5+#CKz_^jxZt zLtC@O*v^+Z7`TpHijRQMlPzQ&4FLZ+lG-IQE|hHYW3OlQsP-C1a(S<)vQ6L`<@Ke!wiPprIINr#%dZxD=s){SqB5(Rgk?zCNaWR5B^}*N76&D8ex5}hZ7a#CN;eiJvRUsJ@=mT zM(T^IgNqX+{4w#1gMmM|y#V`fj^Wa#U}>JEl*k`+9tMHb@MRY{jYve~4tyVnW(5tj z0lYc17;=BiuwA>GaLP810(*Ra{%s~z;~MCmfEkgPNRh;61Y1E;wXoStM({t5$ey@5 z-k-{z7r! z9rA!t1~7VoC!-ji{UJ`z3c+AH0pRJHbvuDOBd&#(W9GvfoI;yB@e$rE?BRg&CTnJ? z8(jSuJEg=M4g#hv&3R%gUrL(LM}uyVO*SA9?fisa!WfX+fIcm=;ra`Ph3$gipv?{u z52~#=vxW~$8;j8zz2Vh528-ILA)0W_#@qQ}*1C%yIsd2PVwy;XP!N5J+jNY*)TV$U zh)=DluuDd|CM+~onf$4Mf?f;g{-dXd>=W~6pqetJhoVQYTqM23k-}71&7T@D?v=<3 zaAs?Y@UwZvrjQ@0b()`!qXJKn5V_{|T@AXJ=>PttBm4y_b z`Yw#-6JR+$1W1Ik%}9%j(;q;ky?~;%ZEbFf;LCs_>l-Txz-jTNjZxdr!Eb9SUwRfkT zr0JtyoByoOk{v;;Wrh-B8O zHQ9O_{D>2bM46WBf;YKsjGiyERAM!b59UomAu>P0!sg&nG|9kW%H&|L54#66``>ww z9M|wnwCOuzForJr{N|h}7rw9T31hA5Exl2h*SMQO+Oyl|A1c2We>+*K4#1%Oa1Yx; z$t+8C5AH^pnfqS6I;zv(|F0~NXX$`c!mp)c=Kf1?+SmUsZ~t$nM7Ji4 z2JQ-S4%D{*_f`Ms3oL8z(VrOlw1X!!{m%oCs4KQ83nf#;)%vIw>f1VNa-K!9wf8-V ztg;IwE}1`Nl{;nevYISBTmO}3oqLkZ`cFk=5NjBO-$iNc-|Fo*v+c<)kGt(}u{<|@ z!XV!0{RAntCk5ea z3;ZE_9S9PmD;G!Z&eT8gO#wz8Q2-=I#6E48Ik5CQPx7u*U@0HfO7>1+DG+~)qnxji_ zQ16jHHwS0#m7)-YxFPdU?lC=g2aoL8!pMJrhnoH_gYvEj3%V0u0Nx{VO+)2<%}q7F z+o_ZUY%RV9ViuPL%S$2q*Gefu@FxhNr~u`QG(hxH42cjH`_zldLgXbBp>oNd0cerC zP)f9b8R>d(TMBoPh(p#6ydDZsu;`xybV^nJ3=;JT_iX)k(7rT7)b6Edm3yg5RRQvn z^%$Q-qH)G6{l`VCAbP0<*ga&TnBKAxe9t{-TD$T})*%Z@)+F0MGmN-@^@45Fi}Jq< zMr2=OQ@QqoDV+myCF?PLDTnx;+tGrq;iw*g?8O`)tx`9f-pv6NX;%VTyf7 zO{@ShI8;t6O|!Ba4Za=XIX&*fbh|>Wpy>Av!KwT=E!(gJG76pv_riWzL!m z3=6{f8b#h&v3tt7Wk*)JbeRniOTu|t*r|;DDj;?I@{~}6tg0k&Z8OJ}qq1Z%W6auf zV~p___kt#IiPE4NdZ7t1Cbg8q5qB+;@*y*nxCI+zNy%_uupzbLM*d3%6g#3Z&SaIR zb=#umJkKASAIB~}PDI(J&`uwXq(-BeGJr`r9;p z6Cm>xG&}k456_xhxE;h4j?gM2I)U6rmJeM|4H``L;}DW__x6~Rqk^X71N3K(n$wDM z(Tyi_7@0K#VRb`IXTTW4L8o}O`)@aHY<`th{ze$Eh)e4_-jev=I^*qpqMo%CW;CVB z8>3l2bHu5%(+mn#@J;v4R0=5Yhp66bpl65&>8@0}WPELU;utUWOE~mUnCUTj{a%lY z^*!6SlvJSbOC&G2*oM6rcYn2|;o^2%V!KonQPd z*FzY84is)5jAIspIVI*44~u+NUrT78yH3#`9wx9>;@I(78!l$;^hLc~h=M<(w6&%U zW<@x&7gGH-oWS8#@81NSTzVJk@7nk~AOm*m^qOa;dG7r-NOEMePQL_Fc2*|JQ=g` zl&45o!~MDFnFD}g>roZ%#&I$wUZf^Ajz^dZ<%63m7N922j@fN$0-`E&ZiLb_Fj=fz z`Dq_5AbyJyC1|KFDjo9T@P;@Y)#e>!dDNCgHN1xRnI(bLrz|_JB?k^m@PHBG?rfd< z`u9(QVwz?ASZ9v;l5$Nd;n@6|2rM;K1Ze~=T$bo0e@n|iooq#8aqc<#mdlsM<6^z1tXAWIOG5D-MNKCLgl`y3y?!VPiM?419SwU=28g>7{u$?!hcK#y zt#bpXlRyDibg9G`^)ai4c~BkRJ#wtaGr*-Lt#X>t`WTUV?2L-C#C0Y$ry$iHAHtD$ zS!+^J2X{WX>Q+nOUQCU<8^?0|&JObBdGx(#;me0k_gvUIdOzFxd6CBTyDKzv>%0D} zR4F4mvtn+d1iAKy6LNZgBd2WSOHBI$>qKO$zX{7$>rL8|${RSi+2NiX#_ah6yXiMg zGg?J2`_glu&hBD@e$r{YKf*S(=67~6K)T9#_#-mvn~L_dr8Iu2hFN0mqEf5@tR|LR z)lW5{NJAPr@ugz93`4TK+Vb;ODH>i=lP7WV+f@Z?j8)VOwm24tQx%quCiHWYxEr)K~$8B z3nhN`-23O{RQyzT{WP?#{Q1m9o#RHH7s+tn&+GlrH*~GzZ05H=9OwKsr1Pf|c*XMO zk0M9QNCsv8N_~p&RQde7Abp@^=(6@SkM9V*Yz8!S524fjVg3sOezq^%hV2Vm;LZ1D z)Tv!4QrAI7`6-g-ZJlv6Eq$CZUND8~+TZ+K+G$34tS5vL>mm!kWjT~v&E_gugXO0b z0*`kUVC+q;gV|*i>9m(@vQE`fIuZ*38kSJvHUY5?4y)7HomiKKi!DxzIPPi6xLiPc z%z;qOOEEb+|6@qb%WrM`sod$!*P)u46#40q!uYriYB3$r#g9&lhqP>yimgCih#WSV z0g#dE(Q>jH`?elQ4WPry9b8v*qy|iuOM7KsD4~CZg7iUzq(*=B2CO@hnFB>>I~<=< z8%A_uT@VS>TA9sWmljkj%VIU23yk#Eq$1R=9H1k!@Uf3hCV-1_b&Gyi*Om^4)0PuA zcqt3P&KO{leO4%MRLOPAg8EqSs>NhM63eVA9d{i{0lj zs}qWEj>p$3r$cxySbT#oF8&A0$8t%BHJYv-Lza)fPY#*)O0X+FnQcFno>|PSP}MJ` zcKKR@tACHa?N&$izj5F`_SwL@Ac^A#w=yjX#;k)A`Ufg)um6PM<;mae ze|9BLFxIGgYMy0SJ>eG6sF0ZZ19x@>Oop0-SMScW=;zVt4t1)g;9{dfW-W)dQ`o2K zxJj+cMHy@}4}6c4J(bf-B`0aeUW~R_Ha|BYeS$8*YpfaSsw)OYZXgJogw;8H;hv|S&)TmtBh!A3ZRPLM@V^Tr zHTtldVYm0@adnUw;Q$;*^zMI?6cYAfBc@1c_tvg5!QKqrrTZhcksk0|pt{DmAEvg; zL$ur>Pc@G;%o<`*_7&Y3Tj!Xq0&p+*%EXr7;H+Z&{S39f{eArxjWYI!&9ogB1Vr~g zcDwu+nX!tag~|Uyw*SW9G+@+mSI|F)p&@@WL7tN7gAJzy#RY?gFSS`%2f@Lll_-?1 z+MLF-2G%$4&h4(B^pw8!a9Cs(S$vhB(8)#7CAbqsE|gfM*#3HjdL^9S=1yP)lnTZ< zd2)E$ZFlZ;U-i6QcG|Hr^xoa0gU~&-pPh>Tsi`LX%A25|;qdtmTxOx|$@!Fwd|geGmFdq!X7>?(jC!eK;{dDS6cP%8!B z!Qg@L;%Fk*!VpON;6N&f*n_CR#!RP=X9Nl4{>JEmfNoE-&w0zK7h z(w>rCsust95z}iI2+SJ?h}3Hj2r4XC(;G5~nccR)FM(>H9n*W*-Uw`sp>{u;w_ZP< zmM>x~l{*0vjeEeJC{M*M7lGon`p;|Fp7LE)Z_$3M^QE&kz5bnRUWmVat>|3=SCsCk zJF3?hJ!QK#-hzP(=*#jQLgD7b^!a^TvkV47od&yUo=P?m6*u)I`MXSX#;yzm1V0gY z`NV+DtXc5oip}9sBHs2lXE^Ek?GVtxNMEpt?u;09vEYWUZt zRXI(WW|3o(&HkleqE*m~k4F)3F(p0sdHP2uJDRLjXr{9{k$(71&Ftt1m!!6D0n+8D ze?l=;T)j9$&*xGr-KMhuS0}k#b>;cvMaHbt18<}8YC7U@vPY5^XE{ohfnNR&1#suK zJWD)jzD}r`I3~zeBi>{j^AyY#ury@QF%xt57eOe}>rlGNWkaW}tFM69Q=8o`ad$Nr z+6slVBKZvJzRlI18DlXDnOths@M|st!zD?I@MqICnGFAEs2(e*>jPjECWzbOT|&V5QKSU8N5%nd&VS0ShVT-1m_$A$ZO<8{kklWnX>jbP%DSVPYcv1O=jFk}-EE`nd$*oQ1TspFN z1!`r2?bSSDT$I#Hwe2k2+&J+uNaVq4jvGPr@&pcR^|d{-Xv>AP;U zjzAfWNcxiH*N`}$qQ;{#Fh6Kw6l0ZZ3Kaj>nFE7LAH>*S7m@CZmdDJun9R3l?!oKX zl*h4n#)m|ws=pp3PSeSguHgoVt&G8LObv|Pmlki$^C?2IkHHs9kIPlDPcyzDU$M!U zy0fu9;G}r86wOj*e8MobiVgapQ?P7mqRkTi@M^J^?WZ8t8#YV9dMvT840|g|)1ug) z-fbkVIqUNG0?kEra0?6kNq4jy(RO@ef$Ww!y3^Rk4%~u-`T!jwxLQ6%YYF4|RL}W! zhk+7U1w&gb<&N9ozfRro3Ofw0QcDTG5^QmXtVr@an9y|ds#J#7eyxId>x%rc;x5pH zv{W#fGjvi6pz>&q7_B`pBL;Sg$aMvz{Xn53KIM0i5wO8f{2K!U+AJyJgX#df#89By zUz$Q*Sj!Xw$@G?3K-o5=t(dYDSr&Uxihb%@(Gar8wH7WV4y!8tj2PZ4$P&sgY<7QH z@?qqb!Si*W#m(d`dLi73T6aNOtaFDZ+)Ip!TJOi3DKMUP2P3E-N82@&RF81AjTL)J zg+o%WKm69Va=gAD#Kj zSprLfzgfC1cYuB>Z-DzEzux6uZ~2mgqit@^eF)hX+J}#D>JWNHzk4QQ5eQcNexBRU zb(!R3wgNxi@Xf*YN>H-`v{ zrTp~C-7L?Yevg;H(Y(8~>JKv|Px1YAl@sJ4o=3>0iA8!dE<3Csg58fbx_l$02rg^# zR?GSzltBnWvWQ^4K)W}+kAE@7AMbOs#L!>$4mke_jr%VYpxRfl!T;)R{#R|&o%-?* z1vt`65xxaVz_AM)q3~xaz~4+L#2%a zlH~@Wm)zg7oNaD!g*F)EnUlS&E5<9IZ6CMG?smJQTtB;yuO0(G_nSgYeh-JaNWHGM z-$S*!?>`X;RAhw-s5Np%Lj%XXWejfgW%(SDjV&faF)KAHepzZ*{yNoAYg}xsGO^JA zHFAqJaX@?aD?{UFowc!rsYgO_vI)zCGOZ1*mPSiMQ=_%f+Vrh6sN?KFgQO_bwn{x% zZaGJAs)~1vyei&sCv{I?$v(6ptzFqOEc*3isq`93Yl@$Aek#>58obfNMPL|4teTMN6tx{CI&F1osVY^A&W5L;Kt zKICI1)*vW)>vcuMMxDy0po(A*gZ24ci1k7X+V3x;eBlV%@8{H3>a+vB?d8gg1!|e8 z(;D-u(MCeG9n-FS+0v93Boji&19Y!A_g0hx?g5Z5vWztAI> zwU?yiV!_CvdSY>-I!1Ry9E8YV*dLq64|~( z>%SHiHjMEy;;z2Lh?n@t*F#~d>M0hpfoC9C$|D0)b#2B-&ehF#7hxSy#aak+Tfv9Q zD7aNM)#e&Fw3LofO4I3rjVYV9w=7AQ`Z1FxS5JTGG-PW_hWbQrTj8>1uN1;&+RW$# zWt)=~?J(tX5VVeM*(;mK;$(L)MSY!=$M(zK!t<==D|fAC#I}QfBORxdlS(=9asjtt zjo#>%qXm<2GU7}|d4LhOVzs8NbKq8}J&DiYj9chYxnr*Sa>b9~aRxtOPoTcLrXT780oy!zVgo3)lNIT_Uq#%O=nuT# zS{tzo`!%Q>;3oYxoNY73=LvIJhgu)EZCGdH0E?EgtK>AoI9BMuNU@gaTHXFk z><@#Ed)scZ3OPS;tP^y6Ri(ktI0G6rl(FGgR#Z=(qC^8LgC20qHtWS>we;#LTQna1sAX*Owyw;d!McACWA*#w7Mp znb~KM@1XCO`GhZ8i~UnGdgxCT0ICF$8+X)Bg8Vf_57E?x)Cr~#Nb7eK{6853F^XrY zI8odnbf{_;U1E;D4M{#1UBr%C=MUVB^f6S>;O2Bk}#1wBWb9a4g(4>ReM5aN~?Ns)btAT^bs&yoR~N!_iz2}la%R5FhJ z!L!YXeq{~Q6A_fKSDb(v2G@~cMO52kAn9<%{O5@DMYz87z|T<0DB z|A$*2=gY0X{%^Pb>^YKY@Pw}tbdCeLOp2r;b>T1S<|e5+?sAG0hGM#BoTxR}E8`rp{8UOh!U}Kti$N=xo4*>#$EqiOYcj88?h??KgmxukHZst;GA1E- zG*>a7Xl5>udxNVS20qkk}y)l%NB%Z*?MZeELn`#lvWo9Avqtm;oJ z??N{b<{a8ibj52FXa2G2zxEAlRfplVSbfeK%uGSL{!d;}?e4P~+!p~C_a8~P|B}@A z@3H@H=2Lk?ZsSY6hf7g^&~YT!4)Kj?_jdrar(jv1uthmBL^x|V~nJ@R6|}RO=UVqyf#s>o7tCzhd_;6$!Zpxc5*>R+EJRT{^WZ zCIvPfIoG01vR;`Vq@Ac)_J9Du^^0$*&m=H~lqlSp1Vb_@_fUQ7?e-?`W{g<+?Ex}q z8=fnF!YxFF*|RP=IIO?uAgrc}{Yq`8qn5n~dG$E;BL@rMHd=%0*3i+l)%wkUA zrT%`gvry3BEI7&kbdD7KM~T>fv0wksgYrMm$&eO|x3cOoKZWe;{5@##_q-+(gjGPw zdv6uZcv5AIVzK^T0Q01Zjo)dBNq6qoJ6KztXr8~cUDVIIIPjrf*uKEJ#64fY7Q797ugU7nH3zT}ey3RQ=jLw^ zZtXLF*0z37L>UJGF0+Nb9Yh6#MzGiVe177fAwm3?=AcvE8G9NaKD-tdfsuRk9vFZD ztA|toSGXcbvvUoU)4mYQagPfsD!3B6#-J@EKV;7pz>R%r*p_KP1LTE|W#s9`zhuK7H9xP4|nH*oN?%WR`wYY zy?w;!EBid?GsAfiy#wgfH3y|m4j4cb^qCPHbcQ}DbcR7zKwZR{LsbOk9yL0ZafJA?rJ%5-%c z*p!@mxmUkF{LzLfukOHY)@oE8(c6!Q?zZcUeg!;G=^~Ujx@LY>@%FFobnuUOTkXgV z{f+E;pvaT6Cjv-R@s8qk$HS-Q3y4(pMqxwGJzCD`FI4eH6fb+`dAb$Ns@#7|8{1X6 z5iao_Mf4fmdc}%Wy4&M447^^r&z{wz)K#z zVV^6_8aJ0R<5{`{{^YIrQfT9ar*>JQtveKtGOp!=5BZ9NMO+#$@drB*B9by04qm(KvwhoU002%d#VuX57v?L)`x-x4>< z00+RV$lEP@$L5ADx7PfhdxBRq@^P=k6WjRIM@_|yo=^p=Uxcsgt-m-G<17g6_IS}B z6a;hZNA6V)@=}8D4HJ;q_G`%Kl4{Ts5$(nCMaVcoqrO>l01!*7jAr=*Nzp89C1C9SDwob-;qLWcJfIQ*I0BjBgx3c7Htg7F>lJC5A~&6 zpYjpN(3QlTM2x0g#=!~7(TJr8bd4Q>Ti2l%d?zSE)ns-s;1IxM31S$UzI=Wq=6cnq z9#vL&6KnrYYbC=kRe$wyK3qkCx(RgvT+{ti1BBT_dF`^E}hR5{{bAs~^tv#3xh zis6)stliyfh8a$^1ZT=no^c=~k*@bEWLE>tHo~2a>m?3gc};#QwfG%kM6f{Gf*^MW z%NRVfw4P}-YF1ue2Vr|$O1&wbGd`?z4cC@5bL$LdinDMGwaC!!JrN5tRG3Nr2ig)L zZe}H`U)1YFEZqB|*|>l zbI*f>2x6N`o3>P!)>Ia)NzN@vEm~6CTI8)-tf+W7pN(pJM2A-)e?Nnq_wWW+7dQt23~*O$IzkBQ(Y zw7g@#R73e9pWNoNF17dj#>ZaXjCaVV{U9nlGmXrOI0J%>AB%l@WrB>0rnQ`cmB45f z+{nOP^HP0!f1TT5K^lb+3a}F+2McB}YOJQt!5|F>U!Sje13;E6ElM+zRyLejd@fpef%t&B2Fz}R{E(k zg^RhrMK{XL6`fdz!TdD5<`NW|%Ilf%!YvjVToa18n+({F;VEu>OCg&T@M|-Chl4(= zxrKs~=KB@~w(FRCL^*`k)5D)^5wL@B3LvBEalTu^LSjMy>#>P;XnlIqaAFqGJ#Z}c z%|!!Lr$nRsOSsm_BP94ML?dvdB22A(<&_y*7&jTUtcu1eL%ODxT&gy(=1d{jkT}cc z3;F9h#+?^%=5WeCe33JD1Hz6nZK^!T3T~{3d!f?glfzqoM|t9~<`muQ7S!w~F#CDb z*OIn(Sy!~G(9jew{rqvlEKz8iq~c|`{mK~+9chXq8fnhv9vHwjT!p_8s)aA)ibNRn zUF`9;9Ga+gB$fpF*Vgt1*+U3{?r+TkBw)u|UI1gkaqoN{s%Y(86E1&7$R@PeG32Vpj9B$me}JX;!3X?V9XAS)zsXAXR`8Pb49;AaOba1w30ZdmfQfbe$DPTKXcJUADo z+cGaqOy0w?bbom~8RP!G;(ixSntA4}4;kdj+1p<5&cpJvP!#Y89JyttuCI-y%8 z37=w2mwqB9(YW@z$%u4=`gp?u*;=K@8jy6Y(&V4-x0|M#O*IXACwcaE5~7e#5Xws! z&g$LxQLV4OhAi3NiKMZu`yBD!4MpDS?7wlsz1`S@KDRq6=_MsDz*4Dw?-k(FJT=+BAc^s8|3u8=;CIEV~Ui4NE}_N zw39+bTtv+4w zjC71!^fVmhVQnz@%HxAS5PjeJN$U4T2N6TUXsdoYYt@MKrk2Si49XLqN{AqbPQfK= zc{9ob4c-rhnYc2-NGm>4MZZ4Ba=l8@%bG!B`7S1h9-jDJ7waC~0%{IPZ@tFiM|L-1 zUE25=;oil1Y73w2$PmLcemP>ln1=*=4aa>eJlZG`kpkZ!VQ@>80=|f19roD30k!Li zbHgz`*&`{`5eh|hqijihu%EAByZRIjRn?5@`+rfO(%Ttu`FB_9IgN)4V`hR-(TFqRarB1SyM|cLb7n42-4Ee;dUl7 z9FipUq`;=StL$fH=tZPS{dV4vjOj(^8( zyqvA?&ZC*Hxc$+s3Hrtndl|5@k5z~;@v-R0Rh;UmS3S|bLYLBPOX6niVms_*HBsFT zprG#my9B$izWWW%x<53_B7OB+qo?7Ss2D`yyHNHYM&sqiN$Q09 zrar1rQdIV;SX}yf1Tgpygev~vdu7eMlfP>GwyWRSwX1L;aS`l_0DI(?tLxt~Kcx{y z5O5Cw($2yudvoNEls*irR*wSxpR^`W)w5fSg+BYHr=3&0vq&vk%Rs~la!-OVZq)d$ zNp0m{&d{$Axqhz~A--(stkqJFoyxAjq~C1Ur0{{I68rhVA|X^v1&tyV^!;b+r>M@l z#M!BmBQfHcT=Y&9>RYJm`fT5Z!a2ui-hDKEo=9!gou;Oo!K+*67U}gG)^}lP&Yc&k z_Z;Eqgf!VNhz!Dd2=Vf?hTRyy_kf^T6oP z##>!$hkE89Y%05q4et;&-4a||aN2z1Qv3Ybe!CDCF0Xjli2AgI;uyo7c(VQUy=cY_ zQ>_%OXm(jedzBCR;}?ORzlt|9^EFvTFTG-K5=_96a2l+lzrg!5}NDN_U zC)_K~x)=8$yggAlqXoYaH_BS+#FK;umw~Zi%MrzgJB6Hk7dLM=)1K8A=I+`_!(Xp2 z{23+-kQ)fd6$I-+VtV&KIti^qpD%HWe#Lac9fsC%KYlW-G1#P-0PP9l|sn7-7;cHD|&+PchQZ zGa)ekMQ!kvJv-Wp2Q{RkYgj(ck@J`E``)(?nqYbZO^T;0?iXoi3PfVD-_3WHT1Log ze*FP~&W*Gm>3_t(wt${Qeo1S#VphSm6KgkF)zT@ky(!Z`7bF#lZ!2WrVrf^*OI!F) z3(qh8xA@5wNj(=e(>*YFN7GVsE7{#$0!a66f>!Q zmtxP2&cTY?)&J(iXBB4$4vH{&usc?}aQd-*FDMHCBsF(>V_sEorisgLSbu^ zC~@#M@>@=^E4zo=K|s$Sr%m2o{UT&D1BP|xB()XFR8^j`tZsUy0mrFJ0cqu<@tHJd z2U0!0cKPReY`-Tr{F3z6Q7qDaYO#PiJJm`Y2UJpgqT0vz#BR9F%Pbw|r_n8l7iV!w z$5LmmZ&;zRg@3N={iO{1{Hho33ID;^+LYI5kWfKDUcU-Z|L=~7{}LYmKN#D;!@>Ut zYHLHPro>U_jx}Bfp-QJW}t_CA>a$Bas_w-O0jXUT$#@ep*`~R#^KO2yIEV- zrG1ih-Rr{hQ}W{t*nzqLANV)y3~Wv8b*vl%_muqXJpn$)x{&-B16&{?fEQjCo(_l0 zUR#(#&Kye&6Uc<%kToCI=Y%ueH*b$9j5B@k1B_R<4;g{sfDvecP^+&16t&ec#4>4{ zv=7;%er44b)&ROVvnZTFw%XdC+5Ew=B`(;*zh}mR|G?Qk5NKu0+xuC7bGcgt=$u9W z{VF!8GGLaaq`X(}Q(kCdq^WKF!nTG!6(ph(fQZDjKdjh)z6z%sAdNgM zi1G9XY!nKMfuXj(u{;DvK}Dq@#{zmo3uy_t_)?mJ*x=5RU za=mO_(KY{OuVTaD{*rT5P#DkqQ17n@)@PY;pN$w!P(jE!I`X)_=r~WG0r41uIHnL3 ztUP8LAeiTkCAy>3`HW!a`$+G{|r~}c{uIC>+$OFaoL}+36IJdWf zY^lC`(V%owH-}<^V3W|grlwRUA~UjKrn#4|Kx>#)yg9Hu+CJ?oW!|v2QM0nL_z+6s z%p)EZ|63BmqlMSNkIj)}ZZNDox3WY-dFmu5qp8%iqx9!RdG_U6!lQNiN*pyC9uGFu+n%ywNr+sogHIMe5te* znH_~H@%`*b8xD5-SeK!xsPUu8MA!up1j5A0Mie6Rt5he=vq4rqt7)a1!TKxg-piKQ zbK9K6gL*=xaZGBCxHvg2H7{Ne;ihN8L#RQIYZN+%)o208sk`Coq5B&sv zk}*SMLrB6oZi96`sh4OHU4rpWl1U`#uZ|`&$B}xmqfvG(}M-ZbGD~65}hTM72qyD`rHs38Q9A_pvxmOpaw36U9+>{b*K- zw|*7EJ-bC4Qqt~<`rmNTdEW63?~~M3mOE(_#;fP&V4iiLlVXS%c5yO^Z{x;mKycmc z#N@enXWyRcITTbLd6Qsx ziivk~`XHHW%B4zobeYh@Og2xEC{+j*75zlQUr{|0ABm+t5dL}h%PqeT=Rtsg$Rhuz zcmMw<{muVd{tM@hrjGt;=9-8wCNR+11txpe0D6m9q(ECFs01to2<21DxS9d`j496L z7@MTfn}=Eqee`!yTBtQEm86X=l9wxd^2|j=M66HV@|z##G_|U8Sk{a)&)25NKkg3> z_*tIbpL&mTye_jlIo}_o(`w-k81YHj$N^s94#alqqAGNM~mv2mHX{lBG~O(!S9$p#{k$JN&&6* z3d(5S+R7^4s>(di5hd=D@?-h4N!pv$RD9mb%7DGIi~@t&68I5zRugdB`6QvuvLC&q z#nZq(yEik|DvZPYjTVVLstk?y5>{&wDY9a-={vpcYvCF3YL+=-Q}ubaCnltZ z$V*KmCeiV6a~8oF(R71p#LRdk72K{#?_>#nswz8{B1WE(CRJhJ2nE?xbCWGCh@p}u zvZhgmS@G#h){G}rpb~P6vm0URv^NP#Je*Q%mY6G4SZBE&;Ysb~pTm>b*C+uhj_=XY zT_6XmQbG&b;^buNfklpE9Y_c7486_Xlxak0GwM;| z0X+idGbN&U#vQ$Si|BV=39SX$JK-!8yy9}#SouK9_H^ok$y`Ah_}8vU-iGq^a%?in ziVm4zP}uXVC?BXl+Uey^9gf;@^$Y$qn;b=#j_=s1%2knqj~XLp3w@zPgec8x*F~^e zNvb20;-0`+RwcQ!yPayeCyyR?b7mj;+bR@!I93KY-3%7Di?|L`w?}&XEZ35t1)PM% zxS5kzYrsUCd)5Z`PId(D0kJLug?8`xk_4JQt+Yc$i(kA@hG*Qk!OylwdC!m&q7Y(N0bs$q4BL7m1 z+s-P<7M=ajX2Ct|Ja!gr{=2-E9)9-KB3 zr*-?>$!x7BFWRXbQh2k5t9|1eOvz1<>st%D0Z!9fzHyDNx*{qf01eH4g4#=wwZg7Y z%k^nvKgNDFEJdYBW$H_Y(|m82Te%W!2~NWwUa4VVYEcj%Vp+C5*%-G# z;f?j-MVFKp)sr`wS6*9DPQF)pcu=9>UGm{&la!Zj9l?!QzYt+0&KI$C{8K+xf`*;# z+#iNaW-Ug^&{J2;J>g6qYD2SV98A^UZoplkh2dq>!3AY;)T0KpsEhguQK)G%aulL5 znQrHLPx+9Rw%$f=$+ZP{X>=@=3CT-xvv@$mP+nCIqrN-`KU~_TsX1^5KW?DNkHq%k zX+;!EV=K8>(&uej^?z7V8JzP zzB$+2vi`ZhwO7?&)zt@m`n)~r9^<*+YZTRN7abCqkL>)O)>`_WbB7in)>xT-+MOx> zUL6~-(kV|6Z=J@_mcti}=RF}YWPJjUd)cBLWTT8JV4usPi1+KL#r~2;!m*UEJd!|K zOa=S2{)fe+2d22ly_)Ww@0ffg3N0sE#!|(Q-Z} z^MW6*A$7+|>5v#>VNkR#yCD(}%!J@$z3KIez)XFGTB!^D)_LnlEZ!tI} zm{br~U!Ya?jUcI-$#xY=H;#&}1C!4kKPSrMb7y6bQ1Vw@?ayytkyM6Mvp}jzOqoV1 z@pV$^hsQ>{BywMwR05juKf>vs-@UFX&sN`?Tc1;USltyWhwia6iEf695e^g zVuG^KPZZ~<;{npoVeAJ@6gM%}V^>LILVzOAgUpm9(T+@N@i)v%ELUZUM2xy-j{V9s zlc$oYJQ`=ikz>>?6R%UEM9GvKRLQSKw*?Jo=TW7T7^RSkC=883t!V z;Oh6#{1i|HNt7I#xPP3p_|#>7KowwUwn?A{5Ufg<2}GPqXAZXG-I3^62WoNI3p zJI|&jzm`trQ92h4W>7dZpkJZ*36|N&bl6ZztE(pJFH@6U(-hgvq*aEkF)CrnsHs2w z8CGT}+m&yyqoX)8e-b%6%EK7NQR&Q=7R@I~3JmfsbI&AQ9z@ZjBP2;FGanm9A_sb_ zEU=#=v$7Z#AMLvY`NWG&11cbF^27+@6=9bkR0enYFPBVmIr!pXv);vN;0{?ezc;c* zy-;7uDp}5LGe+16La_|sjeN4kvdYueyUpR)cW?1UVe-8{R44ODI_k6wunxtT96ievYsoELep@8--IT-c<~p| zeT#Zd*sl03SuM-o?_Wn1zc6&y0@V@n`%8}_KqvD*o_G0Qa3OeQUIu{U9kCbrUx(q| zO@lB#o8(!A(qL#SQMY&FJei}(2p}5fvu#qPK{>Ev?cIvao#fak1z2etShpw7xIRkq zb!n{G(G04n1=0*PaMM?3+@`tX4tp0=j4nHikd2bq} ztEzb=Nlo46a|<1}M-Nf75Bn=jq-> z%HER=>K4#nfrUkDX^SSBN`?s4;Vt>!2_-Co2*(ccU6HoEKD%u>(J^xkP_Al67(zYz9 zF#7Op``Ot`mPLUW6RFPpUK6R>ZVHAmy1-0rN^ERX<0a<3=3$BwvcRLHI2-N!YRIkA zic?V<1}c{E*2crDC5sq!RI8QJju%9UDZ+U1#VJIcCB z>us}nAg&=27j@!FFNhvs1(N}yP~_RQyU9=t!CtP;QTbj15{bW@#%BhL0B^Eq(1%+3 zOn3S#N1ZeVuWsxnJ>F};wX$H;p;RhD=$pO@6m*NoZF9lTYPK1&YS+d4P$iW+ut_;WNs)wS{`GQ)|sE722(2r>l)&l+}LK zx$=T7BWJO{EgX?!7W62#a+OpH=N>N@&6J1B*bR*EIER5|s>MFS`N$Kx@!}vw&{HzC zaLF{+0CJJ!zSf)swB`yjU%pF@cUgBfd-|mE$~QJ4VQ&xoFzVhYD}H$NXxWLB5`|WQ zCL=}#%64mqf^oH)K>FZvC& zsk)c#U^S&OM=+{L-lzhhu9h_P=-ya#eG7D!Ff@aROs3uysff=UH+)hcs&<1$(0aR; zazLoQD7CKZiw4Mw9exLT?OQIJ*)+ews;CgcV}0E5=!6G!>x?aEwlQh z&7ctb$6Z4O3(3rbKT6HR_ob0S7o-$JIHZt@HP&34(`*3wRNVR>%T7WZW#^GT(oDWw zrkfxKkPd@9b)jSRi>kmU4Ixi+$_3~_2d3?jJ{6-&?pmtghZ)K`!@NmEvpnUa?>lv& zXQ((ahP;+>L-rwm0rmS6>9Iw`X%enLgxXMg`0d-?OSmvd^%-7xty(k`mfb(*rKAGNKa6-6_U6k3DB>tR3P0VHQh9l6{sBve+cOah!OkFE8y+AXV5 zn%E3DZ!$akF>vLpTpWi>$RD=b=#=_Pb;-C3%u`zSm~!aJMrpKj8ijEw0e} zIrzK9i{CnBVK>o2nVJjBq6}XDK~K=1kLi8*%i7zGe9^3*~Ker`oaJ_hU4YWBiKy%i*0a7t*)vxkkPZ3RWglO{@N(j(R^;u6{_1&5bd^LD-gz8NI>(O_ z9krGlKXs)}*+mZDwl9ixD9RAI9Hpmc17?0A9Fo?kAaosL=IVJPHnahxGlNExGno>cian}eD65`+K!ho0?i(mEe zRt?R)T&J2mtWjgl_z8!h@y?cKpK>fKhm^u^Yp`#lDiLy`Qc1=UE~+W{tDi0In6ZD) z{xc`0>**_LM%vaVdm8B`v8h8~y{J|bS|q8YsSTb0f?!J~!EAzSWdOnPC;oxC5pZEu zBGamK=LFhuhwQFk)Xa46H22lo5uIzjUgVE>oR*nX+{O^Q90g{HG8j`FkOpLOk&mP2 zT&h6jN7d4vcexrxuyZ-3dz!HH^jQLxots;k#q`FCKngZ|>dJy9rXI?XE+a@-r_VTk7&TPFTpOa~NS?4Whwg7dBjf(2ksizV=pN%~SKrTEj%$+WR z&8Q+ITEhW(tr7CF)_0uPHpthd%H-Jfh*+m z+Wcc`=9;aJDxfb-&Pb;W(@vSRO6rkbf;_=8CL8k+@yOUc$dd`g_O^!{x>EmSgsN~x zUmB&p;remHE6c?l)(C^HKSwnmoV~rQ*k9h8?%~Vj@Ar27yoEC)Rqql1tm07&Us`UH zJwhcG3R-&iy9<|!Ektt|a) z9WlGH*TaWYi+hVW^m+@2h(5X>m785uyiPyYvTU@o0&XHU_*y-~sF;D8YwLSe9uXpF zHdY>nzRfk&_Fcgc>Kh~H>)-86SH63+Hn+G*Zfw?L8wGl(rf)!QH_dBq_Q$YlK-uLz z9#5Hm|3-cfvp+;lTPowD7e%p*5}SU7JDo4nbNLmT99hzfE3w993oh;tN z8O zh+kRZyz!eOOioBx4X|G9(=`xCe?U?paBVr|06bfr9Q|Oq!<#(Q{?|iqEtOf}2J!eN zC9M&+wHP}CO0&n6le^p2HuBkL9YnM|N5j_-?5_U2-QpneZCQEUAtj$^4wfY_eD^Px z9vrWN?HETHBcj1IY^Z|$=E4l4`UByCbe5tNe%ae50Sb=Df+OqoIr|P{yj+{%nDL#_ zo)>!Yx)T-W67hVHSLSluV_^e19L55rh>@g76^)RW#%ho}X$8-ME4?3?g8o`T*B;@}J<2rlcz)%CqKG+xYj;UCvCql_BeJV|F$~ zk^+zkt^sjQTds|gr!UPqf~MK)WRLveuw2JHm)|>{GXO%|x@}JGy`X~JhAb<;iNd_0R8?=;&-d2Pw;iZP@B4*FT#e}lgpLPSmbwuW77_6K zXu86hD#qA<%$l!|Gzsbs+AK~r_Jh^$p{pI%8o)JfMkBkN=|=rhC{kxT14WRrRm~Oher}TM-}*Y6x;bqvgq`#t-UZSLOx1 zvZK2gE8zJp0T;dMl)k8e-rr90bYH2r799{*Qaqum?gFckLToxk&ChC^m+J`n)&wERB~wBd`p4}R zNIdR_A>5ZQXrGwO|NG{`zh!9JySo{i*qJj4ySNy8i?~}@d{&zO%WKVkT4QYglF!E{ zr~F}3L0saDlV9X^wjwcZ{*(ULliuEuv&Sz-D<4b_3zzm5|c ztfEA$21RBW<0X+qyLOAY^jB?iK72g3%Dr|H-^vCK=b(Y&$$Mth0Qc{%V3N(`{B~$4 zf}%Q7!SJ4smVmibsl)=qe1oR@-^@ADBtjIna0~<#L=~uff#@%T zWTCR{UFG|by@U%*$5|o9p0&>7t|9mqzZoTZ7labmwSwiTCYLa%C1VtgVv`7m6K#2u}Z#>}th;d}evS1YSE&3njIPj%|PMu*_ov~p`^zAyLYf@7szJvbdsBBXe z=mb9<753jcs(*{v{5Qw)zuXl1Xj*_GuJusmmmm}^XdF98dWpdhg&|{nWYAEG(4=|> zJFuE8!g;T9-%K#@sJtO(RH_;O7Y}A`55LW*ICMvRORiA|&efwCgDB6Yo%5ug#+-AU zU9R{EqBX?fe#d4tOgY|cQu3vANQvNVqaN+l8Xry z?@x$Ay^9H+Prvm8{Mv}F?)H7(o(zO~XA`a-_o3dNgoOge5{Mr5CEuQug#xA$hzg+8 zo$vtu@Jv%7;@9P0Qv311lwb`Jz99+Xd}HOFzE1m+3fKhOjSzzU3enB>1*Bc*>vAJ} z1-1fPE=<38qA~Y4^Pm$PKo@&w;F>YqlWlAK`Y%#Zcflu=h{)^-84!UZIC!lbp1|Ul z3uFxM!nri|`UVsZS7gpL_lgAa0|4*|tbWlzmhe%Wd?T-9AkF7m%wMZ6K%Q{raF*~{ z*6y)uU-$ykCw{;reBbOf6uhC?6DlAB=T9Oq0nCYoAN3nbdF?1dK_8}zvomqc`6#cud4byh-=8)-T%3<#dQ(1i?rMnig- zTzGNchL$~By~kfwumQGPAV$1~$Nx&2Nc*fv zmg2k;{{toAY|A_Yt3)0rSA94WF%0xj2M>uvZMCjTI2y~)pG$I4rBlvl+uq{Nc(d@V zipk|Z-L+#efY#O`Hk44%NGx7iTr*XNvHFS-D?@x`#d8)P;)j}kfni?Ss?4E^tUPIh z*Hc9Ivhx^TPm-%}R--~klT}k&_P%JBK-z-B)!2y1!*%m)?TuKWeYXBE4FOsF2npfw zn+qRux-`={mkN<57OAZNd3HCQ?FJ;*BtVElVXKt-Xxt`BO8yFCq9-*nj=rkE!aapx z$~M$wJc|Lf&ZzW)buTr^mq9fKHwiS(kPd6pnyhXue?j2+FiierNsjN@qwGiMxysDF zbPJ7RLLG+!A@@*}!GW{Z5!4`wLUPzsJh}IRUM{Jm&mw$UDNulRHLEwfC*NUASuQH zd25$eoOao|hcJyJ9uC6I#_n=~Mf=d#^ojgvmb5wSeHDz>~7Mu|Q zzQ>egME0cu#`)M31&B!9pvTDrJnHwY*_T;)9u2o5ehMnCg85Ru^;yR|2lB*^7p8TJ z!<-gx?Yx@D{1aq%zFeGTqnRj|v9_mfbr!fR0{XipA?OB+y)yDPW?vP__jSro;IrzV zyjU)Nga6)?VN;=3fU98nb*Ce$Y z<7U}IAL?%BOh`YY8PAti(4B{)_~mJ+Nq*gMW;$n)^Ec#VDXoB9(+iyxpg1DdKD@fv z^scX2SC;>h8+zYWJo-B!gN*TAZa%)u@weBe9n*Z9f|m8@O}0nw~We&nAz^he7+yYGm)J zQ{|Db7nmKc!>hAdEV?+!Wn(}8pe5U+&zH8aC_NdQY$l$XE*GLAR3)j3sV!8%G|;*N z-9T2eMAMlyQ+^k~+c0->VrP}@C{aoJbLgA^d;NUtyh&<;sHEy=b&`(wRJ5e;omP03 zhD^-pbB%aK4757{)YGHWSp3dZ=OvTvN3UroX4%p6-nsXkVMkCuoX1i>{?C#Iu7}4rbRPBWT?aCJ<7RoqEq+30 z5LF?LdChY3PDW3yDqwwjso*kTWPA~5#?L2*f<`&soDWNt4dA=aOpmjC(2dITQK4#^ z9y{GJZj_<02E`*2sO9NR%_qNbiAF9~Z7X-XlxDRn@?FjQHFnmVKG(J9Bf`aYYNYC_ z`=f_ugTJS{qpGFET$P-)r@Mx1p}4I@Db&8+(#FZ895M{x!kjs?@D(brGE*I+;{TiN*Krp|8DdVe_KgoND6O-V5pfOVHiu53ZON=oMeTa(IRA4jIH`G6mAg?}k?|#@v zKnpJj8$?AG&y6|i^LEVNW_f(}W3 z19pw%eXud2%uSu-7$P(tQ=;q@!Y<{ARkbR+)E2eG80Ev4egySSjh_hWfc-}%ahpUr zfmo_hm*SEnt4n#0F#KbR2e9 zh_iqwAH;xMNq?MlsU=AuR!Zf$wzeq9cM{w=8fBuO0_Y_?^0RH8GZ)V>84=VgLT4E4 z^o(zUPoCCxTvA&hd&tc)obZK;x#kSXmy6f-!kQ^=v;7}3N0%vP5uq3!=&jOA!W~!} z6qgP>g8I;m5(Sdg(iB!~RW_1boP zjU>vKzAcl{)BG?J2L)@}APClwuAZAy>Vl}>V|>NJo9&Uv|bYmaiVAWR^mQi9Y{%C!)qJI2!qR$*|b-8;1&L(_+>i+$Zx+JZ@&*u`KR z<2~!^8W^$>nT?frpUB>?Jz$AObiV&}1*3bI?7aY#`=U74Z$hrNn%;Si;ucP`t3R`% zrWasJhLe0&hGXl(ueqkjK7R+T>@v38pDcuYT(7q7nq98(z|$A%Shl&LJPH{{@3JSnxMvbw4n?swCcGwb z6T~MGjZfah{SicRwg7z|M13BHo#mCu`-IlmUeBA8+ijvoxS@bFI#;eL{UCgkgrg;?0J3^0`4xFdujhHleBEBsxCzQ^2M9|@8Sshe|LrdpM0;io!S4)Bhu9L9T%C= z1$IuRWT+-azOhV6qoLA$4_yK&4)Nr2=R{hSZ~&$4ix!AOg84m_AhK18 zksThI%sUEZj zeL*SnShi`A`S8{TZ)H=Y>Z ze0)==OjG`i7M_(i4tTScV^cNZUTyRrcWNk!^-F^HS-7b1fj_L3*Q4H~rCZwV(I|?( zk1X<8;jV%o>QWmXOQ^_Uo{B^0b7^EMQNLlX?OLEITCQ-e_Ni?vW;q&E&LAe5ljvHp zp@9cTIE+xIL+sXw3XkM?1841$<;~cbifU>KpX-i?g}9tS9tms_VI%v}l|g=Oi~fqv z^cFQ_0o(-M2uCp~f1`0v{bN0~2bo^kDZUI^b2{8t;uajX|08plHl6C5j|BJ;t|9XW z+Hzg{)O8jow!@-7CEt(5f}#h_CeET%T&i|AKgevtog#Nwu75IZDg!YYr$6J71n|Fm zOgR3Z9}`Cl5l2Tmb7O~pCM7lMx~ll%=zq#REtmkvj3m^IHX@wPS=;9D#Hi5JSRJAI zbD%Ap-p<&(le_0r79Lmwfx$@WFGdD_#OlJ-la0xmABm16 z?y3c`I;$Pb>8|3ATGD7PHPF5iAmyc%6wn_5AZ44`uaA&|z37A_id!Da7blEBw4WZ1 zbAo#m+C@rSJtiGlL95V{y_Z1I47FWOB#p!_QU?tshZO}z^mtI z)4{qCGHERR{ZDd$d()Te4IXG zqh4Lq7t_{HsKr)AzikbuHH}Zq??X({xO=8@rt*#JY5hx0@lKA_y^O6<1-Np!`VYEw ztCs;=_UzyOu0rUvhXUF6{1G{fVd7WowfZ^fbH13YOQBLH0oKJyK{Gxxn|r9_owNkr zn}^l+HD0CkJ7>kP)Ux7NePf()9o_YBUih3No3nqJh^&$5SYYNF;pw@ zgU(RV)Wdw_ma@Y2t+nEO)QpIMU^kR7L?k(Ke;BduEUjJ%_05)~I7z$F^u`Gn*q0$~ z>Y%Cd!lxwWMkS&mqt&EROL&Br%6^TTx?#adD%~HTpep77i_L=LO5(P~v2Ld4HrZ#u z1=I%(bZLhAqH~3$nwwf6ZIN$~fetxAR!s zMhIpTeEPjND)bF0zL<r{osNnerx#QC!JhY!E23*XG}tEL9Wdm`V^rSWU*pBKNJc5EX6@ zwR`zzGuT*By<{j4kI{1hiNW*|=z0jn&`HG_6kNxckEavGPvEsWmKP` z^M%({;L>4-a}8!+%Rb0W?&8BGz1?D?JW&7Z6+pcUTSNZqEZM@BL|X4f?Q*uD?22lu zjUQW;3@n~WWUz@?x#70;)+ucE-3!GtER|>oJ+*k{ikt1grKrd$6OIJC%nJ^^R-*tZ zP8g52B3*KMt*P1FqNK|{6EO=tz{oA-5n{U^j7NK zW_=eJcHHlM@)#tuy!g)nKGV8GfHjKGQv00Z3_mKrCD0ALhvcS)6DM_COVVkIxoTei0#ano1eyaw97tV$An$L1@WOSu$+eWUuqnBZ z2So%tfr~BDg5s951#HxO&+@-hlkrVlo+4Bc;+lRw2j`rNmv<{!V(pE zg2rpt)D4IRzo!}vWQ#@qIdbfAV8t74t91Ar+1E#O6v37?ETw_-Q?$z`vm(C`yg;?z zun@^`55Bkfn_&p1ar!ves?u%R0$8LjHvLwF){zn5&Oe6TQ&{&|GV*tU$sJ#CPaL9O z0)9ZDuLGI~4wKj#PBFiJ1EKMw6r|^ZBhY=qqmmof++V2%okm&rWEyQ8>JJ6!jYB@0 z+GGj0ZD>lF0-Uo=>Ja&S+p=E=yBma6omm1yC*!hm3R$_hFif|BmA`UPzRV^h#lPvY#Xwn9OOZ9_g@N~s2k~?6Kn_Xm0W<4L}X7TJ3{dsxt{etDp7NohK z8EeWLpT%vlpBb}%!M!6bG6H9UgFKlB|pqpgMc&C8B~#$CM!c|%(ss)bGoN6k#m%F(4g zd))<30J51}vc82cYRIXn$F_p7&6ykBTi-o0J+fweZvtq_oyfepB?Jtgo7HM@A#Bdc zueOc@TnKtL?64;}SE=}C*ShDM&oM5|xA;=3!zJ@oJQUg|#2ge^bEgR7vdP}lJYGr~ zXt#j*=4WjAuMN;LAdEz_w?9?Gq40etlm(6EHcpN=D&Ksk%@V?OZerapm&My5Mh`~2 zM|JQ@$t*rJ>(|sL8*o>#eF}bTgcaJr3qUms605r`=mX(nvya8*U7q}!I;VOlg2oEz zJLP8&7SoYQix(P6m@CobODr-qLbS%+Y{kqRZf)}X+EpEW{N`Um{Y>5*7VD+)nmSai zf3OZ!6Ul=k*L4|)V^LS@wXJZu+Z;w8F5PDXmpdvnCGThgMy)pDava>FSKFHg30#}h ztdX-ve}M?Hm<<#{KL0LagSSjro`nzlSa_UeEohA!`JrrhkCwq=Ut$orLbLN2s}olj zn&riNp5jrXDmAc;7CfbcxGgyE6e2mDXu~^sIg(5#XPrCFp79lomUINp8HL5kPv@>m z245a`E{4oH_Pj)F@$~8-`Qb~|?3}z<6lSr=hNz`+PHUA%v&T!hkm@(XxdcO)$Z~aP zb~#^@Hr>UAt`nLJk3y?c0`jZ2t>JU<07=-uLcFr{Upxyjc?!{)a)D8yUr)9lV^ARe zkmvkjDA**Aa?0>4V4}vHS9E41j;gKom4Y?cpiV$DbNRZ_{`ly)(GRk#>gLVt0~s66 z8)K3wjZMNsdgOs^#oN7FNO?oz-;Q@634N+tRLkERlG<8OL8uIx9y*AA*MP3Tmd5x- zd57$pPZ~*T{p@mza=-wONhp$!^_K+2Gol1*9ny_0tSY|{3=)b#;fbW1CzAE%AkRju zSr?P)&l9%#Gl!h}t#*0EG!kIPhielpO^C}ZVfx&P-2}XWqnPtCfqy?F!RM#Ug?HEa zrdmcm61B5%b6WgM%16$>y8#;j_#iu1^+T({~Uz2 zr;lv4bqbrKh3!c`hoXAe4ccb4jj;$yB?xyDL%sCP`u25sPc-lkG(V>$sMkHP8FXX( zipAas!_(NAO185QqT|bQKS(Ic+^HSc*^cA4nVvr1%kllb-oP)r=36qmY1|GYEI6ZT z_>0a{L%%if9jQ=;T}@6f0AZgpq)c3-C=5;m!J4mb1_VXd6OB@hn5FQ_8o%KqeFZD#Q1>izGL2bGuk~8ELuaczO8T#p zWr=OA<_lAt5#UgcFU}n z)Z+k?P#DGn`kPb199?`q3pyfJD~5A62DyQba#l%I2Cos?N53S88J1_5*@gws)5y(_ zG6t$6%emGuvHf>Fe%Zyf+^&!rUT z(p&CJ$k=Le<;rYw)e5TXZnosbTSw{zChhxN{DO}DGAt|sZN_V6aqC-EOBpzO%L!B> zk*j)j#c-+SwYt^6c+*^mUHp11yaXTb&guh!PUq(E! zlOREIm{EUIVAJDQmS=;1Z`-gOBf?|0#=Ex$7$R26=ClG|KUx!z_)9eQW#t*;e~t$9 z`M)LtZmB{uTk%bZl|2EY^6y1J`c7LohZDIt1{ng_rSIhPR#uczD`;aT#Om3~vmibE z3nODfBCKgIBnAtYB^R!4$x5Wpk#inSw0+W(ZXxXXsI}*oW9L|R^`AoYqJd_WxB{+J zQC`+Mug+*MJg=fF70`vp*h#&1Wn~&zh*q{eCu^1g&TlSP-YX(PoVy+qlLY>n;#Ye$ zvPCIhiU(uTTdGGc1SnG?=jFX5p*gzIdBzo{^gV3x4g4s-C;TEP84)3tAYA;wD9%hh zj!mhLO3Oweat5|u_@(!2gp{!Ap{>}VBk3Jq=yrrh6%h_i9Q`yOjB{aB z9_hFkh+D&S2)qm!!PTz337;FjedA6=T8!8dV$;?wOdAn(;C{Kv}#!x zw>l|{!i@j}mo6@S!bC#iTEAGr8*s-{fb!p&vd6&XW@c7~er#}jQ<2*yu6GXLA5i0!|jM-q09#%}*fj{GdOCnT*g zLN-KnQsfuZzt@OZmkoc({>oRVa2_-1f``e~b?Okj`1&B@vF&^ zdoaQ;;5=DkpgpJ}CmF~RG9us!vUk9CVnx`SW@0Mxq3cYs8O3qzuNo4#A*?f`WTx1n ztc<(eu_*j>`^8F3!bvRAs;rD;RHky36hPKEeo<0=t{jK#pJrB%!vq4k;|i>IW{qXy z_R^~I)8D@@y2j`^!e4RBvf~53Bl@y9GQ(R$uP0aAmY1KkTC0Q_>)O<;&mGCt#`DIv zx^!4)-fl^6az&Tdzzj)Qs0>L&=`*q7Po690P`o~CJiL#8-`a~Ws@{)(uN{+tnIT}j zDKLek!WZ17c4VPuI8R}a+vyo$dsUt0-9Yu!OLEv)cqX}OSOc{#_=gq^)W?p9+dKj6 z`Ouo_nt?Vk!t4s4g;Spu3v52 zIZw<|yy~NDU%d+K?gIK__Q<90@x4tCTI3nkujt2+-dxNh;a44o#RUKjiR5Pv& zBQ3ncMypn56+eT|#$JhFpR2iiz1?PY#4Attc56X%LcLRZ3Rs(<26YIu^e@Ky_ga|SIc3cb!)NA-(9D6+~`BJ`L!+v5Nk$nk^ zg0gNNJ0dD$Nrw~A!j!M1lWl)qL{ELepSbl2?eGZB`RqUt!*RH2f=qc5)Z=@2oF;;<-^EClrldp zwj*nZp5GbsCl4=v<~P2R=<8=u?$|luJz&l%C1}d5*Dsjpz)oZBOJ}kxim>^NhrUc_ z7rdA`g_S06io8UI_IH1jFHSQ4~XAJEgT7uhW^$(7JcZC*Y-%>p?EBKV{lXUATW zA7>1n@c!~=(Gd_~w4YLB^tV#P^52jmX@~zTL}pw>lPZET)!0j%^0sT>ts)mtsTzXd z3zSMwOHuyL+75s{3$AT>5eYlN@V<8=gckF*>%@| zKHpiM$LrUEzMe1FB!s0-41XmA1{kIC%?;jx}WtiX1N8Hw;oLYO;&_iCIo@I6oyIMt`{05HY#&h+{!!1b)$z5r5ktEir zh#?=wz+AQ5BMH7&=kK`xn1^zY1IW0ec3)0O4T)HJe?GxTJ|_#?nliPpKs;to5>@Ct z#3Eci2RqEOJkwk?>_DDoZhnZTx7+h*+#-rz<%r`UMtdw_?<$j2Y^%@oS8C~-PXgg; zUSJ>7A&5g74_f(CW_3iOqC_Uc`wIoxH;GAHb5{MMf$=|Yo|5kwhRL7y z(~VO&j9n{M^h~8z`tspP96Al;v8ZkJ+BjR?Y!0#ypL_ttU4aILAnJ% zgv%c;qg&sXPAFj#UOz*ju@GT|MYONS*o+0=b8uikihkilQ~AtpNVx7?uLgG?%57f1 z2l5SJ<JdXKlh-KKbO(3G+5M(CfB@kX>m!ECCjmYW!mIP9?*HLcnWPK;Efm8p0!zKrLe)Xgd>Lq35*VHR7Hke zzM)nKa&lR+CQ-;s&Ln@%($K1eS063al(IK%H1vsD%G9GrmFRn;n~9pbpPVfm885}p z6ptWyQLCe(P8-hKn-hGYrn;zPQmzYI!YZLFZ5b5geVp6g@qb#nY+mR)8Plh-Od4mj zPY7_fF>i3gZ~YvDVDIq5lPrqc+k!nG=70XBJaQza>v*5aqw%-O^WVv)|I-Zg zDLpx5p$Qr+R=U|MW@w%1Hq~<~)=5lg{UCpNvO#;Twa=*y=6}nkH6#uE`m;!5_qqS2 zKP>;qrj1=@NBp4v_xzH#)j!|8N$;dlJIl&>_P1Y-M%$pRxt9jUb z!iRo;m*m;{AFRD&kfjTkG}>Kem)&LCw$)|3%eHOXwr#7+UAAr8zPrzP=bi7~m^m|F z#QjnAd&iF4`DEtGXRQZJ8XY>`f((f6RXVC05h^Q{6H1urlNHr3*JZoP4!(80j`p4k z#mZGFO~s2ai@R0Y*fT@V*!jBihnXjA)wb@btNY`6h0-Q8k3b&sWPbdxPuID#Fo)6e zzhIs}XO3i`vuh$=61FR!`U^*swq0db2imLT%CGlHkYs{9nW`G0$=Te0gpkh`%?_PN zUzK1W!nzMHt2zjgMKY!yAe!(6_PgLy6M;q1LU$>O=Zgco*&0O3dZZXo+-u~Otc&ho zImF8LKG)Pd*b;OLW*=XeaN`T4LWjB@={o}Dna+b?7FYw7NY2W;AJL#V=bDsi^?O?t zhA%@l`GTwpp@mh>LGQsqx|E^ItPB3YyZgQ;EavAaM9-?u-aL%BPxH#YDMuZrvnzn7 zH@wHPBaqA@u?WeqL2GI8`Ch#6iGD@S!L>AWWRU&S$1{xo0VC-5x6sygDkgphC;@P$ zp5VI=?O)M!(vz&7h0|I@4ZvA#!eFo8ple8DQ%Y!(YcETG^ONb-W3-NZk0Kl-^z*yqKUbVnoLQTvx219qs-6oj!N-R#Bc zcY`{Vr; z_Y2LgFRqGb%ducja2^~n?tMFeh?eEUM8Ld8%SJnRUd+F{k4P`LFHcRF>%#{15z;4LfJGKC&v@by}eY+P_Q))RvT;=OS{`8pmD^SY}t2wVhin zkBz9!54qrNXkstZ_egjxqhno`tiADVXVR$}nC4{})%)mvFJKuJ4gN4XRyUeA%Y=CC4wLGlFsh}-Nm65(Q!Xz_dqFivfOTNJbX=cXWd#w^ zF7_NO>RdF{m&>POE&Jia_s*&~C8c~a$C}#MV-A>FnF)W@Yn`rTYRbFaitUO#kaLwg z@shGEOWPosuVhXm;u`yTkLpWTKVPe=_8G+2ab_#ZY;hT-ZP+jEk<+)?uhNQv96O(y?*wOcBN{q|L%|tJb|Wtwv|t0hQ?KNlx$*wD5q*{Ij5KHe zVbY34U73A4EAGyHi%M`v)(v03LoymKnt7Wzj(7WXCLR+9H`5pXYzSP<0@y1#BS}oF zCaP6=f=Dn-5UM!G^czj0;=RL|yzcGHuiy1@#FURX1H?hJ$kCd$f08wd`T|wWa+r{g z$7S>hYvPNgMSL?zdw8-^P09yHFGYStNXUPM_hy$sBmdeGhF66j5t%<{dUKn}Uq&Y4 z9gs~Jfph38?q|jNi?kvBHik;uYGUH~D?#$6UK{0GU*9DBR@WO^PiC8$)*CWk(|)#g-~1qDip(Lx0i*f93f;v{R@w8xr1CG<^6ekc0ZPdy}6gt zA@~J>l!(OPc6kiIIVIv|_roCExnUN9E@yP(q_^_=+eVkP#ubcV!FN$ZUgk z8q!7yuq7Wox3f7COpD%hRV0KRkL0})zqS94Y=_&{xhs$lpuc!V(a&hLowSE(`;Ee7 z2bhdSm5#ZGj?8IOyaOGE#CsJM$J$Eh*i>yLCISb%4r%jfP*p;wIO|oj)#lgL!HM(a zJle>S-V!^kaj#qZ3w&yi#Y{{(^nLVdy`udQKuI6{vVVs2OaCZ<{%=ws1ruw#f19s= zk&nTxz8apadL+N)Y(tRL%wN-aiSV~6qB+Qg|H6C;W`;reMR5pw*1o7H8owcVC`_)O z5m5*H$$>6f57#a|4zn&dI@dm5S8=~&3^YP$L+qNND*&!u&?606pjQW8yCD-~?exLp zN(&+IYYL#nAp(hsLF|Z!qDEbBoAPjt2Ja#YxCjtjF)~Q&35h@%8)gWq52A}0YW9O- z?8+Ky?pzqcjHdh|$56@now8?@C3p~n1TQ{67L=~n406X*OwAjElZ8T;f-^yDPMKzA zrYxMPp=#vjX<^->=nPZNaImk52LEHyjxerO4$%g`Gh3JK@+7Uaw9PVY538wAIc4Cx zph_|JE4O8jfspfuP3pWzCTWQZW>D+z?-E=q3%J&NW_NMy_0~hb- zR&33X^r82>wgC`G8;_@JpWc9oqc8hyN7R>61J>`EQW$H;JCA~MY~bsF7|66uuGmzz z+1x~B1}qi3ax`M z-BvRxd(DHAc?8F}^*KzV_cX0c17;&?21y-C#gh1#zBLkFDF}Q%tFV2!Vvr1u1ScT7 zPjUUP%fQzPMatRJYs50G307uRuPv4x`zDQ;8h3?kE?988E2|JJ?a?-j`l`sGbLR)a zQu0g{pze+v4>fm1p?7S)+NCrfgHsjZ-5gotfXmz%+^jh=DIN5^;J--_H-(``NZNRX zcrnOhU~R0yJC|YAi+1yC7yZ4&LAwOLrwa+mB*`S?Gk2|2lZ9HsBtUD#B=n=^=`_qd z14EtVU~T5$1}dMJhTSG=m!V5}_(2%6%}hYYf_9OP?JG~f-ng{dYflU*hM$46tuw`{ zp0VV4;Am1uBN;l7wYUNDfrMPqkZ8HvUo!=k4Btn*-iW7|AOZ1KzLIZ_nfGN0m%Thi7pncQv_$<&rpO1tjV@ zA1}c!BT%_37W8AV92cL^(x64qxHb9k+jYTJ$D{%=LZ88=BEx_{6&L23umLvG^%l2~ z$bdx>+s@7++el(~czO&la-6!ZSKms?%`?1s>iX{kCpxh-2MSL_jP5(g?!c0`W&TMP zz0B=p!9G9|y}+YMOc(t?zIq2erF0h`(y7AXb1LD>h>KDz;s(9M5~C>;eNA$9r%w92S^_Aw zmd<9@3*?PGu675ay}#gcsB`ZBQ9(1dBEelRaL+|q?d?eF5HJ&Lshx+r02Ai!{dhK< z?fv-x?sHqxZ>!yt;8zZ{A%E56X9~&)WrP$nO`DXUIzg1P<%$9WnBTG$#$5)ef(_jG zscI`C42UQ0NSmVzhznvKGIjXr!d7qB1ufc&_mM$i4hFDCB{OUGNMdgADfgxKQtieC zxEPI_xnX!|2p9wGml%sm(p2)el5zy6$S}j%{!HTline*gxic-3=Bzf(t^Gzap7E$a z28nI`&OFR0S*ujL?t-i7h}xkftA)Z(%?TW#iNI^-p}OBJ*zFBtmKSM_GZCjAdQ|Ee z45CrVYSt`50;XlA>XK`P{YdACya_qY*U76bZ}#H`a7k?!fPCmQ)Nb(%llOpKc?A#q zkK{?L58fC+Sd(h2vSlTr+x7Uat!1%Bv>z6*QT#Xw`ikh5b^Wq5hoX{6Ov9g7bW)bW zl5M?!*6OaM7@I@GCe=JxETs$ceV$oP@TbDzLIiWZ5sO%&`2=PtS+mPJA(fT8R5EQ! zHllLGrWGctt3GOTGFCGY+5Pl%@Ja#-}sk=W{BD>iA={f$R?Wnrn+s_l{{=+-bqZYslJ!bh3h*NqFXpIKiZ6)<= zq}nmz1XuK*umX8QQ#yB%0%St}wt$;xdhsXtGTemckYb5_pj-Y|ZZ zK|v~g4S^2G>g+%lubCt>>PWPYr;@wK%f2O?CwS%#0!FyJ*bovpx^Z*#!$b1Fc{$+q zKK7sdl3g(C515WWfX+8w~mchf-n@|7^^U127;P&nCbG(o9b1Nw8r+AN_cQc9b za^BY!E&YI0clOn;fb^PXB4~*`F7C(&$J89xKYjgD7crI2hRJ}tQiT#Q;&Qtsy778v zgLI^%g=rgoXDbJ!%Fu8&Cqj7wH$hq1*_6tn`=qigbj!4t0vM5~Pyy4p?C}Gjlk{Jz@+yR=ZZe&~n0+W{nUH(-M+b^|!riHx6bR9wf zM@8cqtj;J!mq$Dr0xNm?G*k|*ZgnZ)c>m* z|1UF@pe2LE57=>-`zxhL{TG~23>>xU$_xrhM+tpx$_z;)5o%|)Rj|`(P5!H|uTG%~ zi}n-no7^DdYFdgzprsy*y^+x&`z5=(lbzSc%Lni;nk;(@Tdtko_7?WYY+>Jtqp+hG zf(Ncs^pkfXhZLfWP3hL83Bxr8*XWJt{o7T?e*#Nloau7=dsfhEPI@Zl@-p2Z2~u>w zXX$cm(#V8tHa+55XqNga7V{()cc1rZh6UR!ZjxD1_or2I}paQzSY`Ct; zgvKwy;BAB$by7o#*1fY$cTtg?f^CI@lS#}&76&goKU5`Jv`$Obi%kkxA~x~}Aoew3 zGFd)T?>1{ODugtpvlSbat>AbVH#9SJ(eam*#4Dpu;rcWr|q z8Enp3IRjlc-x&DKX>gxQ#f@>c8Nr;YtgntDn@fN=Za?O6rqm~1c?Owx+2fx^k^gv@ ztSb|2CghyO5|cwp&pQNT{)M4_xs9_N+@O(1FYNEh48~qp&jcKvLTX_cP#G%}W~5D4 z1ychLeyQ&YqfG;AWB=V~j`uQpYo3grgAy4$>4-)EHeM^8?FDdUzf45C;I=C=@Szz0T6>9`qan~okSwLDfpc~pv}+r zHC9yIr$3;emL48#C)Ek#fZOCXfU(gs6X!LH zsVVj2?eQJx7n3sa4~Z$FF$otEUFlwX$Xi5IBC0|70d5S0yWb`vnj+zJat@HQ)PzCO zpcFxY;uI-81%AXZXs+Dgq` zSDA*5my()HQJzG#mv$~g(IJR5FOJyb6*_3iw8f=B0-LbkQ#grZGg&sS0l$noN_fjo zixkic_#}$}EeiJe5RcQN0Rq|z_9(BqGsi<%!R9uMr7Y9N4R?jU!M=UPtpOXGEoS#Q z3(M)#CQG~Zyk7xr>9Cu=sQe0Wd6YqhE#)wl+C&OeQm%+F<5*6l7bcl?P`dGrndq3+A3(+KkdvnxpSDpy{|SC`?0kP49`~VI?c*jP{Smb z+~@U0F^#9VOZ8b98pDR+FA2R?BdwHPAprkI=NG09aL@}QDJG}fmt#{VPd#k$S%-33 zoT$mLPmNNIA53n9OfzEqc{24JrkKo%M5Dxq3@o?%>Q z&VLBMF@bq_EiGDci{4IY;Q+oj}cRvobHjuV`l>G7;umBq(rMD30zGm=!3ZF)8*xWmtW=UC6Xmqe*4FL44*~aBETuo+-W?(F z0%0NT5#tb1iMbMMh_+J$3?V5ImHQxc`9!`v8aU+1h{&eEg9T#5A&7JXp~Uz0BUAm9 z?!hEMF1S|hu@^$;n0rF)%fZ~_nWj*mnHtOAlYR&|JaY_!v`e;2%O$o4?ZeB~ri<4w z%JSZO*80c_p_l1BNJ?U6%Zzr(TolOH7IjZj-c}YDvOc|ha$GW9MipaFScv3mh6r!$ zOOuakCfrn4ul;>XOC?!e#kuaf`}Tu-U#&0njo*gvm?~REo96RToV@xQDxjMQj+3W}4c?qC%R;!E+iOj{41fGNNIqQ~>N= zq=*LXcG~lrr@6ei_VN70bngVxVRgvOwEi0-3@c~eD0^gel|eE(a{Cm71j;zsZKfb5 zGkzNGUXD(83JQ~#n@Q8uFFYAPCpm{HUcOd77v5{&8w|JZ=`3x^I{d>kBm;V_wU7`j z1SbNntI4B2B$)OmHg0JbL4miMQ+5Fdevx+<3Y3wIjjNLsG)4NihQmWSWbwn`&QwSLBt5UtB}E%_vX1S^ENW6qm! zy(XXBGeSn|W_8)xZGYiv@)xkdeqxqGk)RkLb1Bo60}zV@dJBb$3}9{H^}Jja{y}x5yP0O_6Wkt(DzQ|k=nzoyV+T0_KpEmDnls zZeBOy zyA137qI;2Z$HpI^_3XiD{ykkNVyCOiTg)M1X=U(Q^i+Lnh9>qh3!{nTi=cvR8Wd1L zPieZL$@BglHLQh%V2=5UjnF%R_^i3-RP2O{2MB`6GVI4t3aafBI2h_Xor@ZF*5M4) z`YK>&g9?45K(?VXQ5`9|1?%hXgk~LO;B^Ljz3X(4Nj$<1bd&@J$f=%)K0YZQOe_W#)~{ZD*HB+ZY+m$yMiH7W%uzyz*o=dj^qq$YvcKaY{CHVlNPaMaIz^NRB75Ul)g@t)m+=)a6K`({W21VX-_qr$=y%{b!hRN#Zh_L=TgLO_djd*Vej1Bg^iY#*qyp{*CSVbthjvQ{%;44^kZyg602^B&BpY5jS~<4_U9A zpBwfU7NJiNrSeD-0Zl}tnU0$R z@n|MTUp1v62;+x|YJW`7_bgMFNCZe~HzOs1<}9^UFANTv4v?wc8~&maAKWZSmMC{)s?p1P;TxurFQQpwnM%>;RiV=`|fXE`U3 z{Vp2Jm59!ek^thiWS@*`V2XbZ& zi~(`%D{T|bVjUm?TqmH9Yl;f#l$VHWZs|Ndmx~$i1~HbHp`o3nbJ{g}@66amifu`# zNAhqeV3%Q^(P~X>ostw~`(j(1ZaAmV90{4&WV=yU!nr9HBXcrr;wdj_Ty+_{*pR$uY?whVRV@^!qinBLDaoT}914zG8KhSpAQs^+i zX!Y+NnRq+g$vHUCQE>Qo*)yT$ICw0|*4?ePInC@28JM+v5(aon>%*ja|FM$HzrY6F z0I=r%N6x1HUvl=pYj25(8h?_Cp-%;qcFMt#=iHJ&43>@JXH3kpN@R5jWkWR=KumJP~1fgy}GLpa)r5Yc0P|6h9m;p~v z8Tx@Kl*oN_u)Ny0EVmpI$9Xc+MQHa96GLMv1kVMZNRma3uh%Bp+=H%=Pp&t5W z^Axn)j8gY>`p$>UeX>l8Ygrk(*L9f=HS<$P8Wby4on`U1);z{GD_Cc4A3_eChn>J) z02>LUHa=CM6r{acr(HR|ijUs5NI^USKVlB%t+q*PW3K?B7sE>7SpQqcL~nPs$oQam z>IGl0r7|^v?@(lUg_za9+xa5VFTF!zbgXcHWuECOfRsjlIky2uA=8wDu(O;Ja2Px} z{&SUf!=ZSViM4yyG_Cdat#hGDS;M>rrmr&Wx4hi`@qRCXzn)SW|1sbEsfTB9k(7@J z1o_b-?I5UB4tyk98b(3rER7${3pg1mgw=pIACJH$>xx1x5pdHvKyLs19!)+(XyV?o zm5nec2*dfrR*1CX1L5^@5A<4iiE==HUNhVOVv-2T$>AM=g{JRK#1lv#ob90Zn33at z3K52tSxA^QI~KJO{{-t45yRxKZF}=DmZCiZ= zB!XEo$$$K^A{%!%ZPtF=FM}xB^1L4C%~uXr`xm&8ayl~q2olbJ1o!{eNB(EC>fe<8 zA3ier+v2q%KOiHC=?AH6K@SPKD*L0h&(GS(afik7{?16e$aY(`w7@9yJC<784?iZl z4>MLX{y-b(7YraF5uYOBO1%2fn-Oq`ctA`grW!#QM1z5_7C#;F4MR>>?+CR>ROEXm zlp+jtydqsVIjAJMe2w1_l=jbrTyZH&I*0PoyIk#mB_t0qhiiWl5;S{2Lc;#9ghY4$ znKWy22RG#JghVZV)>UW9m6!>^q2c_<@B&%bVZ~*#x-UwiimxG=xNpopxW<6+On#(P zWRFvn7TLvX-1ctQu$t2v!)-3|*Mmc8dgDif<$1ibNTsp0*{(}V3RDOBw{KKl0_*EA zI$nxXb3RL9x)>J}g5}O}*$ZjxVk>*h6t_n@?;5=Y?uM!v(6DlQOsRMe1A7Wg+$4q zqb`?hta*jiSCRWUyLk}xb>RFK?LZ7ST291U_DotDgF|%W#VXLkI;$iXqi7MShy!r; zqeXHf(8z3-G-@Wg-;C1jAZbUqIT?gqj32-^;_%si%8%k z!$S+d%R`p6^21Y%)k~B^#~v(hSQnrpViuSCAjNeM@r}h8H2x$Ts<&)u2$08VTg`a) z)S1j$&-i$Md`15S$lg+7=m>R{dl@020r^NjF<0`HKyO??@_oz{aXy%{!Q4(04DO2I z(=^GfKr_0V!SACOTck`VwV-}FsvrbVg%Gk7k!n9uy_BH8KKUTvm$_ep+TZN!O4hq@u9ObuQbI-0P%1Kd-tn=NM*MgdmBI@sVh}@MKKj#swLy zVJ30duyNd|odDLEv_S2ctkEP(+9(6ya?(o(qNbF^JD6q&tg?W_g6~yAUNv)?QO2lL zlI!W@b7$Ow8f4pQS+P@r@>!}HT$B~z(~D)fhON{@`MKdDD)VgenMDvXD&T+tFs&lJ zyIk1v)R6ge@_hZef^^phfZQ|KgMxtC(&~euvtU?vjDi?J;?jCH_nhU|)X^qjZqibl zXFSA_;HYuBV0d0^J)?7Zs9EZ|k_qdq;yD97>6yZ+52PoUTDFl@fDOSVWLs&8%kA}N zm-CQeJ_b7d5=sW_nP9cm!+R&BuT7LRr^dQ=)q*}onC8Q^eJN9snn#E0U-7iNCED4& z>k^&pvpApMFQ68w#GHRE&zwD#q6mtb;<2SFKi?^uW$@dM)w4PBho%u{(tD-}f95w~ zkD%nV=3k9{00-a3nHwusNkK2YQGM?yIvUvY4d5gYJSp^E8(`G%IIi`<-I@}{T#PD?RyjN z`=fWW>F3@9NjZ*WXtKyV@bf(2i_+29BU&wy<95z3uhG&W75Ru}9^+cEgRMwqz2WBR!1fRbEqVh0W zrs)VV0gv-whOAX)eG9_`H16idwaw?pHGAtn2GtYn=hssLd4d@NoNyNc0>Yir5MqQ; z0%Zb--K2moR+12Ud;%fqbb4zr;vqj}gZ#v;5E)Qo<7&eoG1RzXu3;E~+!FSRd{W}Z zuuDYE)Kwzi@07QFP|lgZBJEY7DrrffNokR08(|(x<@fW{$!B$v>W#(Mq!yw&h@28? zpGHo#EY36);?S3m#Me^Ddv;FYOWmHIrl_VlMc}F3(=vLqhWiFMrW&r zPz+1pk3Ym$^iVL3XrvZ~3qy-NY;El)-;WfXvmV5p_azQAX}5PZpE56+@)vHLSLG<3 zv-q>M7Rs=s)PG$Gi+Yrn)Fti>vioTQlkP7&I2Xhm|&rK!)8`K8o zYGjHk3fKAdhMa)2o)>?9`ZM#uq49T6X5(axOSmmz5g=Sd+V=JC`GTXnG%$2*zlL?N zW4yVtRQ+B}FB1prl(zIG646+ck+fyH>OJ%cG@|vbpnTmuY)cxu4lQ%worIe0q|@FT zF9YvHN=7F0IK2R5s>%N3j{4Bj*DA06+LsH*1!0QW!Z`2Vfc`OgBwzp?FX;ZUAP%8%o~#a^%) z0~zK6j7S8C7V*HQ;B_2VQNju{wSsRlJ#t0-V*{R6tba0Hr=w4vrwt3LjD%;l|8c19 z7wo6Y(cPQUUw-I{_Z9?zBjQM;#9oQ@Rs^sEWlxNG`SA3Hgne0iMwbyouNhbjo)l8hRL35Y^ z=H@sll`_TfBPyBo35y`EE1=J^(!^%v@^-a4xah5k#q!)|RSBW9tovZR<~@xtW-UI= zDxpOTK$T8GQn-yU9E>IwM;iavA>OiDIr7WIi0$!_jBqP^TiS=3CvitYB zl$F4rFhu-p%Rv3z$Y*1);HFisr9svY9UlEI?9(McvUiVWxt_3 zz{WjF-9WCZyx5@qt*7dWK&UO$8+p~^osaZ5{U>A0xN3sBj(DhfAsps#PhWtj zoR(>E(!M8+RSG>38o`e@{b`V4Iy z0R_kx59r|m(;}R6xae;|!o|~ze1td>Qe8fAg`d1|*u;u+k`o5qG3>qf(SJflgp$8s z8L(%?^J_yK^j8&7)qihg!_pE(|;i1Q~Q;06ixJtK8=4Q zWKn@_I*GZdp*2BeznQSl#&P0?b3Bt}T9M^SfoXaJIlXE6OuN+OYuDwgSabuq>EZcE zU2N8V;$N4mCNEY$J3WWS($$`~hj2Y0_{6?Hga>-n{-&k550CVk-a$)$BiKW$^g?(D zXX!S14FA@BbQez3y>y1A0R-Q|;={nj>|^(xkPhfdm&^x>2vYv5;x1Va5=0eVOWc+? zhzviAiMv!XgpET0o zvi_vX`H0HQWj3D7M%&BtDUr$W9UYkVIpS*pn__Db*%DlS?L;^OxI`bJ?!-8K&O{%P z^r5#;js4zN?&Lm~68Kk^P!zx8N(2!vsJByC~I&+UNAkM()VYx+5h)8l3*>wUE%eTqQ7?=K&EEHJ^_&|4!I zK=j}bE4+&`bZGy6nWP8mC32b!b2Sg0CoEnbVF z6R*eFMKtiuLnka#Xjz>0q-e=$`&|BP5Vvs2Ke%uy__c5;I$P5A3va>>o)^9_t{1s5 zZx{IhJ_DT(E;NXr+^!H^Tm-39oA>~uDLBXo*5W;s=1fb~rTTfm0^;IWY*JQBlC(8S zinp)B&nI3V50z&a<;TsF#`*N#ybpC ziSd!03jR#7@?!kuq$*-$Yw95^W38Bh{ecu3;rZtje?g$GL2U}ROw5*-h4?jQW5XY- zVn@`X5x-5U#o;J1-LdyUne0{^BLn5-Rr`k7f$@`6JusPFXKjbmJ7brB)KA08aX{^N zCR_)u0Utsh+FvT}Ne5*(x>^r9e7D-4b>-iv4^PzU21n0BgEY7B($8|nH%6>5>d%e6w<+H(i_P z(9;WS!$(`EZ!NRRBEP-A5KpZsX9ub||Rclp6KkZP*u;Yj%tztZHb|bOxvAiuC zFW1?Z^DG*lGT&bMpts-0zck%r*r!u)9rBWYo({#m9qs8K2gOb~?z!0jjOzbg=I2zcQ>wXQW{ZX0Wia zfV#2_6wy*vjZDIFxF*G9 z(uURvDMCl8V!07YZuZ{GN}$;Bj~7e6F+UrsY><^<0x?CnE3j)fFJFKT(Z3liC_iD|NGzfdrm8gVC)Grvc~d#ai9NXp0wB}wy71&nClALx}xnnQZZ zXv&oI7y9QeJgAD2t)$Uf!!|kwvWgNs-#k=_sanI8Q&`&>S=s0nf}|390M>Gsd{@&evqE7|9!!P7U@?FZeC^u7{~7bCLjsqcQ> zT;|ls2>9`Z*dSM#RffP3q2k8%vvw9qwuh+_zi(KB`X&d;q#9{(W@2tu8?h3wEAtQz z^9(opB`B&k8HAGvlc)WnO}o1AgUe7wqy4)4B~)P0{gi>42GttIMn&m2it{~#Sw|;3 z0`i_K8~5^s71ZHmXA3Cnldz`1`7)K8HIB-Lcn1MR-DPpdeuZ-F$O+@ar@~oiphc{Q zl2`UEjjt~%S{QFBusPt3d63NlTh=F5p_qY~lxc%g9@TB|Pwl-DqAenFtTl61Q7dEe zthRIaPpV06itNfFlWA!~HRm`%oVd=6Myxi9p>Xy_SeN=O^K3)5i*`d92%Dw~!rdaA<~^XKK7M9E9Uj`-W$#9gZaQ7FmWI(H0F?ePO$#=}d@wF%i~WO>kc z%6cESZ*ln-eedxl$ZEojl9h$t(+uA%_DbBZzXi3W=W7RP*d5!IeiYTC!~dKK9J=zV zGJ@4K15IBs%z*nfv^-zmFjWN~d<8am@pa;Y!Xw7MIh>J`H92_LXwg;?H#-nT6ZOTG zRKF?t8ufD|wQ^EAkOBAcyZP$f=65AkJ3n9eYV>-EaIHYodOGsn{r znR{_qLV4v&<;d=F0#_n$xqswwI%Y>x~sNUZ|oYvhdY?Ue=IjIBT)#K1l70`+mfiz3CG8 z-L1$smfFL~;g&AYf;{~(C_H}cP(};DEd@57kZ=C#pdT+fe~j9LMyInm6rj5Ykf{HC zckdsK+JBrjCepRCwj%o1f#XcT7F=NCGVHLS_(boetQI3 zA!OmPa8>wPbbjVi6hG>|wu4oTN>f3+;0 zt^X`ibd%##q0m}Uen`xyLZ4q#cTF#C6nMXJ8=r|V>)G!>oyt((>oYN{H7F0O+EY|m z3@uI-vFek5NHwH1%uH-}tgwNyVA?i!oIzXqnLh$!WNK!$!N6-H*q`xjaLv{%BPQLE zsk67jK%V{wVJayo(r2Mh$hud*GY}$q^!p40FAlOxRq#pD8w1xH>7dS|$;nNBJ8{}4 ztIfEq+Ak2=CzCQ6kzc!X2z-gbWE?^5+0LQ-9uO%hR|!TYjizlVIJRs1)5}?mlt1`7 z3NS3@m&qB;{TV?USX+b!pzQeaPK)$QD@tsssYfoVy!alLv+|Ev)=S0>vZ{`g^sF=0 zrRP*KexcT5XOdiVpn>$$d`IQ@nXD}TYs-)ZS`4(46{yZT5Y@4;iBGQeShB25PoZI) zWtNp(Q{%NCm8lPQIDH~5N=_GLr6RSs*%#A11=AQpY7R#5XbU!K#S;9|ovA4|GfgKB zKKiISTf}z|^%>nV+mv97ez>MxmE;I!ko!nV9(Bjv?f^Xw*9kPA#|%lwBWG01bqeRD zMSZ)R_N)pVE5|0y1bP3*CVkbDucR2Ok9+-42&ka4D_(bV~7N}=zZ{NT%@EPEj) zeFnw&_o5-s`a?ka$rC$lv`DQpsw8e|S6c$Vxe9IB-Noo}@hObIA+XC>&NZrwfKXI& z*_OXtvHtmCed$51d!X#|MkQA+ZP7L)=Kx$!3TX(TFOHeg1&sg6DCuu zX{#|f{h}Dw|3%L^USVI)!X}O2+63F>3l7w_kYTQ%!SJW}$LFi`;brzYnh!Y%eh|J0 z{vf_IS<2A{~W!#ylPPrpCQoY2npO!k(%GYtSKCOecS|qUOCOh^`it&JJZ#H}z zP30A|WcT-Ei=|S{HTG9IK?pR~V>Uq*l%H8e!ZaX7N1seqTZ+ssP zZ{2ro0XW~jR+BzL~=$WRmk z`x;QL~o#$oeCH*ZXJ2ksFTSFa4K5`12DJs=B)E5Dw=g;Q@hY0+b zTUG~r8e72MfB$*^am$^Eg|35`uDyxkpU?dFP5J*I+0K+_6(kkpp>9Ionjhsm%KSaV zE9i4ybd}6e0>TWZ^P0@if(;1r1;OG&h1)MzI#<*m)yo?j6>Z^G^u6ygK9+b(&N~J9 z0wp0_q@xc#NN+a|*PS;zlQ|hb?k6+dzl>Z3V;!zN4M*y%J>6RhVUn zElD`)mPC?tsFtRtuBxuKqRLWdq34PZhhg&5l?G02DJKv6vScZuTJd|YIYrlCGR3!D zU=&e!LsyLfXBV9TIfwEmoK?l|eLN^b2nx19O=0mCTgcIwBANpjXNuRrJSw)RNny7U zntX7h?NpbgPZ%0Xn*ACSUdHap*KJ{xOXYcRX<_9Z)65%qq*&`GcIrv=iUq!h~{btK8nP|E~f?0U$hAh!a&X|mX#9@xM$uD zN}6P%2OGXAScii+_M%FpOwXe>P-%0V{a0K`kqb zHdCf-Yt;#cX~s5_JA#CDO6kKhb`rv}UQg5)KpGdRr-)xnqeruGCR8Icb}n%;OzT~+ zLj2?)%&UFg(A`L^{2n2$!Mcf5@8KX4wd$bTqwJc3c=T9AN5aubav$apRzX$>c~i!M zLMO6bA=c^?F|c2X8m;V9E>DBW*e8XgPV<`0kg2B?VL#wxMXdiwV^yYfY$G37DP^vE z@@pjM>0hnqt9fOT`g?4xl{!@<2%%g?ckLs|Egc>Gfh^K_Wr>eIjfZ}OeL(C9ZFHC zP_W-s__e1_HoBM9u_Dc0kCkk;s%UV&&0;yJiRKjLG0f3>bf}lZCc#nO}%p^^MW;+*;9|VXTis`yZZy7H zstT#sa_SEwdQ9U6yaY^Gv8q?r7L*?o$;G& z6Qh0+!d^MItOYQir3L??deSL5wW>rF zkl<*QL@)_+0hF(0?3lPvx=EJTvnlH}N465V#pSm&>`(DG0y-P1cid9oc$xj#Nq5Xy zli3vxS|O!x2;%kGGg`iVU*kPI3!u92#_dGhLOH`T#jR+xT$yH*`E;v&KG*_KiKfLd z5S>x5#jAeXMjdzrLQ*9(@lw>!5uSKCMfK@&Z3IYrZH#{K#p%8g%N zPxQd_AZF2{><;1uARu&KO3 zG#=9o9g1}qh)y5>aE+)PfiJcIQC8BFhKo2Kn&c5|2SrXeQ3Eb5bcu=pZH~PGJ&jfp zU+s^}xI9^+dilih#Hqd~qEMO~oHX%8O((r3N?&qBZ=;gAUg>5~OixymaoD{({Zv2q z!y22TAmfos!p$LGP0r@Zw$Qck&kt@7{-W~7+=b@?1vr(G;>TRy52&IWZ!nU_wUo8` zIcI1YoS}|jSl+sFO2rLl*yZLcAzlgOJJ@bZ( zTwQ*GPh^lPr1i3@cDSz2xfbn_9lV~ztg*Z8u=C0MINW3hn&9UHFe_3$tZU*|%YU_1 zYKv|7)xf@dVL|<;N7p}&_5a_GuD_qRk#bVjvwZNLNJF-IKW0eC8^}QgCDUz8w!VNP z;(sL|fbYpwD#T%Zh-+DKDQ|@B&O!9{r9`9AbTcI)=*iUSh1-fyZO`?9AbPkOw01fi zza2ZY#^nKO39KZjLbe_EGv6)?goR#|G8lT&>-*&w$wk33h746)lOizR19d{K>0pC_ z_T3^+q=Z$&ANV1z5fobVJW~~PyF`#Kt&-bs`Rq}{Q=s{IH2ts-_+lV)0q-W|r=<*3 zt}9;O15xO*>{aWCKM_2Sin_(I^}c%7(!CHo*fqC6F6ZZO-UJ@fIXAz7Xf>t{DU{}V z<$w`9i066(hCPZ=q%-VPvgk*@1mvY{d^k^mL}3J#Qqo7zS(Oj19I3`}&yMRdqMR|q@9wE9 z-Z=LnS^pot-YL4WXlvK5ifydewylb7+qPY?U9oN3S+R{3Ra{|374u~O=ld_di?i?M zyqm3!-rDG6_VGL~bv+?Rd@yg##Q7r|;FS&Iq1>s;E%@I`PB{9na%%s@)KFR6sVgoPaP7rVLy*o*cZ|kIo;Fky1f2 z0zbu=XxGju>v|qM{))9Ej6xz0h?0yNl+!9bnDpa(vNeCk2mS{%{KiRA3Eal5LCuiU z(FaQ{agWrJxKmhkUSDNrXpEB3R|Xsnv0VOxYMtDmX)ku}h^x*3)Xf&C5jXzOhCZhHT7 z|3C)@ANYwlyTlvtVxLu%KINuqW_<-BePZ zh=9A2-lvV5$&@EPTDddX=7fwHUCW`!87G~ilrtVWtO2}O#gHq$ms8vb4CcwQH24u_ z>D%}wGiJxPlpwA`tPZ4=0YobZ!6+ZLGey%6`acN;XitDYVCnl?<{5}Rn9nE{U{@o3 zuSv+H*rq$VLFP$DXo<9`@n*{`c-NWR^1R|XX0+@)?(;Qx+2>05 z3S1t!{rL@q1jhOeVhoohe}E?fnMh$T594>%AXiyris~>84QV22TWKVl>M#Y({s+d5 zQS_5-v{@FnD0=_Gup`F}pZuEvx}){Jy?jq`lqbgxw0uu_)Fa1D5amZjl#uoQT-H+z zrQotzGGz)!r8TQ~N@Z4ywM!2rm6b~oC6%>H6J@#8V&OzFJCAzG5_@L-#A%kB&0^^U zm^_w^OE_f=JCAD0NEZEqStaEZyLR0KoVyDmR{L}DTj9XgpWL@l}jh3rqwOCeBH8HF{P&Ut(Sb= z>LIRtUcm&VeBH`nR2IK<3YG0GhJ0StgnO2otxGRuHphCAbONO=$9mZWV%D9_ExUYP z-Gqm{hgFkmilDrQwM#MOd)CFOnQ)3g)>g^HOx9M>L~Yhq*#sNAcNyi};^BH0zhufL zN2h2CABVRCEk@&*YGJ5A65LS_1!h_^mFk|?gBoqlc%DL-!h;;Go!MO353W~bT00XV zg)mo;qRJlFgC4EkaGqkAEaiRP6Qv(AhOWW=sWVMJy`m8f78i#Xb(YnM3l05Zb& z+$4@rB;9QgA>=r0tVg}Z0Kw;( zq7>VK7#`V(_`~ZW?vl>sh-HnRXirezg~A3ZIma!LHwPvzTYv~2-G~;qjhxgL^`yJv zoOX0vAYl zcE((&9bp5toS;dhja`JxCZINd%zzgdI?QxS;dGJ*Uw7s=6}w}3PLK%O&ctOquv5*G z(KmB|2{rNBqfye{>APYZFl^Ze^_=P&5QGieV-^xqxrcW4rjqqx?#CKL!`YG3E4|Cb z*%7Q$zdK~_L}8c*wdAdX+X1>-D)`x(3>9_}$4-6lZ?PaaflMhAO5INccY1#W;!hb6 z|A7}QfIX;-J!1;H49L$2YlfbY4+a0M1*Q&B1-p!rA#P-ii$4@3j20vZW(sTMC|hYv z)2Jg;EBnk0h7OB?QbFNlg3I0zNJjh%N{|mxqd^Gm2^p*oRam3i3Q)aX599aq=P_yp zaYNz&EQEqgND9auqEFI@E2tjjn#bA+t~+SJ5{8>>0G1qeoSZpL7%9jP%p4+~q9IwB z>^r_Se0SOa6U+KWGJFwG0ULpo5z;`dcAn zX+6$*9=KKtw0K|)5<-1q3+hLCN*pkReTy5Agb<{>_5}MRzE%XwM|r{yQbfI0hOj6B za(W1W87LT`28}?Zl>jp@EFl6Eb{Qbv0AO@5_>xBIxH)r3WV=f+Id;ePe)+liqmJ$W zmS9`{33&Vl?lk;U_~^>@jGBOmCuLq?R5&_D?}#3_hqxrY_6K`~efuda6hsBfFTQIH zCPaD-7G#L_gc!7g{zMfNi2j5YBt@DiTkuO5Gbjuq4TU>xz!@T*sv&DYr(0`mt{I*y z$YDXIAZ`E(%mAz$l{;6MF9-=N7s3ys3&xIow>^e%zbR?}t8bfec3;brbLJ7hahkF1 zA0I5g88+XOIfMg=6JC%2#3iLuzy1Y=wTCld|2pIB@#Dq_I?DT)o7yB~KMf!R(S0oK5l zSE@|A@~H>ywn(XfT33pUr}C)-?cXC!&`5937~>`S34G|-~I1>BaDPa zg@2E^KqEtMl-=9|;2t;N9~t`mgRd9+5fCp*>choqYl=StVz1g_sH|QT(QZ{C^7gRT zN1A&!83U%d)<+a8F3aD*x}R`n4vrxLM0U+_r|PXp3{ac@?yEb;>i3M+x5TW$J|lwl zQ(jZ<*kBW(-|oOZ=Zn2EeW-%fuaA_!DR)0{^=~i=7KhKl9=ZCw1q(^-{?zxA4t|Nw zx5YO6hxd2C{|&YK>C00}tUA$%OfxL)LG#S!kvZ!7cX1YZ-JI8^4WCh=VG3aoPlPyS zfcm3HKfY9jGsyWaL$p+n_UMM{!Ua2Ui*c^F>)$huhxd2J#vYV(ErZaIU)aU;MhhD#O{oj&uSLIz-#U)3A%L>om zSiBMqWp*{Xx0tdnZjJYhzagkl`G6Dl^X>j)Yv9r8$;O}Q^Dv|eF-CGM=h}5r>)fVT zDWyPQ#s8ak>Cmf!rid-SJK*x_4)k}+J#LW7(OKop| z)56(31-+_K$CRtB-Pji<;v>euV)Be2_Hm~JnNFhqao4r9mAxhY_j0zaDD3a$<)t=) z1Y(7L-v_&*D{)m9%kHB`o3T5Rh9_6|=C(LD&_nG0g*PVt#%;B`d%uO#C(=Am-L|%| z26WkAV|e80c>C17eZDE+{(DIPwt(Tvj$BT zqof#01nx=TGG411cUZ;thy+JIHe&T;J(Bw12X%lVT_MKGL07BK^0X4VCS6Bb&~$#l z{*O{?DxDMa)q%4_lgT$7mD7bM^aZ_5oa>fKR0~8qn1#$UQQTa`du7$Ce;z(t0Xg%x zbwAsOx@VS(EU|$N8&iNbd35@Xnk3_>!iFrAha+p)XLgAc70-uvbqrrzqfgww zwzAr&S6wrNBs|TxiK0l*7_E!-Zjf6_gK;(!(C0vn8AT$Aw|7y9dZvn$qFZ-!2DrDC zHZ49+txIUMQDS)3U|8+dVaSB6nr)v1d)B26Z?us0;a!QIQ3J2S$8Q|&iun$2MXmGe z7V5m#&83ZEKM9so$^px+fe^BLx=1hycd9XgIm6utOnj`qMwQ_8-?`sM8>we{w|7YS zc(*m(?BEs6*>&sP?sYGN_kcRav$K}NB?er(5247`iYMi_|Hwe{hU&~Fv? zB7%%IPSO^v987(sAiug|$X?UmFYn^xeO_!NPLD{l!U+W^Ol!Ca z@jUFqQ7qZ+NZg0XPt>bJ)r6tce==4jU{>)n>{>v5tD`yK`^cOT#L@To(aZlliV5PQ#yYb-g%^yAiWyb1=pWPbbK-j&iT1$<=aSeDDK4lx* zTOQctF}OWNL6eu^;A^qeVOA(^m;KhLm=WGQZV#*FkPE|7jV&l*?b_`kU?rM&x*8_{ z&vvE?5KEsXgUG8bt4;KMWdkCOn&lZ|#6;9#l(naELy43cEothhgI)tU9K}=;fdt|9 zVREtUM@tBbuKi8YaCL3_6N(J|N~2piNs-oZ^!#h#1{rC4o{3&+!}2bP%5Y--NVS6O zzm|pZ1YAj4ao)w?h&h8RRnj~>G zpuySVX4P|`Z`MYHT20o+)#}5<)8QHQi?#Oe`%e7Srz2=RuJeV0+B&ur-ACa`cmY+e!RVy)6vNq?x(5h2(iw-pm#?KD1{*rXP7LcobPS-_A-nW7z|e^BMD6x8CI(Sya1Pw1Ra zOIK?R&=wC&CS~2A$%)B@uC#N1Ak5N6HZE`Xl6Yn75k@SbxgCafN9W|l=$J%4WJ*3wh%6#}~3lB8W)=GHEqJ)u;ke{eAZ_5DSTS640{=Gz?J ze_LD{sX3E{jfaMkK%k2jz&_7Yge)LG0{3a<`!r@SoT-L&ylOuQfEOx6M{Mlc6Jd;< zEb@5$Jk#1*zy!&Rt3qw_z}Win_fm%1q5m-3)GC%gYFmAL;`4bM&dO?eIf{v|*S!#y z=EOJDjKD=G{ zG`wBYXNXI9<5u;eLcOxRSk%q3OuKkLi^a}P-%^}aT5Tw)tu5PGUd4L1wVhA}5Q$ZE z*lMqbEj&HHtq8U9Ji#K>CDOhM0CoB;Qxl~8bHBH@H{+FyY|}fAte-(jmUee_^t5({ zb(qZ1=CfKtX}_d=Y>X)P^tDvg^qBW{_AToFR#}kQ@g!;8xjMbPb#F7`!p(~5E+-gC zGX8~+>EP+TGL5AJ#8Sj&5UqMyN#`vc*^Db6sbzD94IT+%7}-nP4`QI$OG^%7DBVk2 z3}P_fOKS>ZIND3Qb&?&3Ii_Fk#_C5l&sGqMziqO}@nLI9&htQ)p_>p5A2P~#asQ=` zk4qt1FP#l(zaOGJ17?fASu)qP5BdN(NRu_ktwpfYssz@iQ}0){L0BKX zRW`(zc{T0li1+0aDW z%ZBb5f9?1E{eRsP)9mzKTQlTM8fvoUJKWF*!!~yFR)O|rfiy6Pu?Vz9xJ6ul5o$Iz z!bF!FY#McH+~^mMr9U)0nnjB5}i6K8gcz8v96p;?D7Lw+IMx zMNJjc$98Hw)7SsNKjHH0f=RZ&W+^%%bi79jGIqJd?cmLGeUHP{d`k3)RfcPuajP-h z(K(_ilSZ%u(%Oe~XKU@>8pEbjgazDfOJL#&um^Mk4@Qab-cPv5q|PYn6`(SHWT!bC znO2;*w==QRZ#~(dCE0mpEVOG2EqLIKMwO0qRhj#Alf^^AZ_H0dxt|nICr#da`-`_u zVLMts0Ez+DH}B{&19vNEqxfZ6Vbu9BoGC*ytsNpQotxNCn?AfsAXF(;(!}L0U$80N zgopcr{Kx@Z)Tq)vZBa=0wUzD(W3!C@G+ZUjmIwq_1l|T4ThE`%jyfG@PqUYKb&UB! zyCN(LgL{#r;hdDQIB>1*EV1eg-3+f#8G{Rf26j307ItkftiQ~T%uc1@`Iqaw zjZ4@WAem}a(Imvx3i+A%u*;1^H3~Rz+oh9o&I%4nZ1}pt zFJ~<-oB;y%8@O#od>S~Be*{>vDgF^9b9^zJiQY4p31IV+rjI8@e-Xg~}vi;VGPjlycrq!yms@2KNKQFgce^eB`X&gXo>f4Z-X)Uz z7~ENm*%**)My?(fD2GC!@~!HF)n}N%)xJCaJQcMm}=<|MF`Zlh=2j5fCUx%2-vP0m}%{rj=_a zX!N~`Js01gRMCCo?2gKy{v@ze#D}1|1+@adFlE;oXPLOMTmxk`0MRF;yB}5;e3XIT zzt{7pb7MOVc>gEN`n9rRmXZFf z!HNNq@jOLvUSc+%f z@9n+;_?Z`%VF`G`v2SmCbh3RdkvlDoJ2T6fF+l~DHG#O$oi{!Qyy ze^b?+>rhHTw=9cRG&)wds|q(+u!&yK?p~?J{=25s)A8@g^{!~M;EG$RMBMmRM}*3p zkM^0R-3iyJM*pWslberKfL#WUtzGtct$LRvCz^*tnkk zw>68AE_p~ZG4j0iw*hsdVj*Q%xtg(Z>48Q<;+4##g)zpX24tcPE0K&E+-QROv9DgGqF+tH`_ z^qELQN8g?RnI8l$dqm^cs4TNa+|W2GwuHC3Qw44Jfn=|HzxiPyu?gznpoh()LM`I1R=h-86psefkI_t0lw2mj8?goeCT^ncr zpn3Ps>mI}XBaGMz@)Y6Sfhq1FqP%e!KmJaAAOPACC8c6l_TRIAPuw{iF8xUI_G{mt z{E$}*?}_hgs(hdSnmfVdXUrS3HO*l>BoZac7-A>uArByiD`iVnTs<_RxiQI{XP+US zlj8_M_62BdhEiTf2y!xQKrC-5#Vln&tdvV zgPknfQFisve#U>mvAS>tjP(7axxhR0&jmPK{|?KU_*?5zwZKLE4#EFT#*J;IWczu2 zkvfiPfLA6m>8w2A2$maSZp-Tlu~BiX4|P!#ByEz9FlWR6g0V|caH9g4oibu&iqP%> z`*TfebvP2~PAeo-0@dYDUI^zW)WbpjZ?sl;&<;Sd^dtf`!TJ&5L|8Rt*rua`u5xAr3%%vBh-w)hj&A>ND;LyLi zlaNY&4fSqyNh-?w>1)(*~=xyD4I=4MuM;MnbdLBk` z?@9w)HgIjzB9~ac6RmSqFRy*%%x*hqBnz@ev;YKV%A|ziJ-^# zh#oiw*{Nh-Vm`k0%u<}(JW1P{J={+gzV*Y%94p|r@;!h;gz|hmV^OtsTC6T}@Ry<$Xn%s>Aeqtw zPVk4Y1LATR=FP*d6Z%pjN6nezGlXU5+Xj{c9kZd@xL#=?cdzKRiUAW|m{6T4JzYEG z6t`|>Y7g{iXH#-cxIey0v8Qw!9qibX2#7#VHUP>ksb#bzO33z{(8 z+es*X7yZLym`57B>Wr^^#BM4=Nx1uoNVKiys#RPBTAiesoVSucoz1N6q|`~Z^ge~V z?Xk(s0YY_KUm(0W0DtWQRev|=LNY}&5UL+G;rcx;|pH?W=kEkqr^NMh)vMJQ1tg2vpKIXtok8!JaXQ? zieMF)&pJX&w*+x(In@B^w2p<3xk*Qgx>`YBVl0{za!2hkCY=s;OdUwO%NuFtnQ_+tVO>o$d z73Q7VOyvgmXpq)Z6t=5j63TLKd$%CO4@nYRL3f#50(#H-aCR{_EMnJos8fvoLantH zvp|<-_<2T{fiakWO99TQn`Y>kHY0|V@f+Uzjy>R=i9;+eX49R+ey#t)^cnxpMOX_* z<`5E4aU^HUe>9ZMc=}s2JUXoBOYkXT+ODzZdhwI=)C5xX4+Ewi%fd-bI(45J;+nyrrCYPa8wGXVUpu<_4+}j z&Mi;rR`$dV8Fa9+usdFqg$E`4J8^~qKC%N0f5ks1j8_-|-FCi(2Z5SZ}1cHJWtp?PEUiIi9R z8J@j~Z32-(nqq#fsEn@|!xPTE)s<_+oXqldhdmotQDHQ+S@z*LJ`r$W#<$|+s?w%> zq{_T#HErfQu&1^(InfY?TRi_nh^d?-ZZ$L%=S)3JSm9jdK0N`zbdZY$n0|~XP$cHW zUN{|~Jl_pzZT-0NW+hL?Z2?DFPoy6eDH{*7OM(_R0s9pb)u1CMaU&N5Kcei^nE@w+ z^?aS2F?&IpT?&uQ_y})O*SM8`%9xXzFy#-uS;ln3UG5#{9;RfgcM+>&rtfaDup7+e zE>YW2=Obzsmdw3^Ljk8>+@DRTq?etee7pGfI1uFg%P(~ziuIQxWJ!#SQ>2YkG_Dr0 z1rXMXzPtd!cbLBe!+$4t->O5AKD3-U%NxA7C}HHwJD1kPt~^B$Csc(W$0AC#z@R2XVlRBj>3t4m#wqi9fTIZW65D>BiPLEJFsyy} z^t#U~qFxMI$AM}JU651Mt`Ma&c0 zTw@opa!Wca%hTMvB&%|8i5#g8l@%oa@ngSk5@fBgE0;U)*cE_fOXhZhq%X?m{PgTkT4zdn3`-!c zGg1zH+W}MuxJOK`Q#v;`QnbF6&hb$FFr-{0{5rEU>J%0>TSP~wdbI3%<31~Yu>nwL zB=y3_8FQ1Y&H_2o7DVFA;H#lGW`Mn8N)&y*&b1FEsIoQQJRT>Iwc zjJHGXaLhMM@TSuku|s!xm=M+dG_~*W(FYn8Lh)U&t4E^>H_$%k$~efK6OpmNCCtcC z-2GwFap`BGkn0q+B|5KZ)s>5aOqKx7toQcKWrWPj0OjOkDBXV8v4pO@1eiL7n;SOC z1=;bO+kOI0Ll}m!kbuC5-Ha$?!uB>!DT=9+^`}D0tBMdw6@QxL(j0p~u=(a$+&vNe zfp}z~69@oZmsS}Lb@Il|tFGkOFXy0E@hnxW<5sO>u~$(0H!R}Oy+^y4Nb*NJt{v>Y zp&y%Ejv2ostGK%P}zELPZWGVdQTim|5H z7$PhCfp?R)21ATsupn_S$X5^}DoNrsD=wmCIE8sl=|TeoJ+r?^oKLj0iPS5f8DTBA z=rN1Z%6^fAt)~_f;@0O}={7#D!d;TKUoh?)rp@)*Jmm-c<_K2@CP6rOGBANWVn_CS zzVt-+lo_YAO^rlZgM_syVP?3_D^0h`TlO}Upy^!X+`8ik_J!&Vg<|@tUvL2zfXVo- zzD4yzXGdaq=k{crf8$GWLQAign1~EZdX!b3Eh)150NBT^fPjVi4rT06gV_!M3lCMK zWDm7!{lrR&0@8D(%_5jX0fYlwL)judhAQOE-IahYyiv$Kg1;2gy zfAFSG-aibD)8Qj7_&KDmh2npJlstrtztDFnc~A+H zQ+B<1aDJyEJ*Du&_5D)S&dM3RH51IHDQ^~EoKQ$)Y*WT+h%qPWR>f+3J(OEzidmmC zgZ0%-YYLoXWi;m@qNDQ)syX~;j_H`Zvc+WkMB#$fFFU*UIMw;a!jtf+H+y_F4*7=e z-IC~12q43TP1mFUGeJ?LOh~qIsM4sv8olk3$xWPq$uz-xlZQ|7J00d2sR(hGiHk|`^^e@agtQE|wdYp`Sx+)0*6wjXcwq}13`w95cD zAi-JMm#jGbRI@S$D)e4{PGlo1IA8e1fSOu~t6S3em7|KF(Ajtk8Jm;9%-a6~@ZEQh zqqjC8bs_Fmk@R_md}SQm$(^Z(nkZsHo>Z69L^u#8;5YoI58Rtiii|_i*hNVe#-vAq zS@2S1dcI7l8#Y&0<>)HBFUHN>OGbhSr{In|Or8(~(_P~czVn+rN98(=|FnoF1bc7O z#o>k8N=EH{;{m2ZTW!M1IYrzgVwGI)rGQ@}NZ|TQI{DkD&WZs8*w}!7W<*wF1b@S$ z?nk4cL;kkA?207PuWf{vwA)~XKtXXjL7MsAP(6{kG*(yE7|TDdQkx9>pLy!TStxjE(BaDk;fl=mx0n9;+ZMiE zS8@6}H9Cv_oY+==rk#RkFFx50T>)xc=s}~Lv|zrL9=*}jS`D{aiaVVX8Zb8%fZ zh&NUIrap1x!y%{2$W~QFsbTM%9TV$BYOM z$RjA-J)-LCK>N|pLYat{#Q`RkUygcCCuI5xWR6Ul+fRX3U1De|=7R5I1%_sNhCGKk zTidkv;?6h(6GtfgY?fDhk1q0DNhaKR1gu6)fI^kRBJOCOPA;S*Q$iQ~1biC4bK=jf zT`w8t!uxM}>It)_o!d$%W_;qu8@sd!M{4*1*-dWn$b(b&zR3M5Mlx?$m{xyMnPt>O z;w!>gS!5}){VeW|IFsckUFpmm2#?a()yav1U)L&!(O1kFk4*mt(QBqH^R1=GJrD*w z?ggEZc<$O->}6<`?WSF~J5b~oE|TwS{tO^|tg zd;9=*Dn2!j?13yR*iabRQS&DH^rfZ{RxHgzQy|yqQDtR&{Gsy4iPda|`7mgbgofdj zR4*WR!A=y%^>r_q(m64%V0e?Wi|I2vK;Ru_%La|4MTFLFfDJ+$ANv_guz(>P`KQO$ z>PF}kYs}F8UY{9!<9(*;(z@Ux`oG+qI0RV zb(m~FxSG0*5nWIH?i^v|Mj;$iZege(ngj0bmi81h>+;f;$FfSFaj`#s%DSdxSfg;@ z{43GDJZ)h=sF1wZ2ET;Ap9NbOTZN}4=Gv}&SVMA{H47Iy!2bw_`yh#p=r5xBfPt$} z_)S&@*v0_Juv8JX6{|4&YxLd;IT&d3mrisKmna3q_WhOP{kB%k@Jgfzq~YaRy&DvI z1<6b%WI9^>gKX-QmtzCTaX~Q$tN0=f9*=(8Wn)_pOthCl!ROtI#{hIpXpN? zj+TkgW+M${qCe)bW{lD<|GsWdTsWnblhCYVFOQa_F|FM|5{#Bs!Fg9#%DsJTl%_3` z##O63&SzK@os^DmE?N>@t@^nu8nUlPR(c62{|J+vtyNMN&bQDD9c9zBR3-c8@-R@M zWb={{tOVCRsfl?~w8IhzQ$R)GQ8afoNW6heR*2U`{PVH^n!^%(dB7qAS=SiWts;?( zy8zxvn7r#~^k-;E%?6mIfv|rFF>MJ;MG5`gEYG_1Xn};1G%^FVf7?&O_mYuKIq;^BRUK<$<*gp&!z`M!-rMG%jwQLIij7$^OO~1c zRhHCEYa@fDv`-{x^fyv<*QdJjC~ouT?Qp~r_W#sc7@alu>2=6mfp zg@`pEA3z=k;i6)Nsezl!f>m5t*{39w@eA{$vH}v47TsM7sG6u-ajP_jS;4MWH9@N; zT|omG=B3nmb4}PdcM>cpCl&o}3^q03WDjB1Am`MO)Q}Vx?7)f3Xfd>s_m-};!9(?8 zo1j+$7|oAEnwGWR{0P&|EUyCB?F}+;(2~oWijrhc=gnBZi7ZQBylR}r&rv*8DPkO4 z090D6p2VQO>wR>1vlU+q_{K?{enE?O1aJLNGmWEtheha!!QWKy%g!@B&1u4LFOJZ^Xoy@vMm?)AD3;vkAs0?ijJe|6Y@Fb93T0$Q;Ni}W z5^_pAH&~?U;#N(v3%poE5bX0{9H&H~FWUAC36Yo3ukjbBaY2k&XGyZ)56x(=+VSUo zKf=m(0RAn27kBQ3K8rVJmw1OZOd1bb%uXY3$J^U6T;JCMV) zP2185Q!NpGN50@kNpmIRFb6jvTNYbRMBgmoDjVSq4DcPh=)4-~qh@;RBLu zsDBUj34eP-k{0{T))Z4CIIn_RecG8516P$GJKL2ic56i2*vSad7dbEyFO#~?_V1e1 zyJ5*E)b^?I_>b!BX>N*yC5qVn{_iYO@D>McY1IhGn)0~shusz2!i`N%hQ%_b0*Akc z>i=Z=m~-v|yh~X44x8Pa8jY1g`VxODM!;#^k)p z^j#z75t7|Ur5yyLOi5laT2h0d?k_8qdVMG9)zB1*A0PpOg-c9hl4>otlDk>&ZglFx znktl%u7cq`*usJzi;`Y65{^_jy{RK-Gj)&Q1|nC`R4Cga4!<-_kTv&C-(>ERHH&ZO zjrYia#aurRtiYdYy^oNHU+&-1l;o=WGbc!v5c`~Qf~Z)_18DhR6#1=9t{GWRFe_1} z4OfwIpGVv4XzhE^kOl5Cjf#1hJ`DURwF>&eYS;sBb&$xIHFCzQlN5I}Z87zin5k^v~@M!4o~F5|(A zX;wD8SjSV*#u3ePa7@4A+BV_pY~uTs@_HHT`<>G@R}0USnM)K(<(KE9?491nX`aWo zKU=O>0_$_1uOAWLH1@OM$iK370ZGmhqpz&9lTRtJ$Yy@j2V^)zQ%}vY{o_y7vHj{M zSR10$qP8$cR`ABWP=Htz3!^|7R!%lMQxI@|W*+?qWLv0Glstk#!td~gq&UBon!Wf%vIL@yi>A-Ir9EN6}Fj_sm)kloHSlGTmv z!UQxkKQyl!M!PjzpeZ5}1BM;}2&ftuRE8XLVj3btNMPd!UeiI=@0ml^A6UZu9?*wV z*ewX=+N1V@f24jwRDlba8fteI2J!(^uKBiVcOU8bCYn^G=(yuVAOnq#gl=Oz_h`M4 zA8E#!-w2TDo}dH1Mz3!=Kua$r;IWq!5F5d8UmWt&Y+ch!1jxf*zsrxr#y@fWEBJF) zIh+ixZPGvS9ONckw;i{ zB7BvDG=}94F1tqkUd5Ph{kGrr*(H5vU}w^FNt~fsC<4X6P^=tnUSpxW6!8&kSuwD`^3_F<@R#euTh}m;^1j$$5-jW@FiX*eSA|I&X3Y0G z{_Bc(@TYQ#Km02(+JiH9M}jPRV{kSLR;(;=ggY$zE>oJ1rjD%)>*5AdPC0Xsu5mCw z;}dZXii(Q;1fwo(UXLL^=^56;tG<781>sM!Y+{Tiu|aK@w&` z7-i%tT0J|Vd4G}0s1?hs>ZdXj@}mpefyDOmcD8j7PzDCJnb%F#PM*qi@X`VtHU`Y! zjKNar7h8oqfX29fHzyq>TF07?v7_u4vp#_IG*Qu+xE5!*@(kNSFt7rO;Y)+cSoJ2I zHdmuD$(=i60~fl^@B`Bzs76;YT49^n?LquD&U4`c&YhS3C7B-EBUdPnTu+b}mk*A7 zocw66sJ_@BE!hG?Zt$Y5)HIxMjhoTWNGd~d@b8_!Pd+2s;l`?t9xtYewI=sEPw~q( z>E=11OcecCZkx!$Vy;J5wEi~1EtGQMddp*^jJvb8tmOEijR5SLVpSSWv+eUL2g-JQ zN>=exS2_xk5Bd@b$^~2&_fJ91kkB$3qKHx;vIm5!18A@?QcBBfL7#eWBZdI!j_4Lr zof9W(vyA7p$KDW|i#=k|-Fo1&*GiRFuzT%hvp0V3(*4b|em{1zr8WMFvxf8rq>+^S zhE(lTVZ9_D{kz7`vp2Qug1i-OEgFfZPD-Z2?H?-DWbu1fs?Nd23fJ&~n710pfwxXV zvOm1%-gD(#XZ;d)9_EXh0bGabfK34Z!! zId7tpVSxPw{7~v$olNVf+z$!)7xkPYK%_XoDa_J!axKx9l9^0JC^)rT#%=0}t0w32 zYu-IQ-d#~A6yk0UBB0^S)a)h5ADGS5lV^}EZ0G$Et72*y@H+m;$AV_e_dgS!o*b8oZ5%8-czbYXIh=?}3o5t#9@rsC} z_qUfX<~9#1`CSUkI3VC^0T_kif42S-pjWGnuA>FJY zcyk1Y{@BABO2`gS+zJP`r%^O7-qL0&Sfmx|uxv_&+Xz1ck~ec4sy!*>>g~w=$hb7+ z>0)`IFY7M*$>^OasW9$6y5O80$zdi!_u&Ha^G1le*GKre=uLe z*PUFkFB?)S_J7-+zdU5$TuqJbjG0Uw?d;4=-K-rQnEwCBqW>V_s^)Iw|D!}wmQ@@S zM)|;^Ep7d}cj6U1{+%0yV#tEVMCW&FjlWo4bLgKj;l_}{s?Rx(ObYnQ$5-x-BZhI|O6fqZ3%)a;H} zzke{BZ@|S7(f|%8=IIo9xz7+LF(#yqEFBxdkGy}dKzvJP8@?)6+8MXdBFTh#_m3_Q z+>;@}ELbU%TK?V0t}SJ$ML{6?0R5k3njMfRQTd{=DgP_S|6j#b|Fca0ca?f|Vg2w{ z7V?wK>ltd%sS3ykSvaH6@QGOZOe5&1SU|A5!Zy?tUq)lP@#LgKJ*B!uml9}6Sy&~o zJ*h$(atyR=;h%)kvW1V*Q|gys0v$5_0YrXm-PNroab^_P{yZO7e=j~~vbh|8_q1PG z3Oz3j`F<}T{LJ;V{Yvx=1JDg4!UgC8eKC48fmhTW`oJq%ul! x>S5%rTOWMIjz zAmk(6lPP3i(QXRdhZ?XP!%qj8PxIso8CbZ>g!z^l+^=!X1^1x=+@X2$h5RhoC5kBk zIM4;VbcF0tglpm$SU&lpstx>QbQc__AlZr@N^#J9?9AH|fd+uz*gYek^K0*vuD0ivx`+HKQxY0BaAelhEcoQs>x>8UkqtHO`x^qLj1&CDZB1o-lORnw+0xH^R+%WQM5B;uJ9U^%1pvvhNekJT>!=5sJ z&Wb=X35h^5Zjb21_F#kG)20=isCeA5!ypoo7?ScX`=Jv3eRO!!##EXb&F19txP#?UJ#f@t>yn$eKBY3V2yP>2!?)1yqD zbp3K@ts|yu3MLxhNQ{mXK_<^m*E4+R)7@OIAV%Fl{`vy_($mM>NQrF@TRwx%JYue) zuxnh1)H7@(jP%-U5xk2|!!=r&H6P&Qb&xg1v8GeaSS9g#s&QX4`yT#cI|oCl-aFeR+!Glju_Ic zOrLGNqYXVZ#$%~!n-PuJz-41r?cv-rH9=hZl1=(6`b!pVH}jVCRI9u$FFFgnZJa6a zM_&Ec$~1cc)c6iT?YJXS6h7;!^Qht1(Zl?-Q+r0Iy*B}J4dSM$bCre)GxxG5IfZ9I%5RBD8qy7I{t8k zpHY(K8T2dv_z!uXi}oeK$F_N!GwAf#py&?6uX5wZ*hc3-vP4ASAk760kCp82q=bf5 zkrSEkb_aO?)cG!aFPxYdfzSo^RhHESn52Nb(7j^3=~SYQn#|-Z=Elq#HsPQ7}Sn-bja7~=iAqj?+s^uip8e&HS+@q0%Uy?2Hdg$`z%H+$@9TJ}3I zw(#H}CuDXemed8Asba(ADC4+Cw^rg$Hv~IvPec=6J2dD83thvJNMgyMYc7wKCu-+O zLe+SmhtHXBdM5lFxGNx1Y`*g@$8*pq9pNswh3F91ktc-N3sV!|MZ|qa$|AY}a&^B?*V9QkE2~)^4O6plMs_W@#ZVeF?w!9myKyB>T{82(@ z-Bl8H$Vx*qvpioOxH4r#Eg!@PjVq;yTcNE@NRfbAL`{k{n-?_el&6a1GH%3P;*1tT z+i`eBrCSwjRt;V7wEoEZ+nqt!bt=UBS7}OqK%bTyLoDT>B_lXf=v9SsAg%-|enJo6 z)ksd)RSvCjPi}%m32{huZX!_7Mc@IWA?5=*ac&7MB*lp)g4tAUO(cK4Gz4q#72w4} zuYI-5z^1z39;fcNQD=7Ed;(HXpeQ-E^+VP zf~M}HwMsO<_rCw_dz2PRC$VrW+QrEH1IR!s+E;RC1Iv(jtHIliE_f{cARYRekyA+A zQ_M3Fd+IE14&FK~q6KRW(hmfEKf4wL1$``$>vK{ z%zj%P#`3JpLx?KGD7OzM=!Y1X4{FLX^$u~9i(80$Zo>0fks5l zZAe63bgZuVS}tWbsRE({5whcsk%q8V$?_$<^Lm$0m+xD=yC#OUV^PLTd)tAui?CkC4y`B`rrGDP|MR!jk=!hlyS9Kkv4VxZ9WZ=@c#%*jF z!J?{sU?25KMbpp}&9$K^XhgHuR4w+4@$!u}$2_v%aYCKu<45$up!I^8GlSg%SwBu7 zzt9@?!$sekpS>tl`Vunbxgek$qnCp0eu`kh9};a?qbEKekymM&3sZ5?-hIBZ)?QB! z5(|a^I3cPq^}9s+kaHh1byw2*L0ep)e-av(=RF=1fIg^46_rA>1ZQtS1q&dk(!CHpyLXqMYgUs-$l++9HqwIjsb z@jeA5v?645O;P1Ju_bwB$BN~1`D16ai15JOF&GQN;k5>g%T;^!v1E!o(EcV7a_n^N zEbVPyJGZ&o0m*jw?UiWsdq`%zKGIIJ+|FXX(}s1uhv-s9aM@#7`2jA8T_d*l0I}=_ zJ*qcE4Og_EiDU!u%+_ZOE*kSfVoS7Ed-F{14>fvpFBZWL!vvd7Y!6;QWI}8cAaTyv z3MYULKW=&Vv8`L;TWf#(7HMo}e(7d2QbhPAZU6~XRmVp3y5RS|_IK3>W*)9aTOl7g zrwhU?`*uzXa*W=_IelHTavB>ePj7v*EwG(+pu1lLKaNtZ~=p52dga=Rnu0y=AWg-y{c^x8nc7nk}#Wx>?bdVQ%qvl3%5WB zV^5u-_=j|(1LX5rK|Ftb;d2T1)FBZqQ3`yD#=s`6Xxq)kf@pTm9Vm zW3Oe$FrJRS))l;|p8d_pN9n=AD*1Lm%wx^E5N5pewyYwj2Bi($m`I+EVZ+Ne`bL;d z_0TXD*6JhlhAkQMy|SE&j5!po zvoGUC-R8P^Z6)U@Iv}aMMx`_pcmPP5n**Bm$*~}_7}w^c^~<{UyU}p|v~)A@hwi@f z^|TA&$i}Lo{yluaT5ky9kwiiPU;XOW{o+3{`;Xiy9tf}WIt&k~QpKemrM5!oR{?=J zqcNMxcUqH-WvM($@}vpq2XBiw-uje7p_O z!u+bHAKEkY*#_D5^OhH76TGq0yXBbG8yQl?6+W-bzLKOup;E>1a<+hcDbl08ihF%u z>_fR>!GOE@YT}_%x&5DeNll4^%voeAo)$CTGQ@k}x&yZkzKZvRg6dWo{CAxh# zLQ{2rr8sRqMlFM*I4yn@u>f$;YpQdUX4Qd_yZ#9r8GQpu_wTO$^P^ms+_@EcWXPCX zqQD>}XSzZ*6v!R^8?wUMhA4D^+;Jn&KB1W)VGYmh(j@#F9&P_GUJ5jW5wha=YzF6_ z7M13VxO!v1;K(m2`G^>aHKGC|*nkkBBP>2Pi&_;6$D zVpb4Ki*7ae-b*U@wZFcET?C!u2q0laoVDtcZYLSn=aR=e4q}dK%8_4Mq<5a{FmD7=*BF_F$r8wn#?4wKesS$ri)}_D#C%@eEAZ(`6j7l|T+0LO+ z1`260DPHiG|H)*FOO@GYGX2S3hgSU140ATveG4A`##s}O?yteRtDi;gWzsDEPSpW% zdPIKujaxg<2K52NPIj7&qj|syFyRmfX!&|5i)Yj--VHAH$e8SiZ>(<@oSLv#kNQWJ zzW#ZhVQ~tIm?4@S?QZ&!P~CwgnRbuN#;&Rv^@WyiQ(dc&%rL6AU-u6{{-rUC;<2 za{}>;Td&p|{>`Hd-IWf%<~dww+0wN+!^L^SH8(FwOMw07U$FpW>F1;olrLY5dH>eI zqxrve@cxU*`oFqE0*VTsJv{WRWrvecY7R<5`RhQIojw5Hjfn|4i_4EVCs$n!2Ab*s z4MzNDk>>y`RE0N?59&QzigV+wsfqQ~mzVc9$eF>OZ~?R$ut1=2kP09gRiu{d6ZzFAv`kzwvfYCb zgg{NBDSTBY_r=$b%A(Ma;P1jhv>lDo62O6tcA}%4Z$}d&N9%k)#OyLxnCq=gD!+h1 zz`=XN;n>6P%TX6<_2b&WPim}t@7G+d@!$LPk>Nl-a>s1sGo2gI`ze)?K^89dki{@Q znG+Y90`xqAW=dj8%xX-aO8jt+#FS7pL}Ekn0c{Z@V)+j21Op$pDnb_uxRwS6xtqL4 z3XrzQ!x51^3-21lIhcGb>>thoVZzj-_M|7B{IH8jTdi)p;a@MCuwo@M?0C+XO~|W6 zWCN#SG-CNtu_(}K8f8@kszDF&B|%+NAUUcAbiR0;AhDARNgWD`TvAX}CQKH;KrW7N zyM1#zv&&C3D!z+2KS$C7J9V>EegXZbLkh1A&%^a;ir;@V$6t@Zf6Vb;$KXFK^adV~ zh#ruN9*~S4kPaS@grBVQ(_Kjm5erE>3!>e-`-55Hl8`k!Z*v9vYONDlYI(jICfWtx z6-^8}dj1bNJ-b%+f0*fGnSc81;$NaGj$&`qHRQPAMD%C_GY^di*6Fl`3nV>H3{3ZZTP`hYlrSyA8TzBv8W2mUHVAme84 z@-L_V|3Ndx6+a`P09f*zj>r&JO`IDDy_KGvy!cR>ntm81qa4oU;2fn`C9sy9QJVY) z?2URa8!|c`*)pL$mggyp&r?uvdkf^;RC6RH#-4zWcMOs}jgpmF-fS_yuTJ@gd48}v zWT%E_Xb+ZG3=ss#lPOBb6mfXBL2tGPX`o;#6#S~rdkV?+Ef`9~a~#QX2G5%F!V!GP zU7DQwg8V9WL=E(dIqR|Ws=9ZA4*k8pU1HLAXLpyqoHh7E_ay0D&)^D9u@C;U(0taB zh2$-?R56L&2&sOqBVr`}{jbkcI7t5ad^to?F5(vq=aR}aPksRzr4^ic%mIM?X!t~X=itw_XCDH zFeQUV4THB2)1?gi@2K-~{j#Wgh4{yF#U(W|r9Ul0{Z|a*uK`ou*vI>SjDaMp+bhj0 zqkm__L$`4jqhz`z{qK^L3FzU!d z>{r>>xeeF#^&_66jk}GUkJtALsvkJ}7!%Z9RXgE^fs5$cUFcYm|I=^}$)oNojdRl%o_=b%PlmeMSrju@MaSQcx`T@W6vGgg;hFZR-h> z%(yB>?v?wUC0h05<>?EJN;nA*yV5*h4A$j0*vK6dn-Nmmh75DIn&?B{GmMWRoyABH zhKzr`qcxb^jp06RLGUSLaz0UqyCua27?X>tm$iE>rE4|fubJ)cdX(?;NvK32sg9Vq z4Oz2~)`)l4A`mQOKw3$ZmKxwHu&+>A$C1&|2-E(EUNS~1M;o@%xVQ7i(82aX30m&; zYo}cs=>6124}uB=53qP=(-Gnn?xg0jX>dPg=P*vUjSU|5|8eojB&%MgzVQ+<7_;^3 zQn9D+^Wn+r-eB~6)=VQ_XYH?nBJu-V85+`uIZT@(5z^5(2A@ZK&w+Ze**dcmCY6!H zGGP~v^4vm&KZxc}_@hi#-BroB&$23G`c14Dzmnd;{_%A{EF)slr=S1)%YPctw`lkfD^*mzJfQGi&2`D%`O@ae@T!gY!{tAShMC2KJ zdJ|Y;I%~R-^ALfjGn6N~rkN^n7_39yhYI19!aQc?I{Shvzree}VfS|DRnNDpuJo6) zt1Ev{<>;T3z@&&;HQ(ypw1_LUEEQ*sAs!4Fj3aelA)Fzt(Yan1SCm_2!(d-bA`J{< z)>AEw#wz3XaC3%n-V|M}rA9j=YqQl~4dLc&a&Y4sDZ1MKXo|Gd+gYBNotU&|n?sEg z(0ewxHJ;OpqO{aCVa1m!3{eZavcS%&z+<#px4})@wBb40a)%jUN)3>73hW+9w&0!Ola4zBhc^OL#>15D490$n#MfpGkJK)j7Y zAh#=3UGBk`aaUZ$J!^z*(kdXR8-mad%jbKWTtKeZrG&vTs=>lFTh81y9pSPQj*?Yd zAQi`3KsWA(xTr|HkjdpkD8YC=2KsAuuw@(UE!emY`UE_^?Pl@SqF9Q<6M)yp1IPH2 zgW^(gn&ZOYl(F>EORM!BS-TOMgMe?rL@Ek|Y|i?eq(F__N@UYj5BEuKPrs&T*=f%w z*4%ByIH+z2)neELCN^KW633DW%e<5q0gw^}=3SFN5>^@4hN9ZV1G^fNbg8C{X*Jq2 ze=9uBJ$No`ajfeV!ZY_Nx0uYFU{J_oT&hu~O=sa|P0W^W5@RVt%+RfVJY6iMXJ%}a zw7h5=YO**xy*yZjof*H!qncJHc3A=r0M(8PSFD{DBugcw8@CDU(n&P4=W1F4hwZYD_2j4B+}Z6DWWyz_9~z?#yht; zyw-X^#%>RXr@uV)WvYIPja0hs40F47Qn7QiHhPXXuTFV5>7cQ409}1d8OO~P2uB_( z|5#8Q$eZ>zkplRDlLuA?B9@m^9g<1mGm>(#BkOMKQMhhrVHC*7RR(?o9FzqrIyr^S z%y7KuvtX-ff4B@518id@jf1Q+w|F6Xq7?^~^JemSsu!E0AH}OioI1ToAm*h&8%dkN zhbTjXY-JSVOO|&PsepiJi^S!ODk;(+K2>FWkJl%69y~qGZK*4G$}G zBi@?PNn`Ii;u|;*XxJPMnjar2r-T?EsVcUR-NY2wN~P8CPm9dy(RHB~fJo+1{{B;u zCi6o=WYneVZ+V_oz5@iFHT)spu3z=qJ>qP ze{vHHqnzBkGX0_RDqhkOEL2{uABFQ~Ew^Cn#t}P=Q4&ZcgV80T^Gjficeh8_FoUi* zr9IFH)qDi}e%#p20C#dcwJnl)UJr*Md&IQwMTR~(<&Uz(k@R`&O0vbc_1()tMc)ft zf);7t2+#t%M!rDLB=*-if4D$I+h9r#>WUIj<{Xs9ifHuw0rfY_qx8An==MO}oE?Pk zJ(N7G1|zkCNy+}r6_3Uw5p!^bHeS&14@Q)+8>rOB2u>m`phR}Mxd1j0&dYGPIO^N|7X+=Frt+3#<=542+KNu5d43RNr zegqqchzrwE+=IWeMLNGk%g`rS>qhj8rYBcaiHaftli53(N~OWb~+RauHH> znEjNJz<&;?y)I9g456Su$C%>>*|T4fUn)$m=T1Gh>}abIh>|H}yFe^MKq&-wtWOz@ z9iNuSdV};_q8{K|=;0eQ!_&T>|ESmiuok1K z%VOastZ>*VS-BW-10gocz*MGZ49f*mutzXo~C` zW19&)ViqW?OMKbil*Ujxs7^@MmSsDn$t-^&5R|+=6#6Gi9)Sutj(ldwlfSB&{JkRl zPnP_zialCoTz25IVxQMsYg$XaqB5zE4e4`_jI3}+b63IWh&pM4FM=t7k>J|>)Y3(# ze+K!U-`(41TndMbH;@-TgFYa& z^am*gBgqL%e>DqfH3}-(;P!EtafprlcZ=Gh#LRj^@Ce}o*1)gcQg+W!AENF-9mpE~ z|6WBU@l!V}KMVSAe=X?$dgcC4k@H{HYyK2kS6jpoq)OkXNXP0=5#}Uj4rWe7ZV-Wn z79}Q+gHR9;g6o>-l7X~1L|}82{$W$Od2V3KWy7^jt8Z7Sks@8$=2+eG$$8h>(PXoD zS*`wUDwB&z>~P9VT(0Np=hfy@uIn_<)8}+I!KMG+q3)MF5TrvOJd7@d9))pca3=YA za-CAYJ!%dGa254Yj++?BgX)latp_um=%n0FfSSVugg`Z*21=k_COM@B%L+~e*85Ti zjN=FnT=M{Gj|T8p48LgC)qZIJN@*JSn005-yJHw3}s4uAkW+4RP)Apn)wJ-@Ea0k$|bmTdt*NdTD#b#UyuG46(4 zC@Hsb=*dJTqEM3q_G70?;SA!ymWjbMvhIH?|)(D)f_pbU;^`><=d-dU`nfa+q`D|BznGjreMd_~ORXgQIZC*D-32^HUlROR}AdMdf z@R_NpB+N#(v&LKtlU)-J2$|j2&kK53aBVOZ61kee=jjgWCHY-Op+xWKY&BD?!!+M6~yT3sL%BEz`I{i=(5Bl$` zz9-8>mJ**BtHTuCl25H6KfDOghT4hn>d?Efzpo=~1Q=DG?CMeHQyMRI26ph;mC*@4 z1SM0@OUYK)c9i5mP+~5V+X4eBCEg1_vPDfI_O;S+Rx@_$IBhKUBPPfb_j*pNVO#)q zVLsg!aS#oBUK$C%qwA(o{Fr;f__?Z&!ypmxaECxJy#>u%*jBTcBUr zPEZh8J1r7A5hQaMFx?^x>U0L>(zyu)g)k}!8OpqTFVTY>u{^JscUjukwW2mh-^&%O zGy^43J%XEmtWb~BA7Vxz}!g_ZVB(PaQX=bRsrODnFu?R?HJ#-F=CPqH4 z!^kumg{kh6F#3#xo1}EoEG;Z7-ee;;&H3v~2A(`_oWiqX7hAYO4o+ZhhLo1l?3v^6 z6TCZ8{bU^X?fqkzpu{2s_?X}vs+pfB-#CMW=%6HMLRCX53y3qEov+AU3GytRC-6MV zh8+{m=tE`qumY-3Ogx%Q)#!s~K6O5G2r1+J3?~E!7h>C3_)t#C<_;nhFi~AgZ?5dN|#H@JR zZhSYKDXo?GrEe;@vc?e`@uMjn%l&ttjUzjA5dg|dzQas$FpMz_jcGh8A7$=P>3j}X z47ZCmNKbD3y(P(I1$l{06)dsM{X-gdT5w^%<&pcqZMXZL0hG$O7rSoGuF zEtFG7s&{koAS{?#%YmZJ+EMKWZAgJJ&Z>v+yV(A;jdt zk7PP%j^j>W6ym-p=4W6^Mm}zW)QwA%(50sQ4c()!@98g8kbPmR6h7%y>6~V*JdkW;56W4i26TdaA^>=**-GO&C|0Q1{z1=0~HA zy@7oRkKQm*BPHJ&xWLon;7$<+E8c~QkkGSa z*X(OJieCB*J}iTj6qu+)hkhESmN>y71gfFaD#~TEgfEoUS{4S`L5bJcH8+@B4yK3yx*GkYOD ztLse-*2$tF220K2enTCcH{<1|L$@Ud)j5Zhv@#2h>Ob^0>G60vmTJFdN>w{MMF|a$ zX|Zz`MI*$CZhGb8JYI42Ok{Y;)M>4iJ{Q{Lrv54v^dtB976?fVg7NB=$ z*aK4bnNy;c!iyz4F(b7$y7lV`_v@hI1g6TzK20w-d( zNo@zv9O%>;4QvSu)kS224gm$kF~Kuk+s-K9!7^)Tr$W+09sB+zm$SOF!}9&96dY)apR zZ@jKINlT(UPRSl)UE9WX+%|D#Ev?$jx?C-T3lZxOda>+UPBproE};F1eEuW5tBWpX zKo(^c?x!qisw#sOM*QZ|ed_G>M&E_OLQ zVC{QBBjlP;?}4OJake@CNbI&K4>?hY{X@k2XaI9 zEl}DSA-vne&I5%g6#~z=>Cv2Rhcof-SjkEQ~ z+YS8G@GxJ<-pD)B2T2n-*6fl3Q zseZ>;8Bh1;nBWJ|4r-#Ui0Lmdt=rK!GuzD8Dk{a9-+(!ok-+FqoI4}ucd!u@CD6I6qje~wL0crE~nq4=uPr*wxnujz9sQ?TIjq@vZo zAiG}E*DRICN-|y3*0PZ6 zcI>L|tNU1D#g&h^f(TJ!RQ)Pio{R$7vFHQo=mVBioUPuzcD?#>L9pm^es(DKE*x$? zjs^Dj@*6TsOIqel>-=Q&P6&@1kQAZE&*5U7{(odciNLHe}HxoS>qQtGbL8%msRrwSz`1&K3Pp9Mq>e{18-S zRIl3B^2XmgynEw|>4!w|o5&byGEXco)F@)hqs(mD zDi@mXV&Xo*PHRD6Oovg=nTA+-GOo9?EpGAg?}H)FML5zNgfCy{=>FE(CjDDyTfx}r z|DbD=pXeI$2PB>P8(0!H8WN#T85S<>Gg=HKF2Zn7Pj8F<=x#$ui`;Id_ch4pgIg_b z8PL$pbYFkA)%TB=H_#c8@u8kbFf4a01S|#7c|1B(S0%;nOLuI2cP2E^fy`r`C7J%M zb%m=ymOsACsB9=Jq9;2t@~^>c!?#*iD1+rotc+?lf%OdHMF*m&udb<>7TEk!WS9@j z8EYS+H#Sr{fHQ<@sT}9bjPwZ?i#v~Uss)3unM1dV)-q*(41=AzlD;t^NTDjGq=kxF z_>;kuR<-~&wIz@i_^9mmUB=e66MxD3Q?pf};xos<<|=&9jga7*Wa9Osw2Ymalvb%P zXo;-sbp~$!JIOI(Ueyoxd7s|;D}4Fa`}F75{ttYtI%)eKd~D>Wrz;(u1SBc^3y0ZM zO{Urm5NsGqenlE`P+H#iyaVr?o}q<~@NR!H0xgDj5L`@ErJG}F&fiOKCv)&1LedXH{C4b24kK# zVs2}fE9p4+JFX_t7XQ98k|E-s9$?e^ohdUVZIepJN(iZ0&-{f+ ztpl3~avJy@oJrLyURa0DGya9YmvRt1YUSbAO;d?qfZ7Kj9DM82zgXBW8{bD)9P{Oz z?(073#aAt8Vu~U#W9ZM>F(m!yuVYpRxzc*VI5A8^!=sq-vDYTS<7*JZO|0hr-zi@3 zH&fekUA(x~i2mu{X~&S<>wGqUr2mR?{q?i_L=y7xT-O)MK!v=r&6AL=?kDiwu`c9Ajd%Efi6n+o3XimCR?lv98G zxO1=bFXut$2_v3F>Ls3_cWv(5bmLVS^;ui&jHU8*xLajvU`Y5^)+|YxbA@}gsHA%b zao*2C+iy?s`drwN>DNhNMBP9u`MJJS_cTn-gjZMb7+*2Fp?5otN$Xvfe@qUH$SGBvtq)K{O#E39l0ryw13*G_3C6Iz>gaODk06`V5 zOBIfe6^?`HkkjWo_4hz__sA(+PN9E0aDqf5TMVDkV&ktJxW5jAd^LBq_W5VxLBz$y z*!#bGsBz=6U@Yi>KliNjk)G?P27T$#g|?cT=#a31U=DiHyaEdgY2~gc>r$>juxI66 znR7`R(g5EqA^)k1y}Jv7UWypM^WWU}vv;#tWl+ZSGf4fNH+nt4q~+KmYp7gh$LT6Z zfRx>RuhqT8E7#+K_YP&sbUd@{6Rz$2Okc~3g{ej>_>1G`$Y+uwLkx6uB8As)K9X`) zrAx|OWEUc$_084Fi#}o=zhj|av!`pn5IgOjid&a(2$fBUE2&#|hMBMxYi3Y#zN`xA znLbGS^Y~AVh&f4P@dpt=RB4jiD!0?_dJ4Ja2s8pV|>Ah6^&pLnxIb6^pTqs1X{yf&DozhbkN3Zy|wNz3%@qTs9Gqu?jt<%K8uzd-(bVwPN=lL9|iHS+oW@4)>3#hz~eZBM1n zf9#pHn8tRq8g67<9v#};O$ttb3z51PH^|nvzdNG+55CY=k5II-8L*n3{qAe%_~qLK zIB0;f5jitv8uC)6y_&Uo@0ys!4yBz;N&Lw0zsEZXD@*7q+$NjG}JGSurfyOFB?FJfC`i>ccAcM=>^ zve6q)Yr83~&g98=>Hnw5YHMNBH0G`cr}dQQf6Ijo`UzfIrQovku#6@a3b=!kh{>qsvk z)5@~YtB}adC*oJnEr>3xSk3}L9#C*ailvM)$<7(m)ACs=Fj@TuOZv<=Zu)G81dFVl zP@g2PA=|M)#1}+~TyuTxq}BRSDr4Z8Do?S*VYi7xuU}s@=M+!BhNpgVyvIBE#eoyk zn+akC*O2ZZ9iiUMuRa*jKddK3l4Ro5C-~4LNVCMwlOk-9pKuX_YCsv1|H1nw=G@;g zXRCbXGO_>v1;wYI{wL-ns|Y9!FeCdhQZR;yzFdYW1L~0bW{P!D>cmf#KYrLMQV+Qk zhyUKgeg=7?9vm7KSEq&iEs(p_&A096=kE*J)+f9xy(_+}wkyA@M+}Ws?jF*ss37Wt zwSu@Uk8$R8pYNb>_AKF+jIM)&aW;WHXP&X@1(fArLsu%thuM%E)nib%OY#e+YUGKO z#7hMHQIwt2+#URFrT!rONTVq?7AebLpTWsBV@qG#9pf)BsaJDC!ZD~iOW7FpzAbQz z?n;|w%&J8*!+krmv_#msz^}^a)UM&8nOw7CD5y~WLpP)sQ^$9$%X9Vcp#X|`3=*mS z?53>`@sZ!7{)(k+>wNC0GbHHWVl5xPF9iBCmWzJw@Bbe8{#q-jy1D#^hv~mZmZO#X z{=r+b?B^VC3#+4j(D=hz1hDya`Ki5VeZrs#xlX>e*l{5z!hTN{fS~tkC8FpdmzYug zWS__Lu-GzP+9t~V!st9$KMi!ncf#*5a?;Z>uLpWv@36xE4!V(}Z-U+xx5crfW$ELe z7B4bKwkT(v&(u2!m#@XH`q&AVm#o*2G+&0Vo-L_&Objihtz~l#{%9RViH0!~ z)x=8K4RdWpBz=XEBzNof8si+)S37{y$IZ51h5|GTqv{B>+Vd3Wv8+XwMtA<_ldY#- zG)Tq(=;VRWDb^xlK4izgQCIB9kR9uD$4CB}LFqr^^8d=M@jrI_zp_rZE|eGgGD;4J z3{`fHFyy#f&?HW9ux}P_{|;!uXbLj6UfkKKgOW~Lr@-Uo7vP}B#%>$A$8JK~VDw7( zUYjn&*Q`Mg)KaWcD#ferCo>DjEAq|BYz-IG`%!DivcnZpuv;DGkNua zF|l^-hq$z*0J9C!Kx96N6`*ix%-R4z1$ZvzjbBb^1C#KM7G7`z4DhY2UCBUu_*RyU zDJQ6bY zSv6jHzxxWji31PXGV&FG7}ph_WKY^ab#6Hz$F+CJaz7S^L8vRbL985l4@@d;7Zes< z7gkT@wctkH^>@A6YmqATXXKpxYyE4w5Tv`E&%fB~jjhO18ee!@bisg1biojJ*xX^s zkZGgDe#;y2km)`wbi{r~48-~~z>Rze*)oWhc!RmuiN(T?)fZ4|f@sD~LvKOz>)@TejO>u!{8EC0ED zkt5l6>qWtG+tlKV#$9j0*xG^tG?+UvO z8=H*g2wj2CH5&`%N0T!qOiBG5Zt*T&f~}?QI^|k(VW%IB(MJAK^e1K9 zsgtM2zc|tlljr7HgcwWtD*0N;Iz>|!J>;`o`ELW)LL9Dnf%(dB-<58XV!?4ud{AG+!TkRM~7t4TSB<5 z-#+Q#elg+5WGC8**;x8>m9_9TsX>*9Od-?KV&GGik&thmoUnH2P45brU4blAonSGi zl|@=R-3u=I)r%?F+$9#hb||_OMl33&@;)h&Fmp*srti7>Pf`KzGdwm2;t7d2WuES-yZ~Jbq5kYeNT>@)lN*z$s1;u zSsRWg-?q=v8k^Mi6Xm!(bZW@6a07um{2dAXi^55cC0qvML_QM2no(|C&#DfqV}D(*ptCVhjaARe#V;dFLE?|?>TZpT(9R!++~k%~y?3;3 z32)24PKtjAp@}ghqbQ^ z&TC1MwZ+WL%*<#>7Be$5Govl~#LQ?hv&GD0F*7qW%YL8lotfR)H*?=k#0!NK`g!Vf zp6aaX>P!-4CHDO$gySf0^0+He8Cyv1fT`x9*>3Gq} zn*7#6tE{G%C1dY3q+i+*x;)hTom7|`%!@5a$z4z`9T1Mx#)S>ZxKJsC6Kf?-Ew9-M zBC2LdI{J~fzjQ&D-4&$~Sr<16rcuHe=*@!eMf@*)_fG?mNzKUQg{ia;*AnY9i17{0 zijilV3iCQz8zP2;Q`PsBCs;T)M|?!?TwLw>ik%Y>d)j7YG=;L{Pzt;sxPmEHN0Dgi z6{yyQ9xSh$31Ep6XJk!201;DD!EWY-%s7NpG6g)niLriP8BhV6*LlwvF8<16AVLkBi|;9W(tFtm+HtrRDp_H9}pILM&p~Ep~|oLg*644tqwb7M#;##CL8xkj>Gycw(uBdE|IjIX^9-2(50qq z5RCW}YQCj~tBM-6ha3DRimrH$0-I5vB{@a(PD4MwNGGY&ax%!(D{*}<$_mogzvy6L z@B`=agZQHv;AZd+G@l_njoEy|wrKuvac56z-TsX5YKXM-lN5}ArQA`nz{=5PNl1eV zS<8B)-5j@&}NIvHy*Vl4X;d>6h#K4EFbIRGdVrW30?7p40dSxGW+nO z%n4}yoLcfV94PjKMox(I)$|3hz?O4*+a8LJ!Z0Vxq>gBx{D>LKc7#`-Ff`HC6ghxo zSV4Y`9}5Y%8Aqw1rLI!6dLiwEn>6CFVHxH3nMq#i)rL?&fMibg8YSZGW~K6Fkn>3mlHFHU z+a8vF!^DiY#O=bz{~(#YJg}qwBwEtU9kPQS>1ZC*cs0FGwmf6OqTaZqfZMG;y?~nn zO}2wRO3T=psj0(Nb+)Wg!5KA)&T3LEvaVpn-hMBxK(bnYkE3;kFoC7%A&6GFR_}I; zAwini(johd$YZ%s&7}Ll^T9eUaRa~T0933_;Jp#?V=6qk0?cd<2%3rzyF$)$6T%0< zK$tbSP0;?zI~DL_fu}@YZivgkZ@jv*vB6=hS@6IEg}qg-amMpoM0mK z$%Xj{yz#`e8|*Wkp$xyZ(I7l~;AF#7m6sC&lH_5+3BT&|=mZ{Ui!I{2u42ITg$prh zNe^eD!;=b{2MI0BYH2AeDZhLeE5KMW5&MerIC95NypQ3I2m-wrE$D*7hp1{9&2;vB zWj`YF%NXR5_~&@E&qYi8hPY!wpRS1Q7=0x!fFP_D*Y8k4-bLr6vV3!$9VE<>D9sVn z$1l>#Ma4c=Ws^>6GNptwe2yOFo+6X@*)YVU0(uj3&MvV8!i&72SkiJSCJE(HMbcj3 zpiy$|)06s7t(-eO%4cGdR+|nHUyXunN=3xbxp9d+>c;wI!KdI_e)ocH&+X9QQRtbT7wABNL1 zwn=495O2i_LjEgv>eh0y3vwftMhOm`NY;VES-Rs682)(pRkDkXv%SF;2Y|?VQgNf! zPBwk<&Irn*=2N3VZ(%&Ho2hK-<=$95isU|lhpK{SYr)Nl4N|z-M;{3D2;`EKL_L5i z$+Jut9)qABZJe+M1Gr&S|Geo1H3}z;J__FggCb)1TqVjqwd1xPnId5jBh;KT*$Hw* z>_MUAnulA&7(jh2m3E-AUOK0@7{yB zFD=OZ=dc$zAL$%D5=F+LnYRv0(>^AbQJZTI_vHTb!Kh_cZR+o)))_gi7~i4awajwB$6oR0iIw!aDycU=5tu@73z1BgpKkK}2 zo|i#?l?Wf}DDr?XZ}-cCEJC$8bk1|!uhBL1_*sILl%Ddze0L`du3Nv~OC{#=gX)bx zYS5fkjXqlAn-mrqy17w40+IozP<`S}!t9+E1s zM8B3^_`7=YS8S8oTd&>Hj9}rM!+6}|(g@o@+h2cNO*YQpo+3m54(77|RjL1%g{Qic zp@V~|$sdOTf2;J9l77geDgrMattv4@!tWD9(R}(SO*{7p>ob*1qajFLQrKB^{$bt? z5Lt=r&Y;uGWT#`X)0+j0I#7}(vct+uVfPjjc?gGU1V=H;@$ARCndyCy?QDPX_V&Jw z`n4l91uG>rmW$F+&q#@coSB@N-ce{LPehA^)KPOMyk9lU8fB4OC9Fb3j)XB+a!hJ$ zKK~=61_F&nNDhdA7kX=9`G*Hu3lRw+)LxkMT5TcN7qklTt^hR=w0gss0QjZi9DMJ0 ztU<%%e7YeG2n`6$=jPCZkk zg;JX17!AXs%DNg;H^N1w3-PYNJu6PRheS`=nT6nCSOzb(L)GfHu18ZN*egoAr|e9m#3 zYAFfB_LR-BY_3<(du#jPFIbmjy_=+@##-F`$v>xUr41-Dmy$bpR=^>B?PNppDGToc zML|hyV9YWMq^tJ%a?6jm@GB*%X;^K)CJG%XT|NlCwTraD?l0|uT_rl!Ihv?fqNyi) zm=hJr4oOxk9ZA-rWarzyBFj{e^(ra!BgaJ)|IpP9QTY6l($;RfMN*6(TIk!go=Grg z?zjY;G!J6O&b&m5Y$_+VOSW&JdR0ablEUB0Nx0AZX^OJbImfIh`n;p$mLPZ#J|glw zDdd%#eaa1GFA>KJA&$*y!yx&!$AUxW(phVt?f@N<`Bjc2C2jUaD6y=w0%H&>s;%_81k&P}-um!<(jXsSH+4SFnvur$V%hTwqgU@Z| zSJY^=8!%dz%4#by8m;Kg41O4E%i4I-*%(VNsgY{;_Nq0W*jj^X-(z?!DcR6=CmR{U zYP&YhPpD-D>q`H)D(=JJ_@YPug5FzTN#h6M+W&!NuA9wQkY!0^&Hu*{F(CB4Kd?C< zzkj!%cn4cYG~OecOLS}t=Y4iKZUWg=Cq%Mcwq3d%68$j@F%&T@CT;+Ahjqt!2Nvf! z{we(_`RSvl4~ZYCAEnn*;6cUiM=cM%x`)_c+1#cjtR2&9zkVLA^kpAgAwm181Z*OC2X!o~cK4oip zRG`{`vZeUKz9O2Bpr>ZdItl4kponlljwC7#4ic$PdgE-JNo(7(Ti87z_Iu7C4<1dL0}ND{h2|Z%$K3dkqEJHYNj+| z{Kmhmo`6LLWvxPjKDgQ{ovHSu-o1c9a;U1*Od{@Us@$YyUU!-C+j5$A`+ zG2OTSQAc6&*1r8+B{79#QQwYlgau36ZcBYmnRgDpJfNRoSsJBk z)eue@u4tz0x|l?`^#?ch!f0vJ19q}*7efIh?W@#Mi0$@qjvcS;wq#eS&$I^GX&D9R zy4MBALQ-*NnG?;qQq~EA>%o#zCw;?vBkQ^hWJNO%-h>B zp@~;pxOabLw@sV2VjohFa0E|Ta6Q$=VM2jK9)%2cFA~Yt2baYo>5#|FZ201JJ^*x! zWU*s-q|xXLh~JyP8{G1hPe3Q3kN+!HR5Pneafs13c6Tu2V6gnObNMfl8*%-gDdP8z zkAvguRk?Z#lNQNQOp~LhUYXV0NaM%F>in$l2-CTBBG^!;$=+BQ{Dk%KTE51GW`+%q zIGi!iR4S)Nmdx&v3r*L+==W4!GPQ{n-D3=ZId%Q96S0p-f6)gvoHM}w3c)P@34%Rc z{w`VY5BRDl@6$hhy{X%Bi-`P+hyo9dE}O0}{S~XP*pioCx~MI~HbucNfpG4L$ZzZS z^>6p1^qk70kSt#33+~5_?`h9JcUQ5$npR*pTYtrEHaA%A4)>qeBV%5RKm*7y>9EFi zhw6#R-WhZk^ydU?uGKSzoMK=w*%OCU1iwmXdwivajd6(B4$q>G-I%b$4=3$Oy0Y?V zbD{b%J_N@K#yR>KnqahyB+}lal0P@R*SWkdmwFaIu~>=?2NP?~Fvf6hLfVoOu8H2}9MlNjaBY^v9YW?GFtI$c9k91gVW7e9`wWc31{fyQkg4yHd9p~MA2gGWwY`YF2yQ7-__(>K_Dv@BHR zwordF(f^KShPEl3orY(Iqn1=yo$9f`*0vKq*|#vOrE}<1QMveOU>O_ZS47Hn`MD=0 zNWb2_z%MtDZX2wRCP#bw9vQvhf zG8*{opTxmH{ zlfWyw`Laq9Hd9`Fvb?pJTz(GRc%#UBg2+>lI=I@*w~FENVI-+(;V;-N^rM&=s~Ybo!r}_cwGy zKYMCAw27)AC5Z@s4yTYUvG4#w_ZT=pD&x+%%250B!*0xVAP!+a;@gKWiPs3as7C5{ zePdbNz`2)488g4PH$Mmi@Kc-%iyHQ|`I0+*T27<+b zLZHsgh6V(S@N`B!f*(CWt%q@%V=O@t9TT?0(-`A6#_S1iQSGEJ=FhW6YaO&=u*{(C zcyye2(by$ZJB}L=o9UbzNa zOKvqxvL2J^j`-3uvTlTut-@?efT8L?X1wUEjSI(LCXfDBW`MQk#(zy`r6STiL>nJ_ z{Grr=>HW+HDSQ&kR{))2i$0GWNL*m? zHZjbU?hrgzji%S(+Fx@K;+g@u#*NbGJ<|w^J+E7B3CU*@K4n+2qMbKLy6knq*(gHP zUUrq~)*_cC&Lih0dgOH6dHke(^fj6>iGT6&Ol(d`{6Q2HPWsotkLQL7UpZYj>)!yr z*1vqluOqmWB5+L$3!t%AG+%*EiCZe2zLc1aC^i=}*5G%1VaBOUW)>T!Uy)TdL1SGo zw=lo79EeqZb-r7lg5~R08Pm<-<41d~EL1+iB}*N<`0^%vlT3zTW+a*ImlY4E9jtt3 z+fI`})#etyff!adqS1);1EON&i$yXMvbC8Jr;oeL+_(v|qi&c<2i#+W zbDk&UqKZv0`Z=1L1}>UK&m042WBR`loH7+%W&}iV@?RpD?LSDqBKEGpNuj@ESo=47 z_*pblrQ5WI)}Gh~zDmy$z=;9}ZOC4ln-_r9H(Q^TOg_6l-ngpqCHi_Al3g$KPYBzz zkp-#U86J*3AG}U*Z<-w}UVa<;kX&1~-3U>&9Styrr^UITKpv7nfrdLZXwD-AOgpqx z7=H-l!H2_69dHHZbe!)7Ie_1bV|74GfVeOo?G|1$XpS96m-seriyxl6CB?sNW(x=O*}%DwvMts=iYpR=%C$VsnibM!mtxY#Bbs4S8b; zaJdoRw$tpXdn!gQHcOH#pe9Is3(g?)(od8=zGlpdmDdh^L&RevKfD^$kvL_44knk$Mik3Mu zs^XzVGM4E2*q1=FpI@hq(-ySRjE8^zK^{P#5f1eJpZEoO3;nO@ub7R!;opXLqW;Dn zw5e%NZ>Y3<5}WAlnz}!9O0bcE*~|omgY{#5l3lgbWLaOgX^XnA#`@*|-w4ELI$M1n zq1(k+%i)^$q}jn?HKF+jwc09UNyypSJY&_N^ypXNP*Ua~p-Aj0hdx2LU8@#5gAb$- zK(&`gU$>c~Cwrw4KnLdt|Y+eG!ebjTCba_tYH%co$= zc_hmbvwy+LP5cSp%KUaKrl&c_tOEeRpT3#|>N2v-&}GPR;I>8&rRO7~cpTl3ig|qbr z{}S($+FnQ>!dd9Z8El^XF%-aE$#H9{;k*WM>p6w=vAnw0R&hVqrt|7#wHMP)w0YmH zR*ZdcQui#Pdwa&y`-0vXA(_9lf;;=iiJ@|##^V)EGLZc~ludUhq`b4R;I)!pj{9TQ zMjdH5BrQ%=Ur~$~H8sNx*d-B&Wis%tGrgdkxXl_*TQ)_K5#PF`7nLaD)prRBybAKd zTe@E4Tn;^4K_wt_3cEs3A*@CeGu@C$m_B^cjcq3li%OG-FN7>6gXVP)x5llGAWPTc z9+(>HtWzP=4mBZlJ0VOLT?=PpjYEBBt@7)05O6|3MfRj28A>`Qps93DjVifila9L~odQTYX7m8@rjzswML1465o)@J6 z{)NeWvf}%~hW9!5ArQSoAJiQ5d+Gw}K!C;?;IuGm;<5exP)bt#gYN|CgKu(t8hb~A5Q(VCAyeD5vo^%CbJG% z<)M?TY;Ok^cKHHZ&&4{avo(}BpJxuUDRE9aCwcen5P*eSLps^o1yiJ(ai^^s&2x7` za{-G*FUGk5I@XM@bN+$tc73le0J3V6rFm%F*H9w_zwdi~@;(?~=w0wh%zJ(bO_pGJ zo3*Kfns1ve$k?rnew26lwt?O!Vs#-Sdv$=oqP3CDx+Qu#`g04=)1*%Un1(lO-HLg< zH=r!-;*{O>FPnAkXl`)$3e*yX?ZopI3U)0bZO&)0 zN3`S7Vx!1KM;XX1JIKhZ%g9EPW0-CnbI!Erw8Kq}y$%S|QOx8j?rdm*eiiU5M0L@n zshLqDlE-Z^Qr1rBbng(q*@V=Aa)tvC&INx7=l`UX zld-om|L=qWN!tX>o(u~GO;$>)2UyAdgGxwOboD*a`Y7xMKXoHs ziDP%=S_d@dA@Xicq&atTo3q@$4>h5Fkk(*!nEk-%@S{84-xRc|Ew0yqvy}`a-i#n+Hm=tV0?M zmNxikhhOzvRhy2xC*Hv~O;Rs|+;8IQ9y#wDbrE$YZ7Z0`2%k+yroOrJKeLsgn>N#i zhsPE)_A+_uow2#H!D!=7-p0(r9YhlblRx**-{fNELV;-NAqr3H@SXR>E>Tuf_o^zZ zck~-bl3RE2u-v`-VVMAnxfL!0jq5%dU@?aeEauofbMneg=RJvfTj4xy)9i=CLsg7d zt8Iq`y-bgj(t~N(@vVZF;&N#dUt5zLs=$+d3KM|EoVB7beNzn3le4_YfZ| zOKA`NUA@h3f>fXXnoJG{+;p1Ad@xQSZmn!8Lt>i0_=o6bPUq5{9Ndd`KNRtIwv2+& z5;yK|2AS>o0)SI5zwl0c4}dpTM2qtrt!TrdCYRWxFiPaO&6g+P8vq$Xk&z@|Lo5Cy zCxIW(@=zz32kjSm@rZA_XteA@cj`R&Ggq@aNrM%glKIol4(6R3i8(V)0;Q3Y)R0~N z3|HWBDvz`tii#}dD>mR3T}|5r$6i6U`zQ2o_+I?Xi2vP*`0H}+Kgq^Qre^;gy=XrH z(kWFx@K3;Dpum4%Xn-+N1&I(*$adq32>;PWYvycf>whtL!~Y3E$b?Mx$$-QjD4LpP z%e=y|^TUVLyz@58e&XW!`u3FiYlenmO^$&gFcrI_t*O=&Z0X@K-;EZ^Gl0@yq0neC zmE@tPsMhC;2M74Uqhc*Ef$H|rQLv90ss#Gu;v6`tlKN_07)R!^^JkeJ$;;bc+=AL^ ztWjg1VlbwpU_<;2|1@zP;Ul=M2$OQVwgX@c#~3Js)?Q=`N?#?E1h=kAo9cn;D-~!_-3C9X7x}&g!w~I8;103-u?$q$ZlI z-j@8jo3H>QL3;e!V=piP!yx*6Q$NTPvGPrLfjbwer$Hf4$t5+1;CNKJg}ZLAs8zm} zP()3xnf!rVP6o7#5{_AGZ9L@Zg)ZB>XgxqF?PTF!MUkUkGS+ZDlqfW z(xu5QUp4DvbQ&^Z;pPns(|YWz;P#@*32{A-s)ez=l<_ZPbdJ#9+BPUB%o}VpnB>bW zGM_QUst?;%8P!uR{G8UAzq6z!GZ)FffPP%ff0>nyE# z5JZ(I0_1i&9_Hs@J2*IGTG3+lej8$)1p(5HhYL`%`}Oh5!w}>wH~gWgXvbr)b#aCJ zqmy^!ofmA|`M$Ab&iIpll4J>hpS%{T3CPum42WZ|UoV&;qusVtdIt@0gwS?8XsH{lj*_GrQj$rVFJ;Cd{PL17UQzs_y|zKR zUI$=VJEs%i!RafpmA=b(k_D=ucCco=KU@iv|YbVt!kbN`lReGj?JTe>#r(-kek%xbx=2E;=Kz!WSeo-hrakNJ>tr?RhaVcOH5S0eoc$k- zIY>|mBbN{FSR6eLeIg*nUd?uSh{6=Mks7Iewy@X*^d^q8pFeuSA~KsAqR(Ow7z4JY z{%kAzQbmKQ_u5`%jpkM~eq$O#Su2Af_aHhAXX@N`dkCU|UM&5QTr_s4z>#FpFu{Bj z#1FXEVSWe=)2YS0fH86iPY1S@GmYU#TNiIU{hqsF+4$2|8Bd*GzwuPf>X{#{-dV*P zdaOF`N7QJ>^;DHmeJ*0O=8{ztvot@wfB4RpmfgylYRU*#cGHr7Z)^{;{aV+E43_uZ zPBuVD&F zL$tT_Hks%KmA%hc+ezxTvLCDQ7D9MNy0%gO=tds(+&pKV1uxH@q36l|kzuH9e|@|C zM_caUL~*P1hadiX@l>V9rCUdyv<|IS<~_~}`*e{FFq7x5mCp7_%$OGAifxv3uHR|E zrPCW`#(HYWDx0djp4&6YeVUHkuG0IKM<}khjZ8sm2k?0*L0EwuYHpMZ7@6BhfsIEb ztBngM^Yn7iVzI)EHmvHRkE3eviu9kXt>cZO!yN&f0h<~a4zx34B3W4aj`xrdyTjlz zB6}(#awm=6C2SLh9C;JU$zZCp3_c--zk=pxS)v|GC}Lk>eTq@TWmVw;`aR<;}sF~l6Y&1IR-fs$HR8!3t_Nw+Lv*eeDRV{2mo0H$DSrpRJeUh^Ft2XdFB zOUy}=((3@jxIHgPsvciW#e!?ubv_I$b&Fz>QwYtY&y9&Ci}px+4r5Gl40rXZGMKIV za-R{iq;!u9A6K~JemdxC=HSZ7-Yi}O(-B%08zcE1Pu-vkoBW`c2E^fu|LDk3OZ=Hj zMWb|Je@Z8bG3yJDRk8klo}a}j33&3yeIkEeVD=V^et-#~KoTW9>Br2b{E^Ygm~MSO zj%D_;IL;)}Zk2AKgk@7|a^6Z|B_*kAo5{lG{!W9LO_e2ek;&Oo98i{jx`yhf`_kUn z(_|_NR(3s6X)7&%Qa4w4U^kC8?r6{7YYF zrqTt1$$d2lyXXZj0DyCp=k0ieQGd*HHJV3!{3~>-#~dridJNC$#hgxS|LSX^C-B!S>lYm4idDG@6Y<77mB@SCq0K>lo+VVqn4RWOKlPQ zodl2sEQ#HgQNLjAI1>{b3!5=x$HN+~tHHW=_FRCQGZFNLArY|1y5+ja*Jr`E%KR(} z@ZfkvS9{>SaATKv6i0)sxxxGm)f`FEN@*Ze0e?xEffuR&cMXg`*+z%fpYz{BA!?Zf z_|s%BAfM43d7#1aG@zitPzP)2Tp%qp8I<3)j|@0eEqPY@bMS0@dlt} zvsrM60BEz@Ogm%%R@v8bx8>lgr)yOEba5Ijo4sLV{7>JRpm+SJ0!iVb34LN80wFr! zLa?Km@0izyuS5VDSaxPiSTES0;VAk(^brLPIAm^v2`3T{8n9-XNHCFME5Q|G$+2b{ za{X|K-X4LIH}$~jo5jxV0UG47VrLIr*#Hii_)>tL$%m}%U$=wd7>zub`=qh-Mz2@^ zDXi@Q+evWMMjj1>Xlva_4}B^g;eI1MvdP$d^OOtYBy zrw~?%iz-trK#OcCDiq$cWmXURcK}(AzQO%XMzt>CNek|pk_h`ZGZh30lC(oag(yYH zv~Ck&LC7%~+Y2JST&um)Xa%oa z7QydXc=et$UwP{^u|`&=;~I5oarU0{YLS-XiPXY;L8hf+N^In8vt9i$CaRM@H<`FZ z^wYME@K#XGXW0#L^s1E%?}_hSd~#2^X?-AECwGJPGQ+;|dEd4LpLQEXv=Sl`A9Kond{uJ`{qt-7rIJC)orY zq98jtr&{Ixv|h;YIt^W#e$^E0;*Y~SBEGrB)p%Gf7hyeiRs1TFbz4G^JJeQtv)Gvm zqU5#z?!aOdug8iJgHsy6xQiT-0^NF>vbCU;`OE&Rvp(-G)aDGc$#CC|ZL)2l8`0BB zW=$AV$B(*k&|{6Rv?EjAquaF~jL1CWMXrQvzRl&j;#ZGc$0EZ8s9}5aN1J#^W>J3Y zTV-OYMnQepA;?8D4tEsC}l2qgr!Q#`F66G zOgs*@zY2qB1+g@*5{;uHQ{r5(kXDfIkVRS~isQGM03eWyOq+x_ghD?V6s_%xruM|3 z!G?!to$%uxI>6(M^o+s0LdcTJSog~w7>O`qoa51&6y~9kj7v2R?NIB{(sEmQHy|ho zN+umD*+u$`|b>%BRJ~NRy#9EL?}8UEAo) z&J^YHWu!0PgrJx$MNR?qloUV^uruS#=6F&{QEnrAC5S3$wFG#m79W+2T9d>OO_eNy zL!1J0ys4$2%K4{Q)D{wX;Eovfk118llCwR$N=qUs4xBn$MTiNpWmoGnO_;lMQyRUG zMGxoMAi1ak?vk4E{5Ry5>!n^4mb=bra;@6uv2EWh*b<=T92|-YAu3I@Cscww)!mLV zgaIvuEnQ&`dREPI6Egdw70Y-~cNzZEzEN>o2U{3{Qq@x2`Ks!LnTpa_XZBW1I5@)3 z?-?8roE#yzfhss90!?i{lo2*_M=BGAQw++`=UNF5`eN+3spW_IToj46XVt8jZuj@J zAz((2>{(R45`Jhkq~Pey`g-LKRB@nJ;%%{mTF}jb4ElnU+?Mqtv{kd9QEHAN4kxxG zV=G_t#3)td3!7S#8&u;asY6P48(0G_*$H1WU1?>9`qo4+qV(E){6-p!drN<10A$@7 zx!u~w`qqY1fI$*kNQXo2R~=neFkW(}D6Gu}6xVMO9b$PWB(C`)>xh@sP{Q{}XYtL- z&Q@s=42zs#(&To6698K!5BBE7eFkvC_m3)bm6e)oVDmF1)c|*C3`p|v^DBq#-f69~ z*x`GlxkjlC+aOM;IgBtK?2Y)s1(VXwSA(K{9SWMtl#-CGcENReMP8@5;APPa70T;} zZu}MnQxz-Mlr1eVYL$J5uKHG3a=SvT1@d@!zo*pZNes*r(RKno+M{*KV_ihT#Ibdf z(#nYe!}=ATFdp6LUY0Oerv-*s5;wFBiqt7;Pqk!LL*j-Cw|IFaZo&2Jp-I(O>J*1gLYj1XMab>8PI>dH-c!5VFIx+zYl-J42oM%_ z5S;O&Z?~Uway+A`?tqgj>RjZ#f7dHrzbdLdWw~y3GQ^5_6pLwZKlJmD0#Ai47JeP(!-qPc@bAB$di!4&c>jEBT~*h4Rt$~*lq!xs zq8}PMh==$i-cT}7bsaESBs)XFu@oc-h~6w4hNT*=qtz6Am0t;w3&GSomJy!$bO(9& z>k4Kq32m&OBo%vb*IRwXd9uUX&Gqru;;)~OTOb^!K8(A7$PAV;7AqFJ5e<0*C`-el zU~cL$OK>S!QKKU^PmwbtbWYiG3p>*hOtPd;`mtRa*~K!ngd700n_gYo_7=$ylz zjyw7%Y_J4aQL1r^6k~4c@QeuJVo|=pK{V}C*#YSVDC0f}l$S7}Lk4e=qL#Ckya-=X z&6NJ$PmpB(Su2MSkQ4#>tM?>#$$F@GD0-SC!Kj#sZO}yo$f{L{OXaL{s3L-;Ln*ar z2{(nuPm{s|D!$>|ZSwn9uOJ#)wJwX5$Bmxf+IpDrTkTfN%;P3*rOJD-HG&NbtXAHV z7uR+siim(s^VjdGpV&wPotpN!A@te7&%;6!Tqk18vE~M(UsB;aDsjz10P)jS3AlS0 z_N{QGf#y-&<4mZ5Yd-a8Op!}(Bd=i=(+KY4`kqWzbf3$G=70JCf?k&pn9u~EG#l>Z zG~EebR!rfDKlJJ7gAj<}=AsL~AT>zV7$iAvj~2<=i#>$=pr5q6f{@&5!1W4IP9V%% zkgbeOzfZMjn0qqLyL`#!slBv+{S>Bn(xQ*@0>eM9F!sSylUC;3aE}p(MTe9Y%3l&> zvuByWp|v?g1N600| zaE+1=Dr(r@#*atswhVZ0#@3prTmM`fnq-$8dvBw0QBqXs&a78`PLx~dj&zAp^+Z0Q z=&_JuD!_(|As~ajdu$#G3kVB@U4Rl+*T>pbtP{GyO{Y?Rv8>h0-#RuQ#vJ(-?Ll*o z2hE(Ey?{xUu0TO{e0^E`xwK#-){0Q`ds3Uq-bpQt_N`|azw_V23M6m)Dj2RQXQ3-wpuKjlBF zW?MrS3r0CtTT>@XV=~|osiZUTkEywnq0Qe0*Mk#Q?Pdj$0AF|^Im4u9la*G$q$G(# zlGBTbVznHFL(vkIkb+;Mz86&C+3#_o`+nGBM_T--;;4v#g4iugY=$4Ga9+}qx}4?n z%Y*;*b@B32;G@k>6@VT76vxhTY097gNt^>7oYk_VIkD(m27-BPgo&|F6dQv@!az;@ zV+4;>b7_8fd-Q5^h4L&pK0Dp%(r39L{0gejBw9xF{vYE~zL_|?b9lu=Fev*)UAk0F z2+PQqq1lJ{Q)NmZS&! zwQlpn@WBmXtb%yUn>U$J{!^$GjGcna3;ji0I30FLvSgu>nzJHa3&)nzN6a4`&sZPh zr-ipx<&lYeF^I3$55F5cVEe0s)F6FLJ6Tbe@fFq*O*BANr-Wjm`7BP{gHa`mBvTI6 zK?ZH5kw!G78Z;;#K%)jK6jmjsHxbFG@hFOEWQaV_+eMpVq$#T^pR3wMK`X|Lo+r0zJ0^e#>3aJD+t`Y~*rCo;DI=DMn-)u+{eB z)zpSqPlMhmOv~zukYQdx9f~Bnb+j*K^P6bb;klAgRXn&MOuN zc)g?C0h%v)cesW24xn8jK(Qpw->+nxYcXdW7_h}(3LgBAS0e9ZV(MgS^5-l0Cy^k8 zKP}h9AQa)h{ZD})Ygu_Q2)$o|LT$udwN2q+E&7b!U0-8~G#mf{cIZ)5-(F9TQc$M}6>>;g>~Fq~T8 z@4qJ^W;v|1%t*JD%Ksv&M7%7polL6?&>5K5z@o{Uv|6T5=0 zmaHo@P6)~-){kjwu0M0Ete-4as28c{k5x>K06cUs+bf6o7 zEDdYvk6^KOLhpa8A8r0+uz3F*tg^A8jUjMXF*5zz% znizoKK!QXHLb0Jr)PZ^i7<>W+EnjehIEXR;hx_C4N5-{aT!9bw8;buPh9v)Zd?!;g z8&hK!Mj<;-HA5R&Lpw_|Q|G@OaQs&gz!m$O2p150oBi!$^>aHY{}OxuLYxb`hUYzR zt{#s_f0MbIg^n5042m=mtBg%100_IV-?00`wh;ZdZQ<{ShJ;eVFg5=jMB-lt@n5s3 ze-A>~#?aaM?=k#ogk|PxN9OEmWawn>Ovd~t7NeB@*x~szxHkG7CDT7gK<9$P$)@A( zLkGh^!}v;}=3*2)fZL80IW;uob5()uuzGAm4%3QC?33i)jEWNvN(u@x_ReIdaerrZ zUP!+b`1A$5CQ=(uG#X_HMLaFPS8>0j5NSvT>`-|sjB{Zoe4@$)!aaty za2Ks_=U389Y_JilLlDY+=NS#jX}J_wPwUlM72MG%N1u)3>m)IvkG|ygbmCrkzRo26 zn(>r6#l)EYeS@Msgzfwn25c7D#e_jew~3_S#>A*CA(U@bu61K&#ubp9E2f5VuXwmPah+D|cIgcc|$t%1BykuW4Eg$7%r2AGusjH2p|{(5o5d*!crksx%zs?|H{6gX`pbE^Q*|34UUp0!IcVPuyt|wm90iYF< zlZwjUu?|{m{83A7Gc~RV-19Y3ho=-z-IKoRXKupKFr30RnMZtP>z5Zk=J`g#qqW5i zPa@(NrKrS_%nDNp?sN_c7}Q)Zy5znpZ_#OuTu^pV_I$nJk&4gorCNBpC4T!mvGXB$ zZ*$1x1OB4V^wxLfKTbOplLdv<%(JS5nc&Xz#LdH1ITPN4QCZnzC*}oY_O}k z_|CG6Uj}M*5*)uPk3ZVmjdX1_0y}2 z=ihs68tr8K~?PRbVA)*-iZ( zTc9O*Y*&0o8XujM$iY)cn82p0)cS+^J*Xfoe#Wc#PB~sKDG?%TW>UiIqq%IUOWG&A zHxp2Le&;2;aKv|n&Eo~$C8og91zd}k@SD%h2M+F3J0uHVS{Ga)(OFB%T$g=Nuib-5 zOeyruu1z^M1$}I(b~rqn2HI>t3BENdh@`gHco^Qcg>B-5CuaDx45Y03kX~zlMvfWr zrrR*SZ42E*3-^%Su)fyIsHW^>K)X(BpWN}~Ak4IV#(3uK@MPos70Q42&D%bQG$yOa-JU?UXfaJZ@c4X^)W8e!X35s;^~Qp&!~15*)a%@H zE@Q*j6Rdg_)iQrlaSOCHrfLe`3Efqtmfz)SrnSlxX2u& zHlj-yB?DSKn8j%mG8!-yZ0VJs_!aU-?&4WxC-7nCx|+||Xl9MQzuBh57X^{l%Qz>c ziNHgADvnN;ly1o^J+PmnbT*>r&+;m~EHHIzaO7^8buFIcdxa$ZH5rR5-Iv=At4X|P zE^@18P|X}&Q^y-lb)p-msCx`!vLE4!<=`*Hd&`l8DJAW%S=V6%0~n;bFJSRY>RMWBkY^s_j;dt1PMH&w>0 zb)$d6y*V->1`Ok-IG3-MUm%vko!lQAO6Hn<{Fj_3fHDxufZmY#hy^2{%822J=OC~u z7sUngyHJY40a!N7X0gKiAM1r%XKrT*tjnT-+u^_83jeE^`6mLV;^|=ecM|rG-B1}% z73n8}cbX~IQjI^A(Pt*C6y0Gc9O17b&?#X1MFtKD9^7<&EL^;8V`E~T59(L29@|3Y zCc^cj;f#{1Kf@?g2L%k4`PC~T@A7Y3J151ySuvtcXzko?(Px@x^W2ZvT-N41qkkL!8N!gI015Z(qI35 z{?q4tJ?=R7VZShVS+#1`oK>qt0Z|)E7x4$80}Kqzpq|Z<*+V2qMG8*`&rehGJBviL zm~~sJcrW-y?~&D!mRbx7kg6XJK+0jD1!|~?9tSL+s7Bo5@#D`Cz_Qht!nSf*xaju_ zy`Q+M0RC(%+JWd_ZtJ}&0Xn;A^tUQLqJ$Dq4RJA<=t`uKN5~ z1Tt`gg4(EDO8YqsS}FBEA-0HM!BYzi!%Wc#&}#2XROru59rvesY(NB;LaiQ$V^f|I zt4;gG+aAgWL@hXJ>#5WJ!cm(HsmXqPKbnk`22#mjA@qi=_u`d^r(AYQRkWXVfqnjZ z@#LdYz{f{|rx<-(=gE6HVm7OL^MdjzIe?{9{t-qGI8Ulb;u#RTEI;P&$_Z`K+ngK@!hGtBCvV*dINDM7+A(MYX8Fc#jU6gcL z&2ikGP%d7e$4rybWvS@Ey{z}no5N^lRx}b8CIZOs2=m+wwAS%O=r({hUSzmamYKll zY|s>ryz&wr=ZbxbdlN2fMo@oZiibiQK&oJJE3(5eLFIK*B+o28YV^ z*9MPtMm3%*mkIiP#zf32RJ&axgp8%vIgz@20^biT<0k)UC&gE=6 zIjznZVauU9_&9k2{0SsE)kA%$bn3SCX8Mgq^1_TPvRR03BUfh-!4t%+rXL}5_BL%m z{2^6vPo#oS(U_1*QhVY}emPR1@YfP~3I?iSjpK^Z3&jr^OiS!vF#d@B;!}W> z5IGy-J8m!~TU?zu4W%_cW)n%rJ4l~$>wQ4k&cZ|9%|qQ~F}K+os^?Bx9ah2o=1PN4^c)G%GuOc*vEEPgTH>~ zfknN=pSZvkvqpr!3qw@@MjQW(KL7u>kmFKU)WQ9Fz+b>H6ms#eT3+~EOc;ACWCYw| z{oT4pT9^cQb{xH#QQDiMNVl-^OjVz6K};&N$sfZ^+cPitUUEgOv!}K`h2YwN4A`mW$O4cXLuZ0Gma7E zVyu;42ACBr$~Zde8Ed{9py>l@62s@V^f&paSow+7?~qD1^>>eCFAlJVQ%8IeQbA=e zQ2j-bjE53q*4&tl1QG${ zL9;oy5-viy^+|5ZI#1nk=c(_2dZy=x*Cy~(;e0>oDFKKYm8qp46%@YW#}Gln^$Cy2 z5H5L_93Lrrb=y>m#Bk^9wlxS@mflvMk6=Y*{ZMccEr$-T5eV>AI7VZ?F-R2Y|I$p8 zZMv_QleRf-?anT24KI^QzRb;B!SDGF>)WFNll~ZX(a~pzCj8a0glUx3F4S^^4cr`? zzTy}VFXY&jGd73izVZBAWljN7%tpH z?KqR{<2?yU>UYhgx49cNfszcU-Ad?Icaj#Bsxb^GU2xEG&&uqoZcdfa*DoHhd8PeJ z`eFGdOul#?Q5(F2D%)Ngwy7=V>95Kd>M|R6Jb99Rdhew85|zu?&rZAB9KVBE-1u*|g8Vy^|91$n{e#KFHO8IhRndfaSZQmsOPxuW9n0zCp*$N=#Kj?? z)w7uNbU$^7?ItJh*}BChuwTShtJ)L>-F~Mu*w^3 z%?90N6WdR3Q{D>BT&ecQ!_r1j9k|JY3MhkpiSe0^Mkt(3kXr%kgH|@tuorv4V2m_; z#8SrhTMquUWraGxVlBVXPd2G1WzWRk2DS0IS>pjcW1l>zJ4iiB_-m!9>+sWR>J1t_ z8CJ#K7H1KB=vQq#3C!+Xx6Ol@(^#~B+2x9IuK<$8pLzH}IJfszDCFywpB{WHVT>35 zTHuwy=u7o1Ry09bVR4n=A*>bv)SMK+0=RotNER*IMqP->j(!-s6~vm9zf(kl+GLtx_Ot;HXwsiIuIhRIEMnVdw-b8KWf* zg}!FuHIKq>ZofY&e9J#~C1+03sSD9h{M4b8m#D(>scyeqWJM9{N%YGgJQ`$-$2r1} z_J$Xj;|1Rz?;&3kx9?rt!Iw(Q`bE&+6DqL37fMTPVP)q;cv191u44Dde(hq*YGwV{ z8RiP{B4E{{wD6hMK1s#o^v_WrNE0}>H(-}q_}ePk?A#rxyOJvX9)0>oU z%#b9q#eP>^)fTIT&0j0uNI&@&TVGeh6>}V5JGon1^_FP7{)JjXcUs7LM5 z=7qV&(vCZ*fsZ{ihIQ0u5+OS9UWBEV1k8aZ-+|U=87;iqwK_kpyDW5PA|UN=%_&kX za_b)$fej=U)ww=>^$Ldr&}+p8I8@Q(S(7c+OUzE1&{}C)JU8fufoGyQOj!PO4m&~= zxe4be62)g{?!e6GPrl6?)z?dTilZM9Ikbqg?H9D~j3F!gl5)Nwlu_KL1k)Fh4R)sZ%Q#b8@%hl=LocrR((jBo_%{du<2|Fpu>900 zf8`Hvj&td_u71Rr;oR3lo@Bmik8=Ll;k(Nu3YF$QkRXk03yTZVEjYxa$fw`mqR=`r zJz-S8M2qw2-$?wDfaOD+9`eNjEn@_CXueajCm0DLfEf*IGGCmyB#Ee!tO}>9zwcqD-uDm>+{zQEvB^YgL7z`>XH&OWQd06W@S7h;QR=bg8m@-(62w`mL z1+BLD2bhuMH54&g*vs|R6cNbyPGbdCUk9S21#~TB?bBQgYj$+w&wDt+Zr2%sv3XTQ z*WT-&+5&7+m@mNIx6(61TxYo&0nLhj#@#j;m&W@^vY)f1-H`--nep-E4)5S5xKHmT zLzcRN69XLcmO|s3F{eGw=BPP)Ir!2;B-tmd8qdFXP{&5gjj4Mok^cC$d^afvIx?T< z?xNkluGj#nMEkFsMW*Bi^s;HOJE}SIoEA1S`g2l+rfCi15KN}Oq$gL}%1!NNoG03k zd7ZDEj=9H?-@KSwUGK{eSsX8r);PPS#*u8ce|Z?F zaN#e#clHZ#ojKA|%g z)0G_aI%gB2*uQWlIDHzr^gEh@$;^>`CPxubo+&9k`+5Q1Ern~goq157ilz_Ja!8+pD&91%^t=e>3wJH_P3o5!$ycHiwp*k@rdrqq z8ArEi^)`IL)NyM(1vK)WeA9J+{)zmR{6(j%ta5$697{Ezc5|I-}*txAP*m#IwQ zAt2#LC^^hzQ`mQrLRE3>Y_()@$dJ4xY-BZ(@h23k$LiXav5_XlgVobmp6SNdlG#ms zcw_wimGgz~Z*nglZ+|{Oi;^&^t!rJ0_EN$mU^6lJ6c!Yi$o0^D5h2YIq2R+v{*^!h6nH8(rLaPCM_)b+b)Lx`Knt}sV66&1odp6i> zP0Y?CXcH4%R(KN=*!QvbKqrEw+HO07cnuTTm(bcyPt$V@LFrE#?1~a~*Tl&>1_6_w z22i}$L&{o5*;4o#KYebNDGsT2Cx1;deY?k)c5i5^AMS4TZ0p8?gdv67tk?gxMX?qa z1{@B{I1Rf$Hxs)zs9MPB_8FG;a#pq5W&l81{@I*2*x`c;ajQ6W&azrSBjaideS@Oh zlDZ$N7^VBMMss+4W!0(qJ@nl(co-jcd=r#3sMl)L-A9!wOp_`eIyTC69u&yqBp*6~ zrgN>)+Q?u%v|?>!N7A2pSRzr6+Gs~BHz*bwvRb>15Jn%Ev+IXtmW8z&!WACVg&u1An9?F?>@XOG zt*&FSZZ1Iqr2}j-?7L#alEsAIFmaDBVCc%+Mc6a##a^Pj6J(oZq9_5-R}echVpjzU z2Lhw({jsdcca~-=>t#sCB9Yxe2>HT0Ye?3pvujF1AYO%N#}lR(8mg?37BmsbhVVS+ zqhE8)2a#rZZf~ytadDmGdhbLV?5H9C$3V&Tm!l?+yU)3x4fl*Qiqe-jlh(ix+P`tM z7$S_v{T(JFv{5$$XNYZR<1cBGv40_4>kK%xKE(%>m%HR^Nq!Fu6!TR`EpT5PshrM^D z45iRU;M+*5t!gD6uMgD`U!d!zU?LoRit=GqRQNhedccew(!TAi}X*#R7?Hnw`xL$q_v!QnQT1l(59y%lxRz}EKVt@pM=-gmHj4*kg`$B ze`J_Ehr{~`;^mCFW)>d;R3hv2_9Sled2znO+mku`Saw_*;O~$meTRPWNfEmA62Jl% zgnIB!MwfOxJ#u)e&|2dLwe)hwN)|Wc_E@vx*_7rr`8#oS*n`zqsm~t4?Y1{b-(Z&{ zJR&}bY-KU$QhEkY@Xh73|FUH`AXLcpg!cYNH5V@9(jFG8qs}S|V1yQa&qQ99|<=CO0P6&j&vJ=-+e?ba!-bK^W)4nxM%9 z#Pz%WP!^H4EMHfJ0RDxtc~p@L=$W|QHzeD6Pm+*6VG$65{km`V@DNQfpddbribxo$ zrl(;m7T=^K=5lBPpVILNBwl7EzQ2?7Fg=s8@-%&i2~d(+%5bzB63+A=b)XJop`a=r z534DOjZ9N4b*wKMm(^@^D16!|7vyxzqTozkx-qm>wg~-dVS~bqo!8poE^E|8sQo)J z&;Fg5TVd0t$hL=#%`A2gvf;QDE47yuQ_f3UX&jCB&D>E@?_Rw1Ox5}B)KXxbEpJk+ zSpe-?<$n)O%=NKX<15s)RqTLPD(pU18l6cX?n-<&>fH6j17IbAM3+L3Ka2fobSYfa z&UlskW#P3LF#9n%qS461=Zws974WLd;7GF{QabhBVPN?H|9#*3GYG81H0I=DyAjqEM)CK7 zC+j^qC07LO9)@c0z;62uMa|nxrR-^+|D8gdncpshe(JmWe`!(At zYv(3_d2sEJu;zkL^V1ZPS4dyMuuY1dMcEIYePAwM()>XnzuZgf(eY=wIx-1lp(2Qd zd3PSgDs(;1q^^*mldCTXS>1E7rkXXj(7OZ5WM`jZ4G zs~fb#Rc5wSdiwHC=-IoTnMs~f)KN4Aq6`a1LFBW3wQ-b?zYZZ@Ciw=fMASq2ICjiG@*9IKDWNAr6`0wFT`2vJLUja(%8aCp;rjog^rt6e_t`b;Kn>weSS({+}Hddl9B&c}^w z+X+XKFmh?W-tO=Nln)o+8m-@$;$aG}_dDK3K3u`02KYq72~TX-qPpYV@9>EOUV~Ve z89z>(yes--X#)Ui@lqS+-% zyt{C}qhKbxMEs1VQR9#`517oeb{Rwot4MQ@=V+qllBoU0;Xk*oJ!x*^&3l&^qwK9T z4>fSLcstk{}pcw^>V&ioGqjC3WvlP@Yx^(<3z$& zpP;v}#4iaC$YKlG|21_e+jLjnfUMJ4uq9 zcPcQf()hFqR6{Rd7eM0?dh1ucp~9qXtG=#=LVgQH%W$MlPqPW=A{RR?ev&U>MhX-h zY$L^NI~~RXzWBtmNr?U_6~`vBU+V)`Re7go2oP!S2CdQGhGj-i5%jgtf{UBgr|~7( z^c<*3E6Ruxl5!&-!zhrWCA@lwboh)ChC$4p5_hI^qO!H&Qer;0x(8PKvTUuDfr98uc zklj;5xtnu0h`I~|uL`SNbp2`bgB)OVDO`9!i}AF~L&LiiU$8NgSPnTp8%4rhoadVx zcNcRF@}dB+V0;0Nh2WyW$!6fHB(=RY$~O*F@$fSH9xK#YD7iVwO-$!0FRo_M7YJ)w zqC94DMH9~Vi=GiMre2wfLY|)XsSyXp=RT~N3o`d-_m-MnQJp- zmmaTqIC9S-kIcyGl#_~eoJb!##%}t7%f+B(VUG7r=_Gstv{p)GGs=aI zEGF6L1XnwnrkZ4Z@eM=VCiB!!-SPo!X)m*Gk@GV`abU4UfW$>bcE(PGV7K@s?WqIM zbo-euQaFTu{=Q!PA1UeWKXQhD zA;|=cq_f}JE=E$)0KO0+B*+wVK)fz%CJaP_nUh%nUjMp4NLOF|V=1CRFgK0~bK9!p zFz52wJ*_sc=U3n>EL(7^li-2RAo@YZp6M19|PAmv#7Sp87FQ38R-auJFl z$--_69LtUGm&K_#{gc@uk}eV-m*JSeSl{nx_Y0IqgP4lG--(iq6!(I0Iq`0Hse$)vm2?J5^Cw-A{hhS+(4s5|VO>?x9&Y1|cXfYGV6~mab zoLEYdR#l)KQLU(NPRrV4-Knrg*xL7==$f_=y;ns>k&xIj4 z$}N~h=j5)ZZ-b&P#VEm>dLvtIij8?`hu3Z3B zx)PX^Uc!DX-iCY6M&^Uq^S}7eME5qPDrT~`msd=;wRrPT0B-wYtxru{hQrVwQ!w~T z=ojV7J}PMF(kDZ%NA&Se(45cTm6+Ol`Bj+MeU|0VuBv-4NL^afWH}BUMexJ@AV`(= z$J+YwxL!0Wy2KXtEhoV6)PghCFy@*TwP8_n7}wLZirb>}FrgyYFipxizyOsEgy5J-4qFgIX6x^TbR&}d4Vx80&SaP9f%+S19AR9hWa zk*_O7s66|;0TCc=kDT}Vr`;bOt=jwM|BfQ=f1*eZEbOp`e<33fi-&!xI>X=!quvcc zjBJpa`GjBwDxD}hvDXGZ23+4~NO1S&%>|ZiFpQ~f}QjQsEpOnzw-C1zd@zzR$ z+wc$#&+G7WR&u7wazAanzl6|DN7)!BlWjBr=!T< zIX`$RHaFObb8`77^>QZ5$u|t&#D2m*+07zPZ#ZVQt*yFXU+A3ZI;{TAV(S-)*WD)g zlR_BRtXyc!gqL)1+j!h!GA!0seyg#&onu&)d1Aqo9MB!ccCcv?uIww{JjcF2ho!w# z{AMfulNU-O3blE(V6mQN$%bP zrba6<7>X+ZL>w>+hSIiB{az%m16)n?Ey}Z493j7&XpDva`d7J|oR3=C&iikWosJm9 zQC2Ss@-E>6J;{e>PAOK;fWiaK^M#%z`*0LGh=qE*ef)s_E%QfmgwgNRS+wJb4L24f zIewpdHu8m2(+^0y<0>D}rf8I9YO8RHrHZUL3gNbub?MJjaGo@E^{3!4bk!HPMx?^~ zkAg#<+wX#d;GWME`p%z%LyZJ>$BAmFGHRvkAQY~j(6-&-kmc>Mc0LKJVB3ilsr2Wz z@Y55_AG^XAV1DxM%J`zhO2d9OXbxb1c#;%93bn8Cn{K8x|DC`O_r z$yyAz49C}tsU~cxz<1?9{kRg1Xc#6do35s>KtRr%D7#`NlMafsT9HKNQ{%H*<&yHF zDLH1h1#-$cus6Y^A^zGtnG%*t+~dl&b#D(%FB95vvgob6z%G)y^4u6d5g%@&c-Qa#9DMc1F!_GeQDm&2qoYhIRe zmC%F=02F;ZSV^dwJpj)i%R6Ddp~-y#o!A`R58RIR2Z*ocIgr&O<-Ds5}MVAKI1 z`b5J<6GMyD1L{fc%&4Fq)!M~?GN?yX`<;D=jE$H0^3HUj;k$)fsBUMJeP~!B$+esr znKo^fRbBZ&DmP}<)lBND?057JQD1za8@Ato)t&JNz_iyPYhYyjT+lRF-Ss+%l7iKp zFzlAFqyqiRU~Bxl^%3j$bKpEFeR*g>q_KgUsPMQG>M3Z6A@!IC!~w7p0b|DvZI!F6WOq}TB7?~AKrL@Iw^OVeTL3~`s5B>`Jo#3Jage5EE?J9=9KS!b# zkq$)}xad%Orv&T-ie8}w)eZybq~we}bS9Fv|n=~J5=Au(dBojR?Sj8Dh zOxU%lL!jI?*HTc-bf$z$et17)*xhYq^lUUTRk6$l(V;`!{+QdM9CW&n;MZ{nUTKTc z z!$kD)@8bEuEo-zyD%oh_I+z4A_+rFUlBx-L@NKK%u6jZpi+c{x=pHslgO+iDdxd@m zvo1(HZvi_%AQkM}^?qt(R5a_zlTr^uA;;_%t*;yl!%x)#1$nZ4Jz4@6x-=fHDQU92ZHr>W0GSoLWNJEeIya?ev^5gT)vtj35R;Q;$g-38TD#o01%@bj@=E zn{~KlwHyN#uibt@xt%izWLbvMc7l6FE*5afM+UU$G-Fjzoca-4tr)vv($BCNjFTLDm%H_y{vBbar z`lqHB&fUe%A&s}(ibV$F>&?(D_Wi-NTaVX<~gYTzY2Tl| z&X?9KKU}|yaTwKRqU^aZw$M!Xxc8;9*Uu*K>$lp_vYw(BKCSK{No7=Iw!SRk;_5l? z+8qO*O|aL;#urT5;%qNI+D$Y3Mzt4nXVmPVsI&ga?EXlgx8>*wYMK<|GVt0;{sF6$ zt4Qa`)MhLx@x#Y1M!LB$JJmS>cY{RG7@61;XB4^enyN&12ynPT59lZTqX@drO3lX_!tTP~-Evj(z#< z+l-W${u-&eDgWl}MVPvSuB_{mYMte-v7aL~oJNM%Czbm8zLOjMSTfTMi$e?_PT&(I z41`?>l^zhig+RlY1WBuxOavq!*;_fMV_Z-$^L%Cv(b1H8BqNLiM4R!-?|v>)EIx`h z&)21u<3KKw^mIsglB;%Y>)2~v2ivt8zvS7msGk3*5>d@X)?9lzQPEt6T2u(U#jU(& z$1ljcuIf?GMbMo7xd>HhA8eOz{1SR%SFn1j^XtGhUzpFQZg%@lScP=A>jud{k(69d zP;yKRZqIh8DKu~hdQS$0AAg!H@T?%BKJz;!94?+dULEfdcGy>~q)(^;AuCyQaab6~x?H@vVM8Z@5O#4SjB9(=L(Twu9 zt=ugCJDUHD0{u^;CH5MGT6lpd?GEs3(CVU;cOfRo2p%vs>IF3^3NdaKFfq1T27MT5 zPTa>vqQXBJA7Ebk-~=e?7xBvi>p5RjTFhTy;x9W9mK}meC)}sH4}J5myAOrZpI^3| zKfE~_^n;c_H;6G&7B}jPhBg!z6IT!UO=Q!qiocBaPO(jgLh=!_p-4{41S80D&2RH7{Z{ju)#~OpZF9lz(HcB2Mn&W z%y`fSj_<#z-#DRh4;IDuCHC43~4PBP0^Prb|>Qm8qap@fl1T03NGKF$;TQleWE|C zIOQIBtY#Sr7#$B>^-FYA2rR>h=M{>&6t@*44A6RNRGlUA<>?fR9!3tJ=;#)_U|Ey3 z5Pq>tU_{Gh?$A~yT=vakE3;-nSg6wOR#yv_Ren26(q1XHN6_lwK|QJNxJ;2QYUF;D z?V`s#<82|;tixbC&?ex7jg+bxS+ce$S;3=C$F~qeDi`Y%U39qru|i!sYj|O)%Iy1w zljo%-6P!KujB(SCXgzgz#;n3RM>^{M$@!UF#U*+J{8f^RX+npxWYlW7hm~I?tpFNTg3`JYY`(-4N?1q!PCviH zF03a%OcHq=KD@QDXr2o2m-rbwf6@*|7SHU`WIrie7yD?Q5f9cn;>pRO?II;Y%ATyPwJN3W5CPb`EJOi6^ODHfieS6TwMcBymYM#R-A*`|>!MiaLai zt~Y_D$b(C8J(O7i(MbE~td#9hvf_e6;Sx*U<2KBDW`*$uv0maRC9%+rPzmTY@43H! z;hgypgXk<0q#XvoHDmP?PNzI~sHZ7(YvzcFrV8SAo&&DJQQ5i$UB%m(GVNkT+Wd)r zN_a&>-X|x$_&)korAvB!4Hk7oJL2arsNbSqt%(sq8Qd~;{kM_oUkf1qIs5j{{?fl> z$K1F)lo%S!jK13EU2*8RI-6Z{uH-m#jQln%LwXjEZ?#1HNi!bcC4sM5NFrYnFos3Z zq?ie^YHKwXxfa%bjxz@F+YLb0%QB&lOGfFEYe#XYkuODhnuUI)qA-j3N>eGxL^p&> z956Rv-=DPnSVA=F?(bcLlZo5pJ?-OPcrKKzzcu)*3O`4MK6mUv%mA$izk z9S~7XUMWX6riM~xOAjCn$B4;V4>pP?8c3nZ*{;Y>H6F2d>i?^>Uv)z$$4Ue*N3JOv z%C`!9x3jk5yN8%O;MTyG!koIvZ}hbJm02c^R!tb~yubL^maL=3!(}xQg77Du+E7Wk zSO&=}ggaFCR0u)*jst>YUy9kn3L}K4Y)eK-WwGI-7XJ#y`77+;E8*SJha;918dHe4 zKb<5A-61L6OQ~9(M#|DQ-7QRHEE2w9>zM=w>D~?8IFRxAi zpIxudkS+=HXfhC45PA^G6q?(^k!ToG{2Q~HOp-|moN_}`cAUrPmAMy;pv@1gu^#+9 z-cTW1%hCeEIU>dG*5Mxg`D<(KSeax#36I$nTr#WWWoBs%KPl#7Q*Cl^wJSlJI5j=M z>0?c~Vb-yreV&gl%^D7-Ie8WP1=Qpok4{FH1SE9>8rQY@PLw9G?HwjG2X@)Q8Sulg zf%Jp1&oKUb-!^qS-+@}5hgazuPyck4hgQWET!39t{cjD({4b~dUku1j`9G z)C1iBa-6)>$FJ+0_Y>`$hf}!`!=3(b^mTdaZOldo8ltcrNb9T;gAA-lXWyyIi1$@0*+xX zN|rP8hfVh7I8tK76IJ}3CBcFLMZPxBBGOGieW`+zEYcCI*As*wW|Q){Y`8%==`(Ma^7T{*VFZZZE8vjg&VlyxZVQ!6KC~-cPGU9}ws8J< zL_95173L(qUC&R$RZbbYtSf6$U{N?FBz^7zXO}Km@+9t`zBVzCo3U9sJz zp7I^^#NkIX=oTjmNaht5Dbg)5*+NZ`8>#vj1v@2Kss4og{Z`;MhchL>Z{_%Jf%u=_%3s~3|6?R8(t(d+ z-;ult`rd7*(@^Y?Y&q^d8Z{6~x-ElF0&^Rbo*B^Cc)huHd3%T(B#;B-MVtZ5Ai7|H zAVs9ZKDsO&$CsLsCn;vgad>HG%O$9udl)0!zWSRM&$qXeqCIzWuYTWoCeIaMEGB$i z-<&ELf$sI6*unzw%}EPkH)^inCR$ZX?tT8Gj(po<^YZq+a+y4Tpc*VyPjFaOt{Kwa zjUqK1GXiMb``T~$%7;2H%X`C(5W;+75r&gpXYHm}lGo}+f!=DL^(!Ik{m|+pV)HkV zCFxAVEim80!DqDx-(3F3Q1+iK_*Yxu|7byTL2=nw1lmHl*!%O2Bx=iSCjD*&Wh#>A zHzLtjbmefLC(g2SbMH3;>qL6qtnY9Qs0CRCX$K|4V?VG{goQOKdvjqUC0g_;Ce=5t zD*A=lmHm`2zu!Wl%$=Hi3EIAIo{dL6L_J=AN4{}&c<03;MFYQS{#pt@naF)okx??% z9m!=*O>h{VYFFMWby~OM&aXB1J%)qq5t+*197`Xs%$kAEMh@PvJA1hkIL!Ec;qz(Q z8a$qPElWmJ`)VmFO&@!au{=BFhu%$}cAh0C?Q-T{_1t4;Pk*kLwQ!1X{$~r4{W}@B zxtaQ^|AQI+5&tb!l2v+RoX;ZkG(_Rfoe)V~ZgQ$-yvpIVaUr4`44z9!;Ns z|MCF!R8-tp(bW`@nHxab=-i{6FY|`-Mo^NZLzt)y0=@=K_%j;2xVX8?wfGULm3=*~ z9Qh4HW|@6v>OGG4uzZ!9xQ(!byIiD|{CJsNvivpETu9>P2TF2XC&uJa8B6Th1}^!! zQs2PVaY=IcjUJpnEhXyGxsufp3%gh!{bz%R@3~Aye6{^Mz1V9?S;Hfxu?y8m3@TyKavI|AE8@Nbbz@^4oA z>&(3RKQClCCiKcfv13VHHqKk|$G0q2++mhBBGh8zc3>mo7+t||-TeIS*uPBA>`P0^ynDcZ22 zUD~qJ-ftW{E*=wA|6HghX}^$PZ%CYXc9~KAggh_^rd~7^>|oa4n4NcWnZ5#BM&)m9 z_g{*gf5{JVXCF(8f1RZH|0!l9W>xf0kqC+ z!-6q1A)rO$*!Eqq&j3tc#*q0VBSjz;xrV#XW@j;mYwNz97&m8khNY#`s*H56=s*(V z;}@Ce?USuLz~qPLQI@EN$72kdb0z;ddq9vI>#8=TQg{x|xV|p@pW}mGk zc83)wvRaa72AGa~<;Mp^w#Od}ho^Ff#C*4?2$)HcQ;Mj1xbfU4O-c!Co=s%a!n z2)vLaI&36m&W)$YO2-P-n%*H~8dlE+4zrlG1T-B|#Sy?apf>f)_}b$= z!2I@Dp?Vh6?_jgY{jK5tOPc(b$0|E}*!vS#Wis$tWtJyevOtcFr`ynO7miHX6Cygf~He2dJORE#;Px@qP4$r3xN zdue;X{yiHPE`?93ZXZL>$xw%-U}|Id?O0anIDG zU#nkL-JE@V|KU7_@#Bags^DH8wd)id$(WG50&ZfxR%dMd2WT0gj9nO-IKO5bAyHIp z;y7-0HUQ}_OQXwS%r7{Y!Jr`8ZKPO)n8hcTC@qjhMOGt#cQ!}sIGznv;Jw4R>~+-X zTEP4%ype7z(P@CTO&^zz4mFZi2}h(GF%NX-LD=>O4S!nGyDdHxBGjmTy} zotoU}V0^!(mzKPQz1A4|R@_Nb1qP~U3j<~i6Pf|xS7bWTiSnx$k1VyZz3k&UyR70G zy%Py;3jlmG%rSS~rdG^6l34q+o4UpE@!c|O5wy!=Nus{4LDux_Q0+N;6A$(M^T{`C ziF;=#z}~M8Gg)r-zzrk^os2QO{-gU>(EGRE+R=$wE@Mp#Fy1GHx}|vfbW}jqJyl^W zM`%O5A_;^U`>ZGO%BcT`w6~1Pb4%7mgS)%CySux)yX%L$26uON3xwbjT!XtigrLC* z5+K|!Nq4XAy?f={GtL-!^8@%Xt7groylPddsmx)9@)U5At_u(5CdXSR8>&K%ZWI0P ztB3Su^ImA6%M#zcOoN*9T=e{#9%7$+Fa`vQ*^(I?3p&9ZAJQ?fCab~o*FtzT1d?yR zW=9^pol?#L3UuoqeaU|-oBoGT|J&f$e-mtfbv?CtVQc{#Ik6lkx(rZJ6me<><*R7S zD^T?4FU1n*mlUNS{iQJFQDm>9r?IgMP@k*HbH~Xj(^F!bN?z@D*qE?!ZJxHbFVQoJYqfb^O4!iv@tt8p#Z2KHY##20^|U^ z11mPl%v(bc5q-3o$Wzie=_s%Gb@=lb?OrjI7%yT|7*C1L-kL~!N*sQa`Y2}mLEtJYDNwyFcCLD%sm1-vdWbw8iy z)aDF-GK;{iDe6SU{aAT*PKx_M(RqQRIn=S z*cLND3j;smYD}J!J;#;Z-cB_tJ9?qQCCQCd#d_T%p^&dzzSP<`HQZu?Pe*kjJ;~(J*}5*aM$v4B)dli2 zNImU>npqkPTIpVj%m4%Bu4KLo&7_AX4s>lLPq>a4?G-P`N%eY!ZCXi-Y z?Q8xIs-I2c=*hg(s^=r7Icl*@?)Vjzw4`20;E~I(M%)!TMm?wkl2hg|UWuJVA+*!b zptq+tC#QBLkDrbtV^XHrvY#$*g?`!?d(iE^;J~Ee804RjljMIP=dV`<^uyGFAEF?HRlWwBfXb6d$KWl6?u(lIh@EFvR`>1d-}}h+U( zW7;Wm7dnJ+u)?NJNU0W;^`*RZBk3prtVJ-G&2qo(Wy6(bV$A{wZ1SkE0ZD-K|p-T#c*TH$* zA7M~9g5#Gw;Stz`mR_;Dc<|FS%A~MkWbS#e=@KU&6He5878c%L9c?9*Nb-02lL#bw zpe}LGR!1rhxvM1%1LjL-LIDzd^Wh~G0ILqZgT&ynq?X67b;~)(=J&CK2aWYGo3o!6 z(z0#I&Tewn&L$#l+H~9|4*BHu4)DVUM_AUmE)y?J`O$VdC%ya} z>9FqKu{OeA%awv(G|T-l@?NsqPA@cpp}&=>-;=(diPJG7*uCsQm1#ql_3|BryyYA( z%sGRr9;HdV4{!RBU>&qj6Lg?(-*;5MV~pt3XnFx1t^E_wF3fUsgJp4kjqmur?4L&i zZ#2xSYR5^zB%VXhE_3J;@Tu3pS^&5oB@ko@UhI^29OH6e=#sXIdY4ZG)Ca$+h>5s+ z_OqJ5AYAt=2Y}^D2M*)PlgGt5Xu>Xl@O>VPiwBoVC9pS(WNBh^V|HUz9-1_FaArQsyuU}o<_-sY&4c_jw?dV|nC|u$^0}sTlIcwVuFSw5 zxj?bg0UfZJUhore=%LzNL#fZLo1bax=Qa|Fb?Ls7YQ;hksRGtqbgU;sIz_qq^EE^o z2s+FvV~E|o^hZAH_rV8B)&|vjEgb3fOO&dNL~P&GZ2|LZln^-|%JkOK?LNlX<@qvX zbg>?X;+&0gkH!v^Dd>Q!+HA`By8pPEg0-kY@%5SMR-Nwy-Nkh4`)o_;F7QElDErWt zqQ=L8ipi*`f__7^RLj9smWyq8i{(3~zL=5P^I-*2(;*8O1XE&bD*bB|PfgZPK3gqX zZ^D%ZPNBUp&&%8P@4NYOGm4^sHfV-Unsq0b6B+fSig2!j*@;ji)E`c2gbw1$sC`cC z2@LY!qiC}?v$<|}m znWgWT9U??LP}f-`?o^ugxL2)cuDu&R7(*G7J@KBGPWM`(o>zDcqdDNw9zk^)32mIh zV%eMT;zCZbe;f70x&2#i~nQ)~jvoM?cAU-1sSSx}5qSXAJ6bna;M+E64Nc{~~HVzOo-9ET&qr@sb0 z5+@K5rN2f!D$5u#R)+?IEeQVvx>1$MQXKIVrcssM(vPQxY(n-+Cd`*!jr6o@7PY(q z$^M#!nQ2^0A)S7bj}V|z(xQ2cVNyqT9Kr79Imi`x-h=PU7#p6fwFpW1;#suKt&WaS z8kCYEr<46rU^->*6y9~Xnu9%{bf4#W@!4oyXJJ(d#Rx*UYAcL>r932r;Bi|>Mca4M zQm{}M(z(@|k@%39H+^;w8Fwd^EfIeQ5_dqwCkA#)i|ybKq7hc_AVe@#QX#QG&49sS z!I1Wap!Z_CwDpGY4q9;SY`P4?TriP3|8^ClVjPnHVl|FTpx9qmorZ0s%mBb<~l{WF|2 z+yj_%z%Aj-UW!3VJZSK0fu2(nMy9{*es8wYuqmuB@SqMGz1IhISyVbteY*z_e{+~) z60j{)VDx6niPV|gnLbGd{t0?EVO9j6OgMGtjZ83l2sqhSIP=E`+cKi&hBXb_1H8}g zT!V;-rXj+;A1|7?Sf(D}f}C`>liyPFP{`SJFOMm_w_LymOr{uWYnFD=nV)N~ZT%8JlXAE{hQMWskkqkl`xn%MUx&zk;_!SzO=W=*@P-rQ_;SQZg2bBZJ; z7Yy=~2xL4QmIW5X#K+3wLE|gtD2qYmYumG+HjFsoqw!&9N8*>ZA?Dr1kMO64Ps3P2 z*5viDxKH=G`ZPUsq!d0u_Bf%_=^25&?wR5=0a>L43Wcet_fHbJVTpZG{YFeLvI|;>#7b= z<~&sUMy0Kk=_g%(gZ{qBs6<=%2UM5BSNJbuOHk!TNjyZRCtcm&;Hw2gU$}?AQERGx z+WS?~cdev@qzAk#{y+M>#DBc7-@%PIRRg63V3xdbyrrYrPeTwL1MB?^W>Kx9SPF)Yml12J+-fum_b;zRfrmjAEw4t@f_NjfAb!93!nayhrR; zHBl`v)1i*0^MiJ1dJNdwGj-NQ@e!Ei%vMD#kWvH55NJ1?npfHbYIvvG_4Pp(cp&s~fwgJj_l(+P?XI7n!(3LL;2B$#2uE zU?U`^e4{e!JmIo#B_v5>JZEL@+PP+_U9v(8x@iPcuTZ8-HEGx~r;Wc#Q`{iIs}aq2 z8)sIN`=}S9t}$BRSdvu@W|*#^tro?zbv~4&2)NU;>a_TI<-%+ws1~dAOc}s6AkLA- z3|UoKzz6uysu$d`A@hhm85J+MYb0x1WYiLFy@v;m-mnb+_PWk8F!mBtT6FPx^ijWT z?2LKA<)k62!ptOm7)J?Kgv(~89fV`zRE~6d_LF#vViK1&f*6xjpA22kRTcih1 zy1w%Zg6gA~VB~2$w0?P*Y#ysYXxdTP*Wj62WBBRB9iQ{PSp|xTJJ6B)`|ua#e~O8M zy9>bGMcTyG`mZ%E|61Q-p!9QKn%X*J&HEPXD=?J>h-ga1lN13_8hO0iJodnh`fDEw>(T4fmM6U1oWINma^tMqCh+EyU$>IPSk{w~b z4{&{MdqgkP0Dy>M<61>>QVQc6Zu)HHE=x8?&1XdfJZ;mq*{$#W$_}2bAK}Nrp{z`) zz3ANrJJ!aevjO3nPq_XoLMcU?jHsLkSB>Zc{fgLh@4xbepG<$zNv!W$_BCPB!I^d12u2zGaxv2l&f&|e zU}=SW;&?5RZ~tqeW2~nhs`MHZSC#%1Ph)4REzziqyRKs-!EUp4 zD*jVMj7R=WMD+Z$?!scuOtQa@$)BKA4VZ8>z{8o5Jxk+Kn0^D80*58yR=k%b4_qF9 zk5aVh%G*S{m;)3Mw10`nK3I6#>Q525&qHjf$Z<5!hMNOA%d+eIo5qzl*eM`Dp@|wyisJ7UVF{gow z$$7#gmHGNT{FZFuCG*`>bYG`tmoXhvKz~mriNCaVe@Ng@6e-ZZZumco$X_d=epv_q z4MkkN0RRg(;P%H~;PMYlDO39`Fk-C+x-Ae%NN}Xl7_vdN#H6aG#i*n(J5a`^e$F4^ z&yruPTRH~6A9V=5HR`R#rX7YVQKI}ghjx<^c8SW9?eILe-Su!{bdyL(C=i4>#sb5Y z)}gBHy{X6$0Y(h%`8#b@6fS<@ig za2k2*%p~R1Lb8foN=7@kEewR4OykHeRLi1%Q!@od+lNs4t!-+vTwopaoyqzHuPK50 zTUg;n+0QwqURg#T@HX~7VsUrd!1+&J6-0H-y*8>E9yXX{#UF2*Nz((HnIq0?jg3ni zd^g?1OMP-O7)%=vZ3Uc+Vm1lZq*bBT3I=8xQM^2M1opn!vdEiA1KNYEA~n=7H9R<% zTSi&(b1WinbSG_RYUr(4#EMDjO;XlqH6j-*_7C43t|jsEe(s#KsvJ?IJsfd~snweG zj>FR^%7H@YvV_2xuR7&8Uks~Qcl{o{@;T^>kn5u2e*)eQ+a2ugV_vtt9U*Tu8sa8{#SeCJ_aFnrb#YXnCL?piSl^gVVYyd-7 z$pZLLA#-y(%Z;hbBO-@K=x)t5!dK+1nrn=&XpHzpY}J0^)1rRg!L#wHj!qJKY>Q(k zZFofY0BUB@eXJe-0GROqIB|H zNsJThvj!uy3g)mCM*zr*K48u9-|NS}5~=@FvVWUUWg6RB zcoOJ8C_6@z>sHu>ZTb^T71qh2X%=-ERzz)V)WE2f*-e|rSxMk>;x_HMP0vE`ebJi} z7O`Hz6AqhN5Ju{4iJo?nWE?3 zS14hY@}*P=W^|ufcajKn-w(*pTz#4+n|mk-94ZUj{^nSU$CcBg#$2`|0HK9p4Qq*h z1Z#=m1fc`r3hM%#%zP8@RctPS*yja$w+;CN?3L~tkWNzZz&GEHU<*O0z`)O8kROKL zh$SfS8|o1cxVbBgy2SEh>BrvKZ@=D0&z*jE zpRa>Igb(@9V>%)hrZd%BX&hAu|1=jqBqjHVHGr^SE?glBl=Qt8`)ZJ5t0KzZb^`V6SZo#6%6LqH(_RR zkGv8Fk8^iQY+VnNB*isF0yFEaB;D`&9_5|7gLhGtmKZhmqtSQg2t0i=jZmd+snf2X zINg>NSjtFyrgoTkv(MDOYm#d|e1+WBix~78=pj`;Bme|GsO0t+0_^0>8Eh@15-g5! z5ohzDrJ-ZSV`>uHa`gf29}xpOgBp#oi&V&LuGE%LXg@!8ocAK3fnf;q@iPwM!N`nn zr}SM?$`qWk!^mQpL1Y)!G>o9z0LCFswPx4y_3plicH6BlZs$2;PV7VlbN1 zyT%Z}GfH{oyeoMtmsThwRwPu{5N|9=TjJ_Az|Ek=@S)D!MRgG@wY{QAvs$kz#u_f3kh?hTz_n%>?c?yz-rfHct7yPJ{UH)4Aa>yMKB(pKpRX; znk~6-kMegsBY#>^UhXF#0b?3}4@!UKtp5in{Uc|szUeZriTp#cM!ZF(j}ydnuJoC@ z4~5cp4wG832&Sr!0x^+Dcm8fqTm@g{1=wUR<`pC;!nB69Q;r3QP*!5X_F2s87A6Yz zH2!!f4VX)W&%bK5>e+1Uuu8al{`UPI3z*$hICx5<+qQ`9F^aQr))+dZJ+IVOudgwc z7RTiBd9l^8jL>a-)2+a(xkFJd2M3WHENo7o*01|tcG_=Z(bITvUll%4t zlopf{T;BEpTMbumhzi$XFFM@YuX2}7-mY?B!AY(c;8$7{lpPLXVh!6yx8MtUZl*bq ztnD9W6E1I2+{dh)rR_D?+IH;H17MUY`rK+a_wZJviaj&|xwI+`@&(+vxWQ`L^y~Xo z6NSsqzKq8oyFq z!^o*geJK_muyx$F4u2tC<*QjzYc7A{GR_pf z{Y*R2!)@2Gnxkitca~p_oag$DeKwKY96H-_oEJ7lvqcGU7K4w>;Kn>V%3V}n>2jHp zd{wGkRjE!|1IafKbep3Jn`MqU$*2po8kUWL8KoxE%p^m3-Ixd5i;C_6a)=xnivYpi zr2O~=XG9zvw*{~8&q8~lA`J@zPIi`15Mrx+&ZMv%qq54-R>#<`*rj`8x0o11bg9F5 zGHCaB(<)m=pWpZUuX*se`-|pb6SrIH>)SVHr{1fcW?xe+OsK(UqzM&1zb&!&LXk?S zC?L^N!~bdYgbn>zXl3Ad>B|p-)OW$hy9QMl)lmR(+mJ>0#aFucu2oRU=@Td$0y!I_ zPjTBv&kO4+uy(1~VQjMg=)=e+=4O>tJm$59Z}+>Sn{F4BWD8VQJ&=&t1#@8VXwLlIUI68KEdUz#xC` z4l-_R-^>gpAV?WiHk+NY6dFF5^SwQsDXH9WGLLzI(5wR1s#^{>!+xq-wtL14h{5fx zlzG~1oN^tQ!rfDMSeY{{{ zeUKpvklVX*DBGXIp)p#;?ag#TqjE?+>WPbknYb^-R`IeHA+kcOymH0Jd6HLE?iXMT zI9jOEUHEjj@&)O}J<6=@Q@$$1oAZdLkmY3g(^+9rooh7(w`j08fleAKaPCi>zOzs$ zV4BnR?5j*K*Xg>E!(QD_p$;#Ia$_xSiei-2CDYfz8)oX{$ah@uFbXkkBF0o(CL22F~EK{hS-G0p9N+e!6=%Bc1 z$)zReZtOf z0+2cu^Y>%4^AY z66@PgL2)4cNlxx{!tk@As_Hdzjlh-4+;)j7T@tz)PdOpUyF{yfWTIz9qp4Nu-~*GDewI~=IU zb&=kl9|HR?@D~!Kjx@;&v`~?fe1ZrgApKN3#Gdd^?&Sc8XYSNQiM;r@o|V0FBoTLf zO%jQW+-|)^eagbg`1#ORJxP1-i=;IDIBrLyd%_KMU>1-wFuKD7bWm)u6Vs4oU>+e% zzm*V_F3yw8MM2CU5!&prx%qfjso+@;A?6W8IOamuzWnlZh*i4pg@H!JWRPZy0xKB8BOXw4G!NEj%}2^3)>a4zuBw$Yw_6g2RL6 zvDo7cWD2tS4e@*(>nqw_$x5Zd=w4{jk-nbKXIqwHAcb`1UlHk$o7I--8h6>fS{@FQ z6~KNk+|RX=y8E%?P}{vO-POKb)@83Rk+dVYs2@JiCj>d7m$LlC8q0z3Ei*b4g;*@s z7~8houyT4>e&{HIJ%h67b!@@=M6QrCIqF&m7dePDj=yY94oojI1Rls%o7&j#PTmm?O zwJ2D9dw7@uwXadmR4IhCY949RBd=zLGE92^T#sQ#o1P-2nkzfp56`98lhqdeHAH zFDN(kDLW|#eJKnAKq)NX#CjZZ4xxyZY_^|n-$-pCv}LqEJsnB@~AD&<%gd*Oc*Z1jv`RS6nju*rD%3# zojws3jRpTGL)>P%qIU`z75znDpMh$RB%Oq#dVGjGee)YQcey|Fnkr(xA%uXp51zWh z9-R}k!Nc8HKs)ebe<3DOe0SS|~pe*QNN^Wk?A0URbW)z?{KGH@KYs12By183Hi;*jYlaqb7 zH307AQoo89J1*BBLD&`;<+;DR@rS4zbd5EMt-v-5y7H^%n~OrVNQs+spu!ppQGpC0MO$TiYU%{@uaN*NpJP0a#bh+&=n#3=JjN}%LM6gB$BSU5d+Z-S4AL;Gu?eBe4+K#OqL>deKmDm}Q^B19dg@ihV zKr-h3Na$ZE^7k})ctC1;Kype#{wQ1se%_l%elC{%Z?^=Jm>HmGYWT4`P*u~`1xC5@ z_o<7JtYOi>$iTpvS?k`&P(oV%I^O-~cz!+L(d_^)PlNSGxmcNv$m?NV2cm9j{z%yIgSh>3ZOC=otaSIxXeTaQPJ+nNgn%#DZpwI5?j=D!qay)CJ zSFK~xy6S)zFxdz7KwU2p>FJd+k;BDy|LGf~aqx`PPKn$Rf>bqFwKlnW6S|Zj#Vvz7 zBPv<3PQGK)yTSNVt{#881bIsp`=&&lNxfD5(XJ8M$yj9<{TOJG8ri(-wX_*aUOFZH z>|kWdnjN?zZ`;)F6-1$Vcb7W)qO?F4m=MQ|9^{0bx5^3Ybx78rnCnhPDF-}VL1=*>B^MVomha_#(A zi^*304{fwJWv!}d_n{8iU1*dxcg{z$c|zl|p;MDAd>rLS!|Dw?hPKU_+rwC)eX5T3 zB*n7Wm>8Z@POsmZZqh&X+8%C&y7!?JfNSb@vBdD0JooInH)DoT-P$oOn&G2NlrJ^I zVi9m%MOXwT%1=+CUkC>tuN$vt*od5BR#S|v<4RHQhe$CV#geC@j@20iap8iC(Oavlyk=R|4kd?rsdSQ*hVs~#QsP|o zsTr{FctVV#{cL8hY_n`}9ODKP9r}*AZ(&>^MavJZl@TjX)h?6~iu<5!RRR$XC&$Fb z6rR*0U}OXcj8na!rYv6)KNu4>`x zZvVfGf`93nfbGGF&ccPgx!nJ15&U22nu^hJqyMw6`RjO6_Kv0|_VSKqcE4Y`L975Q zu!A^k+d5!DGNC9AuBA$RhSGQ!#xg^K6LA%pt5~b?@(rQvM6#$;w8_@f!@-Mp_ZxfW z$)v+ftVv z1S}>V*9La>&(hNEHv1yl&(;3!JtsR0)6>vo$a~77Bh2wwa{&p7e!uLnBv$3>bRg4v z{>b>hUZ$8Mz{%0U!odyL>EdBy{x837YBC-~1RY|>6=AKXC2JK4j+3{(IaOOmdW7Cm zv+Sev?1`-1-YPx++lcH+2bkb6-G%n{M_PBq*9fw=h_;Bdi2A|n_6V4!RQ!{MMo{nM z9vloggP2_%@x#_8I+QAZz`h8)8;1M#R#)hv-l*RyLCcPFOuQt~b4|;Hk2ADCv$aT_ zg8 zI8upHftmzig`PCU9A}jr&MCQOZ$MUWdW&X%>gRi#%7LC+1oGx*U(nwR=3g-Xt83vu zd1H|93+BVNn@DI<3pmfh@yO{T2V3aG!@#4#Rnu!IDQV;7@p)_@w9NWIxY7Ha)ij3S z0@*l{;>@(prdI^`{<+1xYzaF{#Mc363kiP4R^M>m3Qms6Br z1rywyNapne9_Sr3NMbjtOK7Z1ka1+Gcr~whD`Viv;9va9T=-8OS^$wB*iN3i@Uec;_7sZr2S``j&SHu>cv108NyRd|wQM$h zj`jd@=s#wN(4=m6N7e9_$tkU zLr%6AK2Q?p{irOiM_n6U(L!rHV9kdJvK?g%r1p3-4AKw^bwr78O1DeWf)+m`*9JA3 z2ino;IrDp-RRC^`=rfQ##eZbrUt%cZ;r&1HUL!#cwqF@u!qL9@2i*y&n`vK_s|Ygs z5Cs2eBR|c1ezST#*Bwy(RP0x2;(`=8#`di#tT$p0gtmmZgxR4EAA~lm;*6GSaG*jtx&ph24b2mTk)KjQu9M6XP*qbU`rdc#`LUGG#v&jRKaytR{yyluPqGpn?s^ReMdfJ_R*{B>w@YfE0?KtdO zn0Pd2w}XT(O+b-#K2eFXl#QA!BBUwx`ZXE__s-Dsr|U@f$D91wtN!yTF#QY&Fe#h3 zxLU{qsVkZW`YRx3pEBpHgT@{NM%iXfm6STNC|EgKQ!EBDx~SMbIj7Do z1tAAKn&~p`;=ZKvp@la9Bv&RnPet&t@3r@}&l&$^kLr^v7vRiHVq)UQ)RyZs&%NhV zyRT!w_tTCG5c~dR!&slIw^$TRgm-%=O!=l>*}E3uTG(6WUNO6Xa8K+l3$Fx|;9dTK zCwL-ezu`-s0W5eUR(V#xFCj!7e%8YdM7pyV1FT<@b|nhniO$iTTHpGD!^1_wa1-wo zz7=CNAGDyxMn_IliFh(!`0SAt?2s`_D%xx;+#4DD<~(sWJdc%Ofog{lXS`miH}UP0 zMdiw_6xki@%7ql#xrP(=$6z&XT4Q-(&0ZLg^kAyX%(tH)a#5T#``EZ^cXUe8`6EJu zU8$+P2$5*LKx9B{Aecna64fq~-)cFF_t}}=x$~FqAfRLTMRo?m?SQhv3ZkMB!99S7 z2P-+t2hSz3Bd}>nz^GP$2U=aGYC>PzcYEoI0xIp4MeQo-eDwLM!VnlTV34TdZ_J&a zOxd{UJWWB)eun0ScnLcIYpUE%x7fc3bDBL zK-V944#8v|mw<^pdNP%IdqO-?iUVsU8h z2g)S1GpQAfEdwG(*|qf}f0<|jaTm55vRCpab~~-4V9JE6eeF+_-}~A0W54M~Cm>Y} z>6sMz*pB!FU1#I4Ga~i~Yn(9-ad|PSm6BD}eiwcG!V$CL(`eSn84y!p#V2c})ok1n zta>|LeYj!nUZ65_;?y&w#?Nq~LhxyV{(YUP%oivLS$>SqROIwVta2xnf|Bul6lzfP zXXLH0#Wn-&^2!P?vKGQJc_?^w{W>4(7<=bu! z=keK`nDds z<@vBQPM=NFAl(t*F121o13KFddJMYyJWZ|58#k!!oqUO4?UKffTAxBnTVaEAmw-wj z#$fFe?9sx~=!8%WE-ClN%8r#*``wM}f{~(-9gIDjpU} zt0CHij=K|9(cB25$K%mtC>8pl7&UbqY%hzd8=56PMqDn8mfm~Vm8Y@MihnyW;1RYk zXdTl*+lj#o(-u>cp+;v16Bp9~LuD{E4wRNtlEuah{t%#@No@i?kBG^9y2H!Jt34bW z!pq2`-KPP?EG=iuE&smN& z*0D-jP^P1MVVVtF?Cci6vNFlN7_^~|Gv_^P2Sc_T&cG5IWypljJmFT#Gb|o%nOWTH zu^}t=ZdnUPg0LAiJnAz!TWW%Q>h5`9($r4=oh>{C!+H-5Ipj^LZHUh(PT)HWxt419 zv<(N@_u`dX2jq+gig}Wf?lzbKrcKZjmbeHJ;uiF1NaOSV+vMH50r+_NXd}#l8yfGU zBhS+$7x|h5B-S8tDZ0|WPFKXyCSI^wjy6hbX5CTxt^`n0(QYJaM!>x5=Lx>-m7+PzaDj`gy4cm5M;UOC-5DVizd#)&h#USdEulU@QKifW~;@g*h3#FM`lPi z#|6VXbo{6&?IdZDQ60@qihOM#CaY!rs=DnX|rXu7}&e9C`!cR&>%9!qu zX^qFqF=#$LQ+A-U$kQmG7dshK%afbwT34t=@F{+k90G|{bKj(xq_jiKGzQi4bo?fh z@nKN*CUfPhsG$U2@W$MFBth$k^%L-AzIP5g%h`<@3s+*YH*DXNU+rF$e@WcsxwmI)<7e z2Q#SnxC-kv?p`#!h!yjd^Sg4bJxQ;k8COS3)UrddaF+V=qbIM;KwBxY|UMyvzMURIH{$G1Y5EtLoW1^IUy2 zX|By z?_2@vSvN@m-99Ogm?IlkVkc8cIDk8}snS+iC~yTpn3r(0qcY1|tfySwIV&Xii1_0K zaglGK`-?e$xponJEX}!bbtX?~7OF*c;4@g~E85Z-Y(S0+OvmrV>%K9K!O6f2j{D;U z|1-k<+fM(j0n|-gtStUVTuSU`lIM4x;ct>W|BgTp+1T8~+}O$(vL0LeEjbz( z-LL2xC`d1CFi=;e0KdQYpeX-CoBicJlK&M${D=P$rt*8H??AP(2(n*MF=HeMCMTo5 z$VOtDa$Z^L_8WW?t%36J z&Ts$%A1*8}wi|&7{c$9?A^IqJgET~dDpV}_vv^E zVDoJ9W&EFYZx@Aq#t1M!=+9yj97lSj?e1)I+dQMzrv2E04ExM6o*sXzh;7=Og;rJF zVm!jtcA@R*{>9CF*|VK0&6T^ZZ4ze=Va#cFJEy;Ew_L%ttIBT~wStvcW$R*HEPO)A zHN#R#n^4RnyU!X{%SBJnAHpu*)_I-E(Xnp={s|6Uf|gH+mVrui+!%VcxL?{Bg2t4` zl&7I@mNGIHg-AgL6i+)JCK?@CIvFk0k&a0^wnzZ2LWm+(VS5UuahnEiURAh>RnnfF zN`OqC3a|gjnp4huk<<(z;$KF_pJ2zr{Z7%SZn+2y9S^X_)-4l`$-%+#5hJ< zkDR0fFmXK_^8Ignsja^8Bs9C8G&By5F*`syy!Tvxr*l!^J}LZ(rqGYP(<{K~1hC4H z*dQd5v92|rI`LzlrP;A1$%%L44OVP<%1RadD7wgGI9cq5T7cpRYw>X2sgF!;N#@Y; zQq-ntss?`KZEA_qip(SD2Sx9Wx&x7XFaqM0kU?}yQ~8z(kc--lcD020moZ*9m(A#< z!~Ecw9|c-RU3B;BX7`ixb%Q*u-k8wz@FOOJ!Exp}jX-=0pn z-XLr&*1g+}hm;f35HlFa4o5)nfV4?OquFRxTNS;gJZ%8f8U8Dy$qgj}O*Rwi(m*{;&an?;)01??qW$DWU* z9Wfr}s4*J5R!E{1W>Dn7qSZdV!oB3+ zejT@$mMJgm-3R?*B#%yE@s2pej(U6|nct!eHYFjKit`noE)y>;U$*N1BkY?3^K6%B z|Hig$+qP|^v2B};ZCj0v#APXWu?|O+~+thcIoCYs1M8GYiaRW9Y~D385M(Y=q!ab94nv-#Un1 zN{W`P?&L=n_>2x;fnH@#lw1EkNLtC7cLfZ{HJv|ph(FJ7{2My@ZR`3AM{@jklKo>s ziB`4&DD4n@Q-H+B2^CO9J&p#bRpn5&0lFzrK3saJjNAQtjm*JQ8>bfW8GTYRp(K$f z)4f=xDIs7nteu3+&FoaOob=Jx_wR4u{+c{Z3XS&LeNmv(eZbHSXn2}w4Oc6Di8NgH zyr1ZFCIfc0Vsd~vointT!f|X}iG#4&z!*)8aD>?|_F|$fHtHSViP`Y>*n~?9m1qq^ z1GTRb{lhk(RWKq2q)|Os5hFCf^wPyYD`DEw4OZwGrdfZbKD|@ZF3RAQ{hp$NP9zvI#q$;-4fB3Bqko-P%MpP3CX_u%q^+vrCQvmt~TqXDQ#pc zu;iW<-fBGJG6|j^Rpcueoo1)EHg2iR0Mi_v&bik=b2=_jRdiPo&Mq^UbY9@gA!2b0 zBnGo|^Q}+YZ`NR>x{?KMAc5r=w{sSPpJ}C+--J0TE2S$TbT=abG1}3A904{+3c2h% znDBer8W(F!*v&lSZXL5XyJ0u&-awkqQ-F>bJw-X{h@WV#|A0l2iLN_+Wkifep1i3{ z^zt_ho5zg{hSsAs+%WMXSz@!8-BnUuE1@1l*kp-nLrSEu2uflXBY0p6gEn0a|#18ck zaOBJ@p7uwM!8zrTEsJMMrR7jb&1H(v=2;Ah+KOd3UA&}-XTPdg8Fk1wi=wQf2aZh? zL82R_TST(NBok{}rP`ArNhW)5V&K?~kSwZ+5p^0%XDZkZ+lHPUF)!bD8nYP>tz1z> zCLhcpIlm_e3N&GJE%ts9Rd!ZHIM9*tNK$mj%saGoL5ZWSAFK5U(tTVnC03VQnjx}F z2jYg4wxY7vc)L5c4jD{zOH%tGky&%7@VC_CR;6iz8SoU90e=6x!(;kA2DTQ?76#T9 zp8tVaIGI=jJT8SDO@1O7{|d?9Tle+P3oLK7T0H`{3Pq}nw;M!BLV^@(Z_ju!LQF^k=P{>2QKeTtgJ2ZMNBOD0q|@BJe4R-Ab0TDiz6 zZd2rJceWXUtOzhw&uE_f^SJf`b;X#&MzO!SbM7k;DEz9O)|9??y40? zPdGIyQhC_`YJ`5@=f8$*gB75q?im`GUk)cP)}P+LkE{Qs%Yn0w0~zyNfaw#tZ>-Hpb&z_fSB2HIt#zBVKiW>IXO7tJmN z8Or70Rye-ImL`;G)ue1*^V=r{4qt#b^Ri%77C|HvZKnbkI9EIo(S~>7Q)K?~BmAs2 zNl?%DLzj`zg&foalWG!gM#$L3FcoQc^$?8BcQ18MUyL#VegldJvdj=)o+Ow`FTpYs zqOc?SRZ0jl{En8a$t+6hVRYN&Y4y8+=euG1{VWfc9&xd#=%w{do@eDPyL=3MvgQKS%;FB z$T7&#cd(7|<8u4owtoT(Gjp^zKb;n;|MzA`uksgN!L~9A&P0nQ|TUu z*-nT1udjE2hh3NF)_%FaJ?sS!&wb@~ro1o|6dxua77ER0V%O6j4eP_2Iaq7h`Uke! z0fo(rZX0zkog(=aK#D#@361V>*H_4l#8Y$wLf%oAVWle}XS?Zl=A{#ZHo5J}ZX*Ow= z-r0PJ+%%6;%qc4@CTxg}~b_a#ccML0%EIJ!S50z2+d)My#_cP?qd-MrEnn4XhZDM#fqV>M$)g0J0Wx~q` zSHAvJpAbKX2;XE=)QzWK3}U$0gxnrLHkABv#9;g_8zlc{T~~CmFtYl0S=W{BXF%ay zRX1Dq;mFn}cvdiW`ndW%lmQIbfMa;zp;0Wt)k0R;;hpNCU?=DiP}b*dAoh@2kG2vs z6BA$V-rhgth42Ei0&9ac!=7QRu*M5*iC|fwNl|F51WPKEvxCKp`+S2D9t55xlUT-9 z2NA#P#q}wQksqvPcD_t2U5oyKBG|qEq+{@RIxJp`XTVfAKDd9 zUWbgn{PYye31?u=$R`Iy6D#yxWnXxr%Pa6(-*TFyoX>k_t3xe;2@OgvL+PoQl#=|N z(Jtw|Hb_(Hh-wTr$k<8fZU~GJ+2Sp{Uy1G%Yr5==&gi-d$Q~J*>wx#$w4&jt?Ma>K z9Y(~K1X%A6rBTqmdegqULe{I_E7 z`$yQe)ySoaArvb9pC2yr>2c6ce%qgKUJ(5jDu8RgvzUdovk5?`^>>Z`yTbdZSji(R zBJie)w`|t%sBNK%hy>T10W?eqC_oXCL0~A!b}j4D9WEW~XSR^9C~v4~6ps=M(*P5L zjbeh&8JRL>hf|zxheZuNKfGT)?69FFrb+D?AT>x%kk;kxWTd@~5$y$p^w2?R0fS=H>j5hI>WDa2rf4vMsnI##Qd zsv=)ZfK@hq6z{}>Udj4bK%XX^h!7nNr>?WibMj3_=^+xV1|5hI!~1r5OliP`y{4`Q zEUOdHd!@@kgzba}7hQVLRszGzs*=?!FX)17Q)Lu49c5G1D-P8-^;t0224c>apu}St ziz10Udx3XFVJvQzMKGKg&B+Vm+G!#6@~-L60t_Byd>3QYUNh7+w(3MxrN(9))bQsD z&gPQPns$-oVTwlI^2~3jTBQ4&o88J!MXDe zPQ7GC1pJu!wj_-znKUP+>x|rRoBSw^Qm{t=KHtLT$0u8Up_W|HhlJwKg1EfO;t$d@ z#WWw*y7Nw{+cA9!uQAt#jasqLOJ*ZsOk{2{KfeC5i&rMz;Y|coqLx2)`oH6dG8RtG zvIh45Zu9}zD9OL>y8P@cQL<$D6NUzSA8EsRK`c;V&KHToXOvZo1}m-iv;duDoVCcC zDjG;{ko&!OL?)ZNg`2@)*Q!VVjju1@E{SvPXCvw1<``Vu4z7Eu83jf~e}mpo3j<)6 zK%q(>ar*@7;n^p1`e7oA>XbW2z3}kSg@2 zyxc8_RwjH(YD3o}GWs^|OXWQ(JWE){PoCr4#gJtCjvIKGaLAz&yvK+-l_ptQxumX? zN%T#~0TfL}^>gKv7~(2n1B_M6%7E7Uf=U%7n_PE(SK({Qs)DLU4)`MK+7Cf+5APGe zVG-+)rhEzqG#NI#*df3%v)+CK}? zJ6xl#7FsWnrKoNDNBS?0$c>`N{iKYPw4u^{`zMa(Xz(4AtGa>jRE!Epl1c2yen}1~ zNP_EmFMvk#d$yfzIzE4oHz-52I#vk_&oaYj1Hpc1SbEk2RumSRp?tu?4Yw6NG7OU8 zmUJc*JYRwo0d^AO89CT1NdU)(3y-+e2qlPXD1OnvP^6Rw6PMY&zz7YP>yTjrfqCN# zdSpI(Hl-(8>V^(1&UlNiS^JlnoGEMOsNzpAuW^|P+^P*atLQ2EBP<;lhju{h-X`3O zLdaH?JcTDcS<2POQLoV*CDhA8nG8-Rd-3YmlaZ&-hY6==tBV5H46ReiBxC^#y?YX_*3$HvV3{a5F#m>`(X;;S%g zS{@#it%eLFY#RcahKq2>Vjk6G{0sw?ee{*6C-HVy?V(BpU)!y?8xNC-rFCK=T1h@b zSWL{sz`I9VlRjRoo5Y&;0d(lV1+#WjDOy%(*g~s1Yxvl2aO_zr)WxkE5pCs<`3ia% zbZeK8Obh;HU#`!A4Z6$SyZ2IlctyYfx7^FUZSe7hhOB=8a~r|t0Gl+IbewK@u%?>L zTL}5#t*Kl?%5xdJ?5vFI?8hR)HE8Va;ygKeH7C(BPl<(5bc~^?Ywmr>MO}pL~Y=rp}uMUvf|>Ugee<$%W*!Ra^XlbC28XxO#sIlqgq7 zsYr520$7_w{Q5T1D+S7md^7hYX3B}dbNP-olUvlx^%ceQ9)@?A7J^KoWq0-0J&zl4 z!z)KXoyq%So%u<({eObs?7B+50Af8%*Cn;*4g*O-xFkgRk1yydysgpVz&1F8CyfF3 zmn*om$fYaOm#e;#SQiGtuIj4f6F3x0i@n`z%bPU;NhqbSdg}1o8>rNT16QD;?qWzC z;j)BSt5CaCw-Crn$s`Z7b~oD##vnaCz<8?5v0plw$RTFw!e?(^$)1D?C=~}Zf<02C z6lfA>e6{oSwP|WMKPulusXHVyB8CZZ5>$4@4+LcuFm_i~<|nCeALR?67Ythyqm@!U zsIWe;&$OEo@yAjK4k_Xonu-I=kj3M-e|YTSh6{1yhJ)3_uwWRyb9MnqC0yS|3jytOIV(vY`Q!nWeG zAI%?djmD1t4luO_a{Y5{olvQCRwOV}*yN)IcnL z6IG*p%O9n|Zu#YWs7agTZ4o~IG&8Uao%;O?QdbD2ktyEvfFH~@2*T($VwdeT3Yt&aUeLfwk{mE6^t%E zT%}LOa%2#HiJG{EKKqnKwfHg-m8{+p@|3K#mBznI?-^YjEE;)S$j-mv{WYU)cb;w2 zt`5>K*#<;v`kWyJ)=F2TaT83+pc^2EIipl6MD=(dO;y$73(F#Z)dF;194AAGkZ>g$ z@30X>kZ_p6jQU}Eze>iPFZQFB@ZSAg+6 zC0mW_;0nS}fd(*uE51lqK%srF{iu&2RpN-6Zxs%Tw=_-4W`Ky)p667uv$w(FVT%JC zBRz^Hh}o|CRmGx4dh#Qk2+|(N-~rO-z-Nebh!4|wDl|lpi;d)B5HKL)fDY0fA9k8G zv7Uwn{Ru+JA^K2}c(7VBH-98ZHH!n;5mi7K%x3^mrxd0~8U(VYv4{|+m^Aiop6y^> z$ns{G_`o{nnseL;t%4{|$LV!_Ai^FM%U%8xv)EL93l3R8g}sOGYk{xc7c+rGQuz8X z)2!OPev4z`PxFKqi~+xZRR+h}8hii@Kmhn^{`VVC@Y@^zQ*+=%^56P{CcqWxp9i+l zkl3uz%$2mTYe^dsQ@yJNRCN%vyP=SPfIyXPPEDPFq*HatRu>&ld#;Aow+iOTsF<0I z=(Dbm9)WST8zAwEKjNBYO>|yzULHB+IMrX``@P*z`x9yqdSmnzguf6iiCTw0>WByq z03+&$KPrhR4x~UR4>X69pCerBaT^!4N=Taza_bnAfH08c!$Lw5mg#Jy%)}T#8rVV$ z?;45}G}547I$Z|@!*x>UBLgF~Ghl=U7e#4CIbB1dyS1jp*9CVlr7#liGe?bS^maz6 z9+*@KCSD`VGvOJ?3>kS-?NiH)&3K>!KP@Rnr%jnO z1$CwlYO`S~PqDSDkRG%#Z8b8Rth60$WFOt|Gj5gzTnTetHwD-su+p+Et|&}ILmN59 zTn7VbCHJ~~s2qF6mSWw=vC^v8T?th$XINJ)bsiVzccr^fEH)_4P)<=c@infoJWi>Z z@@(GCh~ahiwLBn6F2vX;_)J+!WR`?yFYd+3YM7-GhCmxExmcI6O%8wTV$q1 zs+Mo7p0xb*)kHn%M?f;-*B)b8inh|lf=V>pVb!NiRpbYn(lYIL-tKau>|&dZMp-u@ zfsbR9j30D4=S@BAN=K-ycq9alDwB|DheJ&Y21pIlx`pp+xA{-!sRguLIwO1C;dmEe zH=QH8-B-&8XrVrFyXyE~`Tup?(J?^>2{G5VMof@o7f z8sfNiqJCV;3Uw$HNq8XeYiSb~UI3!olLL(qm0H^1Efp!uYsWGT!2uU6v`I&D7g$z| zos3&$L{`fwUz5H8%2dS$&fcMz*!1AE?3b&-DG{L-+>WzVNWIf=1ij8n>p7ktw2t@` z{hWMhj)+$MVq_9?j*(kIa4m$d+uDFtiq4#%NNt1#M5}m*xG{Vjo&*}2pERHNE%!)y zu7mU!W@i#$dIZ^$Z>>4&2F{53?l5SP0VE`!5O6T;arX(JjyT_`|IY1-cCQ^-T^MQ{Ty59{FVNMhAc> z4M!L9ogvn54iTK7oIvCvPMT6598KDi&8KtOng^I2F9mjO6DOR(Ce5W|JvR!_)Z#<4 zS%#e~uU7da`vYNV4w(-{7ltkQ6cu_Su%7;q6u%&Nv#AKR>Qo~9!A<5nPz)Rl?Q$!k zZFx*8L|jw=6oIu>vkN8KQyjslXeIqZV|8BFjAdAN|8~fiwYz@Rjm9Hj7BD~SR~GW> zGNLaX4RWH8_@?j_C&wke?-Dx`wFieYr)k)KB|jSqI~VOFJCg-=F=w7jayE~`DvZfe zdLqJHbW|(g;>ePMUMsf*8_7DdALkc)284e$`Mvabw=wQQszeW=)J5mc?pxSCbb@z_ z&}4$~=o7vDuH8?Zr1w*UYB->=>-eK9729uB?T?d-vmF3JG5OcX;-ffD2LiB`WlbxA zHSOF^7!`quV6!4ZH29j17-8Z>~4Dbv4}7z73TsWPH@6ijSDup8?Hhq|;!1mS95~&t~xqTh~|_Y&3aMDk@5` zO(*QRy|&P04GT0lh|PN9*o$PowU(f5PDoT(;l-m;dT~iyktpiITyt+Ir^6uYDcO)p zT4Pk=Hdvk_o|Um;Lh-BVoq5?JDubqOOGwqcE01c7aJ>-$p*A?9iQVulC~Q=SE^#9} z<-NX~gMt1rM2%V#A?r+9iLP$xJi&t-9U~HEBM2Tz5w|R@hkkuxv`mTS#wekJMn(LP zvop<0=vM3TDW*~-+_#TMb0?}cI0xP*`IfW3xXsLYD+qVvi&s{c;L9Dp1+cNxvDdGg z>(jZ;NG^a5(B_XF;C~0!-s9iUz~7_?rvLtjfAHi&{%J4`4MQuM^3@4dG>_;^4C)zz zQ4k3UNk=G!b=N@)jH!sb!J-u8RaJli2|wy5Bt~m(EEpl{d3g&k4DOoG%I)d)0j~{Q z#+k>V=QQOo<@AnG$N3KD$#CqyAMuHZ2fYj9h9NW|%&g4aJwOuPiGmpg9gUw+hSnPY z6EY$NvUP=q?TM=?Hd}l_Q^vfsn=|_z_{Qj&Pp`K)0<;gxS@1AZedQ-VVVE~7#j3{o zOU&mbuCN?fCj!jS=H!&SD%3G!`CvxEWbxFAHWs8hhy)q!#e`mPCo1@ig*c8D1Ar}d zTX;MBskRpk@TAH~VKFwM@MrUueP*KMdvy%5jw5INn0#;U5<+Bv>+6$i-N{SCa{mGu zQGI55OchfGmmE($?%M~d#glxQFOyB|?Vg@#V7;x-cb&xm4+b$Ny*!Yblv$i*v~@IT(Y#j4wH z=@{)Gy&bgyA)~xd0Ha14VO&6%NowSssafgosPy!8J&M06kAcd({kp-blQ2E_8ITfN ze@qF6-@f&~4P*Y!0TPz~$OWUHTHt>I6E6SQfn~OC4zx-_SXhjS=cnzF(5jKh6$t?q zIYO3K5Fm6_Pm;AdF7;}vf8pOnzm+1T2nXI1A@xNujB=8&s5?mCWS_R%c0XFawA=pB zST}0mv~j3$)Y`vm&%&XF190M0VC=P@6@gPl_DGVQC^R+zWrb#USnb2A(QXZqM$6DJ zE#r*7=;lDMrC69bKjl#9hZEki;33B)uLO}{5`y`3g!q7w zlGUa8UPu8|tZ*x6Fb)rldJGQpU3)&{n!}S6v#LfS>BsZM0DAsM3O8U!}%kG%rFsXjrJ3)=T z;|kw>mTaNl7YOcsnb@`j8;Y3gm1u|WiiipM+)_@BD?;VsNYE#80fUjUEyr{2saUjG zWwVY(rF735diX<16OUvJt($&$&2~Ny<@lo9!?W_FY5wX!Q3RJ&sAk6Y1K3;`ZgE}x zCv1(3l$@ErCKX*XtMM5;<=NT!^4(#+cVFLtEPZOfMlKdkIvrxHg5Mb;jCxhqc}%O= z#TvXd-U;4=LMtt>BFQMUX4B49wD#@oo8^2}9plsi2(bX~#LkU@^%fv^F8`Q2ztal- zUWEPjTKp5o0<6LSfV#R=3;sGhRhg<{JdlaDh9XKa1Qq^u{!ADY*mm<*EwmQhjZGx? zTeB-@Z&6{=S&?hepIFuf8n)Kb$BcX20BBH6lU?sjO1hFBMkn8Ie3$H7Tg&Xncos@uW*Dfw^Q;LsVryfZna34Mb=Cv zXBPUJ3o}_QVVku~<{8{I8&$iss=2jBcP#Z+Wy+j zr}vR6RahfB=y~~!@yhHe>t>4S{rwQkEd1*omVUXq^oA#48fHOuSSrW(!j^Tm*}=d~ zq~?BkR^B&`&43>q+7@lKwHf*{jKPtnn^aB)L4talTQfL(7f5j`qb%8=n?z{yYeo<$ z>>f|ix}Vori2byeVLZcCTa{5BS#4KI#15VT;<5%Q@9U}jHLb~biN(G3lIq0JsR)Lx z#-1w9ualzVOthg_pRt%pFN-VCSwk_NeRc;0;L`G;tCL2}-F1N!5jlx3p!WAKCEfV)?> z#|OkooPNLX9maMzKB72Hy&n-*<7e<9Tf_-(!8tyU$Y(py-lyrrB;76$Q3^QuoS8t< zxA`Pv{Zb_XaS1j8YC~AzRFe9@jHIiZ0wR*n*bvm>;eO^0qPp6Y10z@d<7|R=pc{W@ zJ6diPX*M9!RRE%s|2^x`{#Gdg0(d)XR}%#TXLG0jsFVN69CT4!2heE{cu#E3+tp6r z=tlO6Bc-oriNfc|FC#BkSuKrajoGQcscR~r;18JbOJ&(mo6icECNLjO1ZTxgFSGxp zDx;0jz+h|8*H;u$PIPnJ94|U62G<oaQCO5UWNny(>U z_Z6-x==x=hVYwtsC)>;Gy%=JG{e*mAMip8J;NxgUv9K1lQNo7`$4I81l$r@0!ME+Q` zU;$y20bpQq%A)$e1mjIAy|(*o2wkVths}^X?d!Kn&hm{S!yv|^q~*pw+Jgl~Kndmw z&Vje5FA~@h^Srkz_|{s?-w(Ukh0C*&qom@$OI_29Z(dKX9|`AWZqjc0ym>xAcfS*K zYo;($tEBjG;WsHnL=2+sGSc z1;UnoG}Or4G;r?n@-569dA!eH`qsBS^l*-GVjYF=Z?q}VJ8L1-2e)R%(mHi<{fXt< z8A_*5a6E5?0FX+T9n|IZ?QMu$~Zv4!G!&cgK zG6W6W=Mz6fLZ_JMC9>DE!I(qY(1zOe8ABUCvo!X)uiqngz2qFC1bjBhT72Ux9a~Ys zKpt{sNL7_-0ZQTrvH(H*1V3hmaa0POpYKyhj;vn&iP~%kjuAN(m*ilxI9|>&<4V^X#d4gSg1r@D83@~IbKqZ!_N4P07`GhP}*}) z^2STgLbQsC;xjevV8LvY%I!Y5GR5N!X(*GLO@!aJL=DPf!hokTl>c9<^-|C^rX&@e z)=-L@(+~(Mi&&pinr0;C?`YV@ed!84C9UYbqrTUNP=sueQAy`lu2@BwMzP6Q#@VS`T1igQLCloE`0c!*a`~H-b~?ZF)D=mtkA_Fwk)MD z?Q}$|G7eAv%k|GwZxOe z^B-v$DOF?;j+$ewGZ$qOaKwStj`kM=r{?~Wq=`Uv<^6(9Z7CT)vn)Ih2(<&PJTrVO z4jnc^vLBCu?9o+yymRG_PS@;8e%EhjP(* z+p(@{YfFo~u`4gHilsDlw}Z z6_(0YgfLBYf|6~!59VPvYm==;AnMu6)w@DyI{7f-_vHX{LH878Ft#EF`rw|@7(Io^Q1F2MN=;HL2pr69?FjMo6Qh>^8{lT)#( zw(>f_<2%N#K13oM+D1i6ND0+im?8BRNtiBJ;4}$CSy_v`YxKT1d?bNN8uOUn;$5KB zZc@gO+@wTOe=YsJOzvTYOeRo@xg5`RdUdnmVe$R$;d9T2v;frMFb8jx;Zg?_5^u_U znq4c%ZHY&3KAg1w*pl=IrH(rgF=-)uu7H&z|4u6Qy&kG+c;siEnHh_Drao&rMRTt;Wk< zYA!IFRTnnN+$71er0iUGY(gcda;KPK5TWXsQn;jGQF!nvR;s?08D`4hPEH08F1PR$_BMd7uamWlMJj*Td%Dfe zdp!ge0jAJZrtV&hFsI`%8(g?9o@3c0s<5+gxuE5et6N6}v$JgYASSwl{zLd{56>rh z71@5KPU?ji=q}(6naw)mWsV@MJc|lsqJD|3_R>tT^I9`2Z|O1u$=Pmlb=XhnuVNt` z*aa0Zqj{ZEKVnrFMo8D)D!m60o{ie~?JI{6ii6t3(>iKofze7@U=n6^Y&q~|r*WPY zuVfp{vC&}nZfuj?uu@8{O~f&uWo;L`&Ph9Vk82oAPhe{%6;xvE8;_Q^N^GA8RNH-O zkK3Pf?qA!NlylGYq&HY8g7}>q<`Y-1a1*DQVDQspI#u^OI@R{JnX~x%NN*|}Qg5TW z)-)Kid>m-dsAZ=YE4KtMmCR>LjIfg0TFvl_1n;Zm9J%#mBIYa#oY(4#=i%O+iHDam zcIT`ffckH&dUy{eWtermM0x3x-iE#}q0>6-DTvN~#jx77&S{-*Y1g9o#iBiGeM^z7HU32Qb5p|D%}vgC^pl^tXbf znU=;4Pid={9YL;W(Eh|MJR&15PDKRTd%L5?c+~a5@t1%^jO99LmzjTzW0_6NaP`2{ z_Q)e=HGN{^_5BH20D77;$G*zG%Av|B7-yP;jVH=r4tO{z7V6@qj1Yn$*3(~H1Fkmm zVdz)FI(sSc zuyk3I!=#B9NitXD>L^Eg%ptch{wDTjYRr7_6z|*yT=;VXxjJr3u%WeKn1ejIn3RPor-N?PZPA~orgX`WTGZ19D6J%#-Nmt4!ani1Dq2^6 zvb{rQYP<*(GN^jeNQ=Wd#(P)Bw*fEhosT-nq#TyTRt^c{T%fEMXp!t2@28X2_KO`B z!4ZJf%A$HYIb330S+~2=^A7ugimWfihc+6M2yQw$^FwDg_yCp0(o$4TXP8alxI%LZWm^$!}liNZMG#3c&v{6bO0>{2sbyYjJ6 z2ZOVX1F{Qh^+y-}2%~ha^Z-evS6T-}g)87!1b*R8$W2ar{fV|C`-_LmVRU~=INn=c z7#c@&xf*07X~&_#+?s9H>O7EO3RjNJ+Nx;ii)Y<7 z(E4&vG1as3GbLzkD|t{|dzD3x4Q{lGOp>sa#p*_#^2e_^PqrrL@e?W;h3)OAG9#=s z3%#hj599YdF(>DfN2EdvL%GJHlvolC8*9C3trfqM--I+j{~#Ty7QV;OLMdFs)Xf9M z{{Sj1G4F5r)+x_{-D)>EQ$b$`io*`#&%gT7>?Xz))?lUdj18YZ2|lB^>lX`jv>mkv z5%6a<2K@f_9RK^L_WygoWdKfy*8c=P|GC~U!Lu2H>33%~o!=^8wjzq}t>|J}QE!=} zv{+P9rhSC=Go+EA8CVmKSQq0Vw{*Zi%E|JgdiZ*G4JiPt3T=coL@T3_)~Ha&kSvVG z>roUpt*lCshd&4u%Bb>%;u%B9>AN@#v(p7-&yL)P>du*GX&*cFcR#|=)ysjw*yuTNDt;p_#64VM>w>Lvgxq6mxgS8Vpg3#4bcZ8cxK-O~WqA$6rgPg{NkY zq)1)&nB|XwFG2?Jo+&RQp%K>_zpQO-ayYZwgnAQ4Hqkd53r$~r|9%yjf4ffJ|K$a) zl&}#n=G%^|ujj=i$D8$OK4geFZCekH$!{!TQ0c#LW0;k_aktpiKtw0wLd>(tXMBb8 zD*!eHX!-w*+BBI z|1mZe{@Hu|;@kZ*xYhqWVFoRIy0k>4_SzSV`Wztl%_>5YmXL7?OWCq-|Lmd>ZgtAG zq#^j3{+0Zr0P^2kj;iW#f(XU?-K})Mmc!{{GUxL)pAVRV_++5bZa|0$G&R~78Wmav z=ER;M7#b8n4L1mLqgfQ+peQsu1Xv_HfU8??^uuC5IAjy3)^w6-(6zLz!WDi<1b&{| zJj}6Gkqd)pjv;q#LUNoj53>}|9Oz`vip*xb6B;~WxZU?x&W+_ECBmg@pCYF7q3d!` zwwN``(c=fghKz9g+H&(BOGPPQGv_e58F<{pYf5Y}EV(eV@*J2BIAlWW@ukTHvp^hT5Vq|= ztyz$KdXtse=NhASqN<|qBO@lFbUsDQMadHeRUV4jPi{Dp(Las&8bW`zM^7z9U@eE# zs95UucW&0hA^2{8_tz?X>lNO2FG*Ci=xk=Z73SvP5R2pSAJ zH-3~&CSNe=?O*1MZ{M{xpaEH@_D3Gv?}E?&r@+|R?$@B6@*TN z!~3|_OLW1hAvWI&Wy>hzab8XAK=i6nCfs9|Ac0%MoYZxX*I&<$vDB8|+X-^gTGgo{>jm`i@&two5@%pR7+ri&sh*5jyl) zt?0mY3Eucd3xv^x^;3JD7nih98|GI`qrFECKGJUQ+z1ZRD^^07=*sh%GxU4QL#QK$yKKRaL>qQvTTi(yxGWaafm#E=bnBT zqB~?@h3P6n#3DX*Yw!8eqR-pu-|!5&IJ~sZZ@<-8wVlQx+-0Ksd zUw4h#SdbMHc6$P$U0?=Dvt#}((((w~HVge=n_Ro^%xGwTJ&^W4as&Vk4$BEAh!|mgR zG9DU$bjMOAd3%ej4kMZR&=@5clr?+yDb$=d4J1P(!@4D>oMNisegWf>!|cB0Ze&2; z6{p>sJ*j{IFw#y83$8FAM? z$Uk1iSZ53{RbmXqaF6MDpAHB?ha%3}p3(*V&=UIL%?u)3Ob!`qToA0rPH(5G|AmBA zPIt_n`dh2WfGs;~o$?Y3X5`dGS@64AqOAaH4-!{0_z}yK3%R3#aW!;{O$YXj4X^}3 z6zfqrq^B~NEEA(VUp#grbvn2483NR!KN8`*3TnhOTF3a_0O6Cyi8AaBCo%bHG`iQqO>B++-tJ(DcQywvk8`^{^Vayc;dZ?E$pp({+8)iat<{9w8 zJ$0|@|3Zw5?VeW{&Kmk;SnfdQ8Q9Th^}PfN_jA{H98HQIvo(s}Zp@^UMfv6&$v~}U z77bIx0O{#qD1LJrMDU%MW9jly3cl3V_4!EhMv5-AGtk$tPO@o{gNh88A~WgfZ_B=p z>>(u&G|SgGcY%am?A{WpKH5wup>5z@zKai2S*|EhUmx-Mb^5Ol6c-px+^J+z4J&^L z!u1;#T0yE`G2A50$G;Z)5uT$(5ZN|4%(9}g`{*NHR3b`(ftrU6H>vF>_r}>gE1HkC zBU4K^#lXp7-l~0sCT+3f`%+6{{h~IY;iXIjvBBq|4K@TPZ2OANaQ4Hg1LjGvK^Bv% z&t(e^tc{H;z^9>7$e`TwK(mfRN%^rq3PbIkjnHyG1rH`Dp9Kyx^ooM^CMx~|W+O)< zD&{Y zH>mUn2Q;v`)FoG=f}Obw0zASICCco(Ml7zJOAGK<5e$5iEM|jtt1P zlRqL+zXPBDVHyK)ApIL$@=>)>-cUjDt*19;H+C}!7&95HP|&Rxus)TB^{E>yrr87n zqMhTKhm=N}gM>6i274uc{VNQ!6#rY4!UCp#_i9WhBOgVC(9k(M>Egd*ZZ}{X0=B z+}Bxo=ScBsD^uz$RtC_C`s)gS9OF&4SThdcOdu9NixnSDv4k~3FMkTP`j+w}N^xwi zJz5%L)vxSaresZ8==DaA3r=mt*-j zHBeRwUJS&it2<>v=zL4|xLJBc1M9_bg%8 zcYV<#NF6b2Dxcg&zd=>eJ>I_2{q$f6hQuV4TTE#e=+sQ>M7#7M`WtHfvv(06Qnhl0n|B1lL8 z!p-2n1q}sXGce4rHxI9mZyec{rm919u6OS&yr5X#5cD@yR%Ir;M}c4Lcx!x_ED3c` z$(nmo=z4X`ti~aH#vkcpR$3uqOQ)zs?UOx51mjUr;)(4X$1h%8ecQT7+5r8iw+8%Z z4sF=j=`E=%!4ieSXlkHWwBO=B3fY(_F%nznt^(U%D##ewu9GfHe#FJP$O z{lxXt;wbksnSsE^q>ksT;UK4qdg#GO%odc3^vh<|x6a(5MPS@ge@6)puWq9Xo0Y|1 z7Dt)t%Bepsj#%xQQ|v8m{V@wqNL;DN(uvAe?1SUI#^A$f&{!|#mc<=$2Wg}pvR0~v zG)1M0T5x>&3L1%0fPBM^KIoq*==YZ|QhAm132?B)gP3ZARXwrYoVp z^H+bltm(9i-DH>Ve>21~i|2rK0K7jlY&G_eS_n&hvfR3ZR zcH!C~P##z_7N(KkI0F@FD2PJ+sDz*_G)L~#L1$vHYI7fy$3Lh~2v#01;@&raZ<5tP zDzSR%VwDLOa~Zyj01+E9XdRU5yLd$$bKKt*f@2D$z8}fC{u=t z9m*jP5#{E~lNkS)+m&|zcxQ`G=k=YVRa)<|F4D`Lt975~K6IUjqkL?P?%{Rf&|Pz? z)a$lf*XuUd#ul4&OEX(pqH@yNgp`;+k|gcFOVNDu?yNt;P>YswW;LyU&CqARqoktu z5ydTb_hJ2pnjqzrVWAPqE^m`Z1`C-F2L_=^qoyi5rI-ES#`al+>2OMVsSO=h zjI2|tb^BPm5ySlBinYIEcLPvpMu}-A*3Gr8V!3;$?6D7@qOg0DF*&jTlzpKs z%CzE&wLVa#d6LM}1G(*iw#lNw73lr&c{1(xvseXi>N4=5N97oMs^d_2>9%`p@)d>Q z<+VjrKduowA(Yj2#h-kGX~OY75q)T<*}U$JCy$ziXF#!FnJp6vMe!3hyrK9)WdS_3ncV9t0t9X7Hr+aJ1oE`1U`?k?*O->L0X^Pf%iMFKuDcZ@j(Z`1U# zfF5u*EFXinRY6lYf|lGpSI~VP0PuvxjC`CcJF*(~e$#FYw6Q4$qvA}M=K zY;fbltWUm_b{6QhW&2{Qb~dbFk5+C1FRRK>?#|DYC?%n1x*{o|na__or~fARpiQph z4K!L5XPk=o?w^2Vx$9;TSwe^2%7PoMLdA+5=TB-p$04nd@?cno{lvCnRJU|+q2D0f z8^Y$4E*<8SooPd%R(yn?ShhH4gm-;*fcL_-$R-<9zmifjx|wb*gy7e~i!Y~o?4Y0} zuD?B{f<#@IobqW{7kx%ezi~;%v07TF2!)0nNPg^2WKUBm>b-Asa@p0U$ zk5lz{zlf5Ts^a2cDUf{*mwr3ul)XN9)nXZhAz76?trj%Vn6REwxgwpZ`k}PkaX=`!wQdUy33LC z;WXxCE`7F`CQoQpiqZv&j8x>}N0p{|(m(|?COnfyG5&0Je6;Xe<9z4S0H;y}u(e4M zX9k|Sbvt}@@?^U4##m$Kj#BUfS2v3Z%gRje9Ni%U?=wKQ5s@u>65nxIq4gVAP-8!7 zq#*qJt_@Ot(u3X26mKHT&GUvKonk6E@?J(;ou@1;!3ldpz9@$v9$G1Nr1YKVxhkuX z`NTbx@R& zS0pb!OH~GGsdUWih!Odoa6580aaQ=c!nguoSc0UGm;LkT7NvIuH*SY(j@-B;3MM*| z%rP25Qe3{Mjw@U%9fdyJfE-s>Ltoz>^Vh*Hqg)4+u^3WRSWY)~B;&bRk&0@k=wA7m z>XMPWHW+0eC^rLafi;0BbwZAdvEiR>ibjD;`&9Z;@*9{KuxlCU!-hK(`4sca)nl2b z=6zaxVGFAYFKl!NUrK&Z?Tse(93RsZ9>thy^0q;9;ZH(0#*%4pD#3wL3|y?eE9&D#KJ|zaAN9G9|1V zXJ6F@@A*qBp~)w4q4_6WBLVZ8_**gsJT8}7EiEs9R67X0W1n$EZE4xvsfzSrDr2%? zI+5U}(N3f99iT)8qzddYVkq4^L5~U2r9C(yaQUq@)qo^GqTkraRY0XS=ZJmzHa@AB zKbg17u)Ct%CG2z(<5$eWwg)yV^%G?lY+n@%76oy{B|+{mn`Oiox-V zRCYh*28eYuc5WrJK2c7fqkU)bcFH*>?AbydTl|m`q413{`N+&C{!_D8l9I?1+4lYj z55Pdv(wc#@5IELptK&Hwclhdm6Q4Veen8v<7z-%j(@wCnEPjr@LCDQv2fv8 z4q9G^ELG!+KaYTUJ6?m9D$New0j+a(hxQ7EPpw#Fp{CYAFU>-pTF#kHr_=kxH#q0m z>`By)4^nAVp>pwZ15Xh@rcxldU=hp$mx3|%2vIA^rMPOAxd+BQYMO`Ap0pi&12u&} z0iGh4@IMGr{Qm?kZ%-%;b70O8;U77i{}1?`<1hFUt)Z)lu8HvmV?u=s(?9d=12hI1 z42mkSkylF>vk_YuqaS84a_J@)QqFvX9DT@2j?XaP0iu*F{l!ZA+_)_oqpbCXOD)%cI zi`RVZ3uMM2Ld=J-!NI>y3&wMm)(R&_)5tXp3X-Cz7-R__ex{h;hC}K^Tg=rJ6`-yj zGJwN(4C^ySigoc!=u;*5#Z8br{3?PZzzSnD#_X8TH%9>C;V0iEU;u~AGYy)15=`#$ zAQhYIful1BweG;vKoMxcI5#hk7(3pz9E+8dVN1xyAJjonCafV z4^m!rag*xK(9)g4J@3Hc;;?1+XZMm+Y}J3SW5t^}E612qC!cr#SdGBn-OVYdlo!}A zHRFkl{AiJA&u_hI?<>5NHIE~&Ik*CXB8b^1z;}PCF+S1nLkKPG_3{F=#281rBO<#< zRo2b|#;6Esr3ij~eKLm79UV`7m?of?uL5rSLlMx5>C}apBeYO!l;KZLxiKEfwT)TH znlPP%-1$?NMON`xBn6~)*Ze0&PGSsGmVME%?`_gqhlWc(q)O>)&57oHF&Lcp*pun- z{fK~kDyS$ncH6L~gGG3$le4>xw!N`j>g;EbAWZrqEj4C#d+R?0_Jkr@$z(+wb9|WD z-^k>8|KYrW>2osAcVL$x1icSH_VVXoVjJ7=I=s@Tex-aQX5JKYe!sO&o+(rXleK#l zx=*b8*CQSbOcKQ9oW$9uALy-eJWy&`8<^tvRCgIx>-&x{8J^=@cr2 zR&A{>)RSW@RaJnPJD`PTSS34b2U@aMnX}8V+h=_Zvy^j~uc^C^GR%2=jPYz1%=0Q* zb%^yi#a)*<)v(F)NHQ759XgnKnT6BRB=82MZ%F1xU9^A{@XLyNJO|I^c-jtBrpsQ^ z;DzC&f0o|xOnu5eZ$Qu;%XX|jC>liKiIEpVT4*ixmr^_>cNLVe=l{kBvcOAqekGf_ zH1#fFnx#?BXsjeq`Oe_f3}vSmgsQ&=cR5DZcg?Vbv$Xr1sJ%3!a$!7m^E8j#Q?VK8 zs#G=dOn$V|>W<<9vrslzpKw1Q^8D^Dqe}Va2D;6E=nnPwFXj444o>AYi1jo z>x}*s1nppq9(q9QN->Ygo*~jm$6A*mT?OLR{sc$g6Qq<;$$f!9$&>`&>u3P3mtMJR zBW4a?r;ulJroGgJhK9kz*mv(%% z57~|T*|l0*$ZFJd+CRGid)5t#rmGb0hmdD9b<2{AoZ+E|DT^DR1S3XOt9CZaQVxG* z8&72$c!|<`j~@k9W(!aHi|%zg5@lKa_rYc3J2@X!T0YT+Y0OyR)#{i3DC@A2r`F96 zcFt?lUb~iX(^Jz#k+{igWJo}#oOg>Tft~ZD$z7b4}n&PX)DBqPO zPp3VgC;xIHmgYxYcmY3zivjB_zr!FU^7U1>9@JA+Y1 zMzfjTsYeSVfl3IOc|PMD?B{MDpOm~WzbX6{delmcIZ*}`&*sRq$#az4?QstUQFX|!s zP+#BxW*GZ(G=sdqIf{a5}cvyvc!C{dGhv#$DD>5#Fl|9JVQ+#wF^cFqD~{ zeway^#SyS7+U?`%I%*V}?y3$RDhkE>hukGJ7|$)Q1(n#@9j6{SN_+)&l5!c&>2L^! zjQ%O|>a95LkpcJrL-6na79sx6nc=@$zMsmf3fLrCAkj_>E>aIFDe^NO3xu)6z=v-Y zqQS)|4JeGBnd<g1Kmmzk>ndcO+NwN&1Z|bK==UuH;bE&t_#M_Uy0!RZ zu+m9Ajar`n1Y;)+g8RPh-1B`IuMB-rR!yP%fbg~ z{WdP-hRvbH#mm*a_d})Z0IXYXMj1;UM~snQe8;L<<8$;Y4AHGbK1`9ASYl?;Byy6k!J`@Qqnv;TTIYIZN<%>faqZTjqLIL}H7nc*h zipoaCBV!sr;d`9*xu1)Rv{{;87LoM?#h1RhMNN9t%!Q_vSh~tjS>oZQTSy)_fXw~G zhn|DcdFsNMwy=LV={cxihw(~SPR-Nuc9Ar?y+2&tgJUArS8@-X#JKrOl{(ZD#Hs*h z)QbKQoc?{F{eNQA{-aH0`8T_)rmKk$Cf5`KvqqTVr1ARm2L?HvX{TW~Rv^k5u^qOl zFyuPtrE!do%&4-*Nh-UZ5BLJUbv5bh_fXpYxGk*OZx`wn%DqU0!p~Ti&{*`ie>ymO zHh7Nx9`K6T&F!yrl^awJUq!2>?wD=*rEhQfpoZI>+umuJKmsYWZ=c2qilCif)9K7< zZM$Xc+7HLL0~Z8B?hP{%?bS8zXQ?6!_E4v3DG?noPtJ+M5RsHX9;E1CG?tN&Gf14K z)Ki{`^q|m7xax{Xy~>FYl(W$qtO!=51n+1Bex%rA_YHAFG?aaW`D)b?9?R|$57QK{zw}LB{y$0|~w|YD3dkI*1O;000Lq^P5)Ril0xx*Vc^@mX} zp+paTp~S|~w+>~p^`69Im92cSN=BZtP;Bllb#^v{s_*i?831}!=v8Ni-|%8qWu&eO z?f6t*(%iGe$g#sIbV7QY^4GmFLvjdYhMK zG}^}xc8^eHZl7JJ7*3@6P0>8M1gff%J49R(QR2bN;LppBdM?#K^8O$S*<3*ay;;#t4-%W;yi94I-{d`w_cv~gq8Pat` zDl%Js!xs?77IZ3e;bCJ@P`6$W1z^yU{Ve$=(4INclz*HeBp*GG1XVOb!7N~Rh* z?M$ZEa|EGc_WO-ADeH*0+cvgP$C>!|r^bRRna1xnbn3S!>ndcIWmN+GW1@59ZkO>H z52&c6UP1w3IcpG;lx=j}ALrp=j^vnw;Ag)Z?bS}_2(J#sik7Poc;BLmW9nU^u6xVf z9y#Tnz@egF!-Ixx6@(bk%qWVf>2a#I&IAWz_+4d?!$F})Ug5IG^K6EVNh2vG`0>nI zJQ#)A={XuFD!CATWRjfJ?PjRoWTH{s50@Z~{s$kq^JuGn zVsSjb3bfey+1_K4u^ff;ONepRo+j?WSQ`o_XPy*Y;&+ySP-FMyR(e&iXN4P$>waxk zv$2IL>W_RqDGgltnt|W27FK%Zxrm~s45cdW-E@AS=-HN!4Ks(W_tNSVYzg9--PXhG zioEp$R@QuFj?A&V5?DtZ5>Gp(Tp^VK{RngfMLG=>>NjlXm0yi6FShIgLVY5`PxFXm z&4xxGS^VDSnnjm}ynP;!g&J`Frm>6@F^(~d~a8=8xt z@;^epDDO}l%OhX(j$eQiz1%mK9)_qtL%1{8X)QO@m>ai)(8@UBYfh}Xt4s-P0onEr z&QruK6^=r8w;lB~!fw{yH|!&@8%!zTx`lvYj8aF-Cob`Pea z2!FjgOmFkfH;22=qv3Ev-n5EW)!8+xYK}aU*_8BAnI+pJniNDb&Mvo!hK_n!bJ;?U zzG@~JsvzlL(BmE{*%_HjuNTuokGlUpihjH-VR+kWrp2Cq`IGn*r9NfV0G}7sKc3g$ zrAhyt_@wxkqfDWO?H@p4T}+G$mnyiZ(%go4QNbE^`W?_u&bGW`o|8h2CS!3$p*Idm z!D5Hv%T~Ey*&9qiAh9|Yg{-#h(}B(#R=}4r7piGd9Fg#v_qz!7uyA1!gU^tL#yv6 zDLkCdV-A;d_UwOk6Mlq-(THp-EG|WgJSZ|Po^bwAehfYK$#R^onh&iZhEovR2sws8 z`a(hATiUovqmnE(L2*is>jD254%>nJ>Rts+bfgHy#F-1G5S7w&{m8^OxT$*t0)U2; z`~^L9swkP@2x~t2;g`=U?)TA4>yOB%_wnJ&M6ZTT0vPr@w!bB#+DBgDez)o~`r&%} z;&8pX>gpR;mA@CwpUe{R(668W>7Vx5bFd-0LX!#S0zgegf;#_0e7ZIac#-U8Csz1$ z+zCIKfi;x1rg0A2Q(`)mNjbH^qj%@AvVaM(V`hJ*VWCI)!0y0e(bf>S%Vc@STF4;J zm!r$Bw-W)wCv)a7R3+C}y zE1SiesfbCX=ZHvYen^k-CR13fub>@jIF{z@Qe%H%nkKJQSAn4RP1Iy#oE>BGuCiTj zyrnUpu@md(xTTX89q8B1VEAU5d6hJApOT0-aoo`7X8+5gBf>n6qWJ!J8q4h`iM`YQ z8rtE^v7!ZHvkfOYUzg^`(vw96_#=Cc(lUG1D`<$NApM^?m330)v2FXgRgWKxN8PX{ zc=`zUTyqkPA4_pdBE$$Ri#)iGR7oW-9eB-dWv^sY1&PsKsBVWsGk5cZn%vgKahGU+ z=?lZooOVW51Qgw*s}r2-hKT+)y}evqX;k7A;`tVQ^2uh~zirwiqr$C1Y?z+V?Y3eI zSHeJwqPicCTYu;WOwIzYp9l5RiP8@gL#ZTyOh@1$$IRETa_)Y|G`V2>2GK~XS@;+l z6++7ftM6z;SK(sf9Wj#~XuxmHF)pWJhtJ7zXx1c!BVN(7o)>)LnsJXra1sX%3U5(w zoOYj^PHs$BlWWk7gSvPXfLmL!YSo_~VEfqL3!!ZE@X1R!CJTnHSf3p~6RO$hb_MQ! zFZ{LW$iDC$hLWQOWJ*if0J>0vr1pp3qN^QkdFJ{Yl$n)J)_ophH zNA1=Njb5*2I1M%>9T;=@+-LpN=gc(x{vF)py7^)lW2gxfd2*cWT=)+?2bA|{oH`Dc zw>|r=IfLJ6^crcwe=+5W`sN+E?G7cl7H$OZL7q0)Ny8bh)Ypb5j)5E>DcWa~6a*B% ze*b`rHJg6Zy0OTi+C14HbEmhsMkbc9Wg*YyRteUI7=wV;lARmQcV)DVbu(%e5+;7u z8v2zstMr45#yt98EyLw1GtC;C0GX(DTej?~QkBmk&Gub-{ohQKO>_7ZGsi9O($RO zg}g5S#h2E8@=N}OF@#inC$d<$a?`QMNk`9$Ji! zwIz4f`BOI#f(LOf4aHGY4AlisZX7;ssfHwl5+d^bXxe;Dlr%^$*UGO%le``eD!#R6 zHv@kq9{cL0>U_vP-FQn#a-dP{FswH_WQR$t)Tpyr(QdB!DOWpOG?qAJmC|=c)O3Mx zwOO9}%5MqPZmxV`XSnKDm9^Jh$y!}#V{dZd3Uwje>{!gD(?=cSBLju3dGD)9bEU;y zW>>F%X1gCO9x;oM{#%5D(PTea`NS4)TG}y}%IqI?3M_EGmtCbC9+uwKauS#Rp6w7V z>{#>fG6n99tmb{wE#6RBUHd&8=Or;O{}F?<5v@OW*WNoXlj-?U3kAbJcMGC8T8$*c z=2LI$7m$jBtAUf))P1L7w3jn*Fn3Lv1i#pZ8j)|=nIAMmKWK3TB+AIWob~wT!cg^Y zAB&y?W1ZMiF%;MtJy$)mY9Ip?^3;6W4K*7V?0{vksR`R z2j4IE&`Xow0*X-j0LIO08>Gr`9%wT-+NkGbB2DRSvhxEe^BWc?M9SDYDn1#d4q@KJ zPrg+LxD^)g; z*kxRdyLY^Em_OvVpaud*Wb*l0ihJt^q5(dmmp=!V>yVLCw&44E3of1iJ7G!pxBDt- zZEyBp3(6MG4nUPZ7VZ9n!+|Q7f5hAQttQ(05KLoLhQYsSU9aFVz3Q3dNeZe9lTzIP zFPBf^&+T10$&U;v!y{(IUoCdPQo|6cpq7z|wXXE$XA6sf*S8x)VQkVN)*;52RRRz0 z3+D>Qwdx?#K`A&FLH_YyY0gDaI-hy%p|pg_KMS*U3yiZRg)llNvv?YxQi=(jU5-KG z7INw+Hl8%P<7!Z)r5w05;9izfTsAU^0)86*b|g4dwgW1-)-Sn8arUj(|A?(zkinCw zuAt=QuzrhEtC06+5~%fw=+Pm#&);=`=*g|`S9nahnJ z8rbDel#uDCya=C&0+DdAvNg6F;EUD6xSTtZwU^jHKYoTVhyDDFGONW#6NO{u z{@6=6i>Z$%Slp`8eq`IyI{W2HX+V92MK6CHFtBDKHMy8$3YSWqizR>*>)W~h1aW@` zNrx;*`WSWMlNx(!Hn!&^s@0tg2TZo2yqTC(lnV%XIKg>)a6crcJ2#ahehu|W^o5K3 ztC;0tTCxv-VSnvE!hX`fU5@__`~T28{Cj|{4%7tu`}tD|!=x1ET9+18HY{i*3Ts&p zEPhre`DSIXa+BAyG+`lAY+0XK|KnU_>e})#&~UmMs0c@QsP(H)Q<9PC??x5V;da5 z;iHxy9zxCQ(iks2)g3WP!P#rvl;lToDF_}yiSZK}V`(YeCAYQy54`rql0$eJuD&4; zZobhEIJX7+D^+tTfSCgP$6izJWEt$HO!du@wkn5lE{4!Ledzrn zKF1ukZBa9&@5-5MA(v?D2y+hI8rx+i$TiyBI`W5Wu625obrOEIoTd*!Z$sTPHyxGV za_F`*^(bdHW4>M|HN$V>Vte$y@%Wf?JSza_v1>5t>0LKhc@r~yUFptXOhkSamWsX~ z_IP*g#iYA-ggs-8guz;T3;_R(piwSH<||TSl4|B}@*=i8O{{WOq)F{2`0pYmlWu~) zu-7w4rdQP!LacF2ppU~;l&aw!1N{kQ*~pXS9Q@rd$#8zf07~^^ zpi+A1sJ*TjMtG82p1f#I8cP*-d>C)lBcopUv(-7d!Yj_wux(%iDw*D$X>1ZfXQ5VI z{TBGPUHa$+Kfz$P~Q;}3_2uooJ<_rYEOEIT5cm0=AzpRqmQngsrsV2u9K}E}efkId$l=Z$4z2Bx;a<|QK zo}Y-3)q%r_V0G<_h%*rvFQq~8Z-Dymf$&3|4XJ{<(4`y;Dt9M3w${j6Id|-0HXrWi zqcDcAb9g;Fn}0y(U#?R~Y2cZ_)IT~diTv#||Jz*PznXz5*tEjh((SLjT}Asp7p@(_ zc6tA%Xx!pu{|KN7nNkDV*)=Xc)YIk8(+##&!m2o`V^+3(?XQg~NKB9e=g+_kc89cp z^JlmkMg5>zxm+%|JixA8ms20#jcTzon;I>4X9apedO;%4Qm_~WkplCuE%12EX485- z@#cnEZ!uC8aU>%f5&TDhvE?BBJ5L-9lh$d*OFr!0h2;HBv_2Ist))P<=#NTpKw{!W z*pp3HfzpBvlTY^Ug5KqCrA9cVzfR;h^FG@imp4xD6ItSvNmJhE6_E##UwHN-{;nG_ zT70E(9Egbu6x^`!S_07Kwl>-Wwah_*py^ODJ5~1EAt=z9;whlD39_B({fL0qPcN@G zRRCF`x$5$&LJ8!>N%p7KTH7}38#>p9^T)2H@2_zKN>A11P7=?bwC$Tas#gwl^uFqv z%Wz6>D{I{z>|D{D6BRH=0cgn^#*~Eom-Ra=kijV-3gw@ndb*9Q_c6oQeQ?TSkG@A6 zN6lmuz+W`9zC?Kp16TKef2{6*PgDNe zvH!n2AN}>P%2o$@po43Fkv!{&dpw$s>%h$cytFQ!3MNpcM+Faa4%@C2Ms3Zm*m!l~ zqVC7Q3HI+`!xv=#cM8@sIpo)oIlHkpp48zDoIXLJdwUz{d{fyzhpis00k5y84)1Op z6IycmozV;_l($<3YDb+icaert2@IWlC!A7u@sYmdk1FrB3>iDwIodfd6Wn27bY%9v zW0GNs@1@64+~#JY2_w%Z32er6G{e*ZI~bKTAeyL&;9b(-^rnHWqA|0C?cYf;m+r8R z(#fPHnCL{%tBjYV4HeU2If)GC-^Ol#RGOsn>icC$^c z!>3frJ_p#Z%&+NZbbU{9UVWZMdv<-L`&i}p4Gt`;Kof3kI6LZwqmgVlb^_=;8f>JM zRLD(dn}M@zd;Goy*u8-K+l&#deYqhG_-ricr=8z>j5lkh>GqfAPTxJ_w(1sO4-jLg zWz#o0PTRKQI9zWtU@iC^Mmz+zKcrY4B@V#GXz2DoFcm`SPm0&z&V1D9B!ZD#z^crm zHRha*%i;u%)dtbe1Lv|u>=yOW0XpK{;fqxOsj(DttX^;RM|x*jkgM5l2Mt2DaIxWX zm9PKxugaUO$!`qpxegHtlw$Ym6S6ELC|vwgZ!sbq$n!}^b@o3v-T1C29BqWYhfFu6 z>W$^Lb2OVrX#8dd5K+bRpMs=jQ8!hho^#FZ(JK53yvUg^C!V@wmsACo!^9t zmKPs@{z)-2m63-?^tqM@G66JYV>pwp>6*T~fmis>Lv@X`c_g8al-{c-Lay)2nVi`c zB)9M_u3kXXBX~JLR52D_j@rQ{94a$uS8cRTOX}0#Z(@j-OOD?gOUcLuY-QDt{mgNS z41OuFjCVF%zUjQ7_0?9LDo9+^9p91>7GnpIN_=ndg*{4+n6X1rHG2xPkg{SDI#ZS< z?Bm_IWQdLf8x7{Kaz81VT)~Zqk3E`QQ3>FLAl+Je$79S2#l)G!Css)W4;5;rV! z8rYI4hmFNAir1-pdO_N!URy+yr&;w>uX1KI_+-OB;m&-6ad^+HhP#x|?)j6&V6*4g z(6NhHzC`g_gS0WUCn9BD<#-v%Rpk}9AZ3GXNqiBg*|5x{Ah1eIIGJSzx5S&S_${`p zJG@qir6$zV9NU2(pU@{LF~6PrCrz(MpewQ9U+R-tB(kDTaD%x0$EnTV>EFNG8U0lV zlK+*m*Q~^}p2DRitc-wgpIb42t)Q7VzNVE3ptnfBb>xK>VK8f!4OxzGq+UV6Jn8i=ATuHz&`A zi`~<7h%_;UoU~C|{8o=V9AR4Aba)L^b&Jzl!gMqrnlCIIxoq&a&&17?w|JtQrQ0G1mT|`|u^HNp>KM ziNfN_CM4)7*?SrpamWo8$W8wgBwHo?bI1smQ)sC|bc9mEjd5yv^DRV2k7NBf)-!$& z(Ai(RdUnMXEi(#-Qktoj1Yhl#>EInv-hR zAK9ibv^Ki%OrpGgkXv{S7;&MnBKKM)6);~BH%b1CHI}9Y;res-#xn}X>)@C}y!i1N zzJPU5u?(5`JS_AMDt(KLQYr8wI-}_=hUt)cFLa%LBM#|?aw(okAPNz$*o@)U=ClWa zz<{HqfY?>Zebr}66LUS=;$zZ`rL@YqikW+K_Bl*XHuI$KXN`C|2O!_9UKK`?O+e?tginVZvOGrV6b2atV4qcVZ5H+Nr(_Qx_d=Mf^%Om zC|PiYAv2DeFPJMre&EiYqugqShm&RC2tMaWHIKoB=m*KMHlH6KANw`D-aS7nzVjMy zGU@NuhkO96ab0zHi$Ov|#sbE1JhMa)m@MK%zDjq>QUS8KW+MF2shAo_qYZ#e&AD)z z4BR-=7R$X*gNzB^EsW8)7}f^ULTHzv2+)9;Gb`w(vi^k8qr{rBRb( zP;61e0;1DsP}^8(pjR*6YrZ`TW8H%K6tXuFAAX*A{l?W1Yo#4pt-?swjWXSz9GJ+Yc*UM>xaCEA&}oMrCl zI*ccATyDxVNwT3{0gQO*0&yP4uNd?)&W40JP$owvD{X1_wI(Oyp`>D$()$*UQ=Vuz z7E>)d9}O7e1f-jJ#FoC2N(7{K&D?%2fNKwj5UpdkDb^GYu#;My)NBdqS(!)3m`!|I zI}*&m#vTml_T_7eoSd?Hj<^?aEv}8d<#IjCrI%PQvTq8_!4R|-h)MwFVtGm}_9)4_ zhjmCR+a5@M?;bthSyO=Qk_E!I{^_XoF%BWa4t$;O{*e-r_}g{yLNz3kkr!P(l(|9%pLDF2mquDe z<_!zK!4myJb~-K2N5Ws!v9&vFbl|#g2pMd?bFDQ37siSZ9)zFj7|ONEpVh?(7x0Q$ zMwk~3+Kse;K3u-T>v82>K0BLX!VcB1Ae^kGR2cp}dz4U85mt?=s606rA&e?&)lu@! zQ3Vtk$~z`8FH2WiqW;vN+=FX5=p}qZRsf5XXA=P zb)+uUwVYo&!6w&36oNH^J)Fv`NF#uQlmRAo0yXR0qGC(v#o5)`>)+&Fu#<>?Z#55N zPd`Ha0r!huxrqKSNu&5jxc~qESpPNZjsBz8gBM&N3s;A3YIgD?NZM@Lp;JOH(V|gV zSoTRdC?0_xd|Pqn_Vvg?{`Vii6SPnBu~cA}lWZWILyh!QdUor>cx%1O!_?-b5QLnO zPEX5UF}itMr9sYk`oq#K6x(&dDbcQ1Z0r)A1avaP(|NTLv3r zHXU*|uvFsb8N`atIkzaIifE;9=3q`@^SAibR<=!tLE@w!!815T?BI7Ss_-^`6Nnt( z*2GaSZ9VeA&l~hZ)2HbWnp-4A{;h6lv*K>{i_KeT?p$Q2P;(bxb9y^uf(zP+aD4Jr zYi{|iV8eV%z&R1eXMo*z)hcMVQ|mFw?5I%+NCz(~a;ECCxu!>he#TQVvTd>TA~(>2 zdi=P4#_uwC_ZB`s*vuVVLnx<^GXR6?@XpCokZHeW1|HfZnw9T&jQSb#?x>cZ_t#%HgqO)5D~Qz?$+(RlxoI!*GK+J%l)##Mctm-1GMq1d z5C~AYC-goks{Z^^%NXwGzyiWQFI-Zh!C}5l9_6en1e$f_I>c!GkZ`6H`Emv;V)Fqz zOzl_k_q>NlwLXDf3YM!cN_|SYw}@UEVbrqUf z$hov3t2|7UFSsSVwafm-C$kpV{+AXd<9zL{5d2MG{A2f~{(HImpBcNh=3f8l*T6qz zD1U_P&JOlBUF)J?1~tHgyc=Q3Zq+cYujw*dMi*Vt6QZ>OlA%SeNOE44cZv`VR`k=n z9Naf*I=*g#IS!!Rsn%R=m8sZJQA{NryBZ=m8O6AVxi$z!B=<)fCAZZ*=r+9#0yFla zk;|-$A2=Zkl^jFt*t3r-(wc}p5P6k@I`6GlQ|`J~>nYqROsf$~eUl(L6s`yo0sz&n_9eN~jT`YTSx;S=_+J?+H4X%GeR*%*G|&+HP18!$|37 zTHkx2k=h*wAI=-uH51V z7pYnB?|&ah{NYvcU*_L`WO}QbySh34^~mtk&~=%U#CXFn+>7*WMxPgUD41V{NHq{u zroj+ZnFT>Peja3$$Mi{rH5*B$q&i24eMRiJo;5D&6XR1&RkG0eH9L7L<>sxSQngVk z;1hG7rD!?je|Wa?`{jA{^4;XO)NX24!PHu^e9Bz}T*FbV)LK$gRS@ZJPSi8&83k*J zvr|GOFLrSY_k_fkNDq~;txk+Q?i^9TWf95{zTSKbccLg>A}vkY)=nix=d}eM>xKe( zAUgbkV@zah%cEA08`we_k`3N|Cn$)DiB4IfOZzkG&>zpuOYLOMaleQUQnaRfvMR=vzw|z z18~j^xKvX)*fB=1-k5#WSoxN*s#Onco-u2i+sL|4Z9N}L37$E5d7J*3GRBD$(Cj>2 z+u$C%ljv9F&D^@!bA0997v8Mn`-;A>ebtLePiAx)Cjgj$)GQ~Zb;;RRb++&pLQIur zPd~k*adrCOyl6f65eOw1b5@|8gH)sR-Qyz<>C*)j6)CT& zM;eryNHL?`jX*UUYYHOYT2;k&YY*pU~%1Mi|^O2iED&?xvwio>2u>dt!r0N_kJsQ zyJ3R44?(ogE=fu_dBKmHn=ynVfHlUvK?m5L#4?kkJs23KdK2RIh%75?N; zn$$YU21@0!G21FlKOA{&>zL|cvk(yZ+!Ev97?%i}?CqdY+XOrYjoW08i-Ty-2LMz8 z2Q4u9jPxRG;~>Buk{gAD;Ix2Rs?mm7Om60&hln(Fs7~2w>)JvZo7)3jE!Z$C4|w9I zW6uAXFJ~W5f6?mZhy0XI(Fjy>{$*M6CUyw+-)is7drK&l)zFFIznP;pMAJevkOXa@9b};Iub_%Xk{xNbV2EOu z%8(Zn8<5ni97YKYosxToqbk60CEeqW^2>J#BEJBti6%XhO*VU;gE9y)3uiE|YNAca zh>k2-KFe4hV?V+k*Xq1}C{;3Sy{Bf4EjZefUFjyIB=;R=RkI-S- z_0Yn5;4?%oeK8>*nVYbqI4~5KuZR-OpEo=0Z##9p0-HmE3Kvy-G&f8%rTuG8bWhrv zMrDKA37C3$9e4Rj8LhUnO}9_q$N)?I$~Fr*5ZRhz;}ia_P*A`Lte_JfZ!|u8)pW}z zvHV(5(>Lfm`rXxU=Uq2|{D8;MZXxzJ(p~wcX!xZkr9)9h{-ud*qU+UgYV>kd7r-Io zb$}`9Ej$Ez0k!QNL|tP(>n}@%0^SJ=tZ#Q85?q**B`(NNCL z^On%0Un#O`&m)X*r&}p3k)N@bIFGoeX_K63jPS<1qg|;!`kePA0*SUwBwlL9SV9V8 zz2t~Wf{VUQVx}@ePU1D9BLW6PI|2q1k(Re?Si@Cr#z{g@qA%hl;(bI=gaQpbEj)b$ zO#~gDva8BWNuFwiy{fAZYRZmu zINdd5Ho#Z9c>gg)fqw5)`EY;X$ad*C!F}W0si}fb>s9RY{kGwH|LQh%*R>< z&7CP;{gBOLwGg3`289N--7(%u?0CEvonn|5gJMZv!(u95-D3I27=?+)FA9v0(F&Q@ znAuHXHM)!T1BtwiJ70O-VDljj7mj&-<%>_Rv9o!v5wqL+f7x%}Z-w%nC!0TC;XnlE zsjC}&#YJk@M8#6CJS9o`e;7Nb@XEJkZ+9mh&!A)5=-9T+j&0lN*tTukwr$&XI{4CS z?{oH9=XuV0zKeM`ul{3HjjCT&y`@lo%3&ChRPUJO7Wzxwm&Z z2z=ggWm{w>Y1-N=uYnsbXzl_4%I-)%KUDByUYWesLFd~TnmrjJ;5$Y(xn&O;5CCta zl9IQYQ#@KXTfcO?vg=m62{UkQ?Y;x}&nxJ%xQ-XWQ#7(_u%Vx+N_R;xTO0=Tk+$HA< z@)hg~|FjG_NunZA`DU+r_Iy8cl3+z-Y-NU|)M_^K$)p1JsE1hz9T_XI4X!w;QNsty z@T_{h_4Oe`5!Yak`P(wdddF2t*^1c6WhAJF60q4(J2>=hzUM^lS&=F(0rEuBOQ^gK ziOY4HR_#`bfbBSS-LGELk{|kFmZ5pR531Wm)G?&{%?K2^RX>K#p+SVs1jrH<4RGWq zs`QW@-d~1v77t1Yq>Bg70RCh0oV?8tY9+RJk7Z)X9>~?H3*RxzH8al}30A@V%ZDRh z8@hh=zZLXy>%N?MEB2Znm&mLsG#B6Z(J~K&6Hp%4y(u}_H*G?mqpX7GH^*4p+|aCl z?cCpazG1*IU=c}*HyMLr8Zl(I45g+>jsL-?sKMC^Yv0fcuBwHLexqt@jy$nFM|m>l zk>n)a&I7!eZ%8xrkYZy6QZCci&~t{Bq4xfj@2l67)b(yZ09r7R%F8eL*^mK#i}B5H zX~iPYIg=+y@TOk^?>0*4M+mp;LndTRFowaIQg~Yt-ajZ;i|wM3lOp_W5*LZ>WrJ~^Sh5VJMmC! zD6gn3O^J}(GcK0M$|vApE=~))O&lI4rsj1iV2yW+%}hVEMr+ z#-l^O!6UQcdn+K5TGGmv8b!T%(7~}y)YbL13Ye}F&F_jWv`{F*jM8tcqCxxFaUJTV zt%@D+w|UH&fgq&SU`pTZy*H~O9*%-6U{|mx4-}YCg-*t=S2Yk{tu4(rch#$_Jkx~*RlZ6 zXR5v^oRFtdIKj9;VcxjOTDpTFZbtw4T~s!VOv-%7S28@FT$!T1>jg3vj(moQ2voL3 zC#Uq@3}{B$JGZsx!{YjZp-$M=!tBu{u5idkTa*^4upS8Ka32VO^IF919s{OL1;0>& zC0*R=@xq#505U%f=xjMjr)4saJ>V3YGvM^Hd&RD4AW4U4Fx3Fir=30!LU_DD8#fk2 zXEvoh9x&V9r*WofjyPzo3yHxYbj=bm2uB-2LUUr1*f)&9DKyOD&ZlV}hbq2ia3S|7 zmunBC^H@VR=Nx9R%fjSOg`LX~F!e2i3%YPnLsNCD;)(;{aKw;+jzoKg!9UGfmma|7 zFBd7x72#Xch0_~8AeU>7Sx9VsH-;F(8K8&h0XxunvoG!=K#q+IAqo*#j3iCiYfl1+ z+eR>ds@D|$u}SOwy+Ip(Qp>M`3>CXzUGpTZYa1dhEnw$()^xxouhkHKlBnr6{1rOZ zSqhtl6vT+*q6amD7PQ?Jo*Gx8iwP2E&!RC4+O3MyL;8??+^0lNM%uTZyKutwgxltf8-MVn-{4f#)9#so-5al~OsPxYAHyHWDW0K1Xn?q?G zeigCF;ohP*mM3LTgNB4R8yGHsn>=5VNPXbCrqR zcn-_1`Xar}#akzeNKf@Zj%P9!m-l&04AGFB6PvU04wS`=@sR`~4vGWXNXU=B`W2rj zSufH)gWF%9o!tMoK*0Mqf#7dV3;!RzD*v8p{ak-+HA{oYd9hLewXDpn-KD)87uXOB zz8ftiLaHmR1B`Ti(PVi#Tum{U4EkXT58emdcAE<~flNpaJsxve?Qz+>_;~XA^7aPy zi~r2AYJWRA5DPYvljElll#@H9&ayCq=P&58Z_dBwzrfl=YL_lbPad&`6DiI2IZu$b&=Fax&V{!s0kU6ugk$%>;~vkaU9D4mE7INsl^Yo; zTkz2CIa-^MIv>fZxVkcYI-slc1aTUV_mOQs2!l8O$zH?Q$wt4jrU!lGtlBKAQm@vjyft`~^2wmZ1SP3z@^+X{PqJ`QE-?Oi$u z^Z1GJ8eV}G-|8uB>jHYUqa263jp{nHZ)+j@HH=naygO z>K9ry?>oAGcL;V`IXc!7iivSwzfOxqFCMRjB9cn z^$WK@C83bCKLwy{-tD+)zAlD_0%f8&%WXjcNkh2HZixeFAwPw5J%V`RUh#Y5K)kAK zkpOWcJtcQdf_UOxfq8R7zKU*50(pIZD(%|*_Wt7v*4qp6ReWm`=;K<4$?{(32LsNl zvoGvqcNFZ`GsC4pYxdq!)Qh`p5I21V+=(b{7TjF;lCYluMd#))(=**F{kg4rx`QiQ zn2Q+MP+qj#EA(W0RJs%ATo|_AQ+@c(P@SZ0BtRqWTKAPHTIlz9*-1)qxajGws<^~a zRCKfyIK3(ZhF*P6&nLsvQ= zdMp#xK6v9aCAhv$6i*Br#ar}GaX#|Stv>Y5l|Hr3sXoFsoj!KXfj*Pl1d!OhYKR=& zqa;_JAX@!uNbcdx9TT;MF1udmrMbR&{z(-PO3XjB)I{5mX9-`J@5z` z!D6Udx(-j6n$(BSmDXP{r0Wt}a=GREN=ns^YR^J_UasY+!9xgXD&mF(gadayQeczX zW+GLE_H3S?hNg?t_k-hlSy^snW(OP*n3Zha(VCx%l$NcKfrTS|VFzD$Nk!oblAS#I zrX<%laI?+Z*~!l9a#FtK(`OeqG^~9m;$5N4u2EyI&rmRz5jWrEkpa2|>4@Jkl)qz$ z|J~7I?1kK{^*QE+Jvz74@j`JAPCrg%?Lm)LUK-$8YPY6UHCs5i6X57uAcu36u{7g0 z{%|1)4R7jJ*5x@bV&d)Gvr;KsZ>&8eCd0DFMG+rlyjF$?$B4Z+M!H_fKpQ}&;@g5H zzY9%4$`bAK?YmiDFjBPuBTSJ{p@0I0oP2;}>&iIUyvWVo7Z;w`8N79xAcG;9b1Kuc zo+`E4#uA9}y4n_$lU>Vlv=C#n`ff&O3gt3vbrcQWRX%CU9KTpev276kXfkz6rd={- zBKQNOi@Sm|pYjxZ*1S|NT~KN~gWr2{GuAT;M_XsQR6Mc_bG+ecKZ2`!HQVo+P#zKrmbsuBWWh+G3L`Q!v_blhEM02LX zLr~ylw`DbuD)>90?z6R?)#?&zPv@1anwtfy*NTIyA9FPUiC3cGCA3h?Inv@GlxX$T z9IwBgRSsaLTbhh48zjoR>@WAyLPN*UouAeXvzbeCY5v=(0qwin1_uep zH0;|(7a={RX}Qdowobo^zFtyDtgel&xzC~PvaiOza`PnT0u%MB#$=FueF*~NxFgTH z)JhrOx|B6)+ln|$w;}L?nkDTGjNHZb@bL6eVr00H)`vsw4~<}hLbiz<&C2#$=u+(7 zHKuMqheP)TUXMILzV$()#6su7B#>ZETh$$NoakfeAYW$3!NzbeC8BiRoBKBp2?<{5vuQ-AC5hS=$ zO2i5%RMm|M#0^I&sHQ(-4cPaTZZhApA$@zV8I-X>ReY~uU<&I}rV7{jZPIy2`AXot za>)w(VdFQEMI3$z?I&0d|1iTk?S;DUNsYz&ty4>367XByw0!dryu|oA5meY_9kqf} z46v`mI*E%yXKsV$zJ1fSW-DwNRAj3%EFa&-M8+|juk>t$oZ#hX(WAds-mXQpG0Vz;m+OI^nGx%7sI_w8G>}TAEI7z+%pCQKicuHatpFTE zEl*xT6jVEYv!gG<#QnMrNC4D@>z<-&&TPhX?|3Vz-edNLFjTMxV~8IKwj2>^l1F#3 zvAH^+pT5v1^&7F%8E@4t-Y(lvoNN=BQA zhl=v&V!Kv2<(d5XO;`rW+=LxijwxyY>Ls!vUGNqhn^4ZK$-WKnhIo%+2WfmJo%^{+ z4*&6h(hE(k=7?tTTgkD7^O(9utiFN|7XG0rmk>n>!hHXZ@M$GT< zIAxjH=PLAl2BjI6!~xf`EY$LT4VA!FleWl(M*@laDc-|{S;pmvSOIR~+%ai4=FD?@ zVC&DprI9F!BS=GjUKqJ>>|_~Ov!g!#=TdvxaEW_apiB&RR0@)WNWP~ayLQi;k3YeU z2-KI^xKEpAJRH^G#8{t4Z;V8~KtlVD&`QpD6GxO}+0Ks!Z(i%C5&bVf) ztgv#?Z%&Lc(_?=;FGg|$Cv}%Im_k;NJIi;gL8d?|A+E_-%kzr!vI8UO$%T~$ZI^|x zXevf`p^<6E>>weTw3XreqU2B2isWSJpdaQZ&B+M=?ByUwN2V#&`A!54O;#~_ru z(aj2dxdX)C#;~l*qL^a7x$D+uXlqv5aPV2|s_&zbI`K?QZCj|9l(*ODZ7`)$W?8Gp zszJ#;JB8UqkjiwDm}cUF7xxT%M4O0HCGoAJR5ph=-oKj-n4jRh**gWRkHz82g|&WEBdQc_L+uWMsOUc4XB?4uRy*K>n=K^}_VB<@{&r+kvO@nOF=U7g)ZZ=H~2( z#<-Q&u97w2H$<6x+%iT(a4rtNf6e4cCP=X&a6udmDO}VN^bwFOtuC5nVGVDP8tZ zMDhDALhW>^L5-bOpw4u!AqWq|Utjb6#pLV$zFZc4sH`dof_9TYqYWEc`tS|HUS}>f zpfXnCdBnu$M>fkBk1te^uB`AW;XWPSi}>))pTNX8CZ%t83USIS4tRHJD;fOuo{AOj z;nWlujh?=)za~pM`PQ2xkD1}+<9-qplc1AvF3XI#iYcR;63mL6qH$7vUk>Na^2jps z8oa_QxH}^V!`y1C5HoAY22mPsJg^^cBoF2%+yO%^mPxyTWx?vjo4MvmmRO6#xy2~> zW-yb<-;3gTV})}~>Bq6_g4}f&V132jWW3 zjn^RS?6WP5>JgT$^$Pl-c8VMsLirnSotQ#ly&HhaDUWxD|CMN4z4-=CwfO~Z`9^QA zBhuMK3chD-^RM>&3EIv-QyI+v*mO(ux9sKLC^24JdlN$)J^N42o4tYUzf9diPEs22 zb109K0stVWYI+LB$4N23v!}t9z$GSMp3Nc73SM({bzLNGUyn`3?s-T207gLw#{tV# z_6mH3);rHh?B-t-S6p<{m{;WX@_GZ+hI-^%gRvp$(+z6HOW-T>Q%M&HItB;W_@NsX z&;*Rqq9=vhK;5H4H6iPv*#YVgR?Gr^k55FiAg_V1?o!7P5WPLD^uzVOe!#<`p^r=9 z`G0TIP(7}%Ra@m>zs9nt_ZqW4P8LV%l@M0DZ#yF)=Y|W;7I_1T#9MJP_@Srk2qHv^GNyE)EZvO1|tO{Bi)*jMdC=N$MxG9797_r z1X&Xl7IYGcOg1E77sYl$5aJqXh{1=!b{UzEV^u29WRp>d1SL6Gszec@d@}m zSj@w@Cee>kujt!7FRAS~;2YV~FTaArFTp#s$-9gM3hU)tjGDw8(>`lUt6lM(kfV8U za6;wxkx9lLJ$t{!-l=OAg{e3H1;f82hL&&nOl6b*F_ry0SN4B%&{ktjK_CLKU z|FuUpzr6u-MXC4+=*f1co>Bd!xlbV6y(H_`pHa=0B%)DEyT-*|onr7vxxp{vf3`x@ zOA^jWIvReaPliTD2aA`~n_uqWTvhI^S9`HwojBJXs}8n#kqjv3x~!W6fPT!=911`< z;6Sh4Rr?i0x*0!%R}IO)NLUJv$N*L=(H45T@o;WMY~GkwB47n|f1m#h(xJt~G^!5Q z)ib{{W!O`?)A&X9BNvO7F1e1F^SJR5O?CVInMFEG*2fG|VOb<@ZiXvIov9$OxzLdG z2n@KmU&#|O)IM@BBdV3=X3@*HTfFV;>Y zCr^}tMWU;pj~95bi91~(LZp1NPUt@mk#w*!Lhzk4frHrVQJc`G6c{6$~)? zW3|fwLt+LJs2Cw0*1r^n|82(#wI}owKx}cxTLH|4c@Z>LR4r1Q#cYsC%`h&Iydv42 zw{Cx4z)F@AVx~{4PGaVLv94PvwVy8KG6CU8zzb*{RSehLo)nv;`-(K!nY4u_$pMo(~O`REQG)Lih>7`;&+FyA$fQbCY zmo%_zUt+(^@Tp~+rK7_{T+3G66ES7LVvq}9Uar!CTy@g&-LO4*y9MArvN4zEK9Is)MsKE zz74V(Ib)AAjyR9Ys9xhe)to7o<7)MH6y5o^a?m+YS3`6W58ll)WscXdy+Rt4NfF0r zP&z^qTI_fg+*j1&=vS*dcd*sYKHtpdSCF~2F%Y?C@w<#oSzP`WVCjD?+Do}-$dLTJ ztnB~Ta4hn-3;Ta)y8r1r{$GEvG>~%m5A@VQmQxz&^D%@110xwq{Op%co&80uSwuw4 z$0tY#s;ATF^h1BRMmT~!2l*XKt(XBh0r|5rwE`|TN3AW{<<`*9>EZnC8H5K*jd($# zER{61C~2vVC#nHXnA}%U1 zN|;!@2qQ-~5o{zxOt`;Z-7AHVjOlaDB$zA@|HWu;urLVGkVBJx;q;I}H>-pH%uh~k zNREXAHj$}ol{GlTgn;je+-fYG$kyaf=WVV=QlQa*=O9*+3qn8 zi-*t1eIGQ|;v@gz;JUfbffre-7?SYJ0+1TinHJC+Cf-@giU(hLGH$^;UIatURO@6h zDHQNaRPw+pnwZMZJVz9eWnbl_0nV@|gcw-WP@kT>ny&Po`AW;x5mTn>4{*DOz`7!H z!1G`D(+Jqu0fkT3^Jk6S|E9~}CC@S&uNY#& zE(ORUm|uK_P%ghsMo#9%!jaRFM0N5+>3*tic@_f2o996W{Wat%Tb_HqHX zTp?J)5~Bv^;USxp_`WlXSzFfwA|gkQdIJsOWE^A${d_?r*y=_l!5V2#x9I0E-6JBg zI7F%|Y5T47x!P@TVe_x@O+qy=OlB=<^iO9u=_KYS`y1AQ_m>Cm1cP+;YWfN}fi$kg zA9MS1m+E_F6j{y zT0_@zvx-qcnd@ZLL4fthDK@bB1M-aCFsKw zIkX0hSvO38ei^>H+-i>xL@eISo6h8WO22(3q*zV7 z`d5DM>y@(Ef8|^^EK0*`C7+pDD3B5MDD?Y-A5Qi>&}Des%R_*~jH!~!sIr6}eeiuy@6RhWdUpVIL%Y<}FUS-3mJ zT%6vLjT4R6`?WlInY+7@>2OomPsfW)dHBcvqjYztTd*zT2NXHAO!)!1dGVAWSSbi! zrthI9exgUj@Uz$fl@L8k|I(T?Dq(&#K+*l9a7Z!^1Yjx9DbLw%b&n3JeXHwo1$Zmy` znKbmgd=V>8VuJCb zkneG+(QOCkDzd!4%$+g`$Og`Jfl-J-T0GLuqI|^5=-d6WGtJMA>`P+E64IUPV z&|{jEzh4}D#H4=@+~y8BceVv0H!qOEhqp2WMHVgx_Z~Ph)jrreiE6lD$tyn#5%I6_7UfXDai z8&GX9EZ|X4B|sUviWv`oE<}JU0@9phQWAZ&A98QP`mH{Ci~EqT+O|+`b!sW8qqJbc zwTv?zK+rYLPZs|HwZ!oag`7i(h@5 zKQkEb7@Ot66z=zPVa2p2YN-7+n5GoK_f~*__b%U17{P@Wj>hyZZ&2k)sucoAHz``( z50+?Y)z1|w@YZa#4y_4skaE%4=TJPoL-AO5@p+elMhk%OFmZG*7Tl4Q>ItUaei#L-nm z5A(UOcW}Kyo+6I-*1Fc|a+vioW9;qu^fTmDm#^Ud#b=8)!exn)RQGt#R3@h#g!~NrC;5)!IwoVy$R4BwtYC_s`nsQ zeQ0PUnXVsMrtn$j)us<_GM-2D*1NAmFFqnwH2YXOl1Z+Ho;>s+i3As7b`3P=Tr>Da ze=pPaDgEYz*p(CT(HvWMXB2vm=3p^m^FHTnOBT!9a}8MPw(2A;d0m&}G}OQC)!)t$ zPNnLgyOeb!*+ev~zLWTnB@Ub+g|g?PFu33#pl);-fAgQ!6zv3FAx^q@Scr3!$@IE# zY_l%dP7k`MB~>yA^ooj_0ilN`U}bd3Boq_u0XDR1v4NMRKK@|!X}(p*Q{k@Em&3bN zNSe+fD$rkj-IRVuWK&92Y}U}TH2@ddD=^p_Srd>ogrZC3dBg)FN7medsdyBFjgHsO zYO@8ZaR|C5P=ASdU7gpHz~$46p7gL^T0!kg`|3Q@{{wf4OXM1?pSv$N}NbZZ+N;TZIv_! zz&1AGY-~?%LY$dy@*E#Noco@-$xm)>>v?g#-8b8K(r7s8tvn-SYk*aIe6teT#gTO5 z4`L|CBC)zW9B%n~lI`!ppbdt)eujZyb#X59n}mgJGZG^0{O-#{?A7G~gzMr;uEoih zD4uiZw6M!_hGb-^jn76w*jawg?MU9J3OgfqcY<&XRg9E@rMcAO%*^|+tq5v9^u?WA zmo%Aoh1{-RN}H{NGOkhq0Lz+i(JDjXBRH}z>hRF#k>p>FQz{zTuRHT-15kgobynpT zI|>PUseW-`nr|_@7^j)1CUCjrU@3DxIvsXabJ{hl(E$)PXov_?`&U;J#Rar5^A346 zV_l85Zw&ZRU3@cDZ%M}y)#C1Mhe-Wpij|bzuO@Ki-QX1KO$2;qCJS2uk68_;5j%}l zXqR*}>JvsX>j&LU913TIdS^TD{;#j<(etw-aGR={x`RE`gVP#CI>X zJfM7to)~Egzju{SyrZNpDx@tNl6|uehzh2wBO!(igb!#lo>iDeAhB*Lt@xXB^nv0) zs3afU7szb9L-va6WMomow|eM4bVrE-wbwj~3_A~8Z>w>K?@bFLLfLt8w{;`Mel;=q zG60|Quu_IX_rV$rePN-V3zckM3zLkGJ-IU|V&6OQ&??=c=n7jQ{O>7aP5Q+!mYULJ zpn*3x-xP-Buwl+9_^B1J;cK+XAM$&IRs6y(jzG&b+KGWWMg6u!X2fbVqQYp4O!FBQ z;|;66a^AJI3YxmayFpa2d!m4&u?#Eha{pjDiK8L&470PEE5?iLQ}B z`f4zV)H)z*qtJlop1m%=SC+F+gRfAJNsWwr+s_N61w4M?9QF2!DqQ)LqZ3TI&NUd$ z{=xmntO(bJEipeO&*^_0#rnJCDPwE!ALngvr6?!y-(K4yO6?Oi1Lr@fWMQ7NqI$Uq z32K6*{!#`mM+)ArHvcH8ZXn(gZBKAgIfDX%(`V!r$08~a*8*$j*pZ%$H8FU2it&qh zfpSburK-wUXSk=tKLt7%$zlh&B zZ=*?jsmO=}?j;4sh4VNyeSk(6b@LuLKYm$n=+aY5NNqb`pVTQXA<$+sb~^bA2;YBK z#1hv1J2+@7#y~XnD!B_P0yA?rr*G_g7GH+W;bRM6zAoO$AWt9*yC%6FAXLp%LjByq zOcQ-2$-dAbn{7E`jZko2?1)Ny0h+O`iHa`uyNO|k+S(uHUa@Sw&U1@t_=1B~Y$NKP zfK(g)Q{p70_%03OCZ@ap&92CuyFB(lBvkEEd$KBn_B}~KH$^uC(O95AX-!E&Ps&DOpg~(ZRaDU z8w?{rdY^DuDawN=CZ;-x>X(8i`E+2Ni$u;BCCv$Nnn8PcS_NfLlx&GAXHII#O!Ut8lED{x6KXkZQ>!Cl_>fXIlItGtI>qTJ zKQO4{n~0ef@)1guN~VbH>}gxzH0puZNpYn&zPPmakwmT&sw@qH%bFO&Bs=~BH#sp+v$5#^x(g6-xJhxklDt_R52f9oN3X`eS@6Npi%0 zB39FXj9C9yLDASi&+K!e+u7yc^vZC>RXGe{IL=YQQjG=eF)BxMd;ln37!rH+?iak^ zuVA`1!L)F!C`Lrky4R*o?d)<}tcnvTd)*UyH%BOGB(H{oE`;63@^8rn;&1SvEo@eWMvhqb9>~fOLwXv)%;8GW$+Z^80++ed;`qT*`gMv zeIo7acI2@vP>r4h3Zqs5ICNttL9~VV{nnw|z*sgg5xX9sH>T#MO~W+(SJg0BvuyK8 z+D#)g!&g6g9Re@_muM;j9KAW-jALj|KHFTq$V{kA(C)*A3EOzR%7D$C3hztn$zQ{U zFiwEI>tt^r->w*MivW0RRBLA!*i3qv0fdRG>Z~kA){eX^X4Z^lMhe3frgU9bjh6)Z zG??^M$d_30@eLy3kesXG&;2er%66^z)P;leyc>1n_8avUHB^hDi6b*(%Q+oq=`{u@ zJ$KW4r;O5Z(@t_*2N+L*kd~QUnKg4u$8yEa?!;DV5ky8veTFBowRY_IAI{@Wj}Mjp z7BW*zhsb_Fda`2F_LMA0AnVX};{cyajn1qs3Ex$4zsf8MJyzX(^%01VuaA)r7-ZFJ zcj&`WghEL~@SIW0P_}gGb{fg=(%s-LKbTx{Y&%XNv{do68(Z4gI-`H0%4Yd*2r@?z z?j=LZhCgvrNGv}aWgzw zEtBuv)Ypk0kB&W&>ZY??a&3-A;r|2BjiKX&VS`UARt}wuwp$k4OHy&S}0}rs$mX*gM~<#8Vxd zX1hePH8Vfh#a?z|Ogdh)qgyALl>Kt^`^x!`H2o~W|%{Qf{R zU_Q2P4zESNb9?L>oo#!Op+S59ucTIdt*Bz<^L?iO$C4z)-yHvc@gH<7KhbcXJV;YZ z9f`ld{r~0g<3DMkpXCHjtFD#^pxT@#Js{KR3SjbJ90?&BL$OyG9{SwvU9-ooi3^8g zRlyH((IiyKyWC)}xa$rFApw5f)QsPrX>J?Mm5;w8rM144>D2rEgce6PMaP65{UkCX z(w1lCbW!I1N~Rk?lGsY~xdr{P)#6Ql09fnE@m_7x?;3yxc2L*(!ap5St3U{-bsRsv zT3icnT61!GjR2nk?M>2fo0zJ*ZKrW$?iFEZ^XV*2xRVWdX%frWGzTV@SpCa9zso8If0MVp;h_)dKaG)ShDTGA|2D zv=`y<=2i-1zJDEXiw^kIx(7%+EzxX2mZfuF}7W)kS0zNFv zXA5Lkmy`rK6gcyc)Hx_Lze)4L7H=>gqc`Mkf0qo8U_r4N6SQY(Dd)|wpP8_g9&<);r5%qUeOs!hwjpI=a<$#w_K!!z84<+MJ! zWVu8Yq)^{JjjnDuc@Wl!n5Go+&gMSO+J+oYhe6j3*NXhio&E7ssDaI$@7^2pB<5j- z?+jsuKUcs>Y|eT~I%*nhnW#yakwU+Gdz+rfllS}@r9-j-=X5czMv2rVo@hLycr&NAeha*Ff^D}}fFL<1Q~Ho& zUf3D14?T4VM&B5KX0ZIhgaDq-7(Tvc1ckFrx2mk0s`8zQjDP;6d)^tdsWFhQ37Oeu zDBiSjQ-&}NMXq$4a}mWm=mCw+ZCLn4W`Ps5mU`8)W$h5K%lO@ywr)&ppuE!Y41lm+^NBCWJl;Y75s4$eH zHyR0@Jl`{fYCxzTTE0HT9m=>DGpVeCXZZ~8!E<7rNRz-aoLJ@`V{&*i{PKtKnE8)E z9q!*eww#0I-?XuRTU5+n3yT9bO&ZI~&B&0xnNlcNjo?e|De>##5)(s9@-$~^1kg35 zFQ0+oe2BqgAW%PjeI?yV?vMIqwsp|fYLsC&(dPDW^72UZ>(N=Gw-o)wAx@?-Tc=kZ zV+|BFqsckqCyIu>`d!tYE?VdhrK+we^fH*p{@|^Vt4Z=W97{P-T)u8$*^;n=xztq! z?H(CWrtx|GnEW1CBAgOJB2(|8+CFKd4@ALiNWcCL^I1`ss+P$s3+qNJJoiT?SgCrC zQk;0DV`AT00~kVIcEn_uh>iVv06l;Hz7*57!RtIS9mA!XW_qs703DJnFzc5@>nGrae>$)R@c+$m6ni4gm|30 zt}7p1M(GQ<6~WRkR4CaL&Pu zlQlJ0lHZG@oQS@IFWhBkOgA2$d9KtNIki7*e!RX?{F1+-Q;D)s9zdaEKr5xY7=Drq zAj&1{P1qhWe#8$%?`7yc1z6Fv^;@TGn z{xZfr(In$}I6eG;$oc}KcvS&;z1BV=Ow8XGt%LT7}_okYgn$*w;J+u_bfV-cwOzn2?aww|hpWEs7!fFjg z;BHf_PE+Y4+AS@g_5v_7KDEm!ZmX36pG|efwbYdo!81Aapk<;#CkJ0E3f`QyN2hpK z+=i)kC+7MGLsa2QjY$Y-c)AwDNsH$cBD_j9)z-sUM#{LR*nH{Z*az;J3=||*6YPOE{g^bmOaO6Vm%~V-1G19gbLqsQe z`g1gx7+t6ZsanD4Wo)vThJ`%bI3<-%5 zz^VhgX~o|U_21%&)~L}cihmv`>y#I?yyP9bnU`qQFkC|Blaij$asoOqXEZh+#tLM$ zcGjUcJ493U;$~3hqc6=8tSBpP$ti)?wOiXy=9RGqONGxFyU=5u5{khz=9BVSQ*&vP zeuV)Ld*G!>Q#y$<1ijZn@~}lyl)Ur0kwwrQ2%$)4w%A}HLhCRH$f6|_ooHX;km0%T zCBVeuix{9OniRV1Y@ zw~A8m(`07-c1t9j1;#%~hU?-i1%`rh*|9Mz+Ts*YM;s~`YC-^k;DTkB?et) z%Z>$E=13tB(9a|uXl)*mPpU<|EMFewZd;qt-p^XU+ zB%M2YpU<=|1B;9n^&3&R5#15oxV`Jo))zA%opFv*u%rIiC;bSHK}>wRZ%?`S4rQw@ z$V`Ls9cRjfld%5gaZ4I{n?!OF%N6-3Isef8invNZdKZN-VYC%>!`nF^d$Yv;yztvD z00E!}5JrPJ*NxiWZgNh{bP}IZ!4xVHvY5fD7VrwG5SaUkZzv+AXohvCT*yYMSjdiA z6rv9{hgu91GM{Y^_FNLhepJfU>y4w!aUoNWjjq&%2+~3~8 ze}^~uDqzkSi_!0a7T)_YaXis8B`SFW;m`di8#Df#C>-jmH$PW|EwM073T3q)eenUeL7AhTD` zMt}?_WhjZtpCa>@qR^*^lOZjZ5zDUQb0Xg(jguWxBrTmn^S_Wg^7O}+ITD0ySIJH! zy_56TESMTaa`oq{R5((EY*)<&duxKkxgrQ^e3}loda;4tI2o*W9*R77j_pbX*&x39 zfkgF0;_Zd(tg^)q6ba=nDM+DNsLRS#t_#9dri;{(t;?uMapj3*cIDzt4elviS*PG;%KuOAq`R0>5!yQjg3IgCh%r3rsnCK4dRHcO=GK+ocgpL=n}~}vrEuBt&0StBWK4o znB2!prW)8vrW(OY!Nxb~yysN$ETItt>^=Cd z94FJZtas#s9zBl8VIt1?NBhc4Uq1K_wX-jvlT#V~Z^mV^X(>h`E z#~oqx^Vr6>r)D9N{%%nlxZX;}7D-S`lu-5})lUrP&FnRBcm|m`R-cy{;^2pgeL@KX zoAA_;<*Q<@Ti=f6YoXAisPSR3P_EJTAUY?;N6xezVb$-(WTtyFhk%HzOm;PhS3WxiTT5-G&K8GwF%hMBu}&ms}C~3bIhn? zR)_D2I9FQe(@|OSZY2|LC~Br&XCRri7&hU8GHY9JbnIWJVN*=@HM z;IoJijvXUhp*2@ih6#MBAK}{R(1X%dN>c?(O875RDN82i$wTV7unXv_= z+B&z@&O8MrGhk6*ZQlKL{fdooFRRQmBZB|Ah*~#pJvSnPU*elS$k`S?h0dXp z67MUzh)HX(l5pdlY;OTOF{fLZK9aQKJeX#nNZA#j?P35W_H(g@NW)~-hDsh16;i3} zam`vEOU*YQ5tT8G;2`}cJsS1n&y247R0?@M&(|({>MP+$d-0s+;7TGE`8$=VTGuxh$H`j7d5D34MAje2 zVG46vxaltG-gPqhZeR6CQtiwA&@QB#b15&A(tF|I731t+e3ACR*sUt#QYDcTC~n?t z<(IiChjZeOO7&Y%K~O}-6uWQeCW=pHmLUhJshdetI}z#KNH8g-`Hb$&4X2=C3*;z5HITNw|+oHi**|z%_cJ?8hL#=|lzk=nh2*^D;UrF72pyXj#~F0VX%pTeq{XDpeAR~mmB-D1>q4@Hy7FvQu6vf}UU5wS6E zuqXw=>)-E?yBq>`%S2Y-a*Ha{nbMIeJ%^k)z)j)yT7Ll8Z64HZxe&#>Qk_J6l5asP zjfTBVP*G2o6xBY9(1spuhN})Qkt}G=eJwk|C`s(#@2N8orlk4#%M z3F+t)n%QfQMTus4Q+4~cOC=s9^W8J~Sc92st~SlMfyvW4NA(ChGikriIOhM`Fgkg@{CTAf?BOZ zWe)`?ADh;>VZJ2O5yaVt&}qU}byrFG7Ky4h=H5>4mHlWop3*AyDDm2EnC)E9BeBia z!y*lBnJA)Ie>Je-6E@pF&rZ>DuF=!-%adz?>|0L-k9(a)8L**YCVKm6tI4{le_2`U z#Nrm8#=}|CtT(WZ-)olqBYWMs!Z9v-NOO`euzri71Ew*(mFP~2pOU#6>lkNt*yC6) z-kx?N9nNqeg@ut`;P`m(I@!-{%!kNn?4xt)_xYcXafvP#Rd)i0gp=M9PWK0qlKv0X z$BL&cFIE`^4s(nqI5Xg11x$&l$d+3Ki0IxboX<&gTW4JA(_ebz3Tmy7c7XQ6oGR$=_U&{2;u)vIUs3o7@tDH53i-Y3n&x?aFHoEeY#Oz2T zRK2;vnuu@0va}DhR<#nGeZ%Pxmp-Pu7}Yy#ne;Kt{aho|J{+5w(Qu+nJPms{Q?-#l zqrQwUMbE#p${n>g`iU7!G~RKT-oq?I@N;gf#Po?gRP9Qg8TuxNDe4@JLpZG(>6*Ptt|7Ik zY`meY3fqWkG(S+S@rJg<1Pt+_H6 zF)CX_Lfzvj`T}OH>V+b+*?H}|;&i$Mk7pR&sOY!*rmh)D9y-i2 z|9D8@IG|-iADA@76n_{L*cW`@C&NJ!X`{HJxGqurDe$<1rGzQDqwyS(T@JOAan8O0 zWM~$#WZ4};Urodb4!$C&Y=K8?C$BBkHll82lViFsE0U2XbetNs64r>M#w~9oh;>&Y zh;~tr4kR=hDk)g|;+Z?46QojpWM8+&lr)B6<5c0FsTZcz>tq%o)BrNZ>N56OUAGK7 z2l!0N9;&SLhNm2APz^&)Futu=T;>MGGDtF8bGx%GUzC4z;(3e8#-HeUWU~`A9q1p1TXKq1lLAQO;Ilm-KT>JNKGig*ooa zf@K4oan!5hTn@DBaOE#vES@&2pZ2zf6JiTcULF&A8>GBv)}xVM?Q1_p+0vt=?JsJfd0wA-@GkohxLb~7lq^I zE;Ekpeiifub_sH~=UEqc$2iemiN0SEeI--hs-6=v^u4RAGyIows*W~T{PQ#0&WL7D z?C+FIRfcI2*1|El_Dv__3j2c(0ViJBLWTG5+mf)x*ZZ?<0bP)XWyJ;#@ucMw>;=gh z7GU7Do%o0`g8T1sxT10W@WY}DE<`9-lD-~my_R6{RGZ{8 zY)&2}45g}obK@#1$3{KnLXj2-4Hi6QL-*8L$Oqi`{%oi{AR1@RKe_5a5 zS(sp|TKI1=9)O}!)dao^di6S!#|^V~>nVK*$Q2P(U$Sd)oI|Ow z;&Z;SHBRTtH0FfmzYRAx~^ikQF@JmCGe9<*n{)vN~vmQY9T-xt^o&O zaj->@PhCMEl?~V;!dRX~mxx~4Dx?%8wu-@Um+u;54bT%@-a%S;Ceg7^n1lxX^yR^P z^~Sc={G8)uEy^m;ff<&!R{W|mz*pYES~xuV?gzKsKt9B6@g*(&pkZ=jz?xQTUZ#3-)`5(XURFvT$V$s542S-we33U%*r#9Sgr$of9^ zhbDCnv2J;07v(*`l_elo$Pa%1FJh2zN{{&`5QLrkM^uyjAGhxxz59PNra@}-GNx{( zHvgvC`YJ6;k%H99i>9SkS_b-92XM5bZUI%D)ID2ZLhajNd}__6$}?rja*bn8Sg&E% zf`Z*hsWG@6wmqfDN)XL6{~_m7*mG@mhOom4o?I6EDE4pVn)I1e~6owl&+rh@$< z8U*1q;rA4UK4KJj${3KvSjsR)s->*aIFe>69CdMCRTtVXU{im1tM(qa&L84f%lwFDJHq#t(0y-_3E= z4+@}ce`Yemc1cQpksMFB4!aU^qV^j(68l19Z(^Y==yPdsI=9}0Q@KkS=G$7J ziTDPH>=<;7HmwvXwacibmIcT)D6$^^yg8x3>|B;?$Wdz;Q+SnFDAoWbMZ2wHrca5i zeqkAxQZEZ^A~jAu0Z0LbGMaFJ-*dmry#Eye|Ba^r!G$!01hz0{x+;hE*eFT`k-!kL zmu?nJQ90xbmdPYhthdM}+RY;sP6CHMWS;mF@+g12-$}9M?06ZeP&IdQ@suI#%z@YAW*cg|C>Z zR7qKJbZMTy-idX8;qM|Mr*NgDW^#6vn@D){IU}!_3l8QiIo4b| zE!$uYjcHB(>o0(vGe6VdAazvW)Dr5RX9?OmN~CGCsE!^7)T=5s*@vXBk2u1%_XGg2 zhd==RJ(1Gzgt4n{A)|BJ3;!}(_k=!EZ`$@*p#IH;VdWT2TE(E{@p77JUMP%;3u{@0 zRn81y<)I6|iqJbCO5ACYT4@Bm8!I8A>^>GGUxX)e6Q~Gote4@9dDwau#R=I;1~yhd``>fAe7*mAm6JIFP^HV)#lIhdWU$k5X_B6XV$$ zUP<+!T~aX(HBo;qCSz27eX%u^A0IpDfEo?B@cXrKN>Lw!WiEI4j%bzp$;toC9Cz~T z^-muplA}+&J#{$)7x6v%3ppR_XOA;`#9H+gC=b#3LIG4*pS68cSM;eQxb+0?)S|e8 zvmRjGiuqR+W1#ap2)2n>v*cioy;E1c9agL#3)@BG9oBJ(GC5_s4a#x3Q#&A;1DX@8 z?|&W`i0#umG}v8H@nkqao*wvf2bIkZb^|1JG z?EPXhfY#u4CPRB8NsH_9l)eNV;gP+FZdHv zzMLtq2tlYo*cF`l7xj$8Ws$jNFw9{hxY#ErmB46PvjH?Y1|uC3WsV~bcM}uAm@L;8 zaBR`FZb$_6vhK;Cnok#q_>68Chv$qugD4wpU_T4&cBKrm!XkEdm?9S&ul&pshXP=b z@1^*V<=$Uv>s@NdIcr=9VxmisMx^&n=7<BGQ%%{+#~Frk)NbnHcg@ zl8H)TS9zt4lUkLs1{-R*me4Sp49ItU7U%h~u28QVcvIOWUS~p{@Y1>&K1}_f^$`tB zyN7%Fr7%|{!~By7)_qAtD$VH^vlqK0&epo%OsAaP%tPC8s=sIE|*XD*&uju}UC z(+)Z|4{j*GPu^FW8I$1-N^~UXQdtE-Qy1phJx~%mDAQ&r9ro!-&*Mga`^bL5p)G7o zp~@mzh-p~@_OxxHAd8rCvoQL9i1Sc=HdI@_n3#lfQI--wCGB+NoI{f zvu!HfXn4kYOV=CN?cWEVZnU4>$xl20324&lC1ET3RS;U*K=+}hcuR&8gj z9eDA&$Ve(s#Q2WOEj5Y;ckEXaTR~;-yG-~}u*+aH!(hi*|yn32)l`77CFnq zCm@WXj5};1Zy>VU)J;T6d2feT;>_5x-^nO8lZo%iY6=B^oM_j5A(aHtg7^VrZ{+sC zrgbN}kd+!&i}30~QsE4|o@6C_i82@7Mf6VjFZU6%nA!cdr^@_V_eTAc6R$IHSzRkw z&v3m+Ja&-&QeyyjOA?hE=||Qm?m`3t+bF|(epzl3iSgBO6T}xE? z?YuI64Cvnt@~3*|rbFvF<54WWzyEy`VE|Dm^q_2f{>NhM4>$2YErElflk@-X2S9p@ ze>uh|ONCVL_Xa>ylel)2eoHe@q;+l zEET=E0!P{Hrvy^`mP}(YO8S!1af-9(YYwl`yzJk08a4;@cDxKzTdEY?RT-j!jvs|Y z9Mx>CMv)TLhS7jMY7Yta!32`$gHCLdVrHqfG<5DWr1D}fK)Bu${}q6Sw;}EiC+)N= z$>~cdBJ6M`ln38-Lw0OT|68AAw|1uZtdIJ@v}*`JVo1hnLLBOs=TuohN2F3#Dd{oa z{SFzb-;MbQusF(KD0EiNzTV=1=sYAt?!R(7sEt zh2VV$F}a|iJ>$KE)~2ZHHUU*;Y{?2Gm#lIs8gY#m|Q%NQ?dWgkGijd)u6tJRMzVKp*E_ zyk0G4j;)ZjnAVe*oE~tL<*cZ7WZ_@_MvJ&*&&v68e&PcG5*6cHlrU99R<^&I2;m!1 zNBr2H6&`|?hao2}#cEY%I!!6hF4joq@pu)YpszK|MbEhB84( zY3n#^)pD|AOpNeIPY&2b%(P<4X1SzxMQSX^kkYuT@JXxu#--6BPr8O=G5f?gukCvTd41=cr6*7z{g#3ImAd!Ywsd++s^XUlD~NiO9STyJCG*rFX25Z3 zs7MRk^ClFf%@-L${U&AN+_a?Wrq;QIGmZL%Qf$DO3l9Cu zsp9EKVX+)5XSkj9EtuRX)WPJ%Hh)oSZ^8H9r@a`n7L@=x?XW)+80z#w54gyDtTINd=faGg zuN_*)Fu@j+Iz5EvCadZNh2|)ULZ_ zwT#rLJ>&gKNiFB>#PYZ%6s}_f6nc4VGs&dtZ4tUw+dLjbrkD+-KgJQ@xnU28G2aBJ z#W4+q48F+T<__g%1GDf3Z9Ot8V0w~|k`3qDIvkcFIjM5O8mU2k#9>$)aQne${Y# zAonS2cW-H4D_x=aC`sj8fNQ5 zTugRMXN({m;?H;Rt7pEi1y31qoJuQwLxH9?;_HyZZJxb6Fv; zxx*(f3hW0|z4D)S`SVqoDM0PA`D>;_2=gN{rLfVPk7#Bi^#M!2WZ?o;QYf&n>Wp08 zpw*X!_L8@AlOvpU(Nz-scvyYN$LeNe_IPDIY4j4&3+|cf9g7>IlnuW|`JoT+O*7zg zW#zFjoAYv+ zncv^{7h-OB2}qjdcUiZ9u7XiX<59UK2@89mBw@sc??jD#pxPI!h%)Gkfwpafs)@i} zlfR9K$%5`E-7CB{R|BgX_~iwc&@jwU$sL@^2WXS0!M`WIR?|AmggeumsTTE7kFwKP zg9`rqoDC_M%DTUz+8>N`NW!j4f&YS~Xxc^Bvcrki(sA{f(j|+#+8LR{tC0=8=GfEw z_gaFRwIN??&x_#+++rhyJ9Nexz&65SQgT_0Mbf5~=tE%BJ{*0&)!-hRd1|$G^b*X9 z?DqG4*$glza3f;6)lj>7k&xWmx8Wc|&q`>>#%;(>s-~)|XqXF6x0H|Ts5eLisEyFHaj-c`bSf6LC{%D}oKgc`tuoucY|U!5hP;rCWmvHn21In^UV>*T?vUf>rVDA# zu4)QEd%r!96nNsWp0raJOjX5zrzJ^_i4MW{5z`CIT=<_d7|y?62%djbJ^ttmOk>t* z{~Z4mB=QrkVRGAYt-vv!7P{-`$67S;DT+z0&zC#)zK2Q1`vMidV6E~@5?!vryJ6@q z@H)n|Uw=Wn!IcUvzrnSoKc;1he*|h#J6BuNzg^z{Q_cPB>e(vlN`ry0wApg))#OIY zO;aQ#s~MxOQpj=gW8aCqK-H0$w#<*fi*L=<~-|}F~zMjP0?N(8LSYFL5(P4m2d@N%dwk}>P`bI2GUi@qk{&b_=)87OA{MzD8= zF`)Vb#_R{qERrL59ak}n(cJb=4Li%od6Q!t<`^`o6jr&9e{bVua{iFA21OC>KjvBD ze;l8Zse`?y=+{eo>t zstsbMxB*;6kK$9SEpMxv<5{_I$2FKd&m)pR(i9o6uZ@Y!uj6wcQ835i!%AaqXri<8 z#E*;C%^!Brzb3!S`+++^(0F#xYy*QbP!Gu0lBeRY@`7T>8pvs-pIYAG90sK&$MqCO zvd|9=31A$E8tTQx4T4g4DeP^4%vMSer@jcLGOS(*#h59CvxjQ8s*a)qLOpCX+$|qH z%xzNuL`SiKVvBSs%x!c4;tBlmicG||jBT;+Sldr6en=H#4zOL`4f#T ztgNEN6tk`JWoy&8cjrEr*XUvPar=gj#-%c_zHy-*LOXhry+tA>N5-Y?u~Y<`D0;%j zkDBln6)ICcG^dbSAxUe@b=mpl3*|Cwb2cnR;|4a!-fXqCBH8%IA`a87T@G`pEwh7H z#%*=Io$ZGplArrE?}S@zIen;>PMj{Imm6VpC5-@1TF+AVigMC ze+^YE6c-kaB&rgPoYtB6yp3>EY+>49)+o&L80!b<5M{%Hv`0wCHes_Ti%f9hYR|c8DU|}3*N_ofV)M7D_M!x1;w2CmPwaja@QZ- zac&8WfFxPe>M*fL>uE!(Yq!$Zch_kGcPK*c0fV9p7h)0H z4zzqTeyuZwm@~bJBKZ<_~G+W^V3&uUxON2AD_~ zGBg16F{+k=t?B?zh(`=sN{?6(sA{q}fv@O`g)P-2z)|X*Bjsswb=YUsm6meDcb*8l z*oj6G!tc>=ivjtDkj<=8tmWH;PVl;ISdeSxM?B`g>Eebv~rp5 zjCZ;p$O3-oIDR*S0C~24QDhNb^lM)N(n57QG;0%XJm^g~pRk_4ALnZrCXyevU7m-*K8CT>nsE1Ebdz!=6)oi7mZ@*UYt<(l zm%;-t@*RFywOBj86^6qKxu)jSL!3N58ROIbediNA*+k*EI7#`R6Y1*>VMyY6Xs>se#gAC}S zJvMD*Y&pN_G66GyL~UP|_6N`^FrSJEDi!vb;26zF`Y0Z)O-`NRdCj>^JbV_4PK!@l z?S<4+xZBpI-_C4qx8``qC1zf~>+yX%`w#`DiytMeCifmeO^xqfVs`h_dKlRNK!P%> zk_DtV8WI;i%xG!L`>7mp#u%Zm`6efpI#y!#tM5P&SDQ3Ztj8}DE6bbDBm&2f60_*j zTxgMlg zKIuVnN59()OH*e6JtHGMv#ye$T_6Ct;}8Si%c5wBzRZi#{|F8_1cI%fx+!ikw-Ces zvKmcc4sByO?S3phoeMjU6TLqm&gM$hafYak2j41{qFeZ9d3cGwRl)(O#|wCvXEQ?0VWS9C_tVkaRQjhrsoT`0Ga)@uynwZt)lh=wud62?7P-sq zwidCquw6$PanfK42m^ZOEj*n`aY3TmUY`S+{WSBL-L38I? ze7-q)e!Tx{v*eb$1MT;WC-0AS2IoIQ(jNin-!oWybMxOF|Gz57Eag$VSyyngWs6@dnvl{4v zDj)RL(!4J?JCXGLUtW)yKSQ42BMx#6f=4rn2N#Se?s^Pemxh9w(E`S_hPHu>>~xWM z?Dp`3l~LLnO^h+|Q7vo}Z-9aF`>LXzZ~UM#qI6$<{yvT3hds8F@Bpe<675 zeX~uhJO_8B^H0<>G2*vseuu1~>>?|?yJ+T(Ydja;VW*;sJ9eVa0uYw8G7jo}zBu-& zBd{5uxvf;T@_5FwsSnLc3cNEZ%Y6hsys~VODb2YTlOJ72?2@#<|{~!mZ-3OgKDc`7`7IrgzqVN<=W%#eivC zr<^H~tyLgHQkdz~DmlK%8(wrxI4l<8hJwnD(9P)Vj;3i1tL#gDARNTDm0Rs&jFeB?zw!qg@p1u7% z41Erp#Yy}67aBP6#^Z7fx=&GmY&-tjs?x~R#?<6r_(JTo6bK9mmS1pnt#7N6TmK+< z)ngmpZ{45qMZF&eZ=*yPKdm;Yfq*YLK=`T$>Wb3`@p=+ngdO(BZ0{rMvFDqQFStVh z0aONJ6G6jBO^s2q69jy8glh|Z(^jj6QDENm#RoLL{mz~1B_b28fn8-FZog$8#^cLl znv79+rQ2b;;rDN$Z{!H7TnV*f6FA}DOvtdczL9mt?dmuO>NsY>`tmemmw$=uH}%7U z<=x98Si~bk2$1W zcTT0wIsPn+Yvz|qmPUlF8CfbF;blxYwVSq4sz0eBJ}Z=kI%}P@;dY2`;GU!CQ;6Uj zvoOQ+GuIcbLR!Har6{L=1<6a`P>ZEvy%#5Zy#Pm&fs&Ul4wnV4)b}qiKWO<0PY`qp z5Py8te*p9Un_2NM`}ZU@P4!jKBAE$veDG2qCQ1)x2sSi_^eh{YUO)mQDEuXs!!lz; z3>o4hXI*BH%Vu-oOK0=(%}fZzCx{%Um8-q%*_>&jnc3ExfcHeXPk=|O3!lxgtTgYU z?{B|;$-jHKuK%0>=JFo+*&Kq0VGH6t1FVN~mo`{6Obh*%?R$N=r{qCXLUoUhfn8TZ z`zw!-!FHaF-D|!WBoCjVT~b1#+ozzx5+1(YYl4_d&$X><@E9V`pL?Rb(M*nG5d_$G z`4HkA7QMenOn8X%A(6sM`-3DXev#l_)r5*QO@)S}Vq?{!#e`cJwVA<(iU7#+F=7BH zMn!3!$$6qa zk@JAd_BcaJd%!58>J2mmsi%5iFr)H`i}YZ52=$~Wko)v&bas0p3XPJFN1rAS=k=MB z|R3aDlnA27iKjF+l{WuVE%?5{*xd9|KZ2&0we+};pI7We?fU3q>b&$RKA=R zcV+r>1ExFlQZ^D!ln!ymM)W;3qAvu(?4IJN!5+G$0sVFtN0hZZ#feSv!V85nKTW7BESI-0FCwl97A_gJ}OZf7=xnW`?u^{F)g z(#vA+Kx0I(QzI~{Hx6uxRZKgNvzBulsH<4>YWq5s>b zpKAN6(23&AQaJ!i#?~GdCNegKh}`NQ!Qtd+V_mH6z||^RT$M~FzIMp}@_k~2N;|5* zq0Dbp?GT3kZZG=V6c($Cvin5|UM>T3*=0;WJnu)Ch}*54Yoj^eSfyQEa^Vczku1BcmJd!c_Ir1aB!2K#N0!yiQCkdj;!J z;+cxxz(5W=^0U%`Xf)CG10Tmy`dCfGZ0DCW4gnt8gah;tTNcIx%gF4-{CXRcw4E*2 z!LT3L(-Df+^k!-4$5TS0b4@$f7cwWT`CBComaK*x_#@^ZElB=Z5-N&%(VHU^okICI zy%D4or3Tbfi5|cJ?RufK&ykc$1SfrilWntNE)aJbp1Qbf0!1{gziy|ou_XiRMo~nP z+@wvQw;YX^-((f@8}7(VIiq$#7gdSVv!_nI>-^N$;!(Q&N9vH|@oG`(U5a1&p(c&N z_Qt3S@^>UFD-}V)yO8RO;h(q-xt=pIk7D`=w4=Dh(TYc2=){jFHj4By3>l>Pe;mCQJvl#bJO3)_)L#(tDyCCx-4hLrj>DK{MTeDXkunV zM$_)%54mufW>b&Ov<9lTSC5wPXbBa=O|LP3!f_<_Z9CME?c-<)`D#$LMy<;xS@4U8|~oL$V)UWC`*jkcVfro>`$W?GsEVLC>s$` zu2L~&WW8zH;a4D_kN!Lz2?! zho|&3*v|rx5ymVq`FWv`NL#E(|1Frex}X@{t|gZO(rP5Y%?zKFhMT#}`dCeCFCStJ z4Y8>xw>6awXgO-6#Vom8)KJV=7u&bKE_Dm$wO>w#NH*OZ8J=qoGj15HE|mmZq*bQw zM&_8aqur<5y-t5>XNq=(DTtmKxL{o?q zAmD^KEwlL0CCr~&eaetXFIa4M)G@n!kH^J@<#=WG{)`-#7w~s4nkbGCG6-TF^n&Q0 z|1(tm&&9cacbvafKgj;gG*Gow##2T2v-@l#!7hL*6;agKxXQMy0H#I_n;)xC1WaM0 z3$L2b7)MQBJ~Ly(EB15zN$?tek_sSYb{p_>7$DlwcPJ*PIJ}@|D}Mf#thJfF4LN~ld7nACeNfe?2s}@1&ZlugtfG{!XhzN)z?_k&he-Fpx zl$zuy-NnV1QX3Xyy4Kwh**-Md5`tv9HkcV1qLFmXx?sYCDvzHtOu5|zk5CV9*l*X; z)(eQ2!g_e(o5Xc{NH#pVh2IsYPS8%Wnb^Dc>OL@C&vP#r+`!Nstf4NHi+y(dwuie6 zX0NeQZN8c!WzX2Bd<>O^LtheYt%pR#Q=-f}a-_Q<)IKU^DZi*wS8ctvr#a3h%Z1mJ zRK=#F$8=A;?9>bJeVkoiD~p#JF;1adZo&hcDLH41>t?Auck1cYk4Ei&ScT);fQXZn zh49rWpO+Ed&0m}U>GPcAHe36M@O`cIV&MH|R656_!+FliF3Yd007IM$k}PL8%@WRo zGpi^o$DeJBo=J%|NrDfW@}7B?r4R*s4u|b-_1gi4$EQ(NlI|B_7fkT36Tf68yUE+u z%i(MZbSZ|G7ors{IS3W|Z@p}3;Sc3f!2=tb;=b!^C}QV}wr^XsQAS-*f1NKS2x?O` z3r&P?97}Bx7`B}TXga%GocOK7*Jc4Ny=1J`){EGb)K5+3Co()bcADfKGMWsh zo11SmX1}3Ki!XVemVz-q#g;0}+Bc~SEWs;@mlKljL*BOz5Ue}>%1^|(xw03S#&m9d zHe1XY@&K|dcWjH|+)QY_DvN0%0l*@O=!5}k+hylhzWRXfS06?NRng<;223Q zq8xSMl8=wWSMV3Sf#C)CNMS!DP7dFF+<=pzTRHt0A7R2Z_x39$RjnEBI}rheiWhF4 zen&8zEPTaA1W`Kn2#>NRnj;qy|1giY)U=U^$?Kbr@g2kaj@(Rtly6uP@%W@T;wbC^ zYj7r{)JljbE5#CF!a>bQ4iDx-beYzMJl)V*9;fdLvS?rPB#<^}XZnWY^ppze#vtP8 zA*&*wcj!9SEq^g++8Z&}WEXbidl#4xjN%&!kjWEn$-o_Q^WfmM27Yy@9}P9il6gwE;z^^0zC599Qh99FtXa; z&IFV#ZN{&la-Rg`Tl7Ci61o5P77~O67@ImfE1Md-Sb_r0|K(V~`gf<0r6?=kBY^BX zs-CAH__NQVg9Vaj$pJt}Mgd7qNNu5Tx!M*_ZYteWrRbmJ!FauebR~|{oLCx`xz-C> z$Gwy_VD3`tT$9)UmslG(g8X*<|IFjyiy8Htp@8B zdjL;S0{Joo@0Gf?PJ?3H%B)AZ>T~4;fy#2gn^GI5v~hIj{2RnQMqKBNlm zdJiRsGiI#&2D7D+>MAh?-(dY7@Y_}*J}H4(2Z}$o4!;{X5OTxt|AF3^Dlyvi0^rwf zVUY+>iR05Tb#FpCyRN#HWN7Cw5fl;QF<-qUKQTqR#d~y+dL~L$=LK3RqpL@rQQ=3V z5b=?7?>@lbVx!4s$inpI>V&6^CD43$Ly)vY~~fDu+vXG^;NAt zE5qSVl&5`-0F^Y#{jpF7dzmQunVVK$!X zHhf;dM2#1dAtno8m^GygTwOPFlkVEwii!Iqg1TGJKk_t=i7air7Q%eHKH=V(mGxT- z+5H_-?a+0qG0-^LIC~dk5HA``JkI!I7@Lk%KEbXe86f=wyBQXJ$y8VdmCJ7UmkP*g z9BgKX^)3YPW?B@pq+lq|n(Y&*qkVOtjSr#L|O&ft$@#MJ=8!=x-8)K!~&2;5|P^X(q zjAh$!!FCs$VAF-QUD{!R`Ae&waM>kCapF8njilJ0&ezRV0OLQlqRutaj3dJzV|rn8 zL6mKnAGz|P09jFN8O}ttutN}g9HR8d8)GxzJMsg)Y|DnGHphCWbTLL4uwlz19|ar5 zT>V6%5%pCh1*Ar+W~p$zaP4=pk~5sF;|*&Jze-dEmGH4bV!th3)4$Lck>z*c;~uZt zS`^8b@23GG8}2eT{K4S7DRM&@)^|FNU!B?tSWwsZ-P#-<8AX$Y)zZ~&uZ*vR@9_J? z*t54TBzvIhWTdXK!yaeFu}z5a%{atTYQ?b;r=>Go@Ue}a-alZJk>2%3f5Co{O}-EI zzz%!=ivr#}Tf}s-Fml?eBWpH9f{>+X0);QX5dq2XFIStQN;^qs(8W#sBTmfwkIVZ< z`0>AU$G=jAlCCr!Xn38FDpOZY6t+-aRoqx%L944o>01g-B0*$SArizlQH_S9Y2JHQ zx%65NTV({-Z|<*hyiS(O(hY@mu#x3F$;<5ixc2(%9!db52A?$=3!i~AnM2Rf1Ao*M z9dJw>!wA(+>mU;uV7S%~Pfed)aANdcnI4mIGFD%g+eAatHanDgTU?0fB_|LKX9dYR zP$4BXhy?Xju3WgnXq|Y&6IFvh)cBNL)yQ2mBy?6d)J4IPG%``mEg`RE+8<{*I)3vgoBl&;1VW5;gU(Jr@Xin24z+4B9roiwqu6 za^K(QP|kl<=)~^v7vl_B_WzxN_z2%F%Rv$CzWEK8@L^M&4-x1By4#p#N_-EvJNtVIT+vb zee>~q&mlM{Ix4!_uZMwH&I)LGsPHK5y{!pEKx)6j(fa1cvCm8>QBBe9$|7sz+y)yX zLp>K{P_p>l(Bg4HoK}wyO)AwHxTCO10iP9nNZmUMQl)(}GUVw(QBqM6Qt z+dNN%46>zdK*Wy(^jd&Px-cXo`q7UCJf@j$Ow)xHNY$4JJ_X#4{3jWh0yYJ`&rpd8 z71CDjk|0?d?vV@m{ULh5&?#4m$d&a6t@TZ z{m-==!#_U#|7mdWTMN?0-dxnf*wo>7cKP3`&948pBxAtmMJyB5qlP8<^Z=o8${w6r z8W&WkCZml&)I6w#6tim9zKHIhgdHq`cAr4;3iTH57>z2bN8g(E&GR6w)zi)I_4N)y z0K%OPofe%gk(O3dqp8MdJ33ejQh3M|5Lt-;q+vsCu^|m}#5;a6G3@Ckv*St?-S`sW z39@xL&yB1=R{lM;S|I6pF|(QT+nou{hI2S<9lE6DOMoJ_O{hP1 zY60K?ciAsR7uIs&>Lx5|Dq5Zke?DW1%pry9J^OA!2OA(OswLq|VkmgbhDEUbEqU$o z+)>_gxbdt!_tOtC`>$-N6=)F}>^Y;t|HE8s|ivx^O#_6684F(e?;~ze9Zh>bR)i0B&@T-AJQ(s z_eIJ6)XrO6{w7E$H(g9#)n?GMkd=lak;( z^xKUCHaB3O)(k-gFO?Bso@-$+-4!2(?nE(DBVz)vFhm!|Jcw(q`HT|aQsyI7qs7&d zzc`1YqQ~L-8189R@iT=48>DGj8aoV3)?;6WFYP+8q#z~+PFQ7x|gXV)3Nl5krQLX1RW;Z<8lX~2y+tYD}6r+~!o zk_t~JuDG;&ATO1P*kDAQ0VT&A&UU*(p_4K#THbze&1q|qa3D3;?n!)COEHPreR8%F zTI+~opa5sqJaqZDL|bK?AvM}_ky^C=S2cRehRbG%$Ch$fT+xnxO$&7w7htrQe?&B6 z)O&P5;dH84D+3}i(r_Ao4?3`YCVYZ5Nn>F%mOfS-o`=*s$AhtY(oa_q_4a7eT2z-c zVr&I%EX2PO4YvsJ1Lk++&)Og64FZK8r9Vdgf7Y_fMlO=6K;cLFy#uS zV3#dwOhl;sijY=5sSYp=1r4!&+6TVu`@&Y zjN^Fwy})yaBa7r64RD9@9OyBydQro2&`3^~hcU6sX8D89f7zd#+shQ*dGIX0GM#w! zOJ*t}ok%lBv9LIba;`v-d2pvoBMKv4yLBhkhybL6#@) zA21~!5F^fblb7*MJWt&xndcbK%CPr!kjnUjVYs^aVpGR>k+Cd4Gjb_U2hP4o;O+~{ zAgcS4>Etaas`@Du;p~4^h!W1kUiePILWwK<ePf+tVNLIJrORthr^Q9@VQ@}bSdGVGN`mW+b%x3kYK zdKjcFTbiYLc3M`}P_LDzdX8Jo1y^rdLh01pGyFPhSy4d;8flrW{18WhC3g&;(M zGF2OdS5(bKb(8xF_>Ab3cK}Bo%be&zbuC#_d5iF`b0_S#XXozyvF*O?yOzffRNfkJ zFbvIz$G9$$1k&nWpWwx$?>`8ajr%rxZz^46oxP-cE498P&=WZUU9{VFdWblM^qtKm zTuv^ZTBo+r2=-5_p|OR8^pQb)nwX$dS?qYkoKWU6(M9FwxP`k>eJ=3u8KJ?53;6vl zu~o$pI6Gb3%)BS&38jF$@L1XgHLhg=%ozY4CG#U0qGuor%|}_4#?-XqOM~gH55$FlpDT*kY*hzLba{%r*4#xR7vUNv~4`SIGF_GW} z-0zS<-KRlg3kn&3CNPlva0AF*8f&xo5Cdna{4Uu%`FwJeto@MfNwGv_?B-v88EcYUdEb?<9m z8$X16{O)FI2t645#+U8V?UM8S#d7_V+V$AOwhb~H{259wj4b3SKFCDaO}*bi=uN#( zM(9nk9~_!Ncr4UOy6=Qw6Z(=AsvF^|IH*aujd~j$x)b_R70MU((h%Ac`r-l7fS?tz z>kiTrvHOhBiPh(U@J8UzCjzK}_|OiUa_vNfrKhX|9Y|UYAAk`>k7k5n#5Q1`rXRHR z-y$AUKn zd>FHT+8h*{gU$xV8<_%KO^O7j(lZnmoY&{#wO>1{jgXC>o&p`X2HGZIlkvOM6*!HV5A74?ZFN~8Ih-1XFM*aNSy!i!T3{6 ztNICROZKEXZj+VTp!;U*Y{l`ZVeZO>Q94O>TiKgb=r@>qbjc59g&eSjY2Ty*Ztyj$ z7~?p_g*zsA?W6da5T%G4|AA!AVrl5mAL*b}1$VE-`Phn9WJZ+gDQi>rc?uyacrzv_ zqS9#`Gbecn3*!{Q#vOB!q@tu-#j^wZ(h1mh#&_qX?|n$BRfqR}sU!+G%4Le@iS24< zPzvQD$((X`u942UePUG2m{Obun9=zd$R5NO(r8-=>P&*xr@;!2zjK3|TSh15^Wk|) zet$?Si7X+|Av}yBm(otR3HK@y!3(K>Z6PsrQVf{`{dAeJVx9 zsVU2~%o;hT8k55$(|M_hIqi(wx-osSdBKfobGE$nZ5tu--g+^211cS}o-xZVNxvH4 zXt>8?_~syi+TG?-O3vky1-PaHa@w9}klvC(cP70pwJzp}Hn(y7gG|Qc-CHkZ-()@p znd^3^5#M52BU|TS%J*zb*AEh>N@IU!8~oDRgYx8 zK{6&PO>05+tn~G^k{LA!Ck5R0h0r{Akk;%2Z%fj)o3Huey(Sos_?!FQxMDZ=&4fdD z=i$UccjxZJiI+GXNpI-A!!Bh}?RL|s2Y8+_dYWmxf#FzR0y+`P_H z+Ld{;)H$_&#ERob8Kya)J#C&@wf)KxLOduENdb$P`PpzqqydW>`PtA#r2dPX`8rTV z*ANPL_~&PG=901P*`;U#Wg7vDutM*~i7daxS1{Rzh8f=T0uM z?FpqO0$t5uWmiHxSm!Lbh1Z1VM{?)W_)P(&Edpg8{)=ABMSO!Np6BE$xseG;A&LrK zI{=T+%+~OwDu-1M1eE#qHB^dq-T7g!Dt0R6ZRd_??C;uY5!RSNw2isokl4M>?$v|O zx1c^1td)-6OLc!90JwjQDy&(YW)1M0f%akJ!iu zzt8R06-6X=K-_Zt3~CRb`NL}?Cx-pTRQ0}cE7w^N$rzPYO6P&5tSYapdwfDQHN3+= zAPn=W*#Zc=B7Yt|qHSyM-Xw{Lw?%Ox!6j>%hYmbw4-*v|+z}a&W`4b;`L$r{iK}fx z1J;g!&`-2DRPa!B3xV5@?-;L0Y$8_OHv68RuR3&YZC&eJ6H_L4Aau= ze&LnvJ4WUe3^%#x zoN(Q!tQkT*c^Ne>u}E2%*w9QGfy(rz2}BA-K^G~eEFKt zd2IeSq4UK2_xE`(9HR$}l@79N?^R04IKut<#`e%G`?3S(7p*z_8b{&FxT17I+_vV_ zNpA_iPoz*sACVj3bhd-Z_2kWVTTGkFNsW*9J5DeC7wPF&E&3gdBn%t6HACkJfOo(u7!LiK ze$(hBc8@4H27_0xbp$~2XQwg169MHQN?4Wjml_2?g`skj9efipY}^lwWaXwRz?t#b?Tf%AXQvu% z1($Xe?%RNXwwI4p=dXrftnnqv+ojx=%jH+D4P5~Vqi78d1;#-`Mo(3Y?K1*Zm!fEm zHUlL!Mn1F^?~|^pi@f(IMycGD2F-zbg6<^A8`?qBQA%0Nt;U4zv(uhB5+Qk*GpEfv zH>Z!AbIPu5j6~4d1!HRxvejZ4zDv)BNlRnai(%Jma^$orv&#Y>(3mDnoY-L_%lrD$ zwaXk+6u?KHZ_Or_Yq@`7ZmBjN{D|!tUY=hgsqEsc!qf=gIMFFXp^y+7!%$vP2}~+z z!WDXb#7TlvrHs)iHl~i9oYLBa8fL+__dm3hHD zd6wX_nPc^x1qZ-2V!z0N6QJ2|QHA`X`0=V?f>TMAG{mu>ltQE~Z)uV(_fTJE+`w&S zfqy&9l%A(&P zSr$eSN}HEpk{1_W!-N09TWCDs|2*IuohIEJc{ZYpjvLAq zRU+!@*<@~>T@^9lddiQ`F}WYyA=!?axpkkL(7BU;p|#1ULA9;Q>*1utVdd5EtD!pJ z!U1F<17^m5%NFzvKZp9-?EkFvf%U>duOu?jGyvKsj}F1jtI~)XMeYK|xm$i$L{TD5 zuTS{A{)>yf^nE^U@N01b2K|4Npz8t5`mb#0YQUR8s-y(x{=C`bKHC}<@_aL)T`0TY;{LFuY9uL96 z%y6dHVz%(7d*h+|)4!RC$`2CJ>0zLA%Z=&&_~qE_WbTvuh2Hna8<(#GkSDn>95~?d zTQ^d#%l8+ZZ#|H^x8FX90UUla(A?Cv^1sA)5&Y;NJ=J%qfOSz`x_jY3yvYGXet6Jt zLc1lvd`K^Cy>uYn6aXqeKIk{$-5OvYq?e9fK9COzKn=b?Z9waf0)FOwBYtiaI;ea| zGXKzhL--DW0O(_?QY{dY)HL~OC;}-3Xhtp*19E-5A>%ahfT0HecWhERd0QGO(2QFJ zP)z_ZsAin8J_APuek6Co2FjhUb4$bP75h-2W5!DpWc(O%LJHMtUuRY)){qqxM;XOx zRHi=`eKCIUf>SFlopF5CfTC?kxQ8uS0$ceDGw<|af|PNZr^JxSEhLI=FP75gg#GhA zE$NNC9d3<0Lr6PUvtW=Km#D_)J_?YHgxd|%a4wO%S^;;N zx9%uDy(3C*2reGBR+VkfjjUxuuv3|#aFJQb;wGOOPq79U#~^hsh+HX)Ff`eXFabhN z6Ok8bQrmOGFc>#W4waWtEO;(wkZ`Oc>9Q&fhY^p=&j4rHrTZ3!9>@4BHXI(JgZJbF z(k)Zf4Q zr`mPnQdOvzjE!2+ht{p(o4Lu{1=3>in_ehmN%oB@C|WJwGAhCpRi2dh_{8C?%ktgp z3161Vo3T&MhyD~MG%Fb9Q>Z+>*ls&ktjj}&M6W6XY2`K0sWkat=$9@uH=X%ePFtl? zp4PDcl2oP`H>s_%QdsARzr(l1H#;r9#(wDa$$iVkuBR^9Y7)_&Gb$kqUg4md45$q8(U?V(8w;n~l&|av>sOQVUNO+^E782=znda}D@rC95nTGErTe31hmj>_yQzT?zyLh{7GZhK1fPNnv zdP27)d?hJ)j47a*g=LBhw!+z+h0;E~E#GE$*`RAnUScFRsU_jAcAEE3^y4#3f=tqS z)n4_l*&3sUc_}wJNw{I=Bi#nFI}9^U4k#)!u_;pB`gv#58M>KcmH<|zF~}Qc#Vy5l z!4nA=goEU*8lzE!o4~^7Dj^R>b-{WLwu6xrs)|{aoXMk7lQXgBAO!7oa<&F=r#Jf} z$F0o9Y1_6(Ef>;>%3w;CeEn)O0ysho_QjPD48ko%@iW5tfG7(^k)-lg-E4eT<}3$8 z<2A?>mdxg=4at|acAXUk4OlZ7_A^?lge6N_O_7DM?=~V6@fFEDbG+FzW0{((PYW=o zPcnqs$!QI%jVt2`E>7z86|!w#7Viy3FtIrU&B=$|k?IM9 zp()Ko$F)VLYW-fWf`zTsoJ9%>kkO{D%pxo*l5?*xiN9G>cRiWOFls*)St~PDj>0A0 zYw5b6qOglKJSP|~L2GLe+I1RttPm<`d^c{l|N73xRS_afLafJse{T$4S6 zi%qr4+{_m#zwh`F>A)`j&Wwn}WjZkhJ6gL@^PHIXI#w%cC_RqOA}dMkJbTK*$}tJ= z7HHin*u0j1whv9#pc=h&HdZWK*~ELGs|`k4lA6Gt$(XY*E5JcsLNZbz@Nrn`2-(%O zyN74429)}1c^jWTJR6{k7U`Ll-HE%ul@`h!6>4#WbFfk&feH10d?*42xn(sSN(ayk!eb^n@nPfb=4CnZPg9U zt`gt?->v@A$;3W;+~vE68ORN}b7~YCr_LPW1n0F z05n6BZ$V2O?s`1Ry$+2I@fv;#0on`Ys zEZXlUxk?Ysf^Dn)G)1F7VX8*GyM`wJJp%FmTeaN^23rjtL3%Facjhlie>usDaL#mu zbBdG42P<7s<}+JV&5_ zh*f?Dg%7yLsd)n*pg$W>>~e$mUx0LmJx!qla+B9n;PyWnGi2r|2?_(^d7f}ICr6`0 zb08#TIXSxp3xw>#ijXx~%z=RRl+I(8dc-|zFd%95l?6wnys@G%_6{*~x+q5QuoZDs znK|DAK&^Z*-%~gB6_2cDGDj_~W3s_zN-#^Bkg*>m@bzcoo`;Y&2=W~DE z{{M^ikuq~|l-9HTUrPV~ZpZ%plf99Zjq|@no2>pN+O&WgT&>wh8LErl`crdJ+zp=; z5h)3<=nuC4ozBoXZF%VIvYl%~?K7;WS5=%I5ijz~9{oUj13kexq0Qx*$7MP@E%PrS z*IA(egP;STkdeCKx@q}?(iHjrB}VpRY1TTt4W^KFOoU&fts+bn^V=`F8zTv=+z%hg1p!M6&NxFP7Dzi{`W3t&i z$SbDcv#hz|2S!e1n1-s^()Y@fibJvA>6}R_sq4*=cqM*|@p+u$l@91e;<-=#Z$f*8 zWAr2kYH+9E=U${v5;4rhf!T*yq6M<~g;a*r0WA`N?|z|9%dpav`e}uf2wgOF6`YI< zPs)oM%5{t-;~)XfEAgp4`Ga{90%Zd+fe{c>y6FgP{3P!`{ns|b*Zi~sJ4LQ?p?MEjFZtxOIbrw~15+(=65FI5o zmNp>_O9{tY#Zd-m$|(&C-V9FMGCiV!%q>5J%zEBWftldwv#7gpcG?GLav?;-D>$J;;cKijwM9vHns8e|beD9%`ub@a(<1pxqwZ)Gj)|cY8mK=(&Cn-|) zNzSHENB>83Pzf+cdAQmzePJz}Pwp=saem@-FfJQej;|)y5f$Y7>hb#p3WH*hQ--UB zOY#q)-oY4kng8HcWfxgMGuUyU#o2c72NTq&(P6s0IQHFf7&yqcaUAGHc^6s2-q<`4 zF+pzc#y`*F71W5UHMmS<@2QVwvAv9S6>WcIZx$H1d!IGUI9z`*;cPRW(b@|>fl*kv z;edHf^+157A#2@F78NVI_#t|4C0l$ABe4<#ykXYHP!Ox-OMnK?gDd=_PiZXo!wdR- zDFAD`$%p-Qu%4O8&lCn+Tu*jPh*T`=7~R8~WO)mNg0a9oYFQOG3=SVEqz zlfXfdD;xhA`1N}kSwFf;sd33~c4;Wv;&Qo|IVmguO9%h-f_4#}pe37s9Q9}}%}VvJ zcZ2g^-_3s))-tm;b2QVlG;=pH{GVNSRJ^9m0{xdsh_&c^aHN$H@~~A=;JhhI8!i7?GfUoQj@hWlb%QsK*LAwO-Mgl*QoCAl;own7FFQ6aXH=+m3 z*1dv*gXoy*n9-3gQf@6MInmR>b|fsIW%yvz0+fAJ+K|<#w4py_R#!{1ezD8XE|J4dFGOe9T>F&Le8O=?`DCH@4|Y2X!sFj!%W`|1p` zL2gHP43{57yn=r~*(4dR41Ip%IidTD>!eCyOQ4M$L;gyvRK}CqbRMl^V z(Z^H0mPu{?>EX(Sng}SKRhbrkBi7a!VDb=a;=Cj%s&^Zsx6HaC)o=BgMzfQiLgutj zX(}H>k?gbEk+Vl+DSgmXp30xApfl0V2?coqE z7?PRY`~#e`2vc9?Sird|I}fOvf!YGi(Ruc^uTOLosoO7ijW)F|~u zcJ=-tF{#)% zgA;MIaDLDk$;MgS5*2(lX}F?WYI&ibEADk_-cWf$8_b*9k)j0- zL*pi5k*utV7lQ$9VBvFl!%8odM!JLS-97t;Jwx&&iF#doouC9~&bM*qn-ep=@F{1W z_`1H+syd2^jsfn%z&z#H_i^Z$ZUWEyJ8;aH8HQV&J~99RL=S-RxNuHkOcAXJMoZ56 znchOKYI(IslHY(ZpOH9Xd%*UOjKD)AOZ5-c14?e^On3uy{}YKlpZf$ttmP!cPZDUd8c0dKAtLuzI^snt;Js>+AmeBYmp5twA< z1tC$po1UJ`@M7w?dCtM(19F_Rvz_p}*=-Kc26w@3#BQvxt-Z_-oS;RmJ>knX;Mnu3$E8iW3n368{fZ=MmvuB@#jX^{R zWMm3Vw`kU(zhoUy^d1fftl+mr%tCBmN&mr6DGU5ZM3Uun?fX-BDsP`J{LDIEnNSn4@=nCZE`DWS9!@GItVijwpfEKWITP6M0Je?06yqd<{hOi^XQ> zo8IXcYS`^QZ?My2L)MPq5T9yax=`xqTwc_gxjAq=m} zi%07pt=ADPd7gs4R}4Q-#z*rYaD~(6xLHHm+I^9v*9xC7(9dt`M?vJ%(|2>`yn=^6 z6?j#@l+kbEE3xBYc}U+`Dv;vWvCB&hq6k+;H3=EkVzM@zjv$-sbPAuuoLSXvT)h2o z__|%5ktla+UqF{`0O-8yjGMVlQmC#!FTkpimtf0jtb7Fv*xxV>{~0WQ$I|>0ETv%N;AHtPBxTYUy$eYd*_+l)y`B~rp){RRzT!(0 zQdpi!-a?st6-47FwW%)an6#Ej1D4B;#tYtKfbJWpU2#@f-*VdYgJjlO%=yD62|rTm zoHxhwwnMf8Ox!!_UM$1Rz!S!!47c7(q_3Frh=w+bN29|g4H3uLpYTwA@H?Qer( zN&dp%=Y6UmRT{e6h-{!>s4QeBAuNS^V2?I}LBSiRFA?P5K#54NRGO+}hko*751vsW z(&7ahttfSgK4wBeYYsrdyHtzs6!9W8fa5l4pfNydZQdHg0Sk9{%SF*W;s(pN3oJ;? zZ+i(%5whmnn95by!)^P42XfP0jItM^vk`Rt~^kgQ53RSn{A2Z^ySnQ zZR6T`{X5yV%gP|=kM(YntIYB7`sdxMQInRzcD) zzQ>#+xpf-8>nNE+&}ZBJML~|r-e!t=y^v!w>ww5=8q|8FEhT9X=G?qA;SwhD^uV}l zI*W1O7X^U(I+Uei7&_TXlUJ~EGJ-$p=bPQH^}^E{`#3f)dD{tA>M`xxT^u7++P=sAGuVG>Yf(^b6l<3s zG+7r}WX_2K$w{3-zhquNRzxN7SQ{?h-f@|IEXcv5BqLIZjWYm>cj65ytfi+dD;`e=?Xp;Mj9<9 zxWj~9@-Byj$7ZL$vN&A;)&T#%4Xj*ey%v| zpvLwNMS<=qn^#~5nw$?RsRAygtWY)7F1fO0V1E1eU-dkVs7ZLm*MVR9o5Ik49(vyY z_YLv?QzZObR_sff<$r2~C}mC6ue9(1hBQqNlY+9ysliNx;O|S(sDV{lMQFYbA_szH z*_3byKP+RsuLXblen;kckwMp$JAqPLseBfpUD1*-zaKg^uzhlQf93V` z@sxJi^YMP=@-5(ne>gR=f|1v_bqs(JFeay8i!lz5*%1O4GekOObK{Y5fWdg)7l4r4*B$HMO?!keHp zMp%KE1fna?`ssms+*bR0xhnQ~!%(<_ju|)Z;}C2s#tjJ*%nN8C>Pv)DQ95qI?<=CX zX{pM?`yU9RIEXEp^wbJUz$}&nY zZfC^TvTbK{m(iURNwY{~SV?15+a~@d86|RC zsD-&{3>=R8O~`DH)(T>^*nAwBe{WJ_*Db6PAPsz4#%GI>Az1q+(v z*a&+PGnGWzIq;?!$)eg9fQ-u>Syh;^+iY$~>1Gj@L; z46$QtAQwtNDL|LfrW^}j9MmQR<~^|@odUCIXBmPWu{mM8vJ`Q++RapDBE>>Wg1vpN zCBjxVj7TT9Wf0+}gL)<|*V2-F&D^H^a7>|6)f;(zbTVST!e&#EhHcRi=AJ2L#WjQz zu&P~cSF#x89A?9@i2J>3_@te>bR~{r7GkKoFN;6_tM~ilU*%lDP7GZ_W zP-%mvZg<}n`q9Sk#iS2N@@2nXpRh$>o}>UjQ4B9XMT>m8SC_E4-iusuL7d_}wEVnB z1loD)WM`U2G`=$(WeRwN^e z%7btV9%TSJ_T z6?lz-4@zA)ug1h$w{qu`sYRTnj;N+HN{#4~(rl4@wl;!zWjYnRK)e4craK;jqTA*$ z66PHuFTqE9nAR>C;hRY&aaKQF>R?TvMqZvfoKM0Pq{=!S!VKP`COpa3PmSx1pA(q- zRQWhdriA68q+xlwqAu~M%YZs%>@Ku#Q*qY4O`m_E#g!9OWP-nzU>AR5pZ;gY!}(jr z`@fLke=WnDs*^&ZY|t4?SecK zY)!N|l?1gSiFG~n`>KaY<>3XIj?y@o9EMF?yx%{bzV~W3N^R2HP4u?=75i#0}>)scjih6Z=rQ>pG3iTMX0xFm&8?c-_1QUzB_G zAs&!8p}$SKfX>^ju4x{*;Ut7e+XG z_rfjpQBdq6CYH}u8k-R7D*kOsSx9gqkxa)9dqpWZ@y(0s_c}%SDUQ_J`JhRxD30Dk2dWI|F9g^{V^~^+2jB^rKQK=S<5nkBlT2zkaZ}hOF=aw6lB^|Ub+SS_ z83`Lq_D$~&5|i3u-1);rZIvE@jkWC#(Vh38!HRk%qZ~<#6q5-YO{FvA+XIGpa)x*m z9#QoaTK^O}!1nWKUH+FjyJDp!2Y$i_@0hI@SIv2^ z!ttF;n`-<+{Fo3bC_hBvCc}|Ja~`%3>sHiFt`_1ibg!KPXwWeu6W0wB*TA3G#}A-g z7;x5g)+fu$6urE}V0%!IjlLh);53Ukj#?G|vLu1wJu&?qs^zAb=BSiU~dtsiuuo}BL*4tcr+ zz3svdr9|c({~lfct4q~p-Sr~;`okB#p8w?#{pX4NyRyXpur&TsmiV9ERB=P*YYxFP zWd!aF-Ubm$NX|a46X@L4^^4i?JEUPBw7LInkxR3>hH(iQ6OZ95@+UAJ3}F~=`is2z zRBdW&1l^Xla*0eMEjkSh;nyn^zmV4bn;HbD7uFRo8TGRe|+!5Ig z29y!rT2uP)2OJOjj3K;Ihp~MqSSz+0!`#M}hjt4hKntuAt=0#$)~jlR)Plovb#Vj( zrrmPy_&-2mqV8UbA(AnSN735)kv5mI$+Qa^XI3C-&9e(#ZMp%;ekWXrbYkYiq+@3H z-7~6AtGk5o*V%{XyRo@>htk?HkcugBSmnK^u{D7rS5PbPN5YWS=Gg(+A?10V`FJxe zpp`@38r_{*t|RnuHO=|ydn{G3K1ho}aT8%5lSoHY_lbEm`4{net>vC=C~Ovto>`&B z=od1Eqt+Mfjz{Sd@K0$T6jhRm^+kk83{q@`ZWz$ASCs0yR%vpT=t$+5kD2)mW;@(_ zY%eO8Zgh=zWo;ESs$gN33XODdkSVu~l9w=o(g>A88{!D(!~^=Gs=#!`s6yO{aZf)C zN2xpnUTfA>*m!Z$EHTbj7H%s)yyg=#9=^Z1{8ny~bw6(3$9fP(xl(KjogCFmJ@1cw zXn2U5ZbochIAZT#=~gr@)xutn@*)dk7a!z^xePPy!f+i|F5V9#mjHw3G+M6#(!iXB ziXyq-ufdHZ{qr+h(~zeC=e)=xud%_+A0e2o)zP0JSP|wL+u#MHfafLD+gXVk$KaH_ z$o_P47IyeFb3`k{GHf(KKcNV3G|L}>|JtMO0b^~S;ZpYVc%LoHMS&mspX8$g8IN4M zuS@~^H$v)v58l7YqWnvoAnJemW<-_vBNG0f<>cn&1inC;r_iCov*ECz`c`w@R`H<8 zq#b7UdEVzQT)(Lc$o7h!zSeGYv{5IGt@%bbTo`TB*51E-1B80Qf~T!*d6Nscr`S=|;-WzhI%CcU*}uCiJ7 zD?=$zmWte)!cMisE+{MU3Dy(IwnX(-i^6*1CY2OktrPgIkmQnQ5;UYT9-f^EpG@Vv z`&iXww^1r6TFxSYb>vqlAflWY9R<_kuD!#_QwT^AyShY)2mkwVY}(Xdi)(BZeobendy!~$jbzESO4ZQLy=qEkHH_uvauKMKC+_>r+Fdo)o*N6e z1!EnVqFJ5ysXN35@8)A{KCZ6~p;2c7oneOV^t$j$?SfK)9qA6a+t7QH-rM;;6?ToY za6$0Z>YO}ls-R6!9Gv6##n%3j@y~tyQU~&(T84eeSEI}=!-V_npxuQ_hX%Qz4oo|r48@) zM#g?dtoPeO)YWnFSP=VtwyQsrB`S8-+m9>+tM4P@p{`KCD4=oS{*Y42$eBk|K$_kD zG`fAL?YCt5FVKS6A$oh^HOHB{E$|7C*0~pCrb|VCHabqleFW#|wi-+4N?jdQyQO5p zN8x>FuudrorWPt+)0NXC15r6QD9=S*x4x*Ya^d*C=oQ>6v9?VRNb;? zr1!qx%+0+2MQmAc?K0*4%0hUM|Lq#Z|6AQ6ZDZ(UX(VK1{MX*c=3ffnMQU!p6i?zl zqhcSkv+>YD2oVUv;@Jn2Es<^Mqwyh!f7+F3d@g_HM2RCOrfOOz zKm*l8*{G72D!@kBD3kXH)kNW#Ab^Ezu4-B!powy>V45MYfwG=Adj<7KxmY6CBj7^i z7$fjZ=_p4an5P#M?`=7J1L0N@>jM4S7|h}-H2|ABJ4;+(+$lcl!6F23Cb)sWD~v4( z^b{Li5a6Xfgh<`I2sfy#Q$$T4%}Fnj@X#4hPFU*kQizRf^Sz6(3%Y1Y6_^`lS-Lg_ICtw6CDNyYXZKWTV7aA z04{8nYbmb9Ef9f^Tnsuc>K#8p4!nImF58|w0g?bN+MOQ3%(B6EZ&e7o&B&Ou!#=hk zPu#<-DHO~fp2o7J7oAL1nwMxqcq z^JXYZ7mlDG@+C`$3ulyXS|t^?C|Uh4Oj*?k_EoVL060#T@l*G=Dj5rac9u0fsPs#^ z48mJftj{t> ziFW+~t@>(|&x+Z%&*BAK4pX=fg_5qqbLP;`1+$#gICzt%13E{{epv{7*EesG<{iFx zDkDX`eyAS43}{Vb0@eC>V*<;2$i8x4YQte^4|tQ?A(oF;o}EkCm?MWE0e5zT;D|Or zsd^;^bc|YCo$WF@(?FVI-l#-y%spX5Nd(75tEKLFN|K{9v!x;9o@27QUpUXfnE+>s z)8yLGj1kkwKwBk;V29P{oa3!rHE{@Lnys=rw-)6pryGw^bYTOdINE>+fhV&G&fXfT z|9%YwLg**A9_e;C=m#KGUJ93ygnPYh7XIAAd9693$9|t<5FKo3dO8@0vfr8bRv+5irjhBbbT3dgxDb$YNpqS(k5-YqW< z4N9EIU@*Q-D%h#kt&S&|!CC_}z@~`IYtXPxnT4Suk*0WLQcbtoHPngiPCpba1|>M6 zZcnQk{bKTj>8y|--1RD5-NNIZqQKPosGjI-F8*k=(mrm@Kn*5X+8GASsy(N+vdpb? zUxX8FBW^m>XuG&+&R9fCv8Ce6>g2X1Swuu?VtbQjU6s|T1v_PIogdzE=dqC_nYKCS zcqEaQNMo61*}!34A0M3UxO_D0|6GOHu~| zw~f}Jv9xaciB!fA$%4)J``~`wYfLw^7V_XNk@OK)Hn`-}k?>?awrEch>U|uBd5mkN zqmmJHR<}yqoQ(5H0QqQ~xuwNUZMW6du;Ib)Xy*c}1Cbx`eM3{ zN!QwO1&hO?B}Nz{RL`A`KT>4jCU4d`%o`R`ZP(n=LD6Dh{SVHi!$>EJNP;&AP=Yv-`UIGaEV6H&QbPp!ACo<3nZ#^BYY zD~gNuuS zK$~024~wNIQQ4Los!rESxcxXm3~2~ZTs>qjl@ZoMR^BGU0^XpJ{D#y$i?n33vydL- zWeW&Goqv`zo&YU@p)#6j6fjGDCr=KKKRzL4wQikX&4l9i6z)K0at-IeI<0Y_-ELl_;({*u0! zXf*<+($wvzF*+P)XAJtIBX9`vz;r6snB9LG5e=ydXuCXNts7+MbGM6zGel2mF4XbA4=07`%3 zsv+z)Wy5QR0dQldft^U|5_3j@bDx@}^>^9XFdxzsu+8b5xFJrNz#El3fmG8N=t6&j zCew$F|3XP%)L{cnAWf^IQX~80jN^&9w_M6(=iLQ9flgjv0zB2%4p|8gX+x=YVw)n0 zwwBOJhj=HgXIal_LsP&vcWofK_$|wK*fJ-SB3k>=Qn)a(@24*~!B8Cq3M9qC3kw{g z-6OU4wHO^DffJ@^SYI7mZ6~QW7){uDKgISzOf#65zJqSQ_C2@XPo-jh>F8=czoEJD z^J`bHQeIo5-Ezijb4ZYqUs8M5t+e68khep5LkqhjmcT9Jp%%zQq}{}YA<3ad9hN9Y zn+c(8iFUt$`{cfZTab4%AHvaOiHrq!k?DB#)ol?q|FLLhQ@Y91=esGT=7H;8Fu>(4 zvx_G^C~x4p3?!@V5BpkyLOTYcZtPJUL^E2W$#-$?FdUSl726-`Q@Sh@o5jn!r+16; z{VKexDS?$%e)EPwD3%BFfX18+$6wZG(Qid%(+4M;lGzMw$^CxQKjn&@0f0g4T4;@; zT10(=$a-ODj>pqR?q_ZP!BvXUq6pq@MsQ@nDQq)h?T-BY z0r_cu0ljd>0M82#ve`P+JM=p)=(Fqco=)>!uE}?p8E+U5T!Hv|3nXuqF}j#UZQ8oT zvP0DxDzzEy?82O%O*hoI8DWdd>qc~0po`0xGF??Dg-b!_it3A4ItFSonO?)ho-{UwI{QBeL*8(hdU0P`4=_aE0(b zeltOV#X^V1EZ8ecHr>Unu2n?0&g$=LUAdJ`$^=Z4YwN^Kd=b?zqlpfY^4cjbPhKWV z)t%@det1T#C*}4hz}ZvmZsoP-)@M)1Yek7MxOXk`f2s!xbTgUC+SbKBDvGJp;7r-| zI7}}sfo!3*Zql+dU0WU!1j%&%X2(OiTSIpPuE?}!9w z+SC6q6_PW~7}B?a-~GXv&J5JYe2sQ^15MrVDC3YS4;`D5M2fnW{gSD@GPOCRf4SLP z$gYwgmr1UQeLrPOjRM&Cb}~t`&(H~HQJQ_HWaz4dZ(z3uv>jgLip5uSvwL?hW4T+u z5aVJ1anlF!y=YX1yVvNHA8gX#1>V99@n%HPHEl(=FQl!{Em$);2Eg<#e?7n}Lj`~g zo@{7Dnwj%T-CvNIXCHrHFFu3J+)Kb2k^2kzvC54+ffXR!BCqH?I-HdInAK zT$4@1(qBsd;R0836|e~Awd!Cpz5NBl$lA>5>Gu3iA0?WVdNC*+Ega1~g81dE5Q0sG z4H~T+-GolaPDOwd;NOSdZY#qNK1~I2kQk^3s#VQCErzZPKM*P~_8n2ig&>?D=^9hf z4w3^Ug=BI!I9wB5)eZ{-#f2eHA9qEGde=YPqqTa!dz02cAt;(!P?WS|GIrANk;Y(T z)^dQMX=y*1{I#L#q5VcZTKoZi&Zb;gm#V$G<|r$dkb9s-i>ym;)i1TG!kT{4=mT}? zd?{WJOJl6bYJGvA3h_l`4psX7wYPqvpD6Cbc_gO^Agy@UZdhE_h0N}BRL04Z$uVCk z6*jFNeYB$wLDy>32d|Di)LWSP%)MeDb_OQitzny$l8nI|ExUZ9e>z7aitH~T?;2x- zaaf}_W8cJbcbp)or&OBV=fhm}8S~k0ajq}AYPSD6e!h>ylVdEmywyt;Mk1**kj)gR z1C!uk;GxNGRR2Yf4wm&z%-WT^#pIjIbo?n79ji4^qWbsL`BBd z!3&eHJwfuceIL}km+oJ42U7i~bYK6{A;AA@QtI$uC#8SitNhsGZP)X;r$Ayku(_s*>2tP1?*H-Rtim}Vvot4Z#4dU{62JpU&w3pw(EO-B95CJV_ zQf~dVpP~M*OR@jv&%^j78T)UYSB)>7*KauKd=LL0U+)xN>DH}_R;4OQ#kOsY*tTuk zwl!kgwr#Uw+qRQR1(lQe&$aei^PIEi#dk5DaoJmMz2OBY5PVJFAg|smv_7tJbX$_L z4#BD3t+U{Ye0hW4&qPs~k@N`}ZgIm-r)^Rt8vY&5XcvRR3L!%Z zc?EKp^x?-hTuO>;(XGRnxz6>?sjH+7P(7WsZ_j2>292Fw;@V zjDk~Q7fRAJ!qOGHpg7MlxDaYo=Bv47Q3sqMYKdQ&D^&vO!jgq8*xvmzYs<#Y;p-^O zt~c1F!Frlu6r+QwRv_FiT#OoJ$=y!=F;p6wvZq0&#L5K&kFA!gTVj8>-gUbwW#>Xzl?Z-rUvB|1STKTaBSvV(wfBS_Cc`q6ZJ{+mmnf^M_YScRp|>aaNqi~=mNrLO?$CujRfMX_Aj zpK1qY`>dnAH+D}9Qt~!2GRJ<+MHS+<34SPnn%nlI8mwrWWr6|Xujs`G)Dp@T(~Muz zV5P3i?CYL7r-`;h%5vI$uuHD+GmGhr1%}CbqWH0eDO=ii-7<^jVwA2lLBc=2Yy+^J z!yDlhMl{^IKw*&TTfUs4tP_|%nwJhdUNq+G{bsf40-j=F5O?4U^vcQmD$heX!%!|C&@per)Ejx@fv)o7V@Vbf+$$J|H}xJYXI2ioCUA$3Gwh5|^Ts zqI3j)z_?9I9-&VeMP5sR!9RkMl%YUCSVR~*jSZdFtTkA@ZDT=hkQs(VYq>TE3lsW) zX+uzFz zady>nOSseam&@h5I9;#S$$V8ylxha;eOh@$PNIb%?g&vt+}Bc_iRZk8Aq6 zy{=v8HHlv7z7zompwWQKZ;#KxYM-R|WGsd@h@eHO8kqC^`FGTk>bPT!`CnN$m{Q{p z6!mCJ>fE&9rSCnv^(Y{jkfMHnz!-?ZiUmK0d5ec)uQXWP;hW1OS(m+`IjU2dJ{f>{E(1NkevBwt)KX#|tXNqH8w1J_N!Lxt8#v zZ-u|?pWttAyln91w`1Q32lz{ri@$I%Al`b>oe>rx7jdW2T@b|4UGb0O;Wu&im+^Ui zkNA6kDs=xRp6fGF>HTX}@XzJHoP#m3gua7_xs{W#!~enKtV#@gnug6U6^6~OwO?y4 zD)sGDpz3Pj7Sis3bxZHbl?)0M7o3!wus#z81O(vih=#FN?E$|zPH#9~XLWRSb9a4F zHPx&#hn}bx)xf&x51i5gVE$bIS~aq zM)boNQ4r!7RZoQP&^`X;+vG0RErS(qZd&E@4JwI z{jN=EiFx}K4c0v(alEA_4ba{nDAG^>vo|L0_V-TdJOx_F-rgX0*U`4$NVex#KBnK=s&oc|F5_|)Boflq`&Nj zmz#}e6_C}1Z|WrQ#)^Z*6hepL#hGCAB*fm>I&jkaE^Qb$Nlt3wzX#3={WjT)0c>3+ zCyMcgcBFY8XJ@f{PIoc(`1pPrT?4~`o|spb^vqmyH);dxi9u|8m$s$Z)BmCC-i+}fle`+iF_NwOJp8N4C8ZC!QqFp{+1RguKaCm| zxQ~~;_1?U-EMB&{E&@XZL!4TV!2szi1Wb=&#LljfO}SJd>41jwXew?Dt`dbxII=RB z9vbvW1|i8nqta^W)3B;qOQp@-P4 zAV}J%c!yt54djA#@H_fF%zu3w%G%Mo0tmjJ8DOwfcbk8Ql`yQ1eL|4z47CXrAx1`K>H5DIu(qaB#xBNI|CL;V75@k9S2UOx9Gl#{!?!0O zFn*bbLT4RI*>)rMV{cYm+wj0G&=CBYijtOJY46)J`QQahxlk=8Zzdxr52L3G_n-IU zW4bRHZuxuJ;l<&};WVId$XrS{VtxCB$XW=s8$Oq+BobjjhX7qfJBLVVs5^J*C(96H`c zjYqHBHV27pL=j@hyYO%wf#H|L`q!7Va53I=?3OFoPno^?YPIh(-q{ZwHV<>BU<*dZ zz@Z^uS<4P0R(gsW?9LTW;}&VK%UCk`Y44*57k5+oA3O+n(w%Kcw%rfKgIX-bXa$T@ z-gf!1&En2|a$UVRg6u{0NuiKnJ8XlEQ`x zG6gX9=fboSOCMf@Y6Yl?-Os7Qb0~7t;?n#|3G<~kGz3z<0- zfo^}0&#)}_IZ=nnKY_sF^*%nOu=RlT#^KbKZoU@io($=dRbqJ5=$&SjO$YoLXQsq-_SN6|53cQ$rD8R@fl7~|N3D4zos#N6RQ8t zR9zYnZdi*5e{yI%jotSLWw7@L#WRJ(kyrcb4ynK)!hV9vfH;d;YCu|=CsD^WhBuD! z+j`8t7B@D|B&=k2!UccJpv3ALN@R^$yHq{ozQD@e?za8qDZ}Hxn98E@U zzl^`fY&z2U*)Sgrop(JKq4nJDvFg0ZC`zYm!%DkZ<7n6w!8Nr`fWxT1nmwBHObjJpRNw03 zX7|&N(GTkZgfL(kuuW>ws}0!t;Z4QpSD?j#1<aGSb1<)C}dk9^l=SDJ6fv&SJNF=%tN*2(+RsK>242Bp)!?01ii9`le z#_?)Sm++wg83HNh4(aDnBS+{-EI>j^CD@3@D zXMA-^#MSU&pocT6KqXrL466Bzc&hTz5yv_aCEhq($M#{X^`BSu%VIWrOg|<9Ne&X5 zg9E(as2atfR+C`*C5o=Z{YPQp#3r=*T}%*HogPQQ*)In-w@}yL1^m^*gqWHs<#x1N zE;b#)ROO5_BpidPx6Fsk(l45jLv+}MSxXP}Yl?&DWU{Wz_gR(%<`LCOrn!I#X z=LgA$z^U|7*>_=NC0Q7w#cs+@xU^{sTd5SAc!yz>8+Fu&jI?zhoMWXOu4S=m`EXY*bZK^dq9La zx&s;KO@1LZYSC+{aV{tCzxh%XaNZ0~XKmSrGt%iqEoi(fm&tXkN~letf=vZDBM z)3>i^fb$Kq8-X(cGUI_NJF}uaUo&qiRLDWRur|_Y?x1I5VJ2in+=17M*adqMonbGS z?BfgnF7T4LYUzI8uZt!LaH8pWwf-c@gOS5eV{ZLiJ6uv zctl9SDGsq0(o@4KHNLBJH_XM4P;@HDh(JTlQ9@iP~nw8&zib;T0xV$BOOJ!2SndZRa1*AFY_H$+b8q26fy7I|ROd}3U z2Dr;|R39}Wu(}RTw5Hk3{kh8SHuH5YERow{MTbrECK$`%d7MiB@W4-qOe{dF3-5VmlvL8 z0w&PV+i7mfh>Dpb;c~8n2YGd7+gc_;SnyjO3|LFDA+Hb4L&)%pM`ZSvU0$cJx7oX1 zA*)c4$eq`xB8LmDICC_2f7ect>65hcouLh>Q9Qy6jp*12&GC&)3liFGEUrDbbk4u^h~2~t)e2!@pa<_fR;x|RRt;0 z>FmCRxNIPO!?Pg1z~MDmL`#yO+=+UT+S}&(LyPVzsrB3jsZ|-kif@0sGbwx}qt|s{ zLuNw*=x_tkB(5II&^|M%U^z+a#q^I5y(LIlhlP9pvZW`w9uM(cBt$(tQnc2`OK4y0QEL(s9L0Q(VIvLFw0n;L_m&C-;e&hJ_M&;)|{1&B=L59%fu;=%)--u0k z*T|)Uh|UD0;Uxyg?GVtT7m5mI?*LWG5uQ_W+~H}hGsuM*xE(_>4fDjBL@C=|ZCYQd zI?K~GQa7-z+2|=nwf0%Y_L{|pI+IOpwND@c)UlLaYm*7~_^`cxULDxlThw!|r4>pR z^#~IgI`OBcrkY%u=grTLQ1|aIF`IHazE3xnW2R zC9zu)ZO?E#nXaTM5Zf5k=rD-BwgkQm$R8uPQ?lG~-`qO*x`ptOc!I$1{~T)sSp2e~hRhbwymgX$rsA%+dHn$S zYgQMy%~U}K`Q=Li(Z6l}kpI!Th?v_L{ny%0^|v9iD$a*g{pxB~rxm{FDIXL8wb9Q$ ziJ~=HWX))D%y=;~J98AW4!TwU)ih%l7J-V;sx^x)i>7psX586uiTPS_Owct-v##sc z^5(5&Ub%N4SESK8;@^j_N8a1Vo?X|!Ubi|;b=>x-{U)Dfzu3a=)P2cTw&x!bM%k9T zl}6cCz5Ry5t8`0*@`>0)Lg7`uWklgsy@f*QQM#o<=~1~QLzymf(;3Q9eiI(jReqBm z+E#uO9O6}elN{;^l#7ET^jrR32Yrj5ysrn}bOM(T3ka= zlM2S`kqT{AIA-J!0>rmj%*|63{DDF0YEe4O6Ji39Qk zYNQ%O&{&aheQasZK@}wo-yMsJIyC)8q5=L%){N=p_ zcA$AEVn?_^4Baw%*}C!R{bd<3P~AZA;8O=pLAwK7A-qroK}>IBd)Yf>cet)l-4JaN z+`@m(w@U6fWp+voH$zCF@-3GJ-A$?b4Hm-f2ey7*u33WD_XaenuSfP;FAcJDD& z_SIYd6Z(sid_DkDh;3IbTDuT3S(wEIn*}VY^i*hFTGB>Yj&-sg+7o|m(N&GK)BOT> zn)%YdN*HuCNFQUiI?|MbrH)0%APym-FboGLw%KV?$f-pCdNfU?e$#m_NLv-svUr|- z`lP|AabTkU5+#Tu`SL#BJ9=N(75+3%hgOiv(xmF5+0pzIvyynjQmyUOj+V%yaFpl3`1 zRmYnGpQ1JIyKflfvS_hOF1IP#-gf6BWEDN0D8I{uDP0|$gvKgj+NN3&qKZHz3!q?Q zU7AyUC22Tq12zy_>eA`L#}QxEDg~s}kzr=zj~e=9lymIkOG{ewsddS388QkKvx}u- z0XxV(E~D~{_9W|0VwUpxI^)!%s0ijSflIXC z&zNY2E7h6$*W68^#>+fATsGJ0IHh#61{2j|46f^|_8|jaf2`TGPpru&nu$r$9TH;aHAY>(x)|-Cg3MKL#V)3l4<{a5a*X58AlW87rxABIlgbJST_=e0$8@=MLdU zn5iQopAy(a5x1mPv0}Gqywyj>-0RnG6r3pFLm)Tmq}TTHO_c}FimPh7_BK)^>MN7u zMe5adASov=s>{%SIx{q^n3YH(e^*b}gl$mc9%;Z0DN+|%%4k-|W&uEgoDvOnmy}|= zwj_}tpYQ$?Iy22&BEE+6aQIRzIOE z#5RVi>c`0AaL9s2VO|WKZfFe$$!5%&aG(?9L>`r2w9N;~*T!QpO)TOa8IobkV5lp| zRVJ+=`l2{JC%zTP)mr%T!hq(jj#LNSSc$Mr;LPSU45Va)Wa&aiLRyU{>8VD?H0gHi zK(LIT+oVKz4H*yfh)Z*;ot#ovQD=BA%x+J2&d1=AtIp@}@|iM!JkO=X;KCQUvD)K) z9b=iXV5Z?^3tMXi0INL7)1ql_T-vM=Bn| zQ+a(5_WZ7T{ATD?PXBP_3zk|hc~@B*uJHKFfDTA_f9($0u|TMn>04~Op$*vo4f2-s z9S(WdALK0nm%Mixkt-$zCeH)0uk{LRznr|T?ya#*G=BYBoAR+8ZDXj9Lp1uPL8)8Q?k5~ zj1jy0?@!6I15qxXB!@@I2$kdeMMx*KWG0Uom9|tTJnmpvPTTWU1!}Sv^AGOPT#)*) zPae%Z&O&&@BPDOvg)8hHRc(jR>7!f4BS$-+vj~{uH)VfQQSqLny1GBb`^f&GUjF|k zm*M~8{r}Wwa&!M*B29nS0?rBlS<{Vtm})IW)k~%kA|coZUS5#y(ZoFq4Bi-wAr7{z zL<1yDrsrHjwN0}-1k&pKLzF5?8zaE*_{91|N=nMs z2@8cY1F$_m^>vN}N4&%Xyv8d~&Y(Kk6D&8+x?RovtW}0;` z&Ns8vDbCH4gfLXs(!-}egVG`Ibuq9mNt(Iby7$^ zVa9_xJtChhh{a&m3;7a3BSN@A{s9@{H!Jiku5x|HP)+V_P3rMy`@pmu&!bd5`I5Si zc^|jCmKcM*Td-3DVTbVro2yOUtg@+y88ZkdBEtU zfB5nOf)25h9U4M7EEE}$3Ll5aKw+RUsE9BfVAel`FI37e);c(V0s+G|JzxwG)rwCr zI*(7t^&s)%R!M{vjvK%vKvdhRx{~L7 z1t{Sz=`1^x))t25A%TKF$=RL3Oe)OX8UO-<33ZmZsh{DIRT8h&WCwd=fD67AGyd=At+ z;GUUe6*~RsQAC5k$pi>=tmUd$BQs_v+hfaSOrcRTTE=gVn#(DtNt5lZpV2p6UB5Xl z%bi-^F7vOI2KLjjcCiL;yT7pJM#cK1h?x8h|@SDR#s9Q!l)VQCFJNu z`u;61CG6U4OTg0rZj97Om|U?LZZvs$?_G8Vyx1To^85L@yhpSd}v z`T6S<4xte06mIVMKwOXz&VGlvzr?6E=Cc0te@N&w`i1wEpAldGuMz*xJ@h|#ynw&s zzDV`=e}ryCz9mSB_zQ4t>BK)<&S`E2rcxf#bnw}#~oD3;^lXZQ~+xE13 zeX%_FOR;jhhs+1L>~#i2au9@rjPEGx_2yw$N7t*)pEj3MfS+{h+0eWW*Z$DCPUrYO zb&Rc>*Z4js?y`=&+#pI46pm!(7r9+ z+lj%<<$%D(p&)xd&-a<(9s0ly4D9hRlA1g$$S2!T2&5-1Vk$?yK z6C|@TnXf1-(>q@R1);BuXU6eqa}ix3OZ$1gF0N4NAsB#IBkSA2eG}%Rfd#Pbjeu+% zhXn2WP!GRD=lf9e_k?~;B~uMz9Cod=~>fwX2W7yNiHe=q5MFhKtD1hn|va4}We5G&zBFAg^MbOEz$b4S?B2 zn(Z?ai#U7FQ=v(h2#zw-V$ft$I=pT7kR@lZ1Zx`sD0k(I9tz0}nJ4=9YiZc#$rm4;4}k&L$zTX<}WFgs_04c z)8J4boB9AfUO4;VXkpU7IN_)}iiioV$q>NRX76K5? zm`h)NE6I9n_L+|Z3mQL#Q$85t<2O#43(v&~c5xh0rdZF&=W$XSY;H%}k~Mr#>Z+na zO<@LL#^y*;K$Gn0+0aTNaFPg3?8zaXFidd3tNqswWkhWIHFh1zSZZXph!-oFsmYpih3Blm@ueoALvkon7$q?#ki6kPc7QYzD*5z)|WueLXJCb=HoVAo=& zrq!7;(>Kik)7^@wsV`4Ra53x;)!_21H!`%@>5q;%$kSsx=)#PY%xXbdngdou7II zTIs^dg=Oy0>&D&fM(Bsq@uHMT&(O`8L+zwx4}>a4wf><(Yl*_n#ANiq41gMIn8vCn z!lTf=Nv$I`jsI_q(S#ji-Gqbjd};vj@hfd$@e#zu9$#QDR{(T6;|evDQMfANfu9yZ zis%8LOshN=1kt+zo=`y#q=Vf94E#`|jnq`!8cee&-9G4GsDAL|E}xo`XpXQYp9{?;G)Sf5e#-3slM>5=?~d` zdoq$%(EF1Z_*#>9<>vPOOXK=v!z zUlTwpq^LH@Pi?m4PyGM?9s4ixj}G8pku-Mt4vxnE)gHM25AplIk=0R3pJkgI{2$mv zQt59Clqqg~<%>G4i5V?^q4{oxwK84M-9?9feN1H~VF5k;s=od_dyxT#rirX3Z}QQe z5(tT~h9>o(n(Nd(Z+5rY+K$;*= zP_`EAR0nuMUWv~5H(>U&mGScn2r5~~XD07uf@!B`u@w*id0<5CBu3~J6rjZzO7{VC z9Myyc142E~6p`nxQBcs-L-3L$$y{-zKWh4edy?@_)HoPh47!>fq7P}0A-iig4ujd&2kY!1niEJ6gh$N*Je7puYO|8=Az>Nvh;*P_Z zgtBHNsoa3b?=>_>hH~5Mwr^pv>MxoSiDEZXqFm7e8sqjV5qfn&lNCw)pmrgE-P}vR zd+qPCFJH7abZBlp&Iezvhd)3sl>|Dgm5+YPE1U!+Q6U#zZ}U_7j!@g_Yev7jl$1by zxMrivd~h#d4VBZCj<5Dsl|;>!RXMzro_ThlWN|oYK+lJ~B#4ibFxe%bm>**;AED~T zGZxk!{16<$71p^S7QFW1vl+0##p&i&_5bKQHJ#;G%>Wa()zVRa@oyvyz*Y!~j9rX| zKN#Q<$M6iI8fDSK`?<(}-eL8Dy71Klyak?Nz$63XC9)OabIDO#mWH6nj3TH_UvL{+ zMrS4H{`8d_*zIq4J*zHg)fbAsMjBm%HS3nKx)pi?SUX3!WX8@5mCjVFA!x4+t9CmF zYlle^>J+vJUqBCodGHFp6-6a~$hG6QCm_MViUi@62 z^g&r^2fDenfZ6T8?$_Hk{MyB5*{28kZ};n;YG9vor?yru#{a>iIsRSsC8@sp&M%>S zKxUGw91wRR$U*Xh?!#xog07fKYsj)Mm zuNXX|Hwa-D*-7yFj|eM>>1wSQfzT!BBIr)Q#2{3z-cN&2eqh3+Pa?8y{ zx&P`WH4JAjJ{)rs6p43#N9Lw5EPERnnR9!0{9GKVyMutyGdK_M8Coy4mmjWp4u~9o zE{xROR|Mz`3;?S4KK~kM{8Y4w#K?X}o&NcZ;{dG)T{G&2b8M_NVrsmhhsAo^8Od9J z#7jW8Kh5sFpm(#dtNGJAPH8Hd-RXoBafrKvatHzNWbCbuQ# z7U5m(v054gp=~C~F&7mI@9v8F{4Mo7$r7eP8!&rrU(^v}py5KwG{<(f1lBlBr0~+Q z5Gk77B)Y<4%xSZ}fQtmfSq3;t(NIY1GdFu*ynj|e!Kj~Qc)9 zxUhIbo{q<2UFS`XC8Hj?>tX^(xuqYVKR<4ac6layT5TDnY1HEdec##l=FZF3g1yN_ zl{c?0^|YR5Z%KGOfvK%ZV<;>2T+#t(_bd!50j&(b;%T5)e0o$Pvx`2{y5k(`r{mRFtZZt z=FJAbGnY;fvnB;XD1&jCv^kSlMIV%=oX~O{X#^i#H893IH2F%maT!5>1cIa^sSI;v zOU9&6iMy!vzG!T2Yk7Ae8fzLyDYttGxN6Po$E~MyHVqmIipr3HDYc(f?5%0mlk#CU zJ4Wj}=+2T9n>IM*z?W-lANAdC&B8wVt(spfcX)mm3gx-3_v6qgnL_ejxSR7(ndDH> zxy*fD#rdjlIL9>A*2WaZf_cwPm|NC3cD8lUn|NG{1k9RR!(iM;U^SBa0fOmPo8=_F zRj2J5^rRqllp+n!lAfYXDmnx)GGz=NGXp*~mQ%w)VSsqb z!zrDif%+tm^0TM$L=!7fR3YJRJb6#-9Q9ONTvRY3a;p6Nvffak*&)X8%rH=tO(9#k zJOPK%vLo4)v#4nAnO&(Psv5VTq(bwO$)khQ)3ak#w8tmSJMP-xpYi8PbDe0d*SBRcu~1M}(HLxtSZVF6 zs(6xX$~~)fMTGC#Zij3ZE+m^O%(Pe@1q(Z|s&yLF%G0qg zS|V9A(Nn*OI%9ls|x)dEt644H) z;&shbO6I~=9?tw_ltJ4Ri*GG|n`dGONDR(>7N2Qw|Mockb0+d1Jz!xsLt{H9^Z(Wp zic&b2`NU@bDWo>*E;!+7FQr`ob<49^_ga*`AnrjdTiS~>XH_v3REWosNvKKm@ zV<7R83C=C1<7*7Zl}z$zX+xh}a~3xcqr-l7JrppD0MWL5ISlGK&I#|+yvwAR1HSRN zkB_fC97i=kKi!JTGq)c!u_qG23ULoHqsUS-`;6+$dljV>2dZ zLWe=QZm&tN)Ut1;>v9)F^{0hPT)18%afp64G0rU^BwNfySs8j*CX=k&jV7s|C)SJj zx6t;TjjE6}@TY$ng7PRFg$^9tbh&;h98-73O}3bV@O-!z1NFg28ZAAwj*Zt~=8(5Z z)kvUL<7ZPIqd@1_#WuA;6QC;EeNrxqe)PvZ+&{C!OjUY~#ktg4qtP=cqVQ3!@5iR6 z5GdA_zCh^hhc(POZ?WcpaNYJp;!n|3J=ey^)LEqvtk8b-sI!CpQVap+c`VM z)>X!u*S%m)5K)E;gx5a{rG*O;rDBfuJ5;*keaib;!tA*bp&>V z_8)>mI)kr-%|ccYtrUCT5b(mEr3Q})yWww*x!=U>?875?Xu;ah_Y6`aR69$DG3dBX zp7OyAbruMRhs=rHN`f`bxtnDRp+bm+W7J3%fxx(NmV9TAu6$n^uE5+Vqz#ae?a5N#YE z47CmnWco{5X>&Q2vXE!6&$D$@S&}}pt}2paa@lExAqdAOtC32bB#xzCo!QzPm@TGJ zUZw^VcUyqTVNH@=q$V$`U2B(?j>L!nJs1!W737jmj})%e;F_#j|9TMfB7qIs|yZz+d0vaF_%W4w?gn#Mj4hO9}l+`%<4(05%_=w#IA; zOSnYxf;n#g?@KOuuP1Mvx~1Y~>ICe^;wY`H8Rbn%y9Fl5U{F9#q>?JRAEj`l3a z@M{Q|?5|%iB?Tp-z{Eeh`aD?=B;8-^7n#BzUUE0#A|aKCffjc+5r)E>=`89gO@FT`fE|7AmbB>L1vWlpPOPg4+vM zDr~<-kxZgc3Rp*|*dekhLyGLjspQSsWgjakD%P(_m*{v<#doLNL8%Y53*FX1QCW}) z@r@vr^HAa|1VpqK9WD%F`?hN=9H08j%>ss{6$y^HlJ!SEa^VN1Ni>yA8I`UxWb*Uh z-_PVsn4Bp&N}r0r6L;eTS7PT_+2(i?!(ffTP@L_I6898cIOLeIXR(=*ulZ#2hz8PO zk)bvs$-SYTPpzs2xI5ty4nj!13BLc1Gmmj%TiX{oO_R~|Ec~{Nd-|h!RWAwKI|T@B6e1EUah1^Ku#+C6r_B zQUX>BFh$yhBTw1|6fL_vuuYZ4C+kZ}d8}5=4!2W=!8?dRrd=YbL{hE7MhELX1g{9L zelDk5@K6WSHZ*>FG`fqup^?(PWCtYN{0gsGeT@gOsdl4ZsTsB{+5sIA`!?OPXc>Yi z$)ztG=GYOxyy&2xsCE8)`@<YqVt#C#5L_nKK0b**O)sGO{Udv5l$IQQnp^1g+C_4e#;m1s6Urs$zJ3F=<3sm4 zWI+dXb!d39^fYg-tWhnu2O&jrL7Z@Db_MHtHLv(&Gq0MhF0h&H{zA^MHXSEuZ+8vn z><8%;KDuvcbdHvaSH>`<*54DaxBhpqx6f`|^(gAJst^WEY zBPaZKXU_R(yUYUy1||r`;{t~30;WR?CIW_v3+4jGA_4{hlT$fZFZ7-{*y-fG9DcA6 z5I^z8WjRUDR2CbV|8_T6n5k5YoFQ$ZV-kilgMR#$s3ZdBlQYq&)HAnHxR4JYODF)jN5ni8YlU_0bLa>A`P%;9$ASd^7z_UIZPi~|3jejRic+#x#Zrd%q4n3Q zS112oAz~51(wuQqU!+zAp+dl<0YisUNvPYos&%t8xtZy`>&M%_Z7^*Pwc=>e;~vdB zwr_< z-!ob|8czYeze#O_KDIZ=-$#p;-mlXixf+xHvzv;>A>E1@0tW-$>R@84U`L>ZkWjuxFp2 zHXDf@b&@CY3^-Hi!eH-0W^h$I2^^YhGGoeAL4! zrhy|zGDkCNb$U^SQ&v=i-9i0eP`KeS^b51gT&dELw-me}yAFf2(oqwhwP6OP407*d z-V&34r(+-#BGZzj|MnnHOV(J@LWHyYa&+FcFd&Q| zW?>myfNjVP3~YXar%@;L<1yO%w4C5Vx|W!O&=?e+h;_}bB+!_-jo}VE92KP2uijrH zO5uMQ=|7-)sIsEv9U6UDP>e)P81gspb-~Zz8eA9tgd^S^kD`J%bf%7x#@x^jP}K@{ z-hvEdtoD8DOgQBn%m_i}?03uG|8xukTAi`eexR0&aL0?!_0|wewyx_lh~~uNbMyQr zo1=?e;o>`r=|5nML@v(OZ?yAk$$t1z%hwK<^XJ=N^TymDO*o6s@_6{Ot@^)TT>O7t z-2Y@Z;=hy`{H;OsR_JxEQ!qD6lg}Cq?h7J*N>4f?W zlt$G;=b4{?zV=^Z1i3SW3?N1hFF=Nm&*DrJ&|d(V#Y zMxxVzBgmVW4wve<2ewv{%>z|b(iR``y{M#jF}fnrg;aOxNBM_*lmyAOfV$a_&Zev{nzWxSFbG~0@nMARM(Ay|^c#$ME7IKTrgxQ#x{xwqJYxKnAAN18_GjwAl zMiKw=u%T^dO%bB>Et~<&Nepd3I^W@c$jTNeVKBm&Ykk`g%R&IdGqVgTna*Wv1ep$t^%t%qxr)mJfNe2987$Dycv zv0cC@d0Bzp6S^VdZ@*H%FtwwUYhB~;?w?G79FwnfNjDOcrDDMT3OR~ML`)wAS>o0M z=h zcO@08-Yeg#d7II zwOl8ULj}bj@($WaQfF+aP#x~t(kwAYdABwwrBTi~Nd9@S;){U&_}u)wa+M!RmMj+s z|K2%SsWVqskwfTa`_4AGU=;mj@-}25F#)rQK#1Ms@-uj+V_+DG{N(~!lFgsVLW-If z-ULV#@PAx!QT~=Je@-WVjw2Zgs(|-3q<89gFtJ5h(jhG-kP)hHzCTFg1?y0RdBHZU z_$hs&9hUv~w0)qk4TZ2>Z3b}nT&z?Dc#Ijp9%SBR+w!2ctnx>N${vyr*r@k^Z&3swCdi0#61p;i8hW*dhQ;$Oh$njD zHlusBiI$Ocvsru%Hq#z>^;Q@--Ti70qrM*fv?Xtu4i-IA!@_CjC9EvZuEWoYs7MB}t4;U8nO(0H+|N;m zV6w)d@31G98K2@?^((1mm9OWBo$Sb>iZm8>0=VFT5{jT^#6wcR`40n5jA7&vq!6iz z4p7m>1c>!7rgsAvguW{_qLYW0lW{PZTHyvM7fI4kV=YtLtqd$HCLM%+5@x|1`zpsaaD3R>MdQ(W3e%=@HL?&QFDUZ zZp}ABZnd}4YuDndCXj0%X@cj!i?|QT5I4%6qgFUotm3i(``_~tScjLhvF#OM;U^2f zd36`(2+m3aIV{iXrL5n;n*KDY%+T)B{lTVfkKQlP6 zWgtp&f~9Jh>f{XA9ruluF$y~ZTyHtdZW~D#zQU`p`q#t`-f@_1@UL-GP;XFV$EY#F z(@VI)gL(aNqLNUqH#QD1(e-OHkR{AFYsg8y&zcCluWv~RuAG=Qj??~E zO2*jW!3pLM3=~QU_2g<=bKPG(9-mnkndwOvhxEF5AtL=m!&#GiX!#pzJO-*1N?4v? z9{4IT)^j$HeAZRY1JFhi+=|Z3kLFx+eJS9r(Vm>XE_1qjA(>u$mu-z{iH{)MR}%^i zm%rvpI##KRn?!rC2Fp>9v!>f+>EyO+oeI#gkz1&D>f--JtC1vzMYzxrEZvNYkn`Be z?&pa>Wc%$tzvSUciK5Mr6nuQmv2lOiZZ$Kd3>T^;GXr|?*$NuDZ_jQ@<_ z4_>|GAkJhJjvf$R(-J;8c}gYa^(kkYj*sBiW2ol~805?w+)}K5*MBe%(xqb+eE^lt5D?t|KB@eD597DCas5wN|7?1Htr-D-IfTnY z&RC(UmGVuIt0|v;smi4pKtQzQetY}!17;B`>buf+9Q$}yax5l*?jTd-9b=&_0ZgPi7(y~jw7@SO zNLOIfRYuu9KIj0nM2UEt4gUZ^L`6yli3AF3^vt9*;cdUF_AaYOUTw(1q&uSx-o8O*zp;$hU|$#uub{IV11n}a-c9Lau%{^8YzW5JdRezKO{Yq2NoT#9D-w8{%I z=}v>>C9DrZ%x)ipA!be@b=ifOEv@oIT`RdSz0;3Cy7_er3HK+PDmV_o0cJw%!Fv)v{kD>OsY3iE=D>E2zFz z_~%aXC}cC%E!!hfr(3eL+SZ1hWlf`MzidODv5J~=;AFkYzY@$gt4qbj^~vAR+FzWG zu`q0`j%9g&JhT50g5c4f!<(vjphNWg)k8@m$6ZW zYv8! z9664?QJcPlTpBxe2ie%HYHO*k;4$hrroRO~QJTzpUj8M~v+1Z=@j3#--kHxed$Z*V zW87{${cIkTW<_rg+LXbDFTFqESA739zb{GhmeXOOpe%wz>FE3(FNE=nn*%d}Ey}W% zYZ1!ht>~p0W+mFAWf?RYgPVB#5u}hb!$(|E2PX}B`;<~W#$zl?tFVzImKJ4A)yhp7O4(;kKFaVl3ZTR7Y*(FdzH+v0O0`*}lN8wH zG6Bd7SIv8~C45)Ov6DX7P~K_E5PTOFTHOK%_4?6}LeJqzIL@6;cn*d662qg`>H&qnzR3#8ky z!}SyBU64#%@6EO>7Ilgw5shOnutKIg3^}`9Rp5xvdaMwntES$kuL@-=Cd8%%NYSZD zm>VeDgjo^f??u&TGudu2wh@|qP-d|LjC4`(IO9z5R7aSv&h699zuiH~%f5at?C9&G z&h4YNbnw zn7tEAU8(VBz8*|?LS?`AfQ*=Af3E%&*g)3X&B9R{wA$j8;Ru6tQJVEoDjt=(xQt_5 z3Mv}M*|BT%uH`hm#rq$2Ap`le#lJUElYiVqY5(@I{*C$apF;n8?Ew&C{@uP~R>Xd9 zrXRO;XUofZ^R@WyQ%W{eS`Fa%;C!+io1RV)v9U&@Rt?3{hTXs}TfmPpdQR$a)x@-j z9Vw#kV;)}Z-k(hLXSzat6oJjb=3y*$qX*JMKUmNxuS@Wu2qC0vgb|ipB)4& zdG|6r_it0m*g%LPfRiB2vS5%O@`Z+lZc?+?57)j#)?iP_-w*4MHjOHhgc`I~ZpM}Y z`${;wiD0|3gl8}A#AXBn2R$jz!l%v}t?|jU>-dleou-CHgSJ)!0o_bPc_v}s6w^xv z@n7-itX8RGKo$hpmPyK_47D3@75Q>YWiF-jaL0#!P4+wl87Zn{gDt>(Gm}nwo6F&T z0``W$H+e&>rX=Td#S|OO(kZJ5ikbH$xvCUcn+F!d4?@`22SSSGs~h@WQ(!+ElNH#o zP5*7hO=ur^sfZLt(UO5~W1%5@o6%jJJAQKA;;il4jGG@_W=v93FPI?spYim9SM)VSOT9aPaB%zgcB`x&nZQdn+r=H5Fjrx`jQXfHE3UgpTVnqvPOx3dD9M`7e$y_^!QiK>7<_uBnlS>rk)_m4!y#tv3g>c>oy%41DIV$}l zK%C!b`P+X@v0vc=M1?M4rm>Fyc>ENK)0<`n#9;~W_uuCwn!m;2f6CkH8W;!|+8A0H z7+UGO{tKY3P+YOY6oli{AH`7BB56P%0p~-|frzy*-mzhbg&1Pdi>3)V1sPlwY9%n# zEL$_IPpi4Z6;e9;@!L^~)Qp9@?=%;1Ji0B3C8?oWuW>HL!ToJq+!=&+VHv_+OPqOSlV_bE!Nh2 zxEog~Tm~YVV)P!=eNEHF zhX$A#sC?On@}ZbOB1|KlmmE~I+8T0vlJtZQ2dbmgmD-nvo(~vVXuekRvC9;h(rj*Q zTzM8+;cJeLfrOlDr`p+d!JIR+upVn)whS^Ri7jw_F1zu``_+&_HwO7-GX=i9ZC8ic zMgh`RLGLS!vQc6N%r%#(9z(ZCg0!J6O=0BSIwMXh_x*^j>Xqs2;WErMsM`j0*% z4srT)%B4z;&|_ru@=|^15-aFbR|rws{1p?wA(*ok#=>?g(+c=Z-{93nQkTX2n(%P(I{(UM?s7%JU$kT4(vml$#9oB>X|uD~lv9D0}t7+c}89 zoL)l%v~${%7L!#XCs}?C*(AuB9&vYJ=zi35l!7B+yk4GfWgm1Qx>a+X2|Q{wrSdas zFE!8>TZ<3sAk$GYc0RfCMWfd*hU8Lf9W+#dP+yv5${BlpOfP!t&W0aU1#D>qQOpp+ z-iGRA9414MlLf-shvB)a6rA)Fe2*!E8a<8|=p2!8>G)Z5IFfszPHAqGrTz>u-mZ%w z+-G<@jg|YvUrSq1C}v*}!f#F^SpifTy)amZnZH%&b?Ot#&vJ=g*&xY18Gce!pNQ!h z+EH-VFN_ADp)%||xpdIruJC(!qW;`e*lvd%ZTM{a?u%yb*yOTbv-cmOygdSgf<1tCi}jDTvkZSL z4}U`$|EJNmgPp16zdEj`%&-_RFPvwBW$dt7xHw1>f3(09-T??N5Nik&`~{~zCYyvC zpLA5QwjnhnPG2o9(S7o_+-|PfGPk>9N7)NpnxCJR7pr7ytMqi|dRhWnVEBFsAuIuj z4WU+nrOB8#NC}-LvU0`o8)qj(_rl^WS!lvLrE$9%p9j0Q%sh+3*Jw_l${iB$6|zgq zwQr^(9=G$X2kYi1qxf(blkXnW-X>3g?;+wnnKC*xw-{7f&)$Cvjvsu%r=GT;zq)`( z*nmPP)rS4`gQDBxiWYM{Glo1?Exg(OG*FFB`hno6-@9>K%E&`vu?0%=tcf=}19XK6 z{L2Rvh6Dz4Y{X*a6!V}|S}Ov={-x-yjKI2rp*(U7->V76;0S1vU;%|8VCU&{yHx~Tyk45|+==T362?Q@0M*KVliVocchViI=xlgglWBK0 z)%L<5CD3-X%C25VTxdToF|=!0p>g4_U}b3C7Rd5w72DJUrA<*OpR>-nxQ9g1nCT(?upX%vH=$_NRYlm*hdbXzpB?g(AJ=v8$ zD4d2@k5M+r}l}*&$=GHp|V&hHQvNhAArsu^{{u0C?u2Jsri$T}U)Wz86 z=Md)>LTiGPk^%8uDEj`M}|FHMkfrV(vh;PLL0Xvx89_`c2N4Y_%^Bf$yUEhrxn#9Y`L6#dG7RW zXZ87LUwRG}y8!_fPTE~3rqq3A`JI7|Q>4PM0LUfyA9rZx|4}C_ZLF;ftsDS%9ZseO z08QNg*KqgOPya3zvYO&R0G(9*z4?|x)ukeILS0ZxE>vDnE>T=G&PV3fNT!-!Lu?dt zKTDHBadHp56dc}2f%06x8~U|%Gb)z4< zSWJe`*6pC^8SL)ZkVBvZ_i{z3%r(#YSGgNthVb|psx2H>Y(%?__#-MIGey0s7LMRP zRX^Rmh8^^UndK&umbrN`QhS`^%JQ&i-HR->KeH^;!Xk$cUuLgU;!~K2t423u! zCgZW8y61mvSdvQZKa2r|g!ms3G=jf{^3O;5zZL$!z!4>N`xzmm_xQy`{n4cF3lWHF zT|C=_Xc!0xygAXt{oOjDz>K$ihQ`lPNx;tKjdQ$vj|a}@@GZiM-tXM>@KF&xAnJ`=>>4w#?k}7 z+NkvsjhT~f=LRBcOxmD_0Y<(#8^L7y^#z&k*uXTE$xHP08JjPzFj@LSybq58vb-OU z0!NwEhDemX>!b=AgLN#wNYV0a%8XQma^|Z5(AapOCTYtm-D)7e3pKd5V>s1$U3h7~ z_&#d7Nsyn({(j*Bq+E_cgN7cRo@HaHRn9U%6&z~3&~P4_m&n2d)hR|Ipj9g_dP`Hp z_Qb21?A#Qdu?mWuiJAFTaVplD91V-6qh&w$>>B2LXvOLC-Z7_}KKezt)ap8-b&?dv zH->esTc)Dwa|k=xcs*;{;&b_o@(=Ug#+{!Rd^R!nWG*@tz-CX&m7#0`zfv2HGUega zEXwUcJ&Ag2&7Dy(oz_+_v7Q8uy?Ab$Bz}HVq_^~%TPe3QR@_yxIg~VTzJw8Asr$JL zb`QV0>1B5Ug`{3SeVLA-&D?MPu-a^1cm8a6*u2C18W@KkVIc+RhlGH z@J)^3w19@QH&@PX3nH{*ERMPH{Gw!UOWCmfSL0}=dZo67%e|YqY3o25S6L(fYrMtJ z9^~T0P83kJ8RZ?&L5B+pk%l61l#N|xI{C9Md&B9_G{))G;u%h6uqT|xCAEr!mmDXw3Ph#4KZmRn zX2|5OjzMRY(I`mo8a)j!lkMETYW1Ys+Bta{Rc5j)W-&dUz5*$}8!pUJ^(jl9F%7 zkcGg-YH)Yt#6Y1;x8Hw=ZFGM+wvkbKPJC9_L#ek7e|>u0wnAyH&{SS4lUe?2e@@ z_<9WN*129}e5=Jd?xw0{fTQ+;xlF};iy9ks84bh9`vL4RNDm}=+{?vAIc2?ACD3W) z2^4WMCY&N5IY4Mu<>f6SP+fi98AZQ`TyliOJ-=>T8OY`I?DnS6`o^!#7QGd2&^9?Q_7e(hu`=lM@9js_9*~$v;V#=|93wTT58(=;U^Lc1Pb|)Lqp3! z3yTPqAjlu@moNCPB^Lsd7uUao?I_&51q)P{R@$PYCtY2+epa+*%?uNsw;lh>@$Pyw z$YKwC>VpR*Go}ZndRV}x0`8nIidIKcv>N9rnp?cK_9bJDtQo3{qw&k-6iT0o>sxF~ zv@;t1BZ7yCqrIt%ffB1>4D9^r-%eE@qmFl#^w1)8vES$wV&nu87E40+!OIFmmj8}Ue0@- z?x!8D7>T&LpP?VBHed&%UT}YpBnkL+Z2$j$B7a8;|2M~oKRkwYEll11jTA=6Zir*@ z!o3sIX@#RoPT<}NGnvVYS{%iozUaV}e*<5Z!}pEnI@N5bsac8}&8&MXI}YOmJ+l8G zwWSVGWBkf>+Q_i+%Y&iz=3;+>=?B~TuIz~ZdUvj0jUUmMQCKbZVx(w(x`1k=0Vp(~ zseVQd)St9RdZ0AmDFr|Mhs`W61kxe2q3lmtEX&8P!0^;7uo0BXKH5EDr3_41%%sWq z^mTMdkklS^^O;`A>Al#i@h5Av_w{of#0Szdt^L)|4Ot+1BRc5nuvEP{j4txPnbMbmNva} zeM1;&@A^)sRA11E>|jC3*BX~>rIzufW=gZ91UIAWnHNdbJBj3EJ`TBOnWqC^!k~ks zFc+tZ&6;fjbC-Lo!kEsN%;mJ*;OFArCpPKoTTrOvL+%QK-tMmLmCm7aZ9Hd(5cnu+ z8JEGqQCbb1Tj$hp)q%QCmp-=7T3$XE@#7VmJ)F33Y=h2V42{2I{_Dv4G!Zbn#Qx*# zLiNAQF8`T5{}XS6e_>e>a(@Y_|E9iJ@{>_D2I_>yi6vi(5D*s6l0bo!M7wCPi!Ykh zRx+PJK9ZC3Quy?Jxdc#OmcJGY0*Q0gGSnJdv2%NSzI@gRu0XMRK74LE$vYspw(W%Z zstm*k&!*x~g{1RYH<-zRZ*^b}F0BcE8%GIEX~)a0E6OFeDU+C{7C2@l<7U5_Nltor zHT=aIP3THcj635%6gGYWtw`geXH;t+cH6_4;49V@5oame&82W3;2ausS*JDxHh$Ta8Pj8OtNl6gZcnjMAq!yqKD#-PQyJPLUhhPFrF61Z zwO#?V9f9hy&C2~rkT2m=h}Nf&Q7DmD(!P{6%jp5}5izbE)`Jy>1CNczM*xobj@%XY zNE^0*tRZ*V;OmT3PU<4vjSOlDdO@M0IH_2b2s~Ijojy$PnJJ1ebAlkCs30X~znP=v zdxkJU0Vn!EH>(Ql&Q@o(s2Z>Jr`7r!7s@>hnq~1Wq?P~`TZ5> zK{Eh&HektL3+*m`;#z5HO+`Nz?ao0<$&dLxkq9jFXi;qiv2u%*y5D$bd*5W&U1$SA zZ4k@dpG~qj!**llIdaF8Y6W)~1SF$XSEWM*I<%?VCU@SeF0|n+gbFW;_1z_VrBhHC z^j-Agbuj2vmf`w`7*(m$xFfR%gw-exEc;HfiBxVRzEO#j!Jssuq*7Y5$#S7{2wwfE z1vpr}{^Pw))9GT`ej@IED;%L7sk4N7`Ey=}&CgLvXyf-FS_bK8B7hSOI~x72d*lUd zUnm*Fffa1T6>T}0^Nc-TBw7t{ua)OECQ3W$E*tU2XNP-PLzX_Ng@=~3BZ| zJ@w2nTFxBq-dWuk^y!sAmi$)__IUkargmv`v8e{bkeDh6(5aukjbhgBThD42{XjfG13+`*ek^9v&CYqe z@#l+Z1zyJ4s6wEk<`^C@$feNNyBVI+p&TCb z&ohkZzkIidmd7VYOe7a@Hy0mn1GbQBQo39tva!>%e!K_EVL3n?=wWWTv%n^S5O-;!aPEpQXgvgn0 zF5CW?d&>aGeTbOfn9Y&Iby7P7GG5S|SIrY0X=C}K7UM3ZysC+s=EB~Y9Ef<@wvQ?g zXKUF(h3}n2u`|F!3g3h98p{)-^INr+=Z_M1mt(GxPEeqaB!{pULUB%mWe2~-jPf5- zf*yFjnIzd3O&Zp#f+g#1VR~;|h2FugM=(opTJNa8zvtv68rpCG#vlxkm;3Jn828@_ zyr7k%rJ>y)w_5+H@|K2{dH|^F|I2p#7r*>3^T`>=6)V&d!F(AI?)jEArNm=IGCDdi z53=I!Xt*q+Bwo};0P!O!yHgB-3n5}V?GNk83ZKJR2i)_FcKeAG_w&`KyO&YtPm+}8 zF4;MyTRDNLVCww*y#z4g80j2#q~r_*ep`W&lwxcHvtqrHz%xzeIyekq5@Q%uHiJCa zjc5bov>}3hwB~cmL7Xg3tF+-6Oeqcmefr$-RDtD+D@Mg?XP#bN89zpye~CM5yV#c9 zdk5Xhih)>^-jg_uIklS8n*6j#%BLVyPwhDJ9cmoeT^a%fMjfv+I-P;i+!$#}5-8 zV=|(HRyW_^Z?)_gKOFXkvh(+|)@4LtxSlAli**8>tY(IOcA^=CI(jL3)jr{hxCK=^w;|dXYNl;p> znrQTSh4=?@!ygPOi*j^Cl2HR(MU#_|`?`?&=yJdLp!zoxj#c?0QB#QQ_|p2J6WYr5 zpO8Yy%i1C;E3Wo1Ave;M=wYOWqDz!C5i~eoMTS1uXbNh6`g$1}N)#?092Gf7cnkQ~ zc#GYxcRVj-IBXAXt}n(%)4Wz_b(}EJV7!RZ92lu7xx(lw-$cO_#ol7f?*CPIOQ>wm z{ke3Q!m;-*xIei5q|z-xapj&|QQlcoe{D089a-Qu$KO7$-k85s6=!2}+t(+KN8d4w z8sc?ry*f(zX(c5DszFG)&`F46LUO+u#8&m{utsoD>qgLk32)BSSOI3BRrRC5djh7H zzD*qgLD4-zT*i#uT{n<=SZl$vtXZ|b!co7sNO_18J)Ia&n}xDtvr(zJmeMyVnqBi{ z^dqn4V4}RN#^6ONZUTM#^tM-WZ4H`Vg8TNARuiJ6&}!Uk|Ae?XEv@aTSPB4@WM8-$38kdfjSj=CW3^cEB04V?m`;|7AXvy>(@mmmUF9vt97atHp<~cqQkU zA#&?1hVbmHd{})IpJT%bOOc1K;Y`)xv87s^z9sti8T1>a)O4oyfiNB z@|lZH`8ui2$xn!JB!M_XbtwICIn_!?`a~+G@gOG-RYtPe5DbtI){|DjN+I(GM`X{P zxnCfx%x_@-m0A`N@GOM&!=Zq416i$`%onq>!2+E8|aNxFGQs3}f(e zqB@ep>_?LiGmd>3dT}m!1$`&~x%z=V!sUhg0WmTH6p~9sOsY*K%3O38J#IoTZbFtg zJ6$d;pImMzM%h0tp^ILjJZjELRaQbds&M6`yQf@?Tiru5gx#eRFi3jNZ_MgbszxUe z_Sr288Mtob8fe_q6K@$c8?l^*=Vz6ck5Sz_+}YPjj+3u4thPS{JaF3$LpyJ&ehu3E zP~dPNMvU6VUbDLuCh2!!iuN`mte0FS#xzR)bX7{7p5NB0O_A3SvTeU0K(2h&7ySW}g>dnr!m-Rl<|rk|^3 zCk}b9(l(d>2~S66N1E_8hAIM=k4h2?Us*4N-#|=S=8c`Mb+(w*7-Axesfk2ZreksqNQcB} zLGUp|%CVYcg!KJScY9(E+qZ(j5|X?$JlVfuE2bhx$$Zm-M})z1W~$ z{zxE`qqHU22!4m0*k^z3K$ke&y9OcC# zTa}@V?i+)#p;Qtji$;$>ig4*Y{9Tc=^mzM?Pam-^XxqxRp)$Q7a<(bj3bx(HiXT7E zkmKkG^|)40)sz5Iutfc8vh4s($wP&2tz!v6%f!|yR)UwlnQm>CpKgUwaE!r%qHgoh zbgRj2772bQ6QDe9$4C$(;wk|t2)#g{#8BO^T8tXH+<=N9Ni0cF$dKS63EDyD>$+7C zyW6a?nId65LHLB3q!9sH8H^oufN*7MDXyv!BlEMqV!wv|RA!M@)v8^|MAykUrXpAt zD_59Zq1#s8d2QBY8S3TQiYTdI%C35@u5hH3{HFiqgqG`+UmH8HUX;#-IK}|0X zbzg0DT_50OGgHOUq2&pSSzCwFZ&^zW1avc1&=-mRBQdXG-m6-O2=P_A6^_P!BE=DC ziOEt#xjm|zXuFw>@=_92a&}2r$AR>*htfRK#6;;t>5ixQHXQs-2tbT1jiR3u5-jq6 z*e7NWz|KkYDTbePRAuCYn%7*2uB|k+@39ifge-;RTSAQ5FP$t_whEl_n}#t#*lD=b z3hc8CN)eHe976SCN3=AgT$5_mZ%LuShSXsr$QkDY@FPhz*aPA4?cICyCtyLSlxCbf z3xpP4cn^pSy9aeQ--SG_x+(W`=2ZWjg9YEilX)z6FD90jtnC+{1f$8lATVvq(fo{ z3W0kyDrwuCsa-3vw;+yD`XyAmEl5D!kCx^5A7I{}j@+sv1w|IKRe>X3;~r$h};);fZg_adQN3XATush!WWGjJj@4_X3&_}@86N5|i>HH=&Ww^Rcu zpsAoMpp!DQ^-#yhee63lIqJj0ymMEANTTUoQu#Dbmb(=(Rt{ac{5A>HRC|bZ4V6r^ zl~(MuRy1p#O`si5H&-vf9RM78-d<|9dpMOu!kT&cRjB-hMv2TDq&hOc7QbxN!+p?O z`N_|-x^PK!{#L2Vm*b?JB2W=+4%wW31A~HYZV(q!DxY__e-Gz-XZS0TfWr3eAJI{^ zzYRWjqyGDTEI`^#UA=MM&AfdUl3EX01G0lG z6w+4)Y3ez1O(>##sb*a4PFYnO3AfMKJN~;ls;t=!ZCpCT6TL1Mm-fb|*=hrw`L1*y zS70bG6c~kC5naY0LOa%c-BeP1>s)_RMC5QfLS=JQ=>E9@n9iMs+cSQEA)$zR2vbxU z#Yf|4lFJJxbmHfD26&P4;UKQo zXVmrs2Zt_APZ4a>qcS#r)1e*cJ*&Z~#GRMrWAE-7wI0OOs{WxGg3(B-!12%H+atVkJ3lb5kWdRVS(l9Oc8@?j_^g0`A&FGUW4c0h63zattl{s{k@r6=@{@X6mjndJ-t91nmCp@F2e zfun`tpUo=ZUjlt{eBwX&;P*;N%`oD><>r3VQsbdj5$;F7BeVR9K+Gr+OV7dj4(sjr z^-x8}LCE-a9Cxn?v!~)m+i7O)(%Qw!+Ue8H!^+oB3o-Jvc{)lyB|1{wGvCc@nIq8v z7+TGt(bVWRRu~rwDk7YTbT*hV!uNn12*?xNsflelUyT+O4H*c2eY*T@SRqA;&BV}` zrkw514k!-_9R4}@ekFVPK_w&m08 z1Uo7n)-6i1Zu&?1r$D`GeYu-#k(9L2x->LNz!VoEy>)Lor?gU1?l;`IUryLqx#t|T z%7B(WQw?_|ds`tC%bwXu^x=%xDJASSox}OG*cx%;$)wqa$?=SVGp0F-a#7J76ge zgMN4$iuh8@&CA3p*4)R~@IJO5h2tpBN`sXB7gy`G6j*JdFlDT}{Lo$!X;D|ooBlEj z#x_cmL2mJ{oiq_j*an$w{fSErFom~LN~@qL*g=z{&!81;84hN91u#r!mc^#6q6?R5 zL2hibQ7&7(LnxsV@`<`FchUPEbZ->1(hhKFc&exval5JA2?QF5C&Eoewol*-fxVRy zl2IC{M|-7soMM%C^%vZH3)*R&PSl#usIyaoOFZVFww!g|#o1n)EZ93Iz@6T5nL^M3 z1=e|W@?lLdj{Zm_8Te~bEdkm!;Bz4|gRHvYKfs)2p6sL+Sf&OHt-dt}Btl~F?=Di? z?NDEXw-{R3B>r~?PTbEV$;j>8N!ac6(Qu+Sg`ls(FX(3l5F<{&LzesTYH>6=M{fF1A zUSr*;IWIh}jAK9pN22f7I%ENYkicws{_Vjtr*Hg)t!bWcx5}dUL}IzPlCQD@>}v@9 ziG6Q2+V&SI?8f$9@6XRLJfJ745{9~_O+R%70a!*9lLAw@naa#%9WJYe?0zaei=Bu* zN05YGCAmGr7s&!ujfouGIM-WUKH<+$Y+|&Il)7qz^ykM-L9WHC)&?cH4viJ^2s^ zsOl!a@-Hh#Y{ukx zbcdKOS00=XRH73a+K1#9`ST0J=0g*q#^&YtPu&wDUz;K*z!N(rkYRo^EQ!LdUjq+0 zb~-pjuBX2i-t2m@=kTLD^! z8DJ>*?`R$M-v)wzM(h6UCIt$n_K3o8o;&6Mtp)>>;l+&Xj~cYZuw!gqn3 zo0FQ7nga+QlD%z#pfC$elx9siTZMt@7*^&@d0Ul%?(K_#1nq~tyoM6k&}7)ykG#Sp z1MyFyDfyAq296E{d3j$FF|50jt!R?z;GHmL0qUwr?}irI^lq`9+HUHrWd+`Q`qu15i@%N!k*T4`0VO`ru@a)S$~+_f9ZY4buF zr$F-asl&sw#!=ZTbN+C$=g8(%blXx@^oM@)*F==Xa)H!Ai{5fT8$p)dQwR4*T7OLV zh7M3q9jV?Kdz#J*eky7xW=YbWP~yzV*U&0R_R3&wp0WqaQ^kp2Aa0f&ZivR^*LyS2 z0*Ph6esZhx+c4=d7J_q+aR^}2gvk4`GGz=QUy;SQMql`>SK5(md1qo39P|X^rCerL#3ohFokjb#pF7+&qRP^wyhv0OJ*lwo|-KMAFP-4 z`DE~UP-`KHmvT#G$0S}!$lUpv3QU^}mOA>Z#19x)S+;8e<`_HmQL}D;CWU)Sl{#Xf z>SRYeUBsej?HnUe>sx|n7|0vUf{hN$!1N>e{X&&7%$vCCa@2g)1?PQ-o1af&*-)ln zrR|)pn;c06K}zRjV_;-~;^}MrueM$twinotTXG?-Mu_O5df4%wWKc?J7@co~Tvm<1)KkR8(!%tT?CoG>BiVLH~bCUVU0!(2hm=qY> zCadzd3IYvbC@_?m+Rg>Qoa50%^RS&~L*6$dFThC30kgfXX-cq8Voi%9~f)kJo(|?VP_4V#z*{aT@Ib+{mZcV z{7a_c7xSDZku+i{pSEpAxD|*%kMcEIO1yq6(r})06|12L)xey`NhogqhkECk9Wa2v z+kIGS7)Xs-nTdU^nYReZu>8Px*P_Gu0OsN-Y_g>nFzKty>3ZHAi0PaDWpp`f>8T`3 zcHV|th&2+lA>vLQ+RyX`ss?AW2OuGctIy#p2apNLKU8RAdkINon?SM%4haGBP0j|t zCZ#Uh5BjI+TEh19Z;_;@ypez}lO2He{{Y=!mISpu0j((FA6pUr-&zsB_5UQO{@H#! zWhJG1c;Gyt3IfzXx~+Y}I6u)T{K}q+cELCEojR2*9lP`pwZQVpF+O%tzN$b+otKwaRuGy3h!NSgn4GW}!%hTh$ZWBnoVQJ* zIdcn}w`BD^jbO$m#dhCKD(^ZC;q(K3EX8;_a|*&U=^mqpa;c6psb!1KXH|nxF3b@d zsKQcc)~gt_;V19m6J;BsfBil)3cO}y1;5cg(yk=vCy1YX_$RlC!w7ZSs z>le+1ui)1{$nzG=9tN{Hx-y6ezCDOEm?-)=Q_shm&lbB**xDrDUw>Al#ZSEiAB!uu z|GJ>MDhovwhBO!P`?*g!k>@gkG!0U66`o(^H^yhv^(BB%-P6a8LGZL8l>l*QqD`o( zTlN)w2m)a@-nhBIF-Dj!rC2H&i)OEeG?pMTExo_$9pd-Q01>(Aq6%0!5dgvZ?=!>S z54hzFZLIB09jxtK2>~}^28KqaR)&AMNqYXZcuHB5;c2udh8AZhqQI=fx8b3b;2VB& zocuy!DUBx~LMw?jsqK3KOT)1lj(_}@q0hM~K4|Kdp|sP@S*CSo250Ao$2;UE(7;X; zwSfn@!j^k{hDq;N1o|D$M# z)umw4SV)C*c;bppa(c+h9O>>)Zpa#_M&LyO&G|ZL2T0*L>b$VV=shLtSiRi}7mvOT z-Q|+57SQ9yS3n~;KBP*qr+xkXZhQ;~k zQl-&gi|ks%_9cQQY-1cvE}}eieaMw}?xxm_U$(heYaiNG13Bh7R&A=WNqQ+$#B)8L6$Z8L zZw~I!PIlu2#>;kR__%YRZCm|qN+}Q2W-ePQS`5%vxkg6XEvdX7P2VYy%6R$NkdN0n zgAlZbTctWY7VO65h+5@##eGTn)z|9K)E=Vzc~!3kFdH0=6LOZEYr(X4iQlZRfImPC zOs)xL)m61~ATt+Lvn*#aRikM$xyLOv8E%y`)jS=6)f}*|#Iczejl-WSMwaCn>mcVO z+-z2uusvhSW_r@>#?kHEM6yynrfZCvIJAup5q4yt68DPbJ%&pU`#kH7A2`_Vzm^B) z?V(^|2tgxY@Ja?d<`pDO39&Em$9P?{9T_+X&8(PgubhHBpA%+MzRFDFPF+)f@)ndC2>9a+aKEltg8YN z?bx=Hi6*vl^1L|bd%kmiPu0Gw_8)ic+O@jZ>R#P_T^APL+**@StU|8(_scr{cHzYn z13)OUn|iC7;J%$R&rys~^9pR~JBm*bsS=9x?My$R(PNPYiQmcrG(A1O; zNJDL&-cLb6K}G-4Tx9`*c$gwIB{ZqzUfaru3O~Fu?kWRuQT%%D^W7!{ux7{gp}Z)l z5+<~Wuc5-jjB}=|!LKPA1P@zkYm>RQdkBI}GwTMf8#N@!9_y>;L(G>v_h0Sx#{hoI z?#Q<&vX&uW$7AzJ?$+d8<@aYYH+~P2$iWJD;o3MN5Y9N5nzCFK zbFLP9+3WQ=6j-+neJ7L=dZS5Q%Mwv|$X&Zpq6l-U%(G7KD~N)3=3{WMA1Cp->+o^Y z=;;c|BRAe#rp2e7B>6y+S|Xqr{$hX5g}CaVma>y<0J0Gy0SoQBcXnm}jWg)++#DZa zi~c-TAz)~!d{$|$sGT@rJ^o_uDrpwTc=ZXktSFX23to@(k$wruwRd9;dKwcA>V50s z7xUZ$lrqce8w^u6Qq{&PluPK6kdH*BD2ZkW12~u@Hys+%3!v|6Q3DgT)`8>84H89g zb`$H(vht>xc|wx;%CF#W`7nK;dcUw9zpD*J+MRL>AA}}rY%yIySbo6$wlTH{@oQxk z8u8=|FkW;JM4Mod%6KJP!rz5EnEKbKEE4&fz+atnmw)V>v;NJa{stiU|0Bo#&CBs0 zCks@#_;fNPuQh?_Bmv~wuxlPn1@&3jUIs}0JfUyO-#Ij;(l?a zdEP8n(Zu7klGDrEEEi{MD^Ff`cRapqv=FW+y5v2UzO7$iK`_E8bsGZkM#Z^&h07krlcnKK}_sZI_O` zEny3q9)s%~xAh%o9Cw`9bi)lg$aKq-M{_EZ#jKh;LNnz)dOJ?Po%^wuAd37TpF0e; z8Y#cY;Cz18kIOsQN!utjmq@{{ZSxB5;q^=R4pt)DlZhaXE(R(qE|d8BTakDQ-jn$r zrbsuDi*0Bu;oYDK_?;x$6qIi9$e~>4zee(QeEr{=KLh+<`W63kfXDosV<;LK{2!`i ztC-%ucy@w5q*n)v7T9a~T#@<}aeB9ZUMXCnDVbh%g<7nno%b0ldeTA&Uv7gv5_AP| zhWbHyEU&EibY*6zMprkxe4)VdfaX1Q$8y@~yXl=i&JVUGB!Yy?H;tozN~3^K4L3B7 zR0>ywlV$&cZnmx=;j^q79^Uiv9J*yW*jzji67qwD=uw1>d?kRoA`L-DmKEkj zttHb6F>`EGR9U_fP%OCVHY za7ctOZ(MADLj;%Y+kG>sWJ-##g7!PU2?w%(F-53HOQxPN*tvW9!i91okavd&UKcV?^z9 z6NNV#k9>*m3UAHYV-7G?I7w#dn`yq23xkrnb0bV3z%>(LYW>#Bh_TysDb@^^4o3VZ zHFXLKiu5Qi%L6afLvvU*H;_)VIpcP3qOoE}l!eJmj*FI&1VaTq>z*tz0LX z2=NWtw9GuB45yjZ8e|~&<4*X=2vk$NWU-nN6#7(xp$ueqLe#?CUK91x&C1i^hFKoy zjFZX9fq6!C87Q)K^GpU(GxdO^NX7Qomv=zQd$&z=)0o2vG#BlkBw3QkjKsREBqu!C zwVWC+;~GedqXWWKi7{5G3&a}ZYRLO_-)haGX;+ROqGb7_-7&|^C)Av99z+OG)=@~) z-_foRtzofa>a&4lLbwnb^V1?0ta0K#!b@Cjhg~`4e!bx|;4G?0EXucmt29T^)75@* z=wR8U+TSzuO(MOMuvDoQv7?8fo9hSw5J_OEHb~K%29~sPKc={ztG^l@8Z0BLbXa5wWZ>Qc2QbyARTG67%t6sn0dp6U)$08=8pj{{?tn&QwnZer^G*f242w z`<(y(3)^99Z)0HO;PBr$ez?$fq7pp zEb4QFC#xE@Nmi@<$}me2m9w8p%JxW=l%zYa$0yImu|3Ycy*@y6!6KQ-@oBkTkGB)S zwtDOQm7(rnBwRIrVDHg~rPwlsF(I-k@pbgDAWFKP=s_sL=~6D@7Q;=rzr^WWeLvY| ztQR-tS%luSy6a?8#RPFHXfyr2LURZ84!l$lYFc<)8errQ8?T7)e-NSQQ=^(C5Obpm zD~xka&uw69i8wuB>D@k4aG=>NK#t}Dx@u6{#W)G-MY&-q_MO_QnJ=F&fRih#I1AV0 zR}ZWJekqQh9gPd=%N{_JP(Zy5!R7Oc?;^sZ$&Z2 z4P6^#AM|Y*ZU&XI@z!~yhE>dmjhcVtN{)XfJ9^`q#kueTl~y&fzX;kcu~mYfy{a0x zrxIjOiDQf2$>}6gF?`93n$tc!)U$}GQ^>+0DJ&xNpMcg5Ty_BvQiuZ55_EA{>K5-p zI~yh~f0Xahqrc_a$E)gQ8HTc6rWwMs`1g^a9ep`hzkv<4xwH=6=cmx>S-1J?czm`> z9qRl%9_Rk?c;x(B1pGJ8rKqKi{=Yf6|4%e@F>^E(F|#!K>y7`0#Qze)%?UK?Kr4O! zt>{Qr2*EKhgn}{`!c+=TNwj0*@6#@qd-Hke9L~|iV<}D@K70uFAjUznK=N#{j9*W%n_CYnuklbk+YgE7sZIJxt zpVDZ!@QpENU<0AFBPI7!fnh-W)MH6@t)h zyOiY=QD>foSOXhIDX*A&U9o$GWwo&W;_K7j@r;(52+!Y^KC?*is3pLeD(?OO9#^Dg z037wC+5S}64fbzZn3O@%+~y-z5ijdnR7?tOT@a4oY3?5g8C45D#B(b?-Y^W^UCv>z zmG4!2fvTK{Uf*0-NEtqtf)qEm;ElQ$LW?K$A; zr1gwlBeSA-rf(o{PPCD{%=F0aU|#M@y5-2D<};){=0m$kf=Ta5*5VT-g4Rh}y7#@%ZI%Z}Da}Xv$m)(Da$A@b z_+}~V^+3}2zMEaz^;hF*!))iIs|tHNZ-_Mh@^e=SG8Wtp;$u(Mpl9AY-JI;DqwxCt z{1BMYfFx+;;)*A*Rjj}naJ1U6A;n&HY^PC~Kf`bEd!W@E-o1}O#6Wa=9#J28Zp)9} zKC;t$wsF|~h-z-FI!4~ylx)Z4Lgs~+DvJab_9Yw={6uQ^(2MO0tE=jb6rHEJg~{kMr;u~z?#n73W@kp>_gxpj&BpErWAgNkl@-$0 z_iYxIiIl-by{ufJZwJexhwb&~NnHCJ?U~zS)?UR3P17w=p4J<9uadA0vEqFfM68 zy_I{a0gp}~+31&|pzmru+0c)QAU@>VQxLC#zTIe-s-W-cJzLO^8X!It+ZPb8{=PnF zm$IPmKYO~NA5}p0wnASg7SPh|A*nTgcxI({H`BE8_7DbOO z12q?i09g;7mmw}*NB5B!n2g;;*2n!=08QYUwnIT&5gaUZ+4q30ZTN@}>=sHJO2*34 zdxQZtV|NW*vIOZcy{7FT!ujYu(g4TcKFGdP_)`kNONGh7q7u*dVk4jIR6xVWpyeVH zVCDi8hy#x;h%J3(Szdj2w%|U9dO%}$AEbkH7+?Lr#P;@nWlRW}29;%el?#}-qzbYL zseu?2z64STEyH|}^gs|>`VR@UfP1UMh41_sr~&oXfWz#I#`ac(!_?Db(u84Pdd>V& zsJ{&26ITP5PKwtDi(Q7j$XsDkVzk^-6RJl8EQI61uF^fyKQdk(s>#j`&P~pZ-jN9U z0%U{pN6`+}3xWnV!r29Z15*MhY-0ne2gy)2!#<0N@R*X1IJaVt>Yj-K&3g+do4+Pd zx)7EaSs7WGEe(%MIB9n>U0VaHha^co!ID_;;xFkVBpyXh6E3CI7;F}Zy$a4aD??K9_LcXDJ!HYs{WV8fuODm1GyCB{<~O?;-O^G>i-O@AkdhVTCa z#Y$)WSdLI?7&J&fj`L?85|yY>8e0Kac6>1K6Ap^l-(GX2G%$F6M@Wg?Eot@gc4S76 z%vvM>YT!+x^69gI4+-@qS; zcp|zPl7D|^0;j7U{Fy|eYEW%|wzw`-KhfEk<;Gl9XVlP85n0bTv6$I3A#1HZTH=on zU^6VxD$L}hn>NOM<}Gfw_Rcdj>>Mg$NeSH=V;Mf8;<0qS5kItQ(U=kxuY%3IiOcHl zy8KYop&uwpf6y>b+1t=HVWdHHz!?vE3NySQt-L`>w>Z?`YmV|~@w$1vhd=QMD$)wA z(f$g5?}@L{C6b=SR1uI^fjl`n9u&%XwP|+|c67^zr?p!|-H0SPF53kS|nKO~J!NRdt z(Vkv;CNlLjcA6$>>^)NSP?SnG#>qOq7h&MRSw23$M5Jhpw{V=iIz6@bD2Q`^uwc9C zmDm73C9Co7IO$?OY?j4yN}H)F9~(#al%OW-zA{iw_@~Lkn9ZdAOijH{`LML7Y0Hou z5dKau^}3tUn6u&%wk~a-Xc86aC+OfxUj%e>VErRvlx<+ux_WbSSaVNVD->eUIT?Q_ zWuw#bY&O8oW*M<+*}HbYka;i`@^Zkt%bwn%++g!9@9ogb_0Rq7RQ~3HN8?_g)^ROC zoV{~i2d!QpMoY2ar@+Gc;2Z%h6#HkYRf8%G7AyP)j^icsO{xa zcxvdl8Ico54`^7P`I^~8EPS~Rqgy zz8d_{3GliEg@oDadd`M#5RluS;gyV`;(Ex?*c29SqLEF{)-Ql*Tyc-!R}t1^z-V1P zs_V`je#X$nI~74+0U@eiQAO+D!cg|oIWN#A5QQV+DtB~h%`-itEiIYHQJDxW^H#{i z9r5+vJk&JCX1)PD{MnSlr1{p4??3_$pdx&NZ&t5!mytZcl25rIXhlDUI`OA zQZm!FoE9eBn7NmeVhV7NETd2N&P?^=%y=J({uV`jofEsYUyfrz7}+m-xwafS#-BNLdC?)1k_?52!>TzSNRUx1}1uE@exqQgdI+Q*E?KV z`_(vmcZbMFXa4>s;;#8zBd;i;R_w7Cr)KOmSf;_=G%O8o4|M$G6Ol)3p(zqF92JX1 zVMy|yKpAxQ*>7ZA^^g7Q;JRFIDR%;XNjgT7hk0Ysgi}9>HZ~ELf z@{iQyB%T+~xZghazaVm{FII@g&Oun>sVfRpzKK?(++`&uil#1QS4c5J7P&NGKmXuL z*yc(&)TEZOq2+*acrL?MFs7*|@STnO*-pXghd2td!;I83;WG%4bBNvX!{=KF)7uko z)X7vkV;TR{PRRRb*mkG)w|}Q${E9HQ1oIB0;Dhaq?meHo^(#TaIPQPzcjs?Psi4XI zf-yaJeAmhZpr}}fayq9Yf55Rt;{P7Hr0M|aOs2;PSke$iI41U1EWm&fht7%fT;TzX zfi-raX==p>^q`&AglUWCHSa1H9Z}Z}EDF`JgH%fhjE@pdKSig!;!=ImY20pc<-3b(Ost*iS7lyR_=xgX^B<5PU zx%ERti7C3WF31PO7Md4`kUG6@e7qdKE$lh53y2YM!yjt$ZMuS-Z8%3%GJuWDuh|{8 zx^1|I!tFu9csZ0ya=nNUGk27zLfvi~(A`XJLNCmg@VJ$}@S?+Kb`FE*2yj$hpX{GvEKH|oo;`XbFOt` z>M|l~8ubKZF`Y={ALaZ0`+fPYsZ_^oW6RPbVvF37Xg6d#cPg#H!@DnY3&;6!J0}&F zG6GhxEGYClqis>@97n>+XNS}h zAcuii$q5LLyo(?uVg-4S?HyH=6C&%K^GxnGMNxrJ`1&&07mOUoxUK#DyH4?KH7WQT zs25+zwu?+((066M#QL%l39(*WCe5nV@4WvS;WS(NLAB zt)7XZKM%2gKwYe=qiMo<`2{8wan~`#UUvSf_#rs>A%??(iMHgC**MDGr{z>X5f6w7GIN!`dWq75l`Tv zn3o0TNWRla(JSuVamb3i#i|%!Jt^-9BsN#Vl-JM_pR#|i7A{)V#yK>!EUH*G+1~>1 zNG#GqE-7@=$?_9a+Gjf#$kOB~K4Mdj0<_53P_DFM78Q%EnFYY{{%V&3d!Zs){%#P7 zOR=re4IH|Duv2LO$NPL&okdmcVZvnRZQX*Z-=?{&J6L_7b6)J6J_P*D-xPX)Vr%Tq`_oLh<%`rRWXs65nqvzBmntReqAD z99F>=A$#sP8or|cmLpc@jyB>$`trr;Q%&T5?#lgrr1Bpd;Y|N7OnAvlG6;UoS>&&l zFU-1kmtm0Ig{0G0gb4AfEYS>jpR=XKuC-3AxBBS>LWBmYy~yw4q9Yg7KN6TsJ`p@^ zZf5IznP)oG9qn!K$A&V3sz;#ySum3=2-Auu{KP9kxz7|d!%NOI98*S@#Y8*bA?ZD^ z5mR2|aikO0Q^D{ah_&dAf-54qK&p>-8Pom)EvP>(*50T*5$im-@6OQD$qV-fWg=@j zTxw)>jtdVrCS3elY+KBPkOK|hE;okzT|F@{ha!5S=nS5rg>u z)r32U4nT4KEGfDDV@c`nRpx)RsJ)S%;lI0d|FLc2CuIPtDw?5C#SocjpIw798!V6IxKuh|j3cu+PdopiSdS-CTRoC%5aQ2x!a& z=fMsxQ+y0lZW|mQub;?X@k?#Z8M*12GCgTvTM!S91kSm6OYsgAJimEM^$rC*A}jBy zApf^x|ETb=0<_c+V!!CT>9qYg;#5la1rxMmN;A!#ya*5yUs#T>>&gbeaoS@I_9Z;TGK2k5WT0-ZjPys5Q5au{2$@-AcRZG(m(u7pcwYj07Xo(*B)WK|6=U-9(?@cjEe(_EO^yz+ zA%J*Rc_jc=vcIxt6m9&Tw&)pYn)+?Y+pXMeIC$-q`t!CE-h{<6sczW64O%MRqaU7+ z1*t<}*P3#CM5N6gpOr_@eDyf9S!>8)-{~a&X;JbaPX`(9a5bfKy{6Ze`@%I3Nreu% zq8yE-e|5q9F+~px!1&Ah2WE6PfkARJEa9I%Bw)}aWmCBnbjWm;a-ztzt z`B+=wsBD94p+o$ERT|vyNNN>~LV;yiAcIfz);tr5pP=_^I9DkmTkY%U?-4fR6!(WN z6E#nlLNMcc)FP@IC>47RwxwG95k}V@TFAbFfy%j}V)h@H`sR^y{A5K*ESW%awz-wa zZ1ziQzAgqM<6?Q`N`Zii`VFW0UqdvNnjo}YtGz_*o91$0SG#5LT*IfGTSdmP` z?N}baZWuz@DwdDE;1c3J9&IrXwR4Lvc|bE6#k&D_QdkfiR8mdF0 zhaMQ~9{KJi1pSco(#kl)0@LoPQg#KN$8-jJRd@%=)ut&PcedwDq|*DJ&x39==J zcO(Hkk1BnLu9Q0zv|^9gI%2~H@_s>d00>X&A47Ek#9eVU(Odv>0x`-RCLkME?200y z(m+ij8*m;ID=3X#vC%nEdWoq*%RV}`J+HK=%(ThuB+X(uNk2hCS#=`4EslwrKDX_6 zzw^d9BiwMPiT8E-wUMmGJriM$7K|*?U9NMBgS<6}$&E*WU9SH@(JthM{Q@Iwn_#NV z7{2|}Z|PV5Bm4Fs9*|~+llFZZ3&jqwQK(%*)n7AT2aT#>EtJbQ{cp)8w9B!K_g!RK zGI_~gSele?asaKnh6LPE?g@0NlnI4c&kD7~$jS!>aKD6s%2I!CA6hsQ^vSpX%n1q-Ew5yY@+v*4EnQmY1!qS}=W*px| zU`htXG{pV!sfo!O<(BJI(I)an1I{K`$nI6xNUtX8>I1`>6gO?zZJgAj1)m{>~x z#?Kv(JW;2*6T}e;#F$4@bt&mlvARvZhs~!Y54_ZZflX^_>b6{~YPJN2kbH@}2qxM; z8aMzKvo(;wz56y)dSRuwWJI@1yx{>e#a#d-frz8K$Yj2_+X!D`KJp{`PYAuKzZ zCadU2V^BhLNWn*+;6}KIex$QVn?s+Y=yYii{qu~S%!6D)_U0|zwAObMo-^ZPD?Gjc zkUkUW{Wu?8+iNT!!$|L?H2%0Lmo(8-%}L;RZaaZJ4cAZL1desa_C4-x*7h5Vko{0d zrhWyi&GcTee8_<)@3kqyjAZMtNN#EC&Y(XGj2@6wEoJC*aLlUEEXdnasWuUwh7p#*J(H&q>B+3wEg%dC?V>q&WD3 zHQY|ZLjJhufHpsAl92kGYbX6rXHoHU%2pof zUB+h3x><7-1})}~%2<<8X&3{NfspI8l{mOGhg5%ig3u_V zJ^_aYk;mcS-isxEMW?&_^AGhZ0*nA-D2@PhfGlYRIg+qCB(Nl6Ix-~&SwtnoLm*-# zftkAB6$RN4ZQMZp2SJ%1s67m;9!4Ogo<#4biX(3ytzc7)Azmt>^aj%gm{+@ZKb=pc zo@NXF+VAI}LucOL?j2+QW~#VzQD?Vyv-TpvNn5>Re8DbVN7^~2yUHqyr>4WIiVZ;5 zv(M(&{Y9R?g4&2w~6T1>+{+Rav1bQ)iTCRE0r`f}8)te-ojf((J@&_U9F zrt@LW-`V1z*1Pco^bBp0g2wUFjj*0wAmFHmHXqy!nkU?ZE2nM2B=h4C9&~JnR^XO# z;X-Abe6S+K`~Ce{Xc5|!=I++ts>)|ABYPBcQ6pwce|2SWesvd)?ZGP=ssFA*9s)WG zk!OO%4Tu0wVqB4Y{QiJVp2;t2esQ+|A>d&mrZmDGr~s2KK##rZbVVUBw0cK|K48mF zSpX?M`JI@H+&YyQ6)W#s9e5TH?T%4KDVpjds7yi$d6w&k(W!EESh1M-fCJo;#M19Q zg)fA~*R|hXNq>RXE3u0T-Jogt3dF*78eJLO=48iXO#eK?6~QVKo_V#CF1a?U0R@zP zGv^-GBFzpXJw`i@c?5Wv-V_K+3-AbPzQ7z8=-|MD>X##m2VZU@{?!d9HD2+s_)K?U z{}Dce_O}T43B~x2vnHwnG|>*xKM>o;PA3scT=ViMOUBN3!CcBQk^nWZ<~=SW+Hi;p7Xu<|JRQ zZ0%in5%6zcNfLDLUV%sJ+&scZZ(Tp)M{nIcf<^ORKcYtS-#o%bcV9o^Mt9#lLPh&r zXG9-LxJ4au@lQdx%51mlei^$O|Dv+v2BNg{6J&n-Q*>g6!dFzpmyiM+JO%}+pAJ-L zeH>EwJ9f}=PQ9MccP5Wp(m(Vzz3L{2iKdRpC2#=@RTOV=u2b2!*#@KE?S zPrHXV@o8wI6Ial z7Q8-w7HYpp3h_A*-_n%~S9>UkBfN!>&vo@`L53KNHc(mGzSi5&?|_wkG?(h^}|ZEl&5g;TsK$@7pjCXw2*#CWxk9nn+k> zSK|`k@~IAe=@FTs>Shtdl_8DB9#=udG~*!+J42u5l`ZQpmClMP>IK~nS2zHxqcJv6Wh2HL+fOkd+;c3;!sOv zTZrUWg*i*Gl$93js}&m~#CI&Zb!D{OcyhUA^LW*|f4%dfKk*v4g_W2Ytz#*yy2EFf zZy)73lxXG>T}m_-b1Bq9AZzA+`Ou|zn*ip?qyT#%BoWpG0uL{eWLXzB;>c7R)YA2} zYBK55qM{U`xjl`h^Q`*sLacIWbgP=e-37f?aLyf+1`x`d$_1lt(6Xva!l=`0^Mm&a zP#nrMV;yA?>=HXmel8WxRI#>0y?2Hdg#tFTlE+RWji?A>MLpQ0t-$=%We!ozGo30!T&+|40&xi#U z8EPiYMtbxlhATtIcor;991?7iIk!+0J5X=9O`JQ_jZN^nzovjAzJpf}JyRd6LXaJ4 z6$KAldQ3`E5f#Pk>8Lknh5u`P|M0$IsV6ZB8T@MePXRH;64GOr2z}<1gmDC_i7eJm zMG5416ESj5***9;nXkXHEb~`{lQ+`}uMoyiW}wjaVt7;7253YqsI6x0zz1?7R$%3s z+=I{+*=H5}q(W};$Mq&OL((g;KU33jEVj!9&AV+dqW1GewM-=pFOFR~Wk-gp zb&(e~h?fdXG43}6#x<(n&MHz15bC0i#|0I*4npB&j-?gL?Ap0nEwP+9FrE9F8y48X zPT{XIa-y1s&CUpx-EdCz3p)|+0T>N`IEIz{stC2XNak#o0}t8A0;z*I*xJ)Ti;@^J z6cop@W^Bw`CCIQBNR(w_z)Lwd^dwLD3nh;&@9aqMRHoSE^wWI^M=_gJ!Y&1VR2G<@ z=!#TMu(K|fvZKw^xT7nU(C6v85EUsXY8q+F(jG|Ok2k`b=E#b~c9oU-u5jSez^~oVW1-P7QVP^I>7S|HDS);TUA>#^e z%S7RJzk*k~$f`MilQJkBXbYhAf^b^bDy>~lLn&fwK^rgwXCKnhEsr<7Fuv$N) z?&0)R>vVOabePp5`W#Cbco_>CfF$97j79zOI&RkBT#@Z?sG4wy>>+ZxrBIJROG!2P z-OvqHc$`JjSpD+%;uW1<>0Ni2-Me_s?eQ5Lfp4jZZi)EzgsqJm*p@1sw^Y{U1SHuK zX5WwiQ`^{h#?y*^2(p~3M;5o|)n&KRd)sF7O3!7~oh&V}Kvwm#j-lmU%J4b&S)A@y zHOI+Z`|`=f{0Qdm>^!s{3MOj}9_Z}Gb|O!D`}1YsrBeW(yzJvTYgM>wo|KsEO$+sQ zVQmy=-z^@|BBk<@plk62W)V{DtFqkck#K`whgFuy3q%4ZSVJbKw_o#GOC>9dCRbp+ z8Ju?-4h6ByI6~VGf{yB(D2Cn(qAiL(nI0MDYpRYS#tjX%w(N7I?x6XPKo6_Cjs4iA zFzKPUl5ppHZGoENcjD1OqD9%=G^R)5Kynv(-mVarWxU{#g6M-mF$%kR4);)@)6!dQ zp(oBH6?ZxuA4eQnKAjI}l}I8rACPhaAy<72^v*j)*RXohYBEBfu;!IlyY}!?n4wVq zS-mH$iRaJHu(VL!c&ytvGUma?{?p+VeR}ImS7qzU#J6Qc()2Kexs=ZU<1 z9CZJF@)9hBn@25X5Xy|c7wqF-jH3@iGF6m-FJCM^1!w=~axvBam1+HN42Ox4qqLcW zgPFC-e`i*mKINL$7}5CX<>rtU6$Xcvq4MEN8Z42ggyXSLWPeL$-uI-VN*NhiCU8PO zDhsZ%HUyhZy;3%$E95eZ6s+TBJ51f#RoPs4zd_bS8AUbYU2twX(Av#&`JsfR-1c1d z0V&COBYGd0{Oh7QN}kpFJEC!lutjjnb+9pTGEwd51?Z$0)@O;Oc~tO53Nqtqfcx5x z#0E>+UNB4*@R-h8c~aGHnp=WFD}y_>=~Ij5RB(6LTOp0iqh9)y)^OiMnona5DP0m` z+gHp;;~9u&1IGF4!353tV+rTfOnbMTtdjceHGao2**O<693S?(y+uhV+T&EPNb%VL z<(?kI_GKTMhtU_jKWb=D$r{?)nj2gZ$y4o4l!DUbPf(EC&&K;@ehM;s(>szsM2!H2 zaatMc76T1AbW8XlLG>!QCZ(q1&wHuhJH;Fdk7qO{ip4SYz{xs8Ckd#b%4c!OO)?rqsa98h9Vfu*CF$YDcUkkZQCr&SVePh?YS zeH45DvInVt|Cc0rm{G}l-e*-g`yZ>ye^+As$Eq^Ze;WmIV={f8YF)r`xvabrtkCp9 zIZXBnBJ!XBWiN<5Z*@`TBB}J_`4Sy6nn+mwp3ZKA4A2>f{?-w`2fOps)R(8XcaWT5 zJ}^44HArW;QykOu;T%)qXaxwi1e#?(@#&M*bAwZ=Ay~oRExalwtS6|3?f3A#dP|+f zi1UDw6i&=dz8mE&Z8~H$DDNH!RSNLHUglwVpoYjaLC|iLEQ&Zf;s9A9Q~Xe^ESQ9x zORMhT$h^UIn)@%~^gYJdZ@5>iBXg86S*&$t1+46c29qh`GGOl=DV%C(M#N~@%!l*g zi3MFaWPP#>r^t@)-&zjn!0)@g#_|MIQT6T`R2+=B$`aOI1p;@# zgN33pg+d^zk>m3Gms>9iMBR17(>5zT3Z2%4wo9bf3y4&!Zzbkly5?T2=Nye2kZoq9Z0v{$QQ)}lV5{)aE_asXtRlbaXIN1TiF!hUO_FM-o-}c8) zgD0%>MU`RXHT%W>SOd}2mrOSeONg9{av-$3Ng}}JDIS31$^*m{E(t6aQiCbRB0&8S zG{wkE-DmF_3=Mzz6B*~Hxt~keG~9(g=j0_*kZg#qe={R5odOnO*mPf(#&$g1{wBx; z231ZxTs>SgBk$xTI1p1P?Di+}YHCwZFT@419-SbRAcY_%kUQuO4tKdKy)g1p8=1ZL zMaUM|NqpP$BD_4#FzNpyv<&(Xp5Wii#M^U;1cVb52)NnNLvE@q*M4k(=J`5#=@xV- zp^MEc2kLZLilPRRP4g@9QYB~(#S>jSY1{2F5?seWTT%^j%jgjT$bjuVaS0XV!}yxM z!w>gi^hgM#0f&a8)B~e8iorg?N@AIpTn67F?ZE-0|1>u?gODuWHbJQh%4U2G+mS_S z%25`8^3)KZNF%ijPA1g^^GTkD*ow8K`_wR!c`Ob1Q}ewK{!$RoE+1oy>M7f&{fGabKPdht< zP)nO0uunKv;?>!XRtazkRr^Mlg~ME=01t)`yO9u{!5;phcN7gNA zzpTK+uT@S=f5Ud=#0V)<46|Uw;*)aMupUr2ATgOy)Ujfme)v7t-~k`9T1~f}1~C%FI~6d^@Fj*qVwn>6FRG)5H66Bu0NW z4(I;1DJF!knrPeqwJt-srUXRRt=X$<<-NuwO+**y?WDkDn_$tKbv`yUms)x&O(;3Z z5Ldc^0rRAs&Ec~7Ne7c?N}Eu6iYOyxNShsa*b4mGrdiMX9Oj%JVQxt9?WJP}noyoZ zeFKx7c@x|@lV@UiW>R%$xCDQ34L*$`8q8MJyQBX8=3Q;L+VP&2;QM4y`oYT0{tTYS z4a>wgxdZr?7tIb@tY`gJ4a+ep9vm(T(Qp^x)vKUaJi-*widw?q^^20?x3J`;@DTzt zCytrBmdH=(paox?4FP_4$|&sCRul+vkHgIOuzitaS}k^W<0$~z6? z+>*U?gmI>U*T|a)4(_a_#iuy3OD4#_wnlcRF+$E(_+O`p)=_6Jk^r_|dzPSbu57Zt z0_86zY7D@;K|+u`8~1Pc6#>2H>7SB~c_42iq6u9ep4i1Cu}67l=|YrOnOr@TUv+G$ z3kJT*C3Z0JBTJ7eQMyND@7OX*y$`ba_W=U6laTL?-D_g>DaR9Y#AA5{)D!tzrBHWt zBM#(q6ovy&u%e4ZoXNc~%jN>u1_`mzJdCSJl*TpcZjqDQjq8*~5Wbi2jbR4&;(EEs zTfTo~f~BfQyHXm}PBz6jC>=?7+0}I|=I@i_>^Ishu6{(W<{zXgJb6lhmLGLA<7_P{ z0Vqtw5-i{qwcc>&Ce)=|BdZ8_S4WeE7U&8^LIPLKb1*%t=3wJDgC*_R3*2zj)qjge zCIx1WTqhacqD02I#mEWA^9TDYN<3@XbBw%;SEXaLyVL43qiz3Wr=Kklk+FC8(xX@( zaFm|7(1@AE6HlyQ&6A7mOz?Rx5iW`77a@^n!wIJ**{m5gv<^09>=X*m@7;e!j{~jF zG-Ygb4lt{_oaU4)&8ofX(%XA)#I~Yi@4QORWSVpFNw=E+lRp2df~)Gs#Is zV~L?f6&DTWKX%*{R7o1~J-C%fjrcs9D_sP$g-=Z0e=rrXTry$JX7 zn^5+KTiUgrZho56p;@+57jbV|C7J21Fe1QwQVp&92MIHCZ zHlnd(%S8L=j2*MADEKJsUIQB`6zCVJk0Q?uU?;s!O zNd;%GCOYma-yzB_NxsuUQCfas;XL|z$_em*%{H{Om}o2O4lsR>F@&Gh3y z?=v%Xhjz{ms`LjtTf7Eu;k9Bx$_k{4USn0YEw`NAWN!`sxhm*v{ewP>riB?~f}k}; zoB1?05W8i4ujsOh6P{+>7~xhTiI&za^uR7$MG|ze?j__vJB@}inQX(0N;2(*xz!Ot zXLdcg?nS{_r>K=-W_@i#avx>Gi%U}c0ez7SaHRVnKjHNoZ*xjKhGqq}mmsZc$Z7$y zp|#x5!v1@7`>J}JuZ-~^tMM70JF77ri-qDiPUOmoHuX|LN?ju@h}L+9%c)R@%00+qP}nHY;u0 zwr$&$wrx~el{eSk`Wq*9Z;^Zf{8UTudfzIFSJQY9YE4V!(1=@{{CQfgObV;wHl@_ zgRpluj@0G_3f+1F&N_)#n{;iLv`zQV+^mLJ-H)Gktn3ojrKC&EG~GQ$f>6JXSDWec zA9XAmX;My2v0P9z(M*6QR%#Lc^jDWrhZb*3_bi5rOy~x5okyK-+_XqWteZQ?d(h^B z`h&z#6&c=)3zEGA35e{l!%@8qY)z>84y|p*e-Cf-<4?tcYJdb*l&VbZ=w4C2?HCC* zVEorg0g~!P5MwA4fGzFTBWNJG{@%~|)2wDrGm6>>)=(zc)|`M&D8FJYbDP74(eNwb z#F?l0F38ihldwUYXa2dsm{9#V*C=4?!kOL+;)h{_OXRNj_nj0!R7FZv0@#B?pG`}` zu)(0_)rivFX%Tv;+G&|DV3?I`S>sk*Ya%c&L{RJ*4fc6fs9G+VW(V*bnq#7*Vi)y1 zE{i>!_#CO#57+e?W?vy%iv-c_Fx2d(XwhTDnt4XyV~AD$L=M!@$tIe*?7}p@p0a3U zKS^*!i!1W;>w?26>0h5SEpHlaNZbCLhiye*P7af}GSdofeg{CUp zP_<;Nb`hbn=vBLnG#Uu2;s;m7W+0Tejvd$I6Ewxe4QmH!{#cEX$(b>g6gWn=TG>|x z&UG!U04kaU>>=kMBBdH7*uz;`7u%dn{*s*22Dc&O~*K1A?tZ9TQ7V1)BQ$%SKG;dLD0oF_zv z9nklfYQva`9OJ|19+1_|J#fI*DKVSZvAIF)$ODkdMf04TyP~o3>6Z{d#$(1TgN)7P zl+=ICj(<{*-C}3SG5-Eac$_`A5X>S#3TFM#mtE&PRU^^^uG|0m>*l8yWUsvx`p1uB z{C_(4{C#{)#opN9TeADVTM`Dw#)>9RcGiEHOaB+wI$8PkUy6XBkWeNZRfGkOXA#BN zRPKJr;eZKo%F2+WAM9JO^v3Br&-~|lBq2m|!k^Pad=oDrDfY^N)(v(wZr(SUY&R=6 zbUQzy3_Ac~p}FblO?UJCM?rJY(b?iy=uLJth@iFYgo$b_9E2h>B^L8|!Zn>q4dtEhU3A}Ef;`p(BBr?WqPJ< zQY&5FT?~}i6f(T(Ca3HxgL4>onld!MlYtowuWxM=~GCZ$qp0h@MFnkR2%-3C)a9%Zds zv{Cv1574^n8rQyv;D%q4Ahwa8UxKEdOZSCPj3!HpE(m`rnUXA*+ESWfxa?7>FNtv)d1KE$$*xMM6zN zo4z_9j-ezY(4k_Sx7~D-gj^?LYDDQgN0XtTY)s;ay~IBtYf#-8$X4C0Mpr3>>_sZ3 z?d3biD80aaGEpGC`@7>!^1}})b3Au}6T2OaGVbx1L3k*D7y5C{c~@aar^35_1Tu!0 zcBFD3aQZ)#bB9+Qc+H1DTCmbgm#7?1HW0>M-xOtGtj{pb#PRl;XlQ#_(x{@|d1tAa zN2^rH@sWC7M=!#kGL{JxISb{nCeF)!nHQPz8_}F?ilA9+6{;|~Abi&A&}?iEh`Jq} za^Nh@rOiP9;dgX+T*A!=ebS%bVf6N^tV|3?=Hk~f+y9})TX>e}8t9vV38;=E0p&)y zOkUPa)i^wLvFDkJ~9G*$y5J_yCJj6y)uZ8GE?2t*^ zLlVA0;t)#QLle$L;*d&woifCe(58m?EV1t^ahW^>l;AFjC?~ZaN1PKs)R^Edi@1|I z1V`+kj`&MzA4K9q9+6IRUo6300+CK?A4TFr9kEt&pGBgJHau5+zcc}c7%S0Qy8&X{ zG|}31JcVqHc$`$K#e2L&yajN)M6yM2oK>(PI>}ljp<2CxgIFuvm@3)2ka)v_T zI@vnI7>jIeM5+ZM!KT1ClX&g>&y3@&A`LRrp~;s_z`E&hWQ}I`{nN0Afw7M#gRIvR zq3~YmyQ8o6uwI0CGbVD!gKVoxul599oOrY*J4CBux7__uz`aB-RDgIw^fCG@{UhFO znUC<;gzh1`V8EdezT&q9fKsb4-+h7?I>1&WcJK)l_7I)gzWOdv|ASQ-;Gd#-=XYHy z{>5Dxc_RKMi~KyDF}t~t)MmMax1u1b@HQatD1hVD#&|@x+<+4PNMS7zpF;ki30C^G zM7J!ESwp4dbs3)^{;3qRW4@7nmEh;#suZ(BPx!Zm5Lv)5kjzR{R;#uk%MDz}<_0^Y zFXez2t2)4T@E=*bG{8O5{xFxaAeyhvAhLT*$oBPg;61{-ufP?6SZh4s-Z6lgWv?>c z3J@z!nST#%c>QJHIQC~d)#tqJS6Eo%N7t770?;{JB^J4hc8{*%B)5T-~|MCy5cSCErjnV zm&zZn06zE+6~Bou(c)gD4=r)ZuPn(gf)~1gI1r*;;DA0+eKipPNFo3O;4eiKB8j%J zyfkyz0K`khq~AZw4XIUF0`70f54AK#_iJ~-rcn3s6_~u1$o_nMA zN})75G!J3_K~tGJxT;9|B~{~*) z{2NPoPVlpaNc~r*O1;0Zrbi~FY<@zdoyxGNZ!;yU!lSZp3MH%38FK&)bJW7Psx7Xv z2-qB_dy;-9C1?1C2_WqBq@Kbs#-`Z!KEu0cPUh3~UQT(BYzwOF4dxGW{{k@kE!1mp zMO&C=d)iJ(gp@gTs^TN#d|TEHL7NVW+HIw>1IbTCl%4!DK5B)QW?_b^^ z*zkRYQ^F<~7<~|^Jly(JxCdsuVf1G*SLEK4Dc_;m65O$#{R!B`xmFwQ?2<$2B_8qW z(nIMjVELKbhIf0Ra!0_Yh0=SxQohgjp`!AUg8rS>cw)=<;Xd*x|A-U4Abq8L%b^1V z)GFVZ`BqZ?8OT){*0WIldC;kR=Z32=JgWQ|B>D&!AZKCmHU9mb-5(!7n-#EbrQUY^gaWI05q+1H_+1tc`wn6W9W7A7}s z_WbBD=d#H)trb530uN%}yrG4Mr$B+^jOCRXoE|Z*udX@m@@$J&O6g4LNrRh64IMGU(uU3d$E>>zp6NqFZM7w%AmbU53(Ml1p zwgifzWeTeyKmt6;_m>98Fr%yZ%Nl2JoV_|8IB%YGTwpfb2#jP|2a&+fT(J$2Rw-b?qL`7HLDs-g6L_q zve$qUNJ+(W{C@@4KD*@*F-cE_HrR7U=4|PsrzCnUo$SJi5Ax z5nZ!pHJa=pCaq)GMlLpk0LFI|_v;Tj7Fnh%=dK*IjHp}Qb(wp0>)hPFxip}@eLo*I zMsyVhCnZ|Du{^TP{D9NIbWE5PWOFjOCY3DlPh_>CPSQD3li3EZiQgv{VygWismDo- zo4=55y$bTm5QIm5IEuu`E3llGMyVi{>n+wvGv6xtWrp!vmW z)8hkkI)f8Kh&dLQV`l9lh>A39@N+YG>*PCD)ZirSHJSCLA5_9<%WMsNh+3YIcc zm3UWZwQx?QLF-L~RH4qsmZJI1Q4^x8&?Sik^qLkf;1+Pt`(6UL#weLPKCfksW(F-s zAyM-?LQJ+*AH}-1Jav@t=Yt=9OaGG5_{w^a{uRC8B-)`J$q1aQ z$&B6nfSVao#+h_5$BwGis~ae!UaW08)eWyBljI4#;g_z_l*$R~+ z;D)!MIYa9jW?jh+u`L<#qvG7)p&Y$yiE6hX76xTC zSQVG{?Y)>c!pXov+T+u)=)7`=~u+Sj;qZG(ZKTEZ|iO;WGO z*gP0-uVelL1(qOppMm15EyRP7;#9s%ew)w`K#zOw2sMCC$W_9n;U@RYdpzR&@xrU*b@W7Xg z>YX%`go#V5#8p3iGnOGTvmrs4aGvmGOe zMJShwx)4|?C$jfqDu>`vSZ7---IoiTOmifc8d0;1;45Z$n=MI!RKve&P2=l zCM%9z(5;bNBB>6gBQ6=}Ff2X*AZjaSNixv#d zY0O%f*is-<6&uOR?-YBrgjkJhns2=EcgD0-!#RW9y`eHP<&(OxAxbgIf+up0tShx4 z%x{MGH=*Rq<13qrSX?4SZWL_CZR;N2w*+sB!xU;|C~^tvt_z#Q^J^^LlDs7sK+6Ff zs6m_oJH;}@RU;y50I6bSXfD&(ROtCr+3$O1HFK2hFNuP6Du(hovfxI@%w4ncG{r1u z^@mNNB`ci$sy{90=;X@GdGQ&Hr|RHz+%^I8NNlWSMXOtU8Cm!}1XvE5j@wynp}rKv zZ*iT=7W1W-3y-c0dy*RIA=V*MwVIx#`rDync0EZ3pjY;nwNk9JYTRs6e^5pjZ$f$! zVh+8#@cfAdwX))m=nfX?p>Z+r1c&A*JkFw#>JnH&Z2d5;BD-$~RK54HpzV%)eNS=_^t9z^lEAmnk=P8{6=CdcCi zE5l59aD0z_vtOS(RtEk)E-c$9sxJTzjD9|wBYK~e`NT1*0{485Ik9?LN;24HTfCJ^ z9T(c8HT7UkpRBfqfYTR=@-#6gDG6Z>mlGjHlMfOp^IXPXJn_!TzDEH@taFYRYo3yQP4*qMk);kF+Qc!Dm*^Qa|e zZU;A?2Kq-mi350!+%n zvLsL{S)K6;-60xtNtTG2N;5Mtb&voG6If(Rv%E;$3?O$QE9t;lxMg06oL?|imm|u! zK+c`#_ywCm+ew8z*Li@{@`nPV!=)w{an{{P@yjFpXXSR!1kUw96=b6vckz=+e zYk$3+IlsH5t7mWiiXUKtRNlymIBOh)*Ma^&!N+k#5bw<1W_9n-r#>+y z$ODbOS{IZ1=f7Tge%!qFmfXtKaxDcz!;PfqB8l$u^hIt%xpnlX zSfg+_RH_q!r3RLohV;;#yZi^XkzlypY;#L_%ZaXFl5x{jmFx;~GY-N5rqK-L$S{eI zS0g?+Cuw&r;f)LwA-S3qdIA%XKVW$t<6|{t%TvhCk8QY{ZC4Se?UuEofF)!+#&QyM zhu;l7o(6fZ1F}86B736WE=Gt@$ha7u-}y{`S9WWii8Aiq0JZfux<62wd~o$7;Z8IZ z&1%d>4vx!$qFAv0_B#`LU-R#e62MiOh>vh(jq5)o>Y6^pR2#XN-i@B!kg173`8Wwe z2Lwg7hM8Mx-&#dJ@{a-Kvzf5pA1_)YH{EKL2^H5Ck6@4-jg?+7J(Tk_PWLz(se+`^ zu&XK`5}rJOCOp1`!2y-Cqs!%U+oO&(>&fIMb+DQ2ISY2$euzjgz8_^~{NzvV6*h`d zj(0c4wt*jvuR=?RA5*`ARq_>7^LC!%E&%3~Q9Z!!NH)4Q;VzNfdx1%iv;uk<5)Nd( z3Am11@phuYSKRNS>c^BV=QtEGO8%A8Y3?f)N|Cr2HwfY2#U6Hs6K}u89dFu!E9&jc zlSBw$R2~kWhh?lwtj}454KcJ}iU2WRX}7q8y4p?~*VY0KeOP7&TH*bsU&KV(uyqxA zKFv;O63RA@^8BL($>>KnK#Gzd69POFK5wf3==O}FOzS3M#A*4 zhRsg$TNx|m3E%LDlWm=voOZ$#n;f%vLXbieclDWZtTNR$(Kqk(EL_TS9Dy-Ec-S10*)p=3>Z%a*6AzjNX((uL)hWEVW`%6_&*GadLJPN! zVA-4(a_&UuwbHYeBu!uF#*_M{&1{|lelm-1aFq8ZmDsQ(&KVv{Xh0s<)~SANcWC$b zX_<99k)Da9^Qr6#BuPk42z?8}J^sk^3;eSPEd~3wLHObLCPi}K1^PAc@_kr`_J#P0 z>UmJ}8F?1Em-=ZFx?}QCYV$%P5)I|fMgK>uYRL}<-B;{!5K-+2fF0c^|MONnishk? z={>IhWi-qqWEPuzMBp^#T=|56eRKi1r!rFJxfs+6b55`(H(up#wSK>!d`W4U|*3QkD4=A8k0B?qifxN3lm zrHE$z@vNlh+Jo4;L1kr0=(M}1dT@|~SRhV4?sn+WEkzZ-6rEAq0dy&N+Q>1PA23MS zEpE*uz(%{*s#S8$(ytM!U#gtPPXk)rRSkGG&X091=$UJ9Y6jy-Mkwa1X8IepX`)VU ziyV~LVqBIE*BQ>>EjWHR*a1jhYZP7vzkS{L6SEpD*w*Scm}OWmS+{3(cGFJOPt#&M zx^JC?O5;ijC60$$^h3WjIPDcQeJ-1tI!L!C6VS2R(MS#GSY$KT!lzFUogP5X4-Y0R z&2kpD;ZpYyAM0q??MJK7v8vaiNn4-moiHv#yhnp{mp`1d*5TMN_O;$uMX>LWRZ(h57)SM^6n(JP|rUh>hYnj*No){f5uXSFP-*IvQ) z`gG-yYUCfW6TF^Pc%GTB8wH<8y~y^kW#j|g0k4}e$42cSe8mpjpqcw{*%7Wo67fSG z$!_0it4z@Zi8@g-R()otp87GDzq{LHyTDI~Swq|W9KnCo=hrVw1Ny`P>WU_o%xP7N zjTBtG+fxpjaEFI0YzLYt&8s<}gT3;J9c&d&^_`!OUyY3~0I1C+QWaZ3)$(N69#A zhWsGwwi)LWc@LA11ML)m&6PEFyRt?H{Gy%M-|BsV~kY2<(1PFErjJT z;i&A94Gy`%c}CoehTJ_74>bIUBzJygR^M5-j5J~#SkwF*KlehJh&!Wt+e3t$^Wmu~ zPQ_V*Q){EFoUOSo`Ml-1^fr=-V!;jAcs5CEhv~(g&BM|9B;2m}^th}=?WHL@i|DVd zrzpKl%JTTEn=xTOh4{3`)%5f?l8mBREaG*=5PH@@H;TkAp219Q)Eun-lpDpK1Nu4q zV-w15WEY917jHT3P3n2@6P7e5=!kV0=x0R#4l@MXJd^#XEvu*!Y(EZq?mcvBEW64k z)2pgYXufY&31`eaq~YZhZVPRsR_taugl*3x#{@nd(;UYLM+Y8jThrWnn(oVDyxiA? z|4;U2UvmeGb>GzusNPe5zRs+Jsrr#!y7e}6@2ur<$`f#=rzmxrTl2O4r-7k1 z7j})&dB*7j>P@)k;WeZ1=+G&_&}d1@WN4IhNg|nO^sYLz?ZsKv)+^LtANX`w<;adL(~`@-Q#pcyH1ZHL%b*hkb? zp{8nZA9Ea{R7)e~g;@vL2j&8-1b0xWP60-q&^?w*q(i$>+@5UvjC&#TBaee;ruCu8 z4}CwI?uhU7Qk@i`d3RtgSY$E}v-YVMN``iKLiHW!D$hf;v^6#0F0PGEi|-LUu?*;cQIGUWQslw{XhO zat!av39R6|&)TOmRw<+~%1|+7oj{v;@M(u~YI?w3IO7)Y**~>46?&@Uk62e{n092c z)~9A)X4S|nk382mj$ww#?XFUX=P*6`Ly6+4wQ4sZmTRvKg>(3&N) zGL2kTuR~W0Rz@(O8hKRTE$UaKV{TqISQvS&*05_3I3j`cpm&MSRWzgVZbGFR=9nYu z)f5%&qCo043BBFk2IE}?m&1ho&)ON#Zx0MNewN!9c7f9EOM)>lcz>*Jia#Uco2l=8BKVo#z`H!EAvh`qK z5Nz9sulhwrMa`VGEOc~Qwtn6S{b#(={AG*+!;6G>;if*Zv93{}Ir^KOv&u63Q53H$ zZzyHMuN^!Y=oN!3q32xunP-a5hH z3g5cYak=PCGufZm+UNIW)0TF}>i7e`v`@h2%_o!YIc{pfntdb3(SXNkUrp9YZ*Q2E zQIah0xjnf&x{6$QNk3tt3Bls~%a#`ctG6^(Co&b;Y}?m;lOo+Lz&Hy6Ek68KnjI^< z9Ut@;huDzN4Y*?g;)QS@{dUYE1FczoeX-imDtk3MiDahM=x%ircB}=E1a+)b)KbK= z1$!CUJkuQ}6%yYB1Ml@3QyBoa&~`!X;364Nf-4_qcz(8>IhxV3)cR|TLwD6uocnxv z8sdaO+0YbhqK>-_oHXh7!VKWI47_>jZoc$7NQ^{G`Wp(_ zI};^$b`2OGwu=gXyT>)3uDCO0GENM^JN-jMx(QEu;x?|m{ zNB)qh<{yzNUrEJcTf>Z98Z%F8+Sj2TdiVWI5`VjoQq`oYgdOOgz;$nqxE8i=Ls7ao z(EhZCE?&v>+j?3i3wdfy_SQE@4%D2<>W+BEKZ5Tcm z>k;d3rV}k1sswgQzpdqBYIp2o3a~GjwM7_pAa)>8kDi7uCa3bj_67FWkdrz!2y4o> zK1-p(|NkMU{||D<|4clw|3}S5*+yxBAKn*sy#zuaFG3SUFV7NyAm9c>59(zZ6?y24 zj4T=qc%dkf6#9kg1I8 zTG+k?G958@;;puzsGzE#>_AeG1q2;QR{}kef$D%2gcBn%u`ziY^=?Fzs&>L$$UWph zTONKuB&0C_q$hHj;7Wup$Y>n$ELoo3@Q>@twqOB(3_mN%>-~xdFmxn)xnZKH3U_LK zY$8fh2c2Q0P5m%p?41KzA-e`TC01vt73k51w!QS2>h$4)n=_C|D~!g=XhvqT_hWQg zGA60Fh8m5I?mQWklrhgHyWWWW#&`mdwt4=Y)dsS34GJ{w>RtAn6xT}+r+zn=_|2y4 zly)q1l8b2J5{+(UJi8Gm87~5Q8NBJH3I-gT`KFgsTBb(d@qHrYmG3qew&-b3-yFjr zqs>C6TsB6fjiAK|j#(I6nJ>A$nK|CceTSXLnTTx{-TN@3kk&|(#|#~68y6L=aTv?F zycJDgmeD=%_{lk>m-OvU@xbl}!KZ#)gVI4Q6A}cd$;h2fAQUV{#x^Vzx&in_aYD0X zCBeK&Ty%Gn(7nGy<7-S%S$h{}+bOa{{xVz8{3aF2iPmzH(fUIKnJTm{V!OPx6rFu! zT6s$95|5F?fD4Z28I21H5|xTYNN0&{I$nh`1&hh!`5P_*Kq&KLZt%rMj>h47q2QX6 z%V6fiRbP(zDQB2*$Tbi}$+PdS2Vwq3(c{$+PDqT;O%kaJ)-K_IMZgWqTg{<~) zf%#3+auk9vym!jFPm+1`3Y@Fic*%>CKmNJqziJBaEqT$b46#mUN(Xvx9iSw6adpL{ z`|cq3UzNj(a>0uU9oBKkTD>JB>RGZ@`_^Q#CQA?X#z1@4PQi%P%ZSBv|m*p`AP_x&>Isc zO@&@-xrPeq__aXZWp!bWA8IiLrDJx-Xf5 z*hvSVYNcz7(`M+j8 zzv-uv2H#_z{~;f`C@TDA)c2>QS;tcwfPyj){M?@go~|oCDGfatOsO3Bak#YGNGiqp zIis5g;3d4TY7G9&)1K-V3`<{n0M?-Y%6Ry4o&O} zY%CUO{dFM-Cftd(>-+!;IF+e4b^gIo#OoJ#KMp(q+gI~_Qla)W&&j&=OYF#JLS>1z z%BG@&BAeFj=zN=yW`!KoWAaPUQAXFRWs@+2;#at5$Lg!dq&q_k>L&05VXFug!vWu8 z;UbflAx(#QoFd`u)$ng#DvxQ|brkJN4ryqDK}TWK=Hy#^Q>_!`py7$pH+NJL5v`|z z(Vf6LpDs`G!{m7_bLQSJU(_x(WE!I@khsx-7j&Pk=7m&91RC@$sQ*g~MP-iC<(0)G zt*OE7&LYH} zG!1h_L&CHpcmhzFFEwmXdv@buxFIjO(7<9LaAEO^q+muZM_5liwtp zpD=$F%7$H;+{y3d@$`?&1LtqcL&@UXQqDy2TjS8g@jrqvE(-mBQBt#-n~IzH0T3!3 z59Be4?x^nqJn2=C6mD$Rgk@p)pf+nxM?2ctj&G>q^d4FK% z6&h*{$3*DBAUZpsbD}fjSgEg+FF~i3VVWBZ_MuXOQZr+$yZ-8q9H6*?JFvhM`-t!| z%E4w7cfCo;=LQFdX(R$Qkw-2z;68?CNnNHjMam^x$-$c|Y8ISu#8c$Ecz4Kp6x%!e z19pyBlB0*a(?+HfGNNbB2h%lXG%I#T4e??@{t%77r>jaoCa;LxtoO=f5Yhj^M)3Ls z373>3Ea4x}Ak3-EBFyTnWLcIkX`bC-Is>@qCJvQ!;-N%lS&Z*|ii*7E5%M7p(j6{K z@_k!pN89{VhNQHh!S9eG@2H#K!=FRfIy4HvI?*I??DR8p=PwKvV0flEzA?D{e*M3h zsDEdO|KDD%#LWM*PhtLd3iVO^w=LW?I#s0zD5R{hP|?1wzDrmxKIz0{X@f-wsdrbk ziSGOst&UZn9to&ULHL^l($A#3O&ignX;zMt&HJ~k#Ba;i*jxbRJANSrAqgRk!HN(T zn3Rxr%nHoq1@S#kOr$g^qG$^oXeQ2twh|+44T>quHh=|wTn&|>pAV9D z;DbptMQV{+F=)8~^i6N+A}ZZe2JC^9ljCO(R;!lV1P|kv?A*_{3X2&qI%%W(H&0q7q~8p;qqs zyFKm;dSZC&Kua$5U*JX<;VEW3uAB;ETjsI16*WscXODFjHB8hWb`}Bl9ws4|WmNRk zv5=3C3euzq>@wQKo)or}w20-SW2r(Z+>s9%!~+$v*%av>Azp19;5V_hoE>V)ZpFy2JzqXT&&AP{fq z&#lF;AkTnpzy3u*$BwHB`OCA_|GO&xzgHH?-=h03m8bt1^mlUluFLu39 zysl-gSDr`7{0Lwx&Dn3UFS$-R+3gjA7Wi>k1Y1{fY1fdTh?QddW!eto+wqaTKEVWC zzISCo?ZnEWokC_=#M)R0l?xz93BMqUE8S-xuFB;}L_RR5WU&9~;0Az$@JRtU6W%eSzTNs6OcH(qTY#I4{u8xx&?#_cx2 zFxRL$R9-RsY#q6yt@nP4nK;f}jEM$m`uY8w!N4`c?nFtxqdDOpBl_<}SK7kKS;oNr zf1dq!;r;Ii|1J1{B!JAbfErSxYBbhe0JHv zwW0nM-rJ`pL63+R_3PWIOlJcu(KVsn?K+*yY-Xyv_v_;YA}<_t6ez|8Hr6Q6D6w!v zfvzyQaxyMFk)L0ctEV%W1PF9mNlfu6 z?5mUHFYcA*8@hq#7}u)`_cB5!POa9J4ngrXheKcbngU{phBW=|Sxo1=j2nwBw+qG- zQH{!(uHy{x$YG8@Gak91ykIGpVqeP15o*j(%i=5zW3*Kmvd=7le3ar{6|p_hH3DH1 z)sic7`#pU$@t@{?Bcn3MN@8QAL(mBs0e`*XJoJk_*Wo#B()bT zf2Sv`e+;C*D>MIZvXYaLfwh6KiM@%fv5Bpbhw3+H$->U|zd|cmX+suC0a*uZT~;fE zt4xKqfT#sOO5{#4EIfdiv{;xb?ah^wXIZF($%S_44ayfluaD|h9tkh1$9%u?%V^Uq z!aQ!5yUIuF-O=^Q`|I@%$&WjQd2`-gYp~#3kNIC}%snB3kPKN;1pfCH^v3h210o=) z$Wg@2{6lx=xN*h-wx(&uR-K`|J{D2L8dlnX;BIBdx&YIS}Ppj&m?8!nt< z^a;}_t*@|bHrI>Oj=K`2JwvnE(>4s!W?vCD%B{gD)JK0@7^rh3A82fy5;50|rB)bB z3K2lNETJ07eUjNcHMgZg4jW9Q#6aQD5hukYt%s{2Li!7;JU6YXJDX=wNw)FzzAf1b zkXb7RylN(xq_da0h;0Hh&h8q=zkXDCdOinDNXLCovC&Z_qr49;B{C zcp5-t&wcbao-T=O?T?yij-?`YcR_1mIW`j+YZ`qwkaCvLLfLGo!DQ_;J1(pYuel=F zOkJ}O#zJXumA<+2fui}>^FTK^YCpi4C&H8uJ9oE7occaF4Q>Pe)?!3)-gy4 zH}t{Si75@iujBh&0X*itieJX$!M0M8$+=*N89P8*1hK$SLq?2NbO43wlne*g>8x_P#(dMlCT$XHkXruGI#m2U+F>_nJ(CqLCRQ5GqM7 zsw)~O9BE=wMp$aooHc;zAQgX*|icd99pwoCgeVTuSY%FsH z4*q2{Sc_v>A@o;H{>R)y@Hc4wckW^QPe(&Riw;Nt`44DwRYk{+odgk$;b5jRG2z{z zE-|iItB`03)AVK4uG*cj_)sGr$qeOsn8(XZ){VClt zejORN^3V%P;;MKN%^Y&+Bc+=mNle>5G=^?%rB;C!mf6@>^+a4_fLf*!J+yZ*IOTw3 z&7n@=vUOlnxN8GxKsCK)#zv><$QTj_PV6JGND+tUNIJ`NN%5_TfC67~XiBeoOvJudniw{yV?kXXK- z05TN1Sh*iY5(X#b5>rU=gn*ECI6i6zE_N^sMtk(RU4+3)XWent{A2Vv$DsJpN2l*- znmVKm$X2F8Rg(Rm z*tb(ZgWjk!1#i~rE7UoOK6KCR-wpmChreLe|3_G1zhRY@5E7A9644W|H!w0c+0xW< z+89CceXd<~EK#xmEy-rKz&Z)9KMbiBvJh^!B~1(^sl#sttsA4okNAA#N$vJ8GIo1Y zl__D7xLj*I`JH`}H4Jsv-YY^knT>ekQqEbb%YA>ym*leUynpd}A^0KVTG#}3_js|g z!`*Jq2K||-R{Qk+sXfo?WYk)VxjoVC=?`^Nd*iDaIeaeJ)vcjJbCb$e+3fD-M8;cH z3+3SE+-RL~Ei9ktPWj zL!Shgi58i$TKK#(N-bh@y`ggjyz1k!vQfAmnl3W0mIa%7H^!Y8L3WwFluG!tK$Xh4 zU=QHya@0oli)yBnbR97v}u?+$xej5{_8i|K+M59nJ;;ZC15(x-RKa5=sx{*F0U4d$B zI~Y)$Dho)0YDhdx#g4v(-Bu&D{)YLc>a#B5ue{X@gHcEK!FG2u>a)4n7B+Z>@rt>< zelaFG6HzgTeL=n7ij8V5Kb|#i5h?Sz{CfnHAc^RCd3wB|v)}?cHW&mG zIM@ZP<9z$EX`Bp&C~zq1q93FfNcDZE$&)zwll4GayHE2YHCyLcH}q?k6s`kPAqB<5 z+a2H>p=k*g1)y8&3LAub7=sL;{fKyRxlRj$0{m4B@Q zSOLR()A|oczI{coJ*r}zhqyKa zPhP5nv$S}ou8|;y;tZY>RQwM!zgjY|nK>xgx4zRWdDs)%N=lkx`_p@)pUh7S5agG8 zQuAr{;6yaVe%AvGhJbZvAN86V0CC*nTVpavzxf7>0q2l`>XLYqjE92#Z1q9i*$1$+ zAaR$CRXL#5wZ;SmwCMH$Ve@cJHU?+v;PY?>ve5PuBYV{9?um$%jGj`*A=M_%!Rqz{ z5zXP*xMF^69(2KcoOKho2*`UA;c(U#Dp%RVMfrx|j!5Kef)Y+x89!G>CYkS&RG#>MSi2H8*kP=sDwJiY=jDG%C7)x$@dz4E(CXU_b_ZLw zCZ>LBDsX4I=f`)K*T&e$lw(!b;8CgI_gA>vkKge(XXx)F?uyee*O{GvuB-i=>8ZdM zS9*WOtV!ytu6R`S{+XAw`GvU|{0?U6!lv`%w$|TmPZY#x#@Wv-|L_Qt$5LNNM*2!`G~1JfzNboylG@?T`cM^`EwcycgT}(>zvG z_OdiDT=uz&{WPPT{^9;zak{Cmx7M%s6^nY4Wc2Dvhq{itN_xj0v4*{*GJ()WaW$rw zt7g6Y!|-|iZPiDd``sFeY%!8C+fqdH&6~chQTsf@g$b)Qa5%ApG5$pA(vV%+oIAGq zVxkAO%&^@l_Ed$NN!zynbefy=0XECqD_O7KD-9@msV=rG_w>%E9gcFMg?C>4_6uEB zB*{{9Ml|R`?8gwRwU*D;tkXTz*d!z7yW!wZ!5*IHZVzNPaL*Xtnr`T1U7I2p#}Ksr z!zqOm4N?Io54?6yJ1NqC8GsD2oD+NU9w%LIMc28@aFbeCMSN*y=`mhR*@TUY0m*w%`UDK z{@+bJ_KYeP6$R-+GWI^kQ3Y|v3p(~IMZ`D;h1}*d`W)|^uE=$nwUq6S;F&qiMb@W< zSWB~#wKA*|IxmLEak4y1v1JalVIrvAIRg9lS7+fD_${4TZ97-uNX#FNVoB>4UW#qj z^Vg)Wl~6ThJ>w+XPxKe*PAVy7`WVUk{$u=&w|i2!@arB5K6(?)&~maZk$J6V->#Pu z!?|-E?4p+SNR=K}73pvB-1N0o+7|!uNYR$Ayp#&AZT)&-LW^2fT=Ux$S0dCb!@KT> zg0MmHfp_(|%NHCOq@20KNsr6y!!ROZ;)*Yp5%$l_zs+X;sQ62h%Y1n!)(HFbbQabo ziDNlBqN(%iyGd+?KlkkyJ>s3Fch50_S;0YATRMA>T0KW$V6T~#dEY~yM+>((eZLj( zyYbP5NB4)me(Gb}{W)mE7FMGgKwW@=-tk;&@7f8kmKcFYt6E9FMr>D+3tsH zE=bCl9r)(uu6=@A^^0CNbH%==OL$4I%Dg|Vx-(lc$l+2#7-eq1I#rtrt zUX|wu&pb1StKK0$S2BGwOY4YOA*RdXvgp8_-B&I7zc`c@R}^O8S|gwBt^G0E`?P~N zj+L9a8}n?<{6;6G-n8xAIUHT2M&$*?HayvlgJNOW#0|&2>VI3ljam0@+1uE8l1t82 z+Z4S2*&=y&xrNuaV!!H>Wn0#L4Evf`cXZaHoAISmysVc?8BE@K=SQy;=Ih7c zpKEA=5$O4|O)FJp7u(zfA-9AJHHHrscW3QqdfK~e_^Gn>%XJ5q*uGz!c)7XgAj|Eh zRyh(QYj0PzV`cgQmS#lpfh+S*M!-afdxf0Y#H zmHL7RvzR?EPMYzjoEG)2F+8S{BJGj&v~X_7Oy2FA&F`n&Vrg|b54ge`0^MZ&zCZ+k;T^Phy$ecK3JLa2z3|VKD z%ZBT^-^qO8TxoD(fqm-9y7>pUYKYuDdFJBgbHOc}eqP_IGlOsT%yupTwRtYW`t>og zVWA7+5`GB!T5Kp=lHeCa5D;d!y16&%c2?;h=H&xIo!bo>XYVpNdUWZdldp|G1*8@o z*80Md8BCPri+<4Fr|!$(H2alp*^HZV?hj)2Jl`|mS6$Gmv4OpQFXr;YLtX8ww0v{F zG+!1FkJ+-#g+qlU|6~q>M$y9vkK_!utv5{Xn`dw1ufeYT;bPY}Go&K#PEc~T&aOAL zNhGD9vy5pv>rPh?SMch{Kduc??y2e+O4ShWsi>-W`O-b))(zL>oa?{W4tM4{pN8m2uHRgV?tFbQ+j=`v9(x9!dQ zFROGTHE;K^etvx6g}159{L7m&w$4iMz7{Ug6+b(!GE203*Bjs3uukU+k296)-n=p1 zcUseUr|^L2%Ls3;l5A@>#e+9_IoGxoiVD)++FOo)AQ6_c0V5OW3R>p z#C;*g3N{FTdAVMG)oB)=_^mfezDe@GmzQ1HlXlJeLEq2wU!Hxqd|SUZS4Ergm|54` z^SNGUZ@=(03ZH3XB69Dn!{)?;Rfp#daqxwm{Pg9~Zgnn?3rqFFuuFFx(&r9{h%bt* z+jnkltM-1S(p$|l^gLpFk392OboY#>y3jK<-UIa}0dR6ORFp^a$}NFBiL-x1V$|=* zyfe30^fF({(0Wh4Jlk&zp-s{*YAZd;NTqxH-}~gOdenIK3#(h@qpYT@7ZoII=RaxM zA)9Nm@qvHVp}7pw2CGxebGx z8*a`vxurx1XiM+Vd3|^eek~?3^2<4;x957=0*>BW`D)>v*F%DeGQT(SJ;3W9el+`h zrM~FOxI*LfxYZKqyJ%;L>E`T5s*t4Kw~ zvlc(L|18zbJfipWvm#^2hld5r4-0a!==L6&SE^NgFEmsm`D#1I)~?iE$-UW;PME-% z0!ulRPW;5{EMk|jY9O}g?v?Q9xSZ_cCB7r5WbMln-|w-$Oq*>vSMfj$QN6^N?I8Bd z{#ToS?b~2u$*e8*`OVFavYQ*{HXN1MezAii;IyVtGp=2U>F1#1Z~e&FgCt-4-|7f^L{57v+&^Ky`$Dv^1hO)*(?em6n)%f=8dT8}Ju*^L6cg%62pMOkm zG4x7bIFpfmf@|9WANIR!bI-g;;R@7cIV>Ms70^F`H*5dU^L_m{!9OGUmx~wM6Z|^?uz4XJjnLBw)_4oct2A8Rd28Qh7B7^XFRQB5U*Y7w7KP5 zai>R=h&}WAlFl+eezw)8WUks6E8Lz-U~<0OhxrjCcYATqtX5~YFC{f+>V!ITL~|GP zni;O@cKyD5pIbkoY!p!RfRnA^al1_0ClQ(n7S9_O4WJxBx zFRi&?Sh`BgL)o=OL`t9Ky@i>I{E+{J9J#*AquD*Z9YK2g@aL2 zli5L>-3y=XKP#WhW$`f_8Cc5x$uwDX=>dx#&#%Qi^%-U>t9!mhZ^*SdQsdFsBrQ=N zFWqk9nJasq@uplezxBKO9`wM%yX`MAmA`n|oY*Xk&LoiR)ocdyj&hg zTD)x5mi5JQmCaw6Ra7chII73&{~5kcv9rA+^c{iy*J`tw62AGV(!;iu=U(hg;u!pr z+rRl%{mm6R(b2=K^Yvffabaq?6UFp_W8ShaE_H8qB#W%Y`DolOYP>yHAp3+{lL6DI zd6|uUfuc{eTka@u{wON5d9wzKt8eTZ>`f0}q_0qy^972G?XTar=^H#J^mu3sNv&36 z=%?r2r^+>|qLsd$hms|_$~+THUZ{CBe*9&1GQ+lCCFxmKY2AwZ98%@(r+2m}r0`~a zPB!FykX^r7eO0^wTkaak=e^5BIav3s&B#hDd9@{2MZN#Lq;-dGhE``q;EC6~5%W2X z@;+@fVLP>@|D(X#TXlQ$Hf{>teGhweo8o(g*OC=m{H5c0y{tP|`Sf9lJ1?FW^v9bukvo?zIx1f>HLVdu~v8nqRsK&eQ}N@B{7P) zeeX^gg>;l#RW(W>tWq0#bpLVY#cFzEUt{=wUI4Hokayt+pI6^!@m%?=*8}k%UrGRXIJXrl)$Uo}gTBAVQqRXwV*V>KcmNO9p75t&=}7+QmXLESz||T#nl69KUy|viXRI2jNvH4xs~g_yzRvzyw^gR&)!)% zDO=!Pk^E`*in8UI@(+uu)BJl}R2*{p&j`(#m)Q6w>df!P4|R{u3A1jG?*6bKp(RLt zFB|R!XU9NyefoR5Wiw<$23H2Dhvl1oD?QRs$*c8MyHG@I_K~n4)uQX{voDD6+AC*0 z^w6EXJ+G0d6J#{M+qdnS>SLE!B zkKJ>fv;HUFUEI8J$M4sVnaVDGHhQFlBvzx%`t6rPZ`8m;xnx8WHBzbO%b|>6%9iUc z2YA&T?1p5zW@et5zhiBNdB`2tt#*311#At&7c(*}efK6ZvO1N_le3A5uOC<;_pG;CG2#BY$1u=hD|rA<5Uz5|(9pTuEVMjPX2q+D(2< z3vQGAMwRQCF6VlP<#I>QCiAhapGAl)ySS0@o!jNu-}3z@14IwLdwNy>eYW8ppRYRt zPm)^iI<{QB-j{HL*K5g|matQ;mVy`z;kLyFz6&JthVQ1Eu$ z;{>RBwsGobf8{F?U+kQ*u36s3n_X4lfd9U%jy=Vp#V`1}?6@6#2TN<>YvjL?t`DbQ zPIEm_u@2^AmdjP~a*vF@EznDN!PN#O&|S}to&3P!6~3kFwA+`;PMQ6sw&y<7Nf!ou zc$lafFTG1Xut(>NNSTU{YuhX%zVlSCoA!79CTW63 z(D9^FUc<$`UsQUQHf`F`DEMyf8$+DCZoCg`L%n<5gQ|t5o29<-u2^KVxT>UYL9V90 zb>wg6;_r94FMe0~#koAZhWoRA=g3>~y4Eq13Si25Lv?pn7&RD1gBQKPz)^3sE4=|K zBe^PXRp>LGoG;0|&G;rJf0y^JB$eK{NX*TE+Etl*GcP9h7OSW45C1Y_v&@@|*=eRp z1vPa&-!=a94~`Tw?-UbTxBUI`%MVQFi)CB*oEKJ>7j-tM*JTu)eNsmFL|(OxAH1M= z__pQt>kJo8ZGOjLCDboF_i0O8!6(g>qKxnB?Nf|-D-=xjiwVo*FE(5;J5yDm!M<}K z{G(~kcA24dzUrTa&Wg<0s&lHrn*XtR?yA)%)nMXN zbros~@>}QS@t?kXcd2)_Vo>vE5#=5*LQ~g9D7bw_!KS*>#=I@*y1CzP?IdDay&S`^Qn9L9<%H zUr#eV_V=FkcwG|BC;USkn-QtGiRs0nS5_GkPZD{=5?$~a_~;1XSB;fl<-KHvU1PC_ zZyFo;)EGp(=vj@AU2bc9Iqv$qD&udrKT5wRJ-{Upt73!-w2)V`Ta3P%{Zw!po6GLJ zZ@0!Dl?qK8j!B|NR_IioNS%NDg^!v#a@HTTNop1xu@P+{WvSzgebq`d&?gKv4yYW?~DvU(auXVLZCybznOg+9jYeJ?7ZUM=N=gI-T{s(0O>68CH7pJ$nU@A=jIc-tIi zN!X=3Tr)gZDzU}JDgLVS+4vb(%D;KpFV720cuc}&38}7r&K`UH=&to1JFuMzXZg7m ze?`lF*~}eh_i$&@pTu0x?vg}D%`<_AW?zi=%8@YMay;83a$&oA_X45$mI+p8Om(CD zHq`WMD62SYWK=&i{1B|TW!v|Af)`WjrJb(xGL=N_eA_9pJSA{Z0`J|_^yhQ647hE87NdUz;y3<_$?|ybjyP*I6QXZ@O?8+YMxp%E9r0i$$i>4?Otq0oDD)K8` ze*H2%dt|@SyMw&f9c4}}<9erHa;Ruef5@8L_E4kyV%kj0b&HsiWUKG)_p@erwLXG$ zT;%v*M#u@$%N_muvg`zhb=3up?Lf}H&}Gpr!y{FPx*yN~#(~#4 zPIVetftvQIP9xBh9+?GtXU1DxW^S{P^Uibi=?$Ij%DC?)

-7{YiV&Eno-2qaL1gf zmcv!kIjHEp+Zon`ti`Ls-{vRc1#u63eewwgM-sf9Ph(lEg|p9u2Hv|Bq|@o!`lHOA zJ4-CV{Zv$SQRB6$zFnEVMIODESi;kcv^%4fmZ;i>oxDfjGhcXLG~sK{PvuMd=FUl| zxM`SMt#|gSsF82rjrr2T4*PIWW zjhh>fnyXtR%>0y)JJ0e~;^I!*MV>Jyg)QzVh7vr6cwOB>EiNT3wmJA=))Sd6E`Q?h zF&;24(K$2ES|(GL!Gteozvv668c% z6X=67NQJIZxCbsU7z$2DCm#PaYx8nlu8P;+t(%WFRpv8qW9n@#aO+;vd$_wWyr?~j zci;KlJ*&>k4mq>(--#8!&AHGj|6%OS8_%+a?yr0m9er~PM(D`r{4Vz@1BMh%eXXtA zt+sNli+gz|@T-h~WPRMh3i}+n6CbIA_O`@w?&G)%ivKykjLM7oy$!xhd$znapsHW82leGgUHIuA=C%Uth|=>Vq<8Q+IQ|E3R%ib?ZTt_pz1v z-`8IVx!veLY>w|U7GAAgv(xV4KI28@Exis&yN8zU^PMlBbh;sbkKfH4*}`L+?%RCG zOpaf1Fp9fj(Q~#(E7Z8yYF89i3~t`fI#}3!o2&k7>kmtUTCq~{j$;gGiQHl$qPdAd zh6+F56*e*?M65lb5}}o(byWRQlb{oE;VtWqu;B*L^3IEmozH9XOkW*c;2qJ~Pg;?8 zbN-%s3r)5cuZ;`M?L5}21_@WK7meEg!8!Rt*z;#WEqM`7eA2#$eUGnTd&g0i7jxbD zSRKQ2h31RN+j9j=zH;Z3*w4FZ81L$K?)Py_^%yr*9m{_{T)A(x6Pl z$VSxoiPz1CPZuxw-QTnDM*k9ldhdri1Jkg+3x0 z$# z4_wm!eTJ85P8r)i!ToI}DhD!*2(RANYA$~r-SzR==kO_ z%~ds!*S^d`bbk!aP2dT8U7nZQyw9Q~3rhv=J`ua&-MXMfxseOYyPExUxkK7nizR&P zPgtB;>6W^`Lp7l4IWt4mi!4XwRe=p&iWjn9HukRAR+%dCBFu#_=c#X;wp*TfU1i_O z)q#c0%Up`u;x}(k?nw3)%s#{9AX&-ORH)R@X12jSmXRg;@}n-JeEs6xZXaT9i6wn9 z-JUeVM%vsztxwzRJSn9s?g}59Qt0))%^i9_cg+l`dELpn>XiJ~k1rR#-l(O;-C@&z zspP=6bjPYU8*PtQ#9BTkCb{hGvtOno6;+-(Ge=2F64yPf_~BE(hUhi%UqkAQnmG}D z+qp@F23oE1NBiUjbyKr9YAGtbwX$1el}gI_yyQsHv(ir{uR_n2dVdSw$mqdfaC}44 zK#g5QYRHL#rcZN9gqG>2>+?KzdDS+r$(W~giwgf+Ue6V+{@&Jlo!GZeH`pI5o!NJE z=y%pYn%_`(EE5MQH(+q>R*k5G~Zp7*>OsEaNw@e z-Ib{-*RJx*gy-Z3ePtck84))}r+NFfuo^FqH10l*vm7>Cj9rcWTP;~G%|B9aURbXB ze%oH(>J#Ga>@~%TD>H3!c6*4m?LNKuW+Y3W7SEame=^n#%{b(ErDvEWN?h)e&fz^$ z8#}*Sco{3;RxLJCdv@j)cC+oh;rOx{#;cw!lIbz9=R92SLuKu;x!;6cE!r?wgT?Dk zq&`@5Q=q#^fL$Hyrbh7WDCr_LldhepUVc{mL*PP*0*5T-+7r?@K3rV0<9El`AJrA# zhWHE5eC0Mi)Ea+WXuxInim=nkX*QnTC#9-ejai3|tT#WQk+h$CSJoqgxA&h*uy%fz zNVvZ{lDT~Ij^aa71LBDbT66!5#6uraGsJx12pJ0#HP^iyHDu96>PYdr6cW@*jsCk2 zmIMRZ7w0kQh|_3Nu%N=~y8M+L>u*SuxXV6bPYU4Sk$A_;x}!Jd$}`+zg9PF8S}zXY zTE(Mum+1{0D-KL}Y_lWhqb8~J^JcZKx&t~}whYfKN|3JNsq#oke8W;@pEA2@6+6Gh zY`6W)Hf0W5xpy+Q3UR8WMkX@0u=FxtF_3T9OT^Ww?iH02SNycH+`(heG}!30UyIY) z((e`1c0!UTKNLC{Uu93Y7r8~{ zJdRV!pwX$bHR2$v!ig$>+}oV?W$Dj0rTgSpZBJ(D_e($4Y{GWsZcWm#Va`4KjOzy? zdkiq5>W<{24-lMKp*f}ed<#yUvsYYkKRpc$)C@39rPvm)4SiIs} ztL>~>_jv^uJ-+rd?Dm)Td5u)F6Pp3mYzLl(wGjW=T-%8xN3NhXW;PFU+j-MId>i$TJ%R^fTzc0#6J8=7v3)c zmwsc!rO%kjhvCC8!~gUTUDtilT~v!4Uyy-^8y@t>Kfp7@4>xJ0V`Gl~xzFMXtIGhp z0qztKLH!mFe+wXbD`7|KdX?~@5lUvEe?x<*IO4Zv3bnIhFpD8sCxpWB2E+arXEa)y zieh!>c-x2B7z{5L1|x++Nrh75DJUKRL>$s2SQ!N4!pDwkO+{>SxwH}kvR#~m!5E?t z>)?IS6cBB&!NE8p!4&5QwG>vf1DH6FJx>xWGW;6z5zZ7 zT5BZ<+CVX`gh`hkM)JC;?YOUR4Mt!Dy@l3|IM9&8g{>Rwk%y>988-Fg`mO;PVt_>lB?ANO zEm4-i5a)#r@e8)Y1qP7t!2!gG5uq%x1lS5Bj_(OiHF?8d@~4)8vVOtPbwZKVz@+tG z$@H-v8$$vS3kxB7;KuceBa-hla{-LkZx{#yK^P2@Ihgpwul&z#AWwz!)(_GFakoJ4 zp{YKC)1*UO7&9Trl6A6f$y22VpXC;YZqz}HWkMwQmg!HGYRn3eJ$@X4lFppMRodSH z`Z^d$#wc|50BY!#BeRMjj^sha2ZE7U;{3-gMQTK*kyRvli9)X^YMuu^b#Ta9C{ADK zIm*hT!bzS8b7iYD3#d&4$dCi96QB6d=@3C26^e(OILNas`TB0(Yk&>_H7=c?fN$T! zr^B+Eh=(bMJV$6V+mJEXTO%__gW}MLq{d+b*UI3C_PF3+Ji(h}89+b=G^H_o6-Kj zN+1u83&jzFm5h-8AM?}XRny(pwK^8m(h7qDJ>3WIpoVOW_XE?!c|xc~-JMkNemr&$ zMq4WDXgE62cITbc2mpAb_2iqKxP`YRAD~{s17XRePk1^FRO-#;c)erIC(uU|Xd7*g zq}|kjJ;!{j()g4_rSYcEeU~Dm$_9E!vTh6 zFd+|otI45@%03uN44#G#4~N!8R5%!pofwn>jy&ikZInn)&ywdDUF9oz{td{K^xR3oL|Chii$vbF&XiA$ zlZA2R57HQ;n0(Jqm&uIa;TPhGgY6jP4?865;NgcQk*3aXOLzA!SBK8x1q(sjAk!7P zImcGw|L|gkA>JZljbFo`iSD`{BngP@F20v3k?Dec<7rLIMV< z+?)6WL7YG?rv10aP>kdSdv@C)ANmC|6V!)x6#?b{i3ZLY5Giz%@*O|NB2Xd;awll6 zU3dQzmkBW-WUORss`_k?7S&t;t`y)hn$R75E3EoYTxO6Eg3#AvVq0)39f)@md1J)hkRgJ`Hzezx&Ie7OPnVj3#(o7n$4aoy=(|rkx&`Epf2N~mzAb% zF42x>L2e#W1FwdTk)`iDN=tDB z@O<#e=pfE|F1-YnhT~3{ybx)ZhHfo^3C0>iD|8UY3@LZ|grLt&gfhZwwLo0~Fko#a zIt9KRm!g-+8j=%4NODcMiwTpXK9uAJ-rpGn_T|8i&ci=hM3o)y`8OAD=1E?MZy9R# zt%QC`{HLE3<>!-Ycd7C@lHVd>W+?gkvkA-9H3w=^oz;u!58O2`q1FW7H|-scjW z8tCqT_?LJ~a>J2mX{U1_;wBYz}?z{gycsz_Ab^Ek>DD zwIMBV*fHD)`vr> z@H!#}BRhc&-@+rQ!$W@AD!`LMFrc0K<~<(z87Y8JMZtO%&;_gS2i^`wIECS6S``CF;6n2bWt8{f_4 z-3%oAzz?EH?CPdLf{YK!iaNKdIZz7BD-B9!&@M`?fqoM5T6+1kv1uj57(!xGYeX(Q zrH(zGCZ=dhyY{cI^p%EL4m))MvUp1uG&OHmt49>H0Jn+*<)anPJ7*@U9myIqGsvec+(w2g+;PPPFpOUu3^3#P5TjR*Fizy}eQzO1O z_eC#}?Q1cxAV@?_`dkyBi^dZj4wF|^Q)I}KCxER8!l3)ZON1s1vPCr3En_l~Cut_$ zP+J2Xei=p^OhuDE`J&WGkQ8z-Ww-Y_J;OQ@g0h%}Q-K}k(E&_(dSkcaFFgqDyaNx~ z$1Rel1U$Ag0{gx=q}B~C&4)5L@(TH!I+UmfnKXiDbRrbyAv{UwE(5EgzJoR znz_L37{IiKZoj$GvAV%K&-cAN0f4(?C}xZ#-QpFH?{m z@R^`_blaEOP6K`H0-mXEGo6sMNdwS_|7rUJyJ?`SlW%+d{)FvIpxrmXV$j)1ABSn6 zk9y|uZXi$9y8f2y1<0-JhQJ*iJut7PpNc#-bKa}x&%rW%KuKt?FBnG~+X+j=Be_V3 ze1frXzX=5==QHD}k3HzA4Q66==F=(uKaiNg+nvZuQ>XfCeJn_m3lCaTJUeI*K|wMe zDGCK?$Yb{i@v$p|mP+73Cp~*@16Uc1lt~}wJrR6eV$A2n7t3Ce7K{~?*AXWU}h(Met&Wj58rp&0Nz5sp~7HR0nvAu)> zIFb!B2!NIH$b!?_Yt%uJ90KeqCd6L7QdHOuDry218BJ0>26O%<6_&C4k@T@7oTpU) z!4ezn;RAJ+V`q>@r&CHE<@OxUoI(&R6a+&TTb_nmd9qqV1RE7*A~lc5M2R9NhlgV} zB;}i-W6-u*QbCi#j6hjkdrRGRK~)0=nYGa&lVBYc;OVO9mj9|7uz!IWga@6v(!5WF z#vX!JLhy9SF8sPFj*Pup=nq+x3EZovMh5z)7Tg#R#VqRp&IDvYhh=OJ>4F`1yX4)` z(cYvR^#HveT+(7xt2Lm?i=q`FT}w5Zw0U{V7&#a68y;O0uFDfDxT~>50t|rZ>Rwxa z_T>jK{NiBvq0_`4pZ>!ZK=Q^bjU4>Z4GEAKwk69i+f_BzJyUkD{e4PzaugLvqCA(N9))Z-M^AEQM}f zj#63AcPi*c;W&?xvl2#xP&_e!fDnu+jXckdNJq>$kRS~Fo<534@#lZz0XixN0CK{9 zoXG^as=hPWz7?7;Td*MjI zrX%kUHS(j&Q-$4~`N_quBtUC}yq1VFO z_@_ro9)HF*0qtrqGFgtPPImxao;<}XPN>b=-UL4`1hwXiJq%Jnj&4POjiIY@=Fg`F z501+M8w~wo4~3b*IPVCl@l7Jc%^1L2nz8WjlqyyXQ^d7{f%P0HP8Vg>mT*dqvQYoB zTEad*EjFvzPu|zTy&>KJJ#Q&VPm7Iw4;{F_>g*u&5E1e^Xh&lPb*~hK@eTD?7~5GU zQ3)Dz^duOOxm*v1jWkMC`HJMx!N2$^jm2%`TdcK;chdm0xET`F=v|36HQJbVSOVnn z#{)y;FVqao4Zr({C$xVLv>zR~884@ki@Xpe2R`}j2ZhwbqkvKj1Ds}?qV_<)5IFQo znw(^zQY~5<+UbqtDrCV#(SxF&GPDPt=n>+FCH^f|HW>Sdi7J%wu|i%T0_WD7$du9y zJ!6g%=!hOgf&O8IQ?k<~)-}rU9|FRle*&cMhO~*N%hsi)R&W=XoM1E>p~Tz(X>|(y z_E$_JA~AqCUCMa@JYLAw{T{GGV-#hkDQ!wKZvq_6o-SvjsBs@+Kk>k6h~m^Tr_G53 zpVMXRnR~tADJb>}C>C-ClRl3vDKo-$q91H$PFU^OKa6gYQ%E7H@;$ls76>eX4!z(L zwxLah?93sYR4F3$)?P&_v1(y7paU#7TiO)H_;4J|3v}=tsS^5q6a3(wc~i}?+<`XF z=o~mQ0R`hnQ*X3I!|5zN6#|{~9p+qg5%o7W`YBPeNsaBUPo9Lydl|tKzUgPVGj)zo}2%Md4L0`#pb1Z?Omq@&k=YzKLPn_Oj^ zH9=UU?iig@HT0uRG$9&QYdsJ?!xG7xE5c|;kG*z(+9WV9z;GDXA&pg2RADS0IbLuP zl!sQFb>MWVsLI4y8h&sc5bXq+&?9goi6RkPr2s+UUvGkr5y^K@aJucuGhm5O!#^c{ zQVgZZGj?=_^4j*MW_H_c=z3QeaOlca%`jT%6M-c?a$I=!#AXPX9Dzv>HL^S+XfciA zTj0X+9s%Bv752e&le$!c6;a|bVjLR)Biw6zK?S1FTF;x8u!3u*`j32 zj-rojy!!L6Sd(4VgrdpQ54|fn!U|?s28%3og5u-W|42{I!lC!=^BPzMVOD_D=n>!_ z|KDiIizl9dq>pD@tK{*CN97Wb;BE!z=8Y_9a?IWAw;&HVkmyh!HJ)bdrp|$sF^nYXCzHT6S$^Gd%b`v(Sr+ox&)m=5)dk#s zc+iV%-viXKtpaGH%Vyub@eYuWK$oNEROW*e(fyGv65PakZ|s_-k)0Dt{a~?EQN#s^ z7C_&lv%_l-(clMI>NL0X8EP8|gTGY~%!`Dj-X`q!LnKb{Htp zu5=Atb3tMC>OW>FBYH$rk{9g!tDu5Q(33x)C(&jic#0+q6gW{8fg$BAUon0aRN(!>6892rXs-d~|8n-C~;9^yplK zO(G8pSuqgXPRSnEpQlMccG!&mBUPy+kBgjN3#77Y}#vJ5jT$J=0CNI@)mie{*r4#QtlqclZ3=vMd-!mi{`NC=^qI`a1^62Wc< z+)ed2$edc)Fq$Bj8o$PZTE2X1`sx!{kP#-ZDQk{`=d|d_(~|EouFV(JHbQSX!Kl?o zkJ^_1#slsMj*So_$n%9Bu~jVuxq3h@^!QoTMvHG$U(~Ay1TN2+`yRZ|0>~$!_cZL^ z(8Bjug{!PaE(){>@SjN6TY6L2{C&7--KT@)AOD_lBwpPAAd+g0O7iS z2qQlKZ-f+TP~UOmYtn%53wSC6l+EAl{oe>Fl%)=op8pP~xiF9aAuRp&zY#7WKg!mr z<$Y)d$((@H0wrN#-~UF4WGAVW&aaT_ynP8+HDL^{K{5CB|8LBw;WAMzMqUBBFNSh8 zK-*Q|+R$qtm4W}c8B+%o`6l>@zT`n}Y6Q1~u|sL%!{Gni1e=kKOsZuN+l-f}CksJo zEMx`sQ2fEe{}VrXam%j?S@iQj6)9}3>!RogP!~auGn}ecYSSZ+SaB~LfWbTj<)WvN z=Ed}}*^LyEP;!E-Rm!#!V8Rv5Q@4ICpe~diX~yvNJUtB^3?VNAbx~%7z!0QR0*9q= zM}*3pHLhDhDprVN^-(%}qdF~C^jM*IluQnLydAkGF!~=UA1wLrd@x^9HOv{OhwR`a z0j3MaCfYDXmQR}$f`mtu1Vv&+mis9m)8Hbh&ieoTyO{U7PmHz+Q0P^ZjEZD*E z0OUUo8Ev%7Y%2QS=*jc38D)Bw3&3Fz$oiq37vGit$VcsxRQFCO|ZZ&eVy+)o~Q2&@m3W2z+*UMXZLdR|nUN76R8!ivU~;T^yK8SD*30(25bm zU|c3?AA_0SO*hA=x015QhX17<`==>?9~)}qEl|Aiz#;B&;>igfDLY{NY z?%$`8`1Lge2BLYrB^u+jU?uF_PI z$F4XT&o3bsKKf7Om0w<3%6k=bf}D!bMd^lBo-UrTeNLp>2YH#nSk(`OVzD(*^8=Ag z$p)A#(ILBv0zGWBq}g~*!N(D>!R=2OA9oa|hi?!N7y;$+-bA#AwdP?Lqn!s_cIAW(vk!vN)MysfB^jJ}BHZ$^D=2ahyT z`as>^;Fquau>h>v8{!GgAGArbvjIF2z#v-F=|R{xjTShk;8frYvUTVx~*sNFa{BJHT^d zD}_c)Yvu;Qlm3e^7MYE^m_NcDL>Sono-#GLmeXc2(}!Ywe84yb zB@d*Rd_tfd6vprm$*YR#lTh$M#||btodR#I1gjAA=v@fs`e=FUDUy)y6819tlokkm z9)V?{yF~3dZHkE~VO(G2S!!{0ug-zCsz6)lX(yzE9+uGwmipADKybLA3LMB;ZP-tk z^hxZZ%`p=4jM9)7VK?uWGxq@09pKT$;KJ`{!;c)SGsTkVDF(l$qtlLDrc(qqjIK-6*+9Q5iniXi4!LUaMzdt_ENktz~`nE79B$k&1xzI$u zJ<}jUB(XH4mdSm1@^R5wAo>UojG9TG*v}M+;0%Q~ywp$0Vf0V(x>9npwM+$FB|!X+ z79sfyMP#^58o6`%Z!9`?^pR?1QknnZ%ip0xhGAHuhh*$GnnWY>4LvN_1#U^v&>KjM zgJya8lP1eZH6lGM^V1gofwNE}_u!ycBX@q$WEmYS^w1~_<%TQwkI(e9zf_k_PZIAOw2p9y60(f{_EZW7b7! z>iY7+WlI7G0{3}LIT69h_@8)?8b8`G41x3hf@&B$C0tVtj*aQR(2^ge+kY!ocY(nv zBg@7~qaRet@ej+0uN=>Ojoo4cscLWqTBJidB}qs_~1M?l0y7P zU!Y4KRsCvlCsO_(0?kIdEI|;mkILTfkR8@_FC@$bQh%=v0&R zTw2KRenh=saE$ID!4xJBt!4Zw$c)Ga4Cs>Gr($%VQ{C@%4{C6<0Dbs_w$VBno<|3^ zCDwyT;ldJe`P$tQV1_XegQ5-Os3g6}Q1L~+D|ZRc$tVP6H^9;tov)OHyH}=6-6PhA z)W(c*U%)rG=)02CgYnStNLNJ8V$Sr<8ZI`FOCk@uj}s)O)aF{JINtpbD_)=AN)14>?x3NQdXchOFHTuiC+!hDQd~+z(ddu}4#)qY*Ww znJB*FuoDL$=D{P4LbbA?jY{6qe(G*G5D44&$*}t-iNY+hrHqL;8LP;Z62o&)m(9>{`B`XqQ! zg`+mlH4atdkz0IBAOcmG1PR+q@WMoLZVY8Y)Gh)uJ(?k|4hH%i>>h1M3jXAwVOIq% z-9@Th5R)RW1ySSFfDD-Nk3+|!N7I!+@~BW!1vhk2nL8_R>mI!TNM+arT!7NSA-M2{ zoDQ5VEvQZN4|lR0nGd~$+`x(s?`pS^2L$&AwH(Mrmg6-5jzdL$S<&YooKc7>vORkOFPrx8XPlxxPh#QtRb?>Yf*^LG$tek1oDgwu>sX zRRFaXYcDXruL&Lld7le9l6#dx9v0ch84qnCxj}UXy-@q55vF0x2}m)b^%0Um1sx9f z_}~D0WTvAKGHY`8&Ab9`1kX1Wn)d)@=#fnTTCHZ?s)CsdJ(vb9N6&W)k5h&=n3zWG z;gTr|4ki1^drMpBM&wuxI+>Jvk}|ewaB!f3A0B3D8U;`#!mpw$!J?6Lh%Wp&@v%HT z9r#IvI6sP4lly#>J_pzHV*Y?j*Poz*Z!u^77dF*!^4^#sfkP5GhmsAPXui;*|AkMD z0^bSsUlqe3%>m#vM#*P#emZ<`5y~X%pnlmMF0K5WBuov0Fo~g;5mzrvgOj`hUk)5* zwFmF20=ZwbWiP9w3=YlL3kmi?Vnh$h_oqePeeMztib5`IM0=91ddk=nV^IpeQob@H zHu<7;Dxf0=)Il?oJ~tcbK_{<|71)w%NZlzTtOn6-&Tpm%mHP1_&$WtLKG0ss5@Y61 zYBAKgKAw&7^@HoMmGrFa%|=TqrI4=%@RAS;Ywm6FHIH3jpy`0q6F{NKJ^L4pyu6vu zyj!8(9^(n}qL+DfFaKf?y+e>IAd&ko{BYpO{vHw28~ZPYI6u5Up5i2!AMd*3A~X$|_BZpr9nl^dO`t99WcN`F! z;6ZoN=RbcNN}n$GEZB1!k@pxp{P0YC+AvIG=1QYA^ZVxHTY@Z=G2S{2m&mK zan>+XkEHG>xR4bLT~_uGJl5Gm5@w<)@Xc`c6sXY?BjojS?kJZKa%VK$vNOetyiy_y zGwG~}6y&9lrWz5z$&5CZJjstYyj$QEIU`5k(YqomRp}+M01rEfn>-OqqVCf&kOm9V zNTHN+L!A~8dBoyd@iz-0c3cd}G_;x?mPQ;h+Ye zDPva*p(o}in3gE2$vf;Rz>~L8`3Lvzo5Dn730?(OP?J9KYsY~ca72GR0ZSo&rMlzI z)`g%sR2lAAV)%8X&cIN!;t_1c~hUnB$Z(>M*}BNNFjm9*#w*? zEy+J`|NO8Ur2Yi`0Ebp4eL9I$;K;WPA1%s;LigC6Iiq2pky@dVF+FcPhH-OsHz zcmXiap>61dQbrghsQ=O_I}`>_NJ6K)flfgi^V0|lJmh7M%s#8}2cneWFRM0LNw~~-I!R~+)A84Oj{F4bo^1%9G;T^d|Y=o2)mJmRYqELC% z*X5N2cN}8Op;55Po%F#Ep<&2(^`X*l3bQ~cxQ}y+-@A%+xr1k6FXU?T9Q z!h>#s7sEJ$Q4nkoQ3#@DJz%pdg5Ng4BLUCEM`+gg?~UO_qV90<{D?wEt~h|6C*Pdh zIq4$7fRPRlIzkj={0kRC7&XeafNeH;T`?17_Tw!Wfu@yI<9$Ss5D;46k}{O6+wU?~8Y zilvL|!jIu{r72co$1??BH9pvfS`?DIZ@-}oz_36th#o>O`Or}0RgtxI&$15aTYZoR z9Xc2ZjsG6zcX)Hrm@}c!$>s;vmA(a>mH*%ng~{SjxDsQ7hG7ztt@sKLdLlk3G8G3G zB;^s{2M!JwL|(XT;ereBp7N1f0T)2yCO+D8r-H$&;Ynj~$XD`!&JH1%elQvy$Qht} z{IHKoSfdvx)0iGcq!DITSFto+97oKPbd61W~qFWLqI|Yb* zOKQb?yJ7+0EIjC_bMN9Q02Exq)2O8xN$~6I{~)Z9n}R^WXpZuoE>3}8BQ6fz1}XU| z2o%hvswHk=0Q@=;9`s25J{cjL!mUFo10^;g@Y7^?(CfNB9}LyJp!}ZIG#fYtfVM(E3Rit^)C3mGej5Y-zR9GeovuOgDuW~ zcmThbgXIkR_x5m*Xf!$*;iu50mpN0_UV!w-<;Bt{eR%1O{~m_@*gMaZOmGCVU`SuU z4ypxyBLy8xR-50-0<53#pr_>wee#%amkoLK8cS;H!26~!KS4xvL>^&45ejM9=y>)D zAxQb*y^tjd5rh}V z#|r-_Oru?&qaB(638_$Bh{Eo0qKQr3zD;_ZuvXxI2Vr7WMM1JSQ-wqVHGh00&Xf9l z+w;+I?OSLryya@j(SR$i)4@l^KV0KZ3MNq+_)lj`UPG4rKmK82#7>t9?#zVc4<)AA z&!a<;S9D-t?n3V_tn#3biF{v}GIHo3^%3yWVa$9~Uwrl?hdbuJDFn-9I#N%L0SZ!= zjgAO+dQm|m-!!E^vc~=Z1TTf0Qt=b*O$CgykIi{*M|fDwbGcMZ|8zf-SZnMNz>5V!>!A3lgPBQBh;8iCtqa*OFMHSfjBd zs94Y#(b#*BCPrh|VE_MScVW*wd(N{<-v7S$dGGz)T)s1B&YU@OdaUW1E>(+a_Ve7z z6eYZn95^!pGbiZ@kX!bzydyFefwms)lY9PwDLOPc?CSmftLbGku+H z1d})BvkeJ~wpnXfYQNc-|E}IZ8+F^yumjowS(ff2#|a5N&UJ@xvzD~{Hd4c=PFA7WRbm>x{* z!8QLUv2r&E{4mjJD%{{L+@LuVHzwsjiIa;t_2aQtbR9D8&D5;D4Bn6?H~L+-!e z=?e#}CuP44kyq%RA(hlN|S>GD=IpN)(J zqz$vmKSy;0m`@0FG$iBjttnVhd5}9Au=!IZY~a14pY4aROIE_tkNtOHihQVKt9i-8 zF`wX!%I0n7#qahCle^^MISE@fq8Pa3T)SBXJHxF;O>jwds~IqBeuri-7Ot~sSBKRm;%!i?=73V>x>-fqs-6mg`gXH!) zKlNsCGbnQs_TuBLFK);=mX=3e6&4$lh}#mBqaU}s$J^01suE}NNj2HXCWO*5c(r-GXxgI?@t%w$}8l@l4!>S8#5;+7VHBD6wu@nRw# zys<-sWid|h1O0AteNg*RwD?&7*$U58(%;*bD0kr*&-3rB17<&zf;(4-+y$-sEWw3o z_Rylb&5WgXxSJD&2m@+_&vjq@ASZ$%Vh4tIGDIkAe$fN>YjjD{GAr+L$)6KB37E%I zdGMq^eu;4ae?N@IE245i+cM-8@%*pkgS4!06ki?YP6>_@W~2cd45Px_uoc+25y2-O z_b`XiPFU?0+RnwsOr?7F2i}ZJbUSt?4ppw%TU$HUPYZe6&^dW)F?}-f zBXb$#Qv zZjfa8WtP;JKu&36QL)d9o-0;@NDF8@uW41nqs9M^)?c zpSOEDWgYY{3mNq-_Q!wfSV@m>UL){{jUN&ck5_8s)>#p9Dq|B^Pw?@1i@?e{bW3QH zW3TU8QGzzew1rW)(!tJpSd+c=Z>#kVGF06NSTw=1mjPKU|*60W|$2CwNcn_=uuwwi;NXa8ewdrwXsH3cx)I#yQe$yaQ1@F8gcu|8KGtBlY#C_KlB0 zs(Bl;Ry=k7x#@qCE%&?*+sAIFoiZm;PM*(w+tO}1xDyQzDaM&BtFq?vnnC+OJh%n?CIYp}+eg
    zk}>zFa-jPa9}n;5P)j8o#JAN(C5k_0q{D3+uivm+G>f-(2l~9b9w`NLsu<4L z-j?(y{_Z?}o%uKo6peU_Ls%HOp9cK$s8A*?ilyEElUK!jFf(Y#NkDxN13apfRV=2ft@nfn@np9@+lZb$lM z1KMCz$!TO5>rW%}Iz(E=+wqS*C~AfLoNWVu%OcvBJ>M_dk67Iskvd=*zvH&ks}>?k z6YXZ+*buRp3m;;3BF7Qgl5R(kf3DXs*rTsw;u{=c;qY8$E1)*Ce;M?}%K2xp=ifpp0(UWXCDhd*nbbhBZOEEx>=Q;R9NLUmY@DSY0TdPj9T zgi=UvY)6`!jY@c2-Aq3sMXo`2V_iHMCrHhUS{hR)6@{RB&VZ88j)`(0Er@PN5=Bmu zOqDIe!dz}dX8mKlai;x;!Fsx%sBN@I#IA`PY>ga}6#6&XEDcQWLm`+?jFRfsg?f_4 zNTP4PQ2rN%>k`*=3?jpRuqhXwW|0I;u(+?D!09*LwWf1YWA?^3%y6zCy8>zE(@3!5 zU^LhiRV1l!E?}jn(x`6_g-JZRR7*9 zRAfSJ!@2s=3d5bSr{a$aff)n6+_?OVAqEI$TD|M^EE0D;|4U-cLbzG192U7oqij9V z(9d9BJGbOSR>s0*He<6*-ZRc#z4!e%|T|c+$?CTId&!*CMaJpn)tef^k)|g<{xwC{|9sPs+$Cu=8k| z%B#w{ol3KfNdgE&$t&jY-+PPg&GaA5X2XUYK+@ZiAjrQKba~V$%8je62>#POZE;{BRfhRAMv8r zhB%?AEG-pe3_Z%vF0mXfgnB*{k?@5_bh!EG;i;1ZQ(ercgE3u~706kFa}t4jMLt-Uo(1(_z+}VbWMsn>46EZvP6A#7 z?tVc9D&%R3S;;5a#)WpuAiQNh3BcA4BSHH<$t&vmdIo+s=~5|v#Dv-6a(<7dAImtJ zKA}t#FO?FlKTzyRNN&xVME$b4gb-f%Z5CJ%EW|YkrvHHV;fXp$#g#cn#5D?5N10}_ z1EmA~6aQK{O$R(~L{ShHT<`;^^7CIe>kwh|^X&hFNcC2j-&}GyrhHNH?&5mNphl

    Zmrz`$u4g6mR|F&@Wx-))mUDAsC7{#t~p9(lu5)2!kh#fk`bmm5`U~Gd4 zf-KNue^Mb=VeDJ@+SJPfUjcuh1-t_?@nirruSn^FEE(b&P?TkGl*3;U;~YV|6kqrz zu;dW;I?BkZ1H;CEbFyn zay{%9zuun3JQ+ZNkM)C9vocR>8Wf(Un z;pF}`U#q6S>{W07weFv3voyESj5r7yD8@=5Cl0oy(pxGvpYnoMYs&sFqckPA(4g?1bFvoI5%Kk;X|9DdIAWmg&B_WJf_s1OR%>r?9Ks$L0%2TWmf zNJE)S%zX<>qB3AL3)ef?jzlnd@V9UH^7#cJ?OV;ml156sH^UFUYGH@%L1e4g=#{{S z4X+|4N`(v1GC)aA?@MjYlH6HOIG76Lx3l*_o1$M?`@K^8c+mXnN zxHK}NQbc1@?l;%nL9*li$@J4~akr^O*&P4yJc{0Y{r%19)|1TNWt_@NJEz6#eE}FF zYtM33cut?`z@;%(kj#5{*h3dhD+m5%#{rk!ZswRK@saoD*T{!sQz5_hvZw!3+GaJa zhh2yO03aw106_Ww**fSM*;!lvX3~En>Q4XNM&M{U9<<$U@`9!$Mjb|pyeAuv!pn@; z>e#%YO6|y4T1tY(#|1Nr%z;?^rp{I1;-wCOIQbnDvq-i9e!JU9wPvcam8(rBhyp|l z=TFJ;Hh#`u@5{bQE0LMP!+6^}IJvFx5G#*4_^k0cXBhi9jpXa6j~hW?Dxhr@D`bkO){Cpo=oOATR429t z7)=rhpU|fEu|X6wQZzrz{jA|Ly3-=TvGZo_SMt(y@E8AhQt0r`-A0P|08fJ%>q7<}HH?NST;-OMPxoCM!FVmcXgd=>+Y#V;>7i0UG$T zO}*0bE?<*1;i-K8KI{O!tq4_SyR5yM*1noQTGu$Ja9QB&<+jzu$^CRleJrD22ib4( za9yz2&dOoSuQ5EhJlgQvGR;8 zG{`mT!I)S{$4#hVfIL$+$2iPTDpeYI19P+ZHS$j;ELm5ZfrdE)06 zqxj>aa+>CM+i=L7WxiF;eUhyIVNfhyxCz!j716?A>-bOlXjy@#{r*9-5N zwL~d#gXsV?m4(4QJ!`DGJPIX!C_HamgOR|C*=+I?2(xBti^a};4*w5??F;!I2wU(P z_r&i~>C+)wWbhulmg6EB6awZ_WM(!uFI&1^J7p>PVzaQEVlV=M|E%wwN>dKgu#7N`NNPaz@LLzp)C%fUeqX0FixEu`7p1d4D zUbfl??6f#pQ`oJyx)I;9y%^y`z$?Xdm`dN%$nX77^YC|WJQ5uflMa@@JR53bRUI{H zu>>cE7Zd2zbFGNoQbRemak<^Kmd!-KW;`agXtq`FRk7J;d8(C(%0Jml0Ef~N^J!!D zeL>*kJYc}nl;7HE3c`bNBB0p$HgjF&9WdfveWU{5#ZP{Q`O;P-i?@W#jyzx`>%Sbf z0iucGX~>QSx9n!r0V;}(@3@DAH+CLNpo>HZ%Q%GtNJtxNHAn!QBaZ?s^ZR*vRV!1= ztJtBZz>AWqo?1i(W-q0lk9Z8rBq|3L5B$C^?+K4J*xCQ4LBn8z#FeM`ZKn@`A z-JZsdZB?23n5}ED9vP+RT5ySM1u)bsHa13E4 zrCI8Xt1!I(@K{AFn4z#H)T*_8k*CYOLRS6M7I5Q~`5|1#K4Ih`H=R&!jGF`jkg8$+ zPn`Wr%BgE(6aYrJPgwf0n*OQ3z3^G*oum-h_vm0ALyiIRzdeDpLnp#W5Ck85bvJ&h zspWuXPK|){-_q4>!mtwk2c#&L;R6}4tq1kE=VQ1uuT8+!2uA-Ygy?i_G-P8)7OR_j zykGGE$Y@UxOco_c%NuB=|@vaC2!#HmRgjl>YC^b84zSP+WPj1z3BMv+ITR>?MD3Ckuk zc8@~d7MCzJnzVc^xP-JNghZXZJ2lJzk}5KfTM}^KoD4f8d-(t?4>S%mFM<9>w3i&C zgdU=f>BLZLib%IXGnzy#k5IAVsh?*)327Jzly{T+!|*$`k~W3-_W$~oNLqsw@TV&> z0j|FKPa*z}xIii!li7ry+m_M>0xiV7V}lX@z^1-`r-8t@^LIqKa|e!myC;J)l?Y5Z zAO<;)lEl*h*$xAhS$LH_jhXiW*=Nxmyl|h^A~Q4w^UM)~&|Xw3KcE8} zX{C3xOX^0AJ4wQBXi}@1R&=UhbK!=sLFRn_=LeeI36*cRfs)eIvJ0wEGSwR?M1f32)cOm>TbK;LXVY=vIO4d4e~7C z{poMum3|M8;i{SKpRoFi;5*HtHMB||<;UCFZ}Gry!zc54H9V$W3-|mF$Hl)y)4xD? zfl?TS6LE4Px(ZWkU+@k^N*Qt9AxHR5cHS(lu^2=9E|NkFwV)D94O-AKT8j4kHv;iD zFMR=QD3#DrAO={#8}U(K-I-A;nGl%*w~3Zip+z$#49x^7FSV+~?0ZtCRA&uY-BVU& zsul^6$3BoSC#C2C!uN$Fy2scgSJQ2mz26w2M;iLW8UuQhR?9|b|EGA6ic8PgJ7aBHSSDb<| zu)}Zy;20enkJ%ThKN6#dnK=}l)zh>8NCkFh-4Kr6?(#1i9rAnV4e}xRvVtZ>(V-3m zrhQ_;9Nrh3m|Ii<%z}jM1D@vl7HF@{p>FeQTYKl11#BSnslW{%pr{&u8nLIO2&!iO z)t1|C?q(%zP;em>dEPn^ndL+7`y6=;0FtP&G%8Ekq&8H1kZiJh?1c0@xY@vi_aSJ)ssFDN>jKyK02=I1LCMt6b+J<5gM}lgLG95Z8BSO zw68>fq=ZL__R;k0!+L%T&xwPON)G}v3t{a9v^_nvu+VCXqmX}^D6ZamIsF)t&}`+s z@D;~r=GZOsUwb2?Y-i8j?^=N zF~x`&aB)+l>;{%k1rO!UBhve_iPA&H*8J}@Zo>!k_)#>kw<@h{3U?I`Yg*$Q#e0Y{ z$3O@>DM+HJ2OLiC6sf*zLg1XtKu`KOzY>Ght}dKJ+c$CdA?GXzvljm5NCqBjmWAQSkK|&$kqLk09ImQWI!FCdtsQ-1m zmT;Di;?j{mS}ym$>a&24**!68kT&rOyF`$*24JioLJu8h$BPD0gZV1L^Gq#CG>3@iICtJr;-9dC{NU>7X0i3Qx2vWS zjK4%*m>%3-7Bx%B#$?(!x;uJ$_z2cuRl7dyV=r`>>m~5%>}V(A%fi)S?PgJ+^g5)R zvW-e}A}B;n1gh1_yHa@#Ifw^;=pa70?omEcPL%8zE)GC4VIvlslL*yyOts;1<(l zFd6FkEqVgP@%s%C3-sV|DlU2+SCH0k+>Nc>t`xA#|wlHvB>`c zJ^+LoQm);v&L?n-WL{o6OeNM_`hlZ*xTa06E~Po!5mR(TOw!;pf%L$b71_9=8p9O=iA341X#F{@N|E_4$Q$)S8qevE zTx2w^*-vGx*l4utK_TTh=x=}lQoWc*LYu(?dy|oXV;QE_||(T4s@p0P1&Ap}{di z0scA!q~?Bk+~GkInh(K{MfF(foB!`h`t*L$^37?aR$<6UU;Yp4+3ceF@rwo8m+)8> zpPE8%Pvr5xb47r0wquB_y_>Lq3{wyizXh6TX=!!5>3QC#`y9Ev76KU@zE2t7D}Z!v zd7{hz7m{*}K|v$`jyQ040RVnE>HpOx{zomZ_a8{AXJ+7R;%4CS-zkUBg(G%r!mZt` z62Vj}VRCqp2^YIgT7n7JJmuy>{h@KIHr;YKaDoxM*=c&d@wG(7GvPDoGw3s*>#lL` z;RUpR{}#ng(cw@$as-Y(7HnSU{I)sEGv9Boz4NzSsoXFHF8KcA)4|E8Si$_YtDlGa z(iJbKAJO;IZyCnyuN9;Dkof03^cHXAXd&3^YJVx=EaqDJxPtUbB-=*+fo!588 z2V(x!zl(DtuI8O9_wvHaRJb*?3>=79KEm!!OPg9i#_mHj+Aoh+vt{0 zhNoIMEHQ<&2?m?oS+WMg_s^@m_=#8xbi&Js@3G%%Iy~t`fsfA6QzjCy)OQO4uh>6a zxQyvMo~s@wy6xyf?8&Rw(x1D2pQRkxJnxzAd++%i+&pM{Q9SIJSM!TsbiGslKz!cx zhL|`k3C?I^#v~>i2!-l!a9HRP=Ow3zV4pSkNd;pi3h22Io}sG7;OSXjaaiWxqRIm zUSF5%j7h9q940Fte7HF}S=n8_E{`Z!<|TK^Y%1$^UR#rNZszg4ktEkNSuMr$o^O1m!@9@xfd&VhS)yF-dN799>Z*0KO!@Z(CndxFB+5 zWC%-+%22W>GB8-VM(>eO81G6C>$I-tlEDr~HlXAXE*40_+i)=6DiD+b1C2a9%7@H8 zMfAHuT>VbDX9BRScE!a?3uDC5vgRpWAO9FH0X;83>s+p$AFACBlga+ejB!SkrdfwY=`zy{1aOW_3TyWg&}= z4JMcfO(@;JyokKTSW<}k(tshmYO;f^t&Jc{+)MbSUrM!9B*>l~;zJ>#x~cSUk8zHQ z+nQD3o~%>GLBAPQ*asbP!t@3_pUSmU0*Pgq&d-c+cMC}mun@k`9Am5nx^e&p1Hf2= zf9<)5c=L)y`n1M`|tGH?MbrH57UGB)820Pp;nmB9$CGdi~*nF?zvEk9mZtq6w#r1 zINu%TV@Lk4RJyJdrlqHoklZJ1sAn(q4b^=^9zgeivxg|vO)TECZim`;92nZ1r@3T_ zz^u3!fh%x7evEv{zYK>tjLy3`_{~RqHE*7`wK3hmRc(=XA8ucSFNFGOEhbR%) z3!Ybp51(|Bo1@D-hpp53$qyZL5Y>OPLX{0q<5fH{T~HuPY2D#n76jDur_tp!{Vl__ z)GfNi;wgxd%|QjiyplEkK zSaJ#jshU3s^aNNjTrX5fuJA?d#4nl0>(!TAFuQW~1(<9!nI5v~T2HUvGnUmgG z*0vpw>>3g$Bj)=|d(8=s(cIF}I{D*Dj*NK%0tlVaQT_*V_-#xMq(*S+FdakDjyS^- zCyE4HJg(P=(i;FR$a(eR@&tCJ@nun2O|(DfU4RIPPb3~VLypNoh}hOBu7$K2hlFhj z&{`2;L(ltrJ1G0to57kLrvvqy((7YR@hAU#f$x$y3G~|udj|rN6bg2mQz4e#b!efQ ziyl%ncL~lbjT!_lP>jG+P^t<&DHyHKN_-Va$TvlhnSh$t+lf_(HOLTv`u+S6nY;Hv z6{oLQFzMzG3H*_9au$qN4Pi$!nmUF7af2QD68L)?YsuJVhq;BwtAV%y^2o2qg|OoW zbCZ-?4G=GaQe}>BhRnc@asGl!nkswuM#-)BoAQJWlKRfK)ijHQc}m}r!5zP@xLA2#`?a6@`ZO?{x1FHRf7W+}-w0k5h>;{5Og zJyc$sY$N*b%1NPh+IEMjb@a{3>~{Z{DYm z8_-@gJdiTrmZdcc3}sjsTw}y99ZG&9NRpHM`1q7ih<~qp#+I8au!w;qyyo5j%3pqA z@LcJ&HP6Gwo}ABn%hc>?1xHBRpA%y*+?eMp@5JkVet?=-6`2u$EQ#X{aee2`(;qoF zZHole{`I@VfHXYWhhZ#F;Vdvf^{r_qF zaa00y6c1MPTyWAaD~TX+PR0VzI|!~Ai$%_}Ioop=HDD;rxMjHgn0s3{Bvv2jFv661 z8ceHHVYa7fLBs6G%2vf-aqbjB-tmGm-K>13GSitpQ|sB;;GxEFdF!d(!hGM4C0F3C zrFN~(i1`=TAkRmmj(q`Yj*1Sf2ey;NYwWUe6%e32f-hkm&0Ck^EJUtcEb|yBfZ92a zNWn^sKvs{&VvX8ZHz!3G0ZpDRYfmNd=ubp22+|A{CqiN_o>-rRYA<)w7#y8^SQXLb*ZN_w zGep<2kyM(IBAts1?%B|dVzzRjfn38Ge$wSe@oYev^oCeUJ|GwVtnjM>Hv15hCVDk> z_0_H>`uqe65nd^Y{H9McdOhv4dn7e(G1)Intn0sy^xz0)z$Y<7%I7BTO6xq&?s)e{ zAh;nA52)FbJ*Fj`j;H{>8WIQNt>J?49|LRO(!rx7XN_CN#+^cxHB3}V99x2q>%l?89P_> z5@@BBp`JvQJdK>%=m}F_gWVSd66H4=J3Y2MEhfedoGII$-kSQ9Y7TZ^;MCXDoa~g| zcc@?;jm_T}JZRZRTSSe0e)|mcdGeP9%+*dTD8K+efiOSTsyxVuFT&n#z|8^(Oi9~m zNi%W(Fa@EA(;D0tNOOjO4Ec-d-k>9T z95gfl3_G7`rZsr$D`!i@WIVCBC55d?I;8}E#6ds-udc``@hO)GLMq)rACGOk@Yf3y z>MHLeT0aGQI5Xom$vA&KWX|vk)|<5t74fyzw<>j*pjgLO{6Ad4;gz*~1ss_$Vx*rK z(U(6^kSncTbJOmjISWyip=dXk`1Vhcncr;f)2A0K=27&i9M;1c;R zcyi2DD8j(d@2sMTL8=hVP%G=s4Y0y8GE+liF9;0@Y#Vh_`b%Rvg9T$gxyytaNgSt< zsE!N(m9W4h6GEDrtCPf16Io;+aoh{N34*l`eO&WvbHbRu#i|Tu&yIt_BbBz{dZ0iK z5M&4taJDMRJ=g`>Apt6^m;8g-$1=>o~QC$5#K)N^X#MEcu%ifeUEbRyc2m;zd1x&zCyO*! zIO{8#{{-ScasqzNFfAlEJ;XEN*TRR z3-I3ps-Hg(D>_sr=zVRkeJ$DbAXjZ(}dyq&{S;=EEmc#;_#dW;QZS zg)@Dad$$BJq3L%N)EUP&w&5p@bELD^ka7*pwkcv&vmUT{EHH*ydio4>3{X&*! zLXlEK-1Y>*ZPo|JOn9w#0#4j3d&sNpx6hFlU@SY}psP0K48$n651hVB8!!_g10Edk zp--HmQPkh7p4t=E$8ajfXr$j-IFhLL*OUxlXrikTGp;kuM;XC#%3GL=rWbew9uzcT zl1_>!fzf7KP)|1-+icrqxpZ5CuBJ`9I zi6&CVA51@2+t14`dK{(Vxf~yBnI(%9jKMeGmT^dF z*c6vC3DIW&4H|o60L=(!e{!6=;7{Tzm^gj}+A~RvkS^^HcoaHj?Af3`lgB-+!EdW? z5mwJP7#~Da<->}tAMAlUq<@xSJ-Q9&m!i98$b%@EVoDKd<}T)wf?Dp&4*iP9&6MwC z%wO`axPm-h;I5K%kk$fyp#}Vyi4K4k1z$8>^pwn_&Qan%A(4T@fYg#&{b-5a!LH6% zL+@1?*6f>%r09yn|8jmPQ8chuOL>`vlHOk~Y zL9zdobN8v#-l*7iCGj=KXG4-sHTeJlMU$*z*snO%&K%>N+)GOOmLD4z|IL!eRtQb1 zg+QoQ=>+|al-(6y2aWwE%VT5l4V02H?f;;8&Yx-rQ;^W4)F5cn3uKJ2QC8w%e-7qV z4W(}7n`;3x03TQe#JdF-zAxV|FivsyUjq9UJ0QSrwVaC(c6cW+kfZ z=q*{Jd1g(N=ZvFynur6`V~=q-25W1@>!k7y!7)QVWWA!aK8l+ILwgsV23h3Spw3(- z#=~!Hu}j3ucU^&T?3!;LW!e9fPc zW@4tP>4yBL7ev4j%0O+ag&wXgy5xR$Z-lT@JT{xm(g!d7~KCs~6>F9^R>4!d_* zI0nP>uQMN}8-V1ViipTOw+S|d(-1`q?kP#{L_NK!8A12PsnadF_>*jy{pq=A>X%V8 zug<@GTmsrI-~il^1=F6Z2N67)k61MGZ-gXdXhAov9N?1`^w~vRO9bvf(h#Pbe-MFH zqvBK~flZYHhwS!4bq%z4U%yIqj4}H)bR9+hKt2fE7gJvb{2QXXBSIm`P zswZa73bX+G7+G%sHzSUm7#(51sflF3VhTvDtR%Dl)&l>*_V6^HcQvzGgjO(3Bsp_! z^92_NO~>v+wgub2x*|~Jd`MpAsX=%nYOQ?G)&$ZyMTB43X0oMER%wq}Yoc6AuXzlp zWNC7M3=F^34>7f4e&r|`D0hI3PI_$Q_{$TlQ$|bxg=|mG`8(Jg?AMGl7=F<#o2#UvWgf_XHUtqK+Fqd zoP~56(tIbN!eavQ6oSHilyFY2^kt?K0X$XXrL@skupvb=0lb8g$3ISawCF%G^Jl8c z@jLG8wAnG3Z-WZ{7XBMJ<&bj~lJk-gCOyT32z%&*&T>EW6)h5i$g$OuBEIOw^IT;# z5Y*cu?J;bgWI3A}koDF{XkwP?u?Sr&?e)dW46rF9iv|874$h1`Yl^i5xO8t&n1Qnb zk~=d=bdhj)b8y{%v@3FAzeMd$e5bL0P{wwluKkofxhA}^qqh zz?OC|ts}EP?0z7_uP^vO2B$I?HnR$A9~PI2i<6w>PWe3R^&InOv(?j2pM2FiHFM&H z?y%$r`n23fq%a2$cQ6&|An^6Lyx%|)U5(ASib|C`>nB7+^o}VJ#^$8T;BO4p$o|AD zKM{qZjVLO?TuCJ!SKW7aPegdaQ#}}aV(MS#Ln+s_*?DwBhd*QU6hnOjQC%=wu}NM$ z@i{+aEg3_iC`!AfuS6&16rHP9HV1DE_u-Y8+<|D%P`sCQILq@luhkd~z7ea9Kz)&( zl2v%)KEBZNN*6lckYj?xzmon1j>V_h_M!N!P6&1cUB>Z^0fg_1!azy!kV+?P>~?(Y zPcQs!#0+)iVv_yQNqYUeMbKI>ezFTX^S*W7&VexT)DT*x3S{UJ-9f}P5qL3kKQX_M za3k8U_M#bmxN-ZQhW{wyP>GJ}!~o>JNYCr{>Jzhm_o0bPVe(I1_#u7Q!QHC!oKuo| zDAmiZuqW>s#l+D^#AJu)3B0xc%B90ZBD36r`Q;Odpy9_NT|hI&J{3Qtg7H&Iw~FJE z8Zt`8N#r+E=g5@Tz>JLmR0%ym#Y*~9I^&}`2BdwPk0!~wPn9Z5sdLNyVu80*eY{9{ zsk@~hnms@!Hhi6U;gJ^hslT!1acgtJ`?*;!v?kM#6;{l`;SSQ`4j@5pKBqGgLswm& zs?sgc^q8_ooi1Gv$EwFo#Ox~M7VcT^;<$>l+0ruGa1fd{3+w^gDB|!)M>-jik9;+R zd%Ne1t@+Kbx0&d^Q*cl=7;Kwn?9ge0TISESPbL6-qJk+Gv^1JXy@dig$z>G9i6?1G zyr)gJ)B{D6931h-Tox0r!qBbVtT*qB-+kY{@6i5oJeTO%Ny`K>oD|P3K7X_y62;~+ zR5m5m?6!2HcgPp8x-3cb!i7dk#>eTTie)A9JTLj{&+fR+ue6w%9iLC|3#-j%oP2Bz ztVEfPs!>aES`KZ~wsO%ajfz4-Sr_k5{iJJ?1lac(2pFxeqCz$KNAf6?G9eQId_5}< z${w=OJp#a$=7WNR^xX(CF*|>2u=D8rxVV0YMSi7B)<8N<${4DCHHaswykT0sBLvA} zjU=7$$pGU&$YIYjb5BsqVbs>BJl_6*a`-A>C`}}iLz;rYHd%aS!gS8qD87AH$M8dv zHAxL&wj!mfR1#xDGvIMF?c|F=S?GjC68nxQqyPsen&@B!k~+8?`T=oa0!>Z~0*Ap% zpz}e@<6e{dJ?7kGSX@WZ)+}D8ETGw-xFrEknBjvK?>{ob2FX$ICF1hy1xe5f4k!f> zE|gX{OZ`6Byl5S&$|}CP{I8zq=V_KU*2yZRii4~sqEgW&Sa!#8pE%C!c-EkO$VfWg zmaDR+t~s+_0ZYLbd8)2-)f{Ba9|Ib%>hx0y9768L!GKthA{2*k@Lvl-Qq1MR%VX5=z9 zSth8l98Q=ntYaV>rL~=9hYEefn38e)0b2U{Lf*cBafi#Wb?e}21)wUETy%L=aAE5z z`h_w%RE)eL!SSIAJz%K3>82wo;6?+ODs^h67%AL z6Wt^vC~DWpBtYm#Gz&VJ!E*x4@c^JJ7l#A*;;Kr^5xjk3d?5z-5gPls%5#rOre(7X zlmDXr`SBoJ%`!PzZacA-Vz?=&>~gb0WY<|o=vGnFsfl$vuW7W-^|MXg{{2pro(W8H zk{>Q3QW`QSBredPKC^=0*PXHUBk;Ht?W(Z$Q**yKKyk&o>_2`-n!ycL+ZCjGGalii zbaV!%q6=D!F}ctmvQ_SAGZN#vFN+i0j?|bJ?o!3j8J5NQF?hN~+)Y6Cp!yj-#MC?w zX`j3)p7mcY)lUOkh6$<^bE0&!Hzif;FfY+sP1v~WexAUtY91d_U{x-6#Sdp#fxWHe zZtSir)tbt!R?-?(ui#8&$NEp57%Cle8uMjHBV~@Nj1^Uvb+X+G&DWm?4$B)7qXSww z7M6ZbXGe=IsF79gp%+!zk)!6B8EQPgE#M;5HOf za~Xs=6S8)gEuf62j?ZjwB~^vlV+i;@Vxi=cp+PEBtnq;#lmKw8LWO00{npD$OcdWfmudASY?ur2OC>X?n=jOB)U)e^ ztLsT-3Nj#tdN$H>DtVP39ojep;E}~ijCH5dnJBl25J||0vJpV%pfuv2Md%VyT7lfM zU*l}VRjjNp{j;n+o>x-USFp<%nNtBg&(2s5=5%50nelWqgP{UHAX$}xS>IyONd}y| z!?7h*S9hix5kYR7#nj*66iK~t88wSLpi4F!1o5C=H+PJm1*vw-%lj>zDkED!LE)fZ z-qjbf>z1^7i&B{-n2a4x3V*HIUhlt_?O^P{U$*Z^qb%%W!$!3?^eB1~;?HO=N--p@ z7ZhbC!8FRih`w$=L%5uMN80!94IfR=Z8d|G4!VM)J4WP=3NcZFh$W*I(9iftBg#S< z;h?uxHm@w@@H>1fublnUMuyC^Z9#nRwn0)b+)1f5;phTGp~#K8?;6*b#h255O};oO zT(Uv&{OOy~zva?|&H=Uax}(*|Q>z#>nO7Xy0%h{r#hhM3i#0s4;7hRGX7cxuBc6$8 zd~W#$gsEtyWrD6J^F5P`x@1SoGBpn%O`&!-JoQ@i!9%sRhp4r>OG$AEh^*BXB(Kcz zH=zd_t`c5F=gGYRW2CDV7+m&{=ul62vfKszGj-aQqlQt`IRQ8Y8EwOkr~LPw*NXPf zz|@vetD@pKb`yfr`_$pfG$rBiejr?fQ!g=8x%f%E58y_11%XG?wKOm#nQ3(w7)AqozC`E@k_xD9ECL;W0*X z4F(yinYu;!yQo{?Mhcz$_Tg{$U2-#Gva)|KrE*h3oWp|2SUy;*E3#+Lx-56~G!UFf z=F%t?mM*(CF*d_J{R?HkBb`tnuaFkUl)BFe({_-w2ehrxxMqe1eviC@6uMtY8t*$4@rg;O-G88uw z_CMV6Vs2a<5>=6vQz)TK+PtxPni6F_{sgTMHZ}n_ECAK@SF(a$D1WE;p7mzm^|S&r zfogbFIA{dh>T1c@?FG+=tU@UjmA~%~vL$@(8y@PMXf8#aybDxD8mKr~% zDWAUuC;PNExhpZsntj4li1T>vP+YT&&xFLaQDdT)D3((8o10e4q84uQicqk2{a4K? zN52Z)uKHPwE%Xe%DvQRh;Q!zNd`qjf*MFcj6}M->GR5$Yf=>Alc+FqU3Aos#Yz*X& z(YXIj*4{Y2`%vn7%UsMbte{S5ElNrbf^Mt>JW{3G9c$7m;)>qTo+zY`vP$1lMPWpn zI@e;H-nLBl&fh@S=dPk=yo9T!p^AAkWMR`?BGWFS5xwx$QiqP7+{v@6nHw=t-0dFz zev@dixrM|Sc;As?-c*4T()kNqB8!Bzqkz1t@*9wYzgwGH&>otVkv&GUQHI(`8G=0j z*Nv(*G=XWr;Y!PQ?%kg3Vo=;j0exq)NAxOFZ;NLgZHiC|{&H3A+LQdDztI%SfqeS+ z1jUnEA`#g1QCnBe#Y?B~rG)9Kl=62m#4K>Zd^Dvyri!|nP(0+^9lcW3Gm%EhzBW}p zQMV`mT8xMD)8YPE9e()FMb7)KYj40_k`RW9?w@Oojh?v-3*`oT3y!A5?uD&et80Y^ zXTkeAWhLuMeiffpu(eYbd1ZsO^%^22TLZVsNM7@`*Q!qy2SYKrD5RWppb%NqA4c>0 zc6=Cjla>|UX?TT8buQ@KYbfYo=0~L_tt7+eDz$({J9yKK{6>3e zNoqVkjfB+1=m*CJUit^2jsz2(9odzWE9P{%nU(pxl>B-gIwhXYfN72DzAGHJkuxww zr^uz^%T6U1ARI7Y49LP!9ND4>Xbp-6l8v_efd<_E1q5LRQ$HJ|9|aHmk~SC#1u0Yn zssaXr0>EnAO^WS?&#`OEwKAr>T0eF5z*}f+C2-|bwmkT@l|55do(WLQd6n+=W%u=A z-f0y5om*&TCX)iJ_N>mqHu{jvw2u>xMexuzC=z`1C1s(+b{Lk!uot7Vvmz$E`<*m6 z%`@)l;^>>+qA9TADAVu)`?2{i8As9Sxom1oUqjbAx`O@@5D~n~E*MzQUUd!cIVDSd zoil~gi}RYhxkBb~NG1%M>dwRWrP87=lF=*6(Eft%dxH?S2Dq=`P!l#5V})TxRffM~ zY`T@)G)|nHTu+|JC?amfpKj+%@&5+F3RGOs46L-S)NlzA5@)S4nNXDX>Y8`zUB3^~ zCJ962W3RW7_jh<+oo;lmWrZ=N=_r|g@KB@ip1)(bI!+zVc@q}~ap&=0iCk*}+_w*6 z{ZBonM!y4~(~rQe!)AW-GtoPZgTFmZ@}1G;9h`co$1G3|QOd{@v5Ea$iMCt#3N)qo z12^>(T=>UYnE#WVktwj)N|m>v;L4ZoG+Kqf_4$!>-8*yMN{=GrnI0|Rq%LQi%)QZui0NTzbPhm0D)g)zuXG|byy^si z4YdQo^5lxF*$lEnzEK};TK?ICEPYgPyp?@R!a30vX!pWQ}$3!{#is%vG;JH{^^x z+saZxO5Q3Zr?tY3gYXi9`{_cp4|E!qXm_P1@6dv>E^Hp$TyA0X`RS_2tnhSKBDT3-qdBk*Q05Ugd@36Cs!K$Hdl!@@n>8%MHN83D)u^p z*I6NyXI3I@i^Adt+U3vVsuhsQ&M#(mJ;=vzIoqpw^}J_nG7O4RVe@j$hq5<80=1I{e%IATmDWx?&OS?fr-Bnz;F zix6|yBG*Xl(Ed+KjQoZyw}M=xN)W;6qF{*pBC@T_)k7anE;f=@&d}pXY;nOq&Z2Nn zOKAIpQt(Gu^-Z^iOth!aOjE_Tpl7Ame+JQ*@=n!l~1~m$G1Nxb+#{Vyjw3PPuIgzl3~-PFtuTyE9?ba zA!*@Fu;xdEFJtO?2=S9_01^Ir_owIU-}lU*%^gMu@B8i#)b~s82!W7}zTT=_(}Gh` zOp!e||M=gGsLxv?Gawcmm%f-YHczw1aQjg&%Lk4O_|IL#3A@(LpY_Z_^01RIyCr0) zhdNYvWvVrKjy8)k=gf9e_+5}qC>lk1VBay?bMv;Qx-B#0e)Nwmaq_r52HgI{+8!!u z{%`)KvJLDX8}QL8A!#Qjb5_3Db?9l@dEmZfDwKmqR02>*{Bu%XDkAV#iO{dH#M$`0DfS7AZU*QtUp^R6T23k6JEPR8$WBtjY-fbFBVnlY5xv(=eePBCR2@ z>PaBA=Tb!y4K-qd?8vb`v@gyd6bQkT;IBug$_>H|QhOiQf6tFH1Kacazdw_CbZ*f? z4^2HjK05z&c=5J9P5b)U-JT=@??+mQ%g(sP!o7qqJ7nGH-Ehu)>*s!+b@q=EFEs1L z3rG^pK^eYss}u-olXna*b-)Z#6~+Z!qpbig3!|f1bB$95Mf^jhTmY{*$KAyJM9R> zCnXXC3%-7}0s5KE3p%us|J?08A)ZX3ExYS`nmWYK^uA9mP7W^(Uh8`N+&lelOW;|G ziOr*MZV=-TLd{QZb_j(v7lzMR*8N(XICwAq{JF*#gajm*RhCWoZmJYIFZy4znDr~W z5f3t^VauF6W7<6t$dN_yraE%Ut&u_p=!J73c;IJ*NYBc?I>0E$X13T~MeoCnenhi^ zfU%UJwbd)oCEr&nGayrfV=o?esse$w>#fmfvw0d@Z={3DAg}%UH7S*siHQ29(O4h~ha-GtnRH?l4H4U;b#x?SK+C~HBYdpPB?y(EBUzkz2dj=wY4GF z^46xi0Cc<5#g&ihie8&hO=8_NtVtF!Z-sdrts_fy8($H9RSXyEjKuP}Y(mrU)-Q=hAL9b*Z6DEx8N@;{m_A}pBVz|VR>^SB zI~*aWI6R zy9+4ki0SSROX1?kGF7=MwFqm(9hSQQENo_GW4iH}2yucpzHrZpfM%^OIGVf(CSARN zJ?(!N+ss`s6iT(d%NGS?uwLM)fy8KNfe~>~!31U-ne)ylqWEu)L7x&h zh6xb}>(Tr_%=?v)u*epHgPVy|QRu~FSTpC}R z5?7cOa;BK#eNYJS*1jxpJwTypQ&$^Tgdx3=;ic`Jcb~Q-4xv_rx#J9Kiq-ZS)ffz> z*KfPsC1<4kNwM<0qk}O%KO6sezc`2Bo<9%+b#u{$r*<4|;Ck|RV=x>%zY4uOJPr*9 zBniH*NrKFT7xx#UN==@Z1ulWXNo*z&Xsh`o%ump5tH4h}e-S1c$%b!gr~*I9qGX?%B3NNS?(1i^?Y4T+HQh|2e&V_@v%fU{)w z_zifHK`B9YzlI)uuXJ2w9o_1)crrHEf~FQD)l2iQh)8Ifb3$pCA@Qb0Rd3~MMizA*IiF7LcCEx~YbMQb^@y@~x z6Y`x(8Lftp7$zuYX;?ZWnl)CA0k>RX0Nc2ji}kpXh8Hu~pYZ7%%;4kboir2^ zuttlUI_4>1MG@l(Az-y_3e$v0B5E@mIR4rok~N~=`d{TF2ICE0`2`J!Sq z){zX|;|{9#*6OP?86sn26PTPQW#3Cy{EW_3Yt6qbY%Wy28al}o^DBE)lu~%H8_xHm zJ8Nc?S%+ zbw}i(;?P&~e>0@-Eq(ff9HXp;0_kHVZ*U)>lig$SAt#hQv}j;IF~h zGRA*;Zm6OSfByhkHGI3xtK=Z{B4aU!)#${#M@PBNgS}}PPTxXB z_<^RC1BYskOATj02BoDOLBYg{Ab{UOJQirghYPVIU56vP<^%`8{zz|!3q1zlLE$>< zl%w zcf<#kvP5x?P6XjuseqmXKs69KFluN@_(s{w|4X2CEZA*j(&;b&v#vc87G`!Q0KOJ` z+o2Bw06e^+JMNKl?ohB+rGt;g%X1STjZ!v-^+t}Zq$Fz-Hf&)v-9aF)pMcB|t94T) zzoabvshA%$!2%#2nPChM>n_av`G2TPj-HQ$lwH#?v{1xa0OT8Ko)qb_fOtsDqzk%3O8Gw5b^oc1{R}+ z$7EsU42L58GMLHEGQ`X-9=eWrwX+w&z?c}=C60F|Gzh#=e_M6KVco2%6>J~MQUQl> zyNL=-k_=*_Bmr@)v0%$Z2s*nHjXHeKyh9KWcfJ6|F4V~E{CL5eV8o@)OakIcOiMgK zH1!Ij3jfa5VmE~tDN@_4e?b@q<2=sxDmu~3)MSqI4F&5OZktF=wYgNC!ucw)6(~M| zeQqkzDS>OMnEt>#jf%A`Ant~_Sk3hNZa6in%f{M6{`jMRbUR4KV8uCex4aP3+=9E1 zd_V@>$2JeoY_amOp`qRA{v!GsV(L4RB8>0E;8r!pu76PBW7tqw`HqaEMYeJIFwr|P znoYts`CxnU$vgcAo8p64IK;n6Lwn#=tB4`P%cegOA)!x{-A`VOH6@wPx~ewL^_N>^ zC?J{=%Zb?Tqf${1s-GMs7cJ@YM8^^ZDm10DG+%e;MT=}!XRuHO?&WTF1dprIf z={iA=>%=$Oa(LZ62$b?29GFpSeGHEKCCqN|bbCL3WfUPFtbY^S4<%_-G&JMo=(E!y=LwWiJgh!u8}A03iACn4&kF5!kNS#UOl@G zNg=gG!Fuz&RaBOQNu5lAvRM(OHGK9uJ4=yN$GEd$#nWfO?E^haOzS+0neXm91tb$l zxOiP8k_l)<5=3T@qx3J<{6?oo1+RRTP)*(nj)1qQE@wShw)-kgJXU=X$pez zJbvOQc7J|T)O3zigacVJY;nHv*#%0W&y>ZBGzJxN=NO1gTw8+J$P`4n^xQW|N>^fAzLbrEc?F-Eh>RlK>4kaeG3XFaEb|0& zSUTXLQ?2OLjJijfzK2k{C8BL^JUWl&a8A zkQZa8bmhpf25G{O&4jRwkO~_%>kq>_`B;C`*~qPH=OK)HgZEL1bc!*w&nJbGw|v&C z;5g)-#Zj5RS;=GL`|@8OG5lB@$*jUh{1wJq5zcOVX;<7Y=BZj5Qv1)0oG^iP)oD~U zXuSfSjW%AV<10D0AKS*yBc{z~pgf-8{_)c4at}=LxmX`$&?Q0!2oc+lOnA(shgmn? zEP5EIg6a;jK0$RG$77iHB(q1K6?He<`JN!o1M%SV`V@ocAQ1bqQKordt^?Db=9C>& zwoYJ)x*MdP$Gkf-zPrY98^bSAOva|$7q{XfToLmPFEp<$%6%T@c;k!@>e-=!VEx@Z zKa1o=oWwK*J2P+$d5brjo&aqLLFT@+`iKa=sv~hn~KRB;~Z{Pea+8p ztnNRUI*DES0x<5TVH0{@MuxP`$TyEwOx;&aZ14|7cmQPld8viR?}birBR0>jKXv)1 zYoPzNtO)q}z3zRg=+#mBZT{lhT;t>B+8~1b2m`J6y6yJ!6qIwFh6o!W9NoLilmE9V zAV9uDc$s_Y%@t4Pd$n1pEt0+&r%a5xY&$QyWoh>#0!MmefmJ%TJ=Fg7#+|4nH@lWB zDMAV!FzhpY6@TB&!+`hJHwcL@i(SoE3l*2CG_%kHTNj=rv?aHOmMmQrb*fA04>wv! z+V%+p)9QWw6*~{btHTZcJK=brYUhZm8g<7u9^!}&KH>1;xNGq5qH&MkCf;GY{o;2n zTsIFdsYKWaV)}bK?T*{m=-%xF&lY@`C#Md=$b9?8p4*qfk6n_t2X32;v4SSXZD2_T zI-wQqAS12+=G}73(Hido%EKJ*=5j{~v#Y&wqxo)G`0(mY=(Nc|^y;Iwsnx~zqr{Wv zAoX3b5|`>kP=l;c7>;R%?QsaqPd0BEfQ#Y>c@y@Fy`lsTH(LTl#5qrm`9;+>g$f;i zCuJc;S6d8jf{}+b<={!&keAp+HaXC;6wM8dyrN{47R2f{Z(e`PeMZcWd7lCEFulI< z(EQ&*&CR!?@pm)s$9HbpqhDLvH)DI+H=6n#;4jul9g|&Gm;HfBH#AMgWRX(BlX}gz z>$1L!eocZdR{Qr$PpIGb1sYOXVBLcM3G;mJ;~qgU04qW@k-;NnM~>(-;z<6tW*6(5 z^!;|z$`wo8P|*9gU0wZwjn9MSZ$If%19(eXs8CO^)d2v{R5XLC1GB7Y!RfggqXL5oW$NivpPUnMDCW zz(>kS-|rbke*Z3u3ICS2yJvuxdd(nQZ-~Wv8&4xKeKyhHg%S{37JLj~a?4)QQ;bHN z;r-x*izK5o1bwIc#39oJW(z#O!Ty`gTx0)!&xr72250|Ygs?xgV|x#0i{Ezt*^J>{ zDQ!sJbMg*Vwnj}s$Y8B*fki6~rw?j_HLPAr6e(5Lu{EQa#LP-!-CU_{iEPST!ft|Y zc8~Fmxy(eHyhRD3w6>Q!DQTRaon4=u)x6nXRUEh$eoS~IxsOp1ouo;WR6#J)q&_X! z4b;bWj2xbDZnc#BwX#dHlNH&c*zYbb$Br>7wx1jtx}6W5`$bdAlz9sTPXYhcy z>`%*RLxN0*8_^!SOnw-yW(hpE-cK$9seKV)@6jrdi_?tB>y} z`LEaks;!gjO1E)i%g38KL(P)*_j{ek9I*GAu=K_;Y=f4CZKJ&W#Yv9YjqOLPi|gj6 z#~(NgqE)nmzb@;SE)4Q=TC-{_4vsKiJBvJBav%2vb6^lfWOABPEnmKQ;+AE=TjrCZ zgVQPzgw@Iz@{@@DwE{VkIPG*82ytuXC?Fk@T!@1#EXqlF)agb)?cr26i^UIK@OuK( zXrKTU95FLGG~6QoG;H<$!fZG#r{#$-;H~OQ^nR>+J1?^(GK&R@*mBOl(6|fGZ!bIF z`Jc!1Pem&xf03KiMRmt0BG&-ZW-DYaFQK6bPEu(S-K_Ze5HBduxukB04-yBEhShz` zM=QU365MW5boV;2i~4DMXrGAW7V*PM43*YNiY7dZ-uE|hepl>tRjSC`aHcu?pC0Sv zED(3xt!WfWR^AiA<41cQO>K55L=uBz*Hg3Qqz8m%0uQt}N{#8P^8&G-YqInc?UV&C`;6nQlnc6}r=ksR}`G8;s(KDPyatji#B? z7=h$S@H*FrRJPiORS1Yb%C}(z;$YFQVZ-6WA_A3?yZmny?d-)ldALi*0w#uY+D#pu zW=teF@faM~A-=cyu>--aMoA)trmOW%Qw2zNAm0pisg7|JDadYVP#s zXJA~Wq)h(awIJ8%Fl_Z55-lsV1^9EF*ELPr>+Jd8TDxbL2UuxCiFoH}I1%6<{u= zX|VdCh8RqqrJSHOgIZzu4%d|j@ID(oxTE%8@Q&<2YT!!`djeSJlS zFC^H^Q(#sVM>I-Jn1r`PyT$|8?%uN8iTy^eshyQZxpNd39R@Yg!?7zTEgsblKFHwC z_42|5_23IzO@5L-Cy}MTQ3H^8qtk_(_+**DeXywd&()-am2*KK`S8?8_o+3eLPQ>5tm@{s=7-pMzalh30pgOSHMCB~{ z2Suob@I@Z}G2qU!Nv6)0YL@nbTd-Bk560bN;|kJM=Se@0Odk{ z#X*zPoY+yfLyglYQrEN*Ek_ehf=NNQcC`PKX=_Z_(6U@WGpIYdnhIJAnF2BWs#h>D zCI4GTCubfLwdYnm9~5b?>7P52v2gOP(`l|`8tP2(xD|Q71ry^S2eiEOi!BnNfkcJ}1bd7Wg2XwpZ_oX}pw%e>1bt$tKCNhLJ ztCTQLl#WExTPBXwd?!`SffSWk#zq7bB)w@kC=nFM{e5k^2{EcJR0`2wyEZyp@aBTy zx2H>Cvi%KYwV6htrcw<#q~Z^rLckKQu?%Ane%nAyXrDId$r3aG|6y+l>YSUPJhdz< z&U|@aY^U+Z(gDsZNY-7tcZ)bU)}ewrcuvSJMR{QKMW9=E3l$S%bwh}25)O!hTLC2; zn<<|Gm?yzmP&6gpY;LTi~>V^)GXMFDMWp9-v= zl)Mo$x(}M}9qMlQU|x=maBg+S7X<7ch3FadOJ>aqZnDU?E3$nZ$ceP#Y19)a0mxV5 zNPkr!LU9=E2a2q_^JXa=ob80$l6Zsdk&2}L*5e10^q&MOayb1cUw#3AThatRR*4|UXiw8d}9`tcopEi zc4|>e$oE_bU@}hWEBcz*Uentp9&&OPO%vHBRQijCW)igtf~YRw__duvG3AjHUN#qk z8)GW#jO_VNj*||MaUR~e<<+AG4Rs!j6DT2;uNwBtGebTZPeW6SP3*$U%e&UL$I0u) zLVppmmras3gUp5R8|1K@>b5db@Fd{`~;oWv0B9bxHYifl>HY>5xVg zVG-tGTdX!{=bSpGr#>yYnnKq^NL~oteZ3~$#o?_&Y=-wm`5bGY_4o{8nM&j*(94@0 zrp;413A017nnPvz>m-@2i7%po*x)n(5uy7@dr2)nQ{;*no6qWE&3WGi{`fE^`r)CWyO=V)aK$cdHm0mv_%?-hQ zP#6g~T)&Kqn)4haw8P|(=Fpg_$6rWcO9|J6_0>@eVx3BX{2!EbG_K1S$t0rDc=hHz4E z;dPZ#pmM%Z#f0TDgf*NeV)NpD!-s!)1?-!B-JG&&B@OdF7VoXjcdnc;hlA(iugB(j zV&SpWM}gGzMULIYCA(#g%f?~a^AY%4nTNBU-&QhNcQZlnj|gb*j!VeA&sxBOrwG(Y zlWe#mmf7iLj~a$*y?EVAX&9OV7xm17jI#D$yn}XYl}zzQd#f%j=N@G*(j>Ocq$xvw zbw!OsV)$?`*gk#w1~_hjq7*4A(kkz*6%=+*dPSEpQ|}vTiI>%YTVL~60HbY%3YWkX zBf-{hXYfQzJffKrYd!+2`np^b)iEif4`SNG+9ohm#-!JJZ9^BvFJH+5jy-Vl!-rQaom(%+ zI=n#1tyvV;Jz;hO51=h@r0d?{8Vetmgx_?z;l5x3hwVY*Fu@^>grx`hv9TW6yJn~J zpE|RzwEizFR-skAIQ8O#`E)=d*Ycxv{c5Ii)34o$)h4!ws`ahak4p(9TZpgmbf`wf z_7G%xipg-=5owsy@!X~b?GHAZJ!mitkdIi#U79~%BH%lB^X>M5asB+|R`c-n4Y8B7 zxl7A^;s5a0e(fM*pl}S>vRD7tl!>|5hpDM~M|+1@tf1u0j6k5kHihJ)M8jL$y38WaJ|Qt(w>mkWb&fF6E|6-JUC4)ovFzY=0GK z&}hLcK~w^Di7Mn({HcoYPJxdQr?8lS+_ zGL3Hkyii@tq#x`M$BSYz=DQoEY=}BLj}=+?AoKI22ZZ}a#`lo^82HTSx_Zo6?NWP$ zi%r7`cqo6Z2@BnQyx*Vb=pU@g3kt0*nh2n=JTit{zq5}RrKX^_wx5{yi9QNzyc%^~ zvwoIAB8bGrCo_Q(3#wZ^I|oLBhnRNx)xcS1#vWO_p7{I>Uw->0-;n=3Yrg3n$A3cu z0m=SIfN=i*#~4nghK|M-{~0qizkkL|lz+In1%*0dWrWa?M8E4sKo6@$=(TSsiqf`tOnWFA|^7dO~(RjGcX(@;J{}iSFU9F zv5Gt~f{(Z1MoK%T_#|zqk!{6MdBwu+_sUY{iy_k3Fb-eS%gxTk85Nwu3zyRw2%|_Y zOCGkWZ`PX|d?9^7vV^4zS5zBtY_~NQb&tI5ygg9t2+G)kcxr6L4P0##MHR!-fCJG=T7CkhfNnGm3!R#K6jB>{&@@DsZbY^e*!szw5`)D#MnYj`KhB zD?m86H#!izpO=T{l{rcElvBM_cnGXDPTwDj2nU*tA(c2%bpbDYy&&sVn@JY6@){HnGzS)k(r%;YH z*3m&tuDf@RgR#l>RPl$86qAGr)U)v5Z-7{MluEeZKLESZ-lHM_*1nGJNi=Ym7>+Y~ zS<<96aD*I^ja>Pb#$d!kA!m|1y_h7oBEOZ&?Cmu31G0g`pA7iks^1RT@BCr{v$b{6 z9Ppx+Emh-asAt&Oj3;5DXmuUvv0c8r{q=bK(+=BKw?00a&V)`k&CXKh*mL9mb4(&# z&N59BRxfXAj^I&m)IS^AkDZ!ipL#FXdsb~^l{J+Eiq>EuKxNIpnotIjku8Q0i{g&x z`1Fbu2T!ObJFeom6D)37CCC*GSHCa%w;vhU+h1q4xd={nb>)SdI+N!XN5uR`+Ow+~ zs_7i6(R)cGjH}+BdMsqAUEf&&E&o72Hn0{C@T5D8n$o@0CGc-FO3f3LZ{55n9ZPzQ zk3+sof!+Ym=y<=7Gc{)?;rw-bp-+X7h5+o`E+)X#BEwXUg|mvI3}y59q28qD(PWMZ zwqu*jp5U9y8=h!e)I^m0F4@Z3zFOY#51$J-Nm7!zA8LwG0H=CQ2Ap`@DqMK)r zY5y$w0g4(FlmP^z?UwbY3FKf$fKE=2r5MiQmYaLAy` zh=@-MX-Os1WsZSDYr9qS{df)T%#444o<6Ae=v6{D0Eq1mLwm*tE?N*k7~MaLMt(;8 z@m!M(c?u^lo1l=+m|FymqopFhEgpZAepKzZV|9qA9XON(k zcK1{<%dF)(mMM^Ff0~k}Lz5cUFc)QR6?gkZ#q6$5u4bAF;X-m}Rb-<`pQ>zVad<=& zY797nniucaSO+6D!7iMg+EfrNuMeS?VH&0whOoRNM1L^NAK$FweI%h+DeApQ5|9&xB9P~r6oo*D za->I`0E6Ym0)v(!KMH|5K>38CCq@*yo&=ZxXNrQt^Ka71a9V<7gXmg+^WPk*KYR<(i>cEZB zNfO_hYPrH!BhfHMl^SCC(}z;(NN0g_UHGDzAAzzH3DekOwLm1=#XDr0)g+|+sAd-0 z6t3gM`p^nDWVi+c4TLR?b9ar8jgGTbK>3Hk^fwCO;w1Yl-(|<{194%~kJ}&hiWK3} z`t7NiTAJg(ns2SGBS85G2DyQ4%836`m^e!DCq6TuF!a3y71PlRWtht78;xUAupZre zq~aLY6+10DTwtc;m#d!cLyMsx0)$5kus1PeG&~BjE^z63cO@Ea(JO}f_$T{O`py-w zS@+a_VxNg`-gzL89RGwc?ejk3`MGq?58%%#bP1804l@~iAH!JC6Uri(XnlYokB>GZ zTfFFL7UD_es|0@a2;G=?4*6j#=@=c-d4xuRG+?hUfHFeRt5_1=qL9&$I^?Q){Sm5T zwL7`Bd=8CSjTx?Nt!*z;kSZ25SD0dt0!5itUZV8~D#%Gn4}XFS61;5?5qaFL{a)1L zNW2m0z?BN<00v60qg3mJ(TbCdC~oTxv&hG~d0zdO+TH!xWqF0cN?J=sr{f!bG1*^8 zy2uvi48JG+%N@FSRjo-FFJY56HY8*B-acFL&SKOpn(lmt|Z0 zTxW=n*|OKg4FQwdY&S58nUW)m9bZ6nS`-3o^3@Gwb(?%k=SdJHHtAayvV3V~9yb;3a>Pnp>Hk(f%?ZDj{ALz~K-=9tdCKwYq;^Rd5iEHE^gq_>Ci&mJd%i0d_ z5MMvbXU*7Lm`Du=L1;mu&*tD_8yoHTPhfvrofXQ=$hbS(xFjQeV@)A9Th>lP>6RR^ zDoCm{QCvbTx#=ZPFRS1^nNTM0BwNu#ex;vsqDQSjd>{Qr`u9D*>-SM4gq>_0aJ5wc z?spXK?h0i4Srd!3eR%OWrT8%cqpOQpveoAM+3e*v7|b-f;aI68k7+x6wV6mrm9&6E)(|ZRY>z zPRWpIOE)VgqzwPVPge(@GKY@y;?j36qRVIm&YSs{fZyZGGbQ7TStWTj#LqHeh7SDZ z$GO$5Q?56KZ&9%Sv{`NI86$4UE*yM@D#CFZDG_V`_NLiF->)GmdMtBM#<^Id7XEPY z{C?5aQylT9ASkks+P9STUjIDzUCJZ{pproCq8#W_BA zoa9uuWSajn2kK6SDK@;d#+56V$f18g-n3tS=D!TIxLgZ-PcU|O>v*qV0=l#r7y8^( zdp?RDY)`!O5j%!RTuj@%S^=DA99{C5EoZvz8hpi?NgrDS##%8!Jjqs5D}QKL zt_hmC9+>$!NOR|}nk~Oo8aq_q$8E4B=HO8b$>VI+Z?lT;Tof+Mw9|JVoV>x*_ z)>dyY&c+Zwj&H}z2f9byTJLXJ3}|v_T@kr#ztW8cNA@4C4U^7owRtfKUrTtaa6sM%y68eVOmIy^fSi!=pOS zUkGMQ6+siZo;$i+&jF-6!6%ea9edfAb_*y@G#>iVWGgu{urrhhMFU$amdy5+k3W>H z%t^bTn{uwWcEXM3Gm}-*pp!pY24Qj3wk+0GvJ%|_t#(`=^XlNXEd{WZ3(b$!Bjy_G zA>F`HMgm@ z)BeUYY$%)OI)^&f>$;Ii>6g@Ds_GiCzLiHf%p7pqM56FXsf-*k68m_+j%Lwsi#Hnk zaMEHX3Z2X{PQ2~K7Ysr!zHaAXuJH`|YqQtZA^v*yYE|`5uBSaw?nmCV7k6kq`)IuU zeV&2tDIbbUTbQ3SH8EdgaF9NEC^)#~q$bdAG%iE6WY~n`jho?a>k~lrA}Ay_?y$N8 z<_~cT@z|UIn(0O@<=yX|5p1Ww0_wF#WVOj1PF>DE$yw25_X-X`0G>P?1%^;`pmKD` zr(oQ96r<{*)ZXmSRNLN6}J$=N6QbL#qtDXzu?B)Xc? zj{sz;7~n$OALJUvf)k0D9|9tO`(%VitP3;PYYMx%U+jqspeX6Op_l$jOn0z6acUq3zX~2%0t?+G}E9(s8T#IR}3}!e0&+5TMd*JXS-9dH4n<}fWG=FR1&*qOJBr(?v*GB;#qYD8w0BWu-hnvNdX*^v67l|`4s%l|oP&%YSr z#Zm-9nxmR*!Inl>}UIRd!2BhZ^wKQgYcd|M0Jo#va?b_OFzs(i&O9@3SOy?pTy81 z?iKlt#59=+*qYlKh{%n3k83L+n#Ij|GpvE0dEwuuvmv!vE=9=JX-n#T7%jb_3?zgZ zuGftd8+F=~cSxD`$C`}B6hTFUV}k@7Z+_bj*azXn#XD5bdsZTJYa21s6zo*tYg~@z z_FhQ;1wk2pz_U{uh+}5VU-mYOCJX@gPmw{Ad1rO~72&%FVc(isSifa61`WgA&^yz1 zdOsFHHoL|WPH&9n6MiM^V$BKk@ffcZSDdufOJAGQRKeSlbZ9eb{!i+GvEB&E86}A# zu;0+U5!u>gn~bp5Q}vb|=u9Z5p?z@^5g<~Xhocs5iysOS(9$bBu>vKhTTM3PQsPT^ zs(^(hMQsdnY9*6(Nm^+eY~6<^gdeUYJV=40g^fpg%aXDnVJhO(i-5pX1|9iPnWtUq zgK$!fEq}sE-PZ>6Nzy`w{;GoMOlyKrWTy4N^sK}>rdSBlYt$Ep{ew-vgJVNl4{pV5 zP2t^Ml21iygl`dj^<3e7L3xvE;CF&z1DB~LnDj&RWx@gFC(Qzl21YU@^*4{)I+pGK z$9A{o*Z?auQ_hvczD{$1h5s@kBfdAx&2uAixoVI%%6B_dmVZ@-Oa|3#tWKFaE@ogU zvHKT?r~st_qd|z|ReVB zf2ayA3*?@z5zWzEM!aB=a_EqVD*ei@jM+<=icFOc37RW~jk$uJ*>Prm2(TjA$XGIJ z)KOm}0Z^lzcok2*xO3@hpbQC@ebayofr7jBPt&|;%61k?8@Lk{BXDsjq3^R9=-4=B z{bOras{d1)RSQlBG^a;3c0w||3Jj|TOz)_K4WnGFNNf#+=V(AF3Zk`D_YimlD(9+e zw@qCKVN7efgiw9*%cMJ7h!Q#-OkWW^UF3Aw^pH0+X{L!w56u0ju^ZIdpdFb+qPK(q z4(1f_fC+RK%)*P)d@AUN(`DwBag5+FkxuIs;k8ib8a)PCz!Pyn<8$BLq~T6CE@TNa z9D2YqH@`xu@W)FF4DOHbl8ehg&HEjb0#skC2P+)14djTeMi6;hz~`_fsDd1Z^o{Bc zsN+(n5GD1&GFxGiiK+>A^eEIk8j&(!WGUz-qtD&WqVQ3|N3d+J$ah;CoN`owF5yhFt;p;Cng&jOg7m{O5s~C{vYcZ-l7&TObQPhoKStw zY=~v*JO2Pv+t*4uu)Bl})JqQGZ(Rvs7bPyk7^BNg>j>~`l3uFK$q-rP#*s9= zM#ii%LDij(r2T`#fz{Butd8|rwj`$nqNW-rc6eAAqS~FzK8Apk4ke~ia~KNbgknAU zM2K`OaJst&Ax_TAPO73+6>&%!PPVFPT}g7LT%~?Zo-|&;I#0s>kKyEH5+Y`x7hVFo znEu9?g^TCD`Z40e%{&+6odoiI7OeyXOgjH^Ml(xf%sMD+j33FuH_0(4Do^~SJuC*u z`X5;Y-HeBr@=7Q=F+aazCbGRt_O{ppCCyUbO>_ud9isx1A%(X*MRPp1lA1r>1lRV? zX`#0h0%H&_nCh{Knc=%aw#t>=2qe4M&4cL$4xe_Zj)5uhK-lDET{AUhhtkvc$F&O- z>{S+lR!P}4bqZUlneqO-hb48bTr#1%+uh*5Q6Nu!JhIdI?!osEvYgc*G$iZCRd-m{ zUMMcY(w{uiLon!y;N&NnFF$v4y%#lH%96Uv53Y_Ffv|b4_NWkxmEkwC(fN?Fs_N$AJU} zF=67rdE#^)?)9 za?7+lVCYNwY^s;UMuqnBlSqHGD2-b#&Fg|dflIcqHv_@ja-B1+?euiA1L(Ej8QGi+ z)+D>&5%qF^EFH`E!q_$IY{yLb(`Il|q2Pt{PJOJR0BvIywSdO8SCQMgTS5X8rK3 z&9(KE#T#?teQBngrH$s~r_!TeVJ8NoSYKvt>+)88;6~x9J)Qq;u57%COYD$kG}byN zc^x$X<0UhNs10+ue@PBILX=SlCWh5ojDZ!VXURKUlImSr(oM6iZWk3#m?14%k|}Ez zNhEc)Tzr1jWJj@38mXL>%b3-Ii46zSit8VV+GkQ{-H21xQc3rQQY#MKM)3VJTK)(Our$ zIj34WOsp&fmiW5TnFd@M1C|}+?+RZbpU;?E;`s4yKw^eEIMDBS>3L*qV+W!g?|ZS* z+or{!_T6cpfe3pDS@vLCC^SCK1UKR>agZntCo8K2;ZXCNngW5OV!6)+Q{6)T(|4kg zkYxK*xLhkvD$B@v9^shMLz&%6A2{-DzPiDC;q!uw%Ip;hrgzQw)N6)kwe{ZU*_lvZ zUZczXV*l7L6g1VHUtD%osGH!nhDM2XPo zrL}1qU*!WO#8uv?3;(9mNY?JLj@gL*GPc7`@i?ytsnettY}%*eAlO2wDi^~cl7|I9 zZn{JlThk^bRnwZyYhL=4*}p3t>sxyXG2m+*_pimWHqnZo!wAPjeI3ncTxa&+()F}2rUuWM> zU_ebZ1j>L$%Eu~vqHB(oU9w{Mq%yMv*CJ+;5POSb{a4V-%ZeF6NO2)8yzQ(U&U(Uh zkTE_x`KbSESv>TV>hwu=Ta`ntJ`+C2ij5eaU)i$N_(@tO1SS|Bpam`%EhyrKRP>@W z<&zFDQNUp2`Fq}}F$kgD=*V%(AXvbd{sr56=z(M#EG_`0oPKzo4OWM|9PT`|>9?Fd zSn~J-P;QgH%__stgFes)yVG*Q#8$bL@(;Jfu1j~P1H$hPtt$4rHT1wW* zgVHN*O}J~^;N&h~OO8%unS_BMQ!yoG*Y_lsTETG`5~zbGmSt8PWmgi33jm<3l|* zl_ZJf)(oBQODyvdZS$w##Eii<%7F16J|ee~L8^$MeI;=V$lQ`eV@SnA?+;&MZ|Q42 zJiV^b$t5%}`T%$I*Qwc(04&rRYO}Rr*RMpW-(u`qR{J|}faPSG`**s(0Z|(k14Vya zz=?DGemTn~B!JPxMT5vM!w6kXE*qwC-E23W*y(ZkB)(4#lX=3I9dc=1Y>z~TFwytt z(6o3BUjp9uA$hPh_btIr>1rtBIwq zKx2^4Yzm80sunn*^LeDj7%$CqF;?99STlRIGh#3a-Iq<+mwyi%O2GO*B*R7x>Bct) zk*#i&rRtZM>J&Pmzx;E)7k8Pc6H8q;RGv081LQw?FzUWxJBUhy>;rh-y$Kg_5w%aB zg|#?}cPx-b4x12da>@_v26>W9b6}gUKPYHH*i)v+6BVLQ7_8YV=yE|s;_u&RR_)d- z;xT@yV{dmdgWEHg_u3PbD@o%(ZK1Bj&(CJH6QKjq5(bw`mkuw5`y>)*our+~B#Y=^ znlY>HV3P>v=(1W!nd(&47qihNAgwB)3xy_OOF1q0tS9fIN*u!sx`7FuiM5+2@}Ycs z?#Umk3k$Qt{tGw<ckTWl(ooJW+|Di?=fB|F8Y-`Lbns#^(w-7&(X5TT98H&)#QmHEcjKFnlp_!KTCbe zXZ_cP(b{2HmD1H!CB>b7ZU+TmJ?yDHR;7_&LH|bHr2+%x8erL$1f7*ufDOMG9TVdM z)<&(;0?$Lc3PI^L1w$cAiwH73<^stMxfL?FQdsI@Tq93YZl#ha;7+YaHTY4)g^7~e zf4@;EeY1L9@3GvjeZ*fT+P}m&b4|3DUf(@vxG*V{@I2{--B9F8R3|UZlMrD6Ym43T zdzp0$woaX4a5Hbt+hGFBVP1z@{1YWKiC8}moJ6}ee@YrgojZIXS9Wv#k zGZ*-H+Ojr#<2pahi3@_N69gHTKLmc|#_rE>NBKv6nM2jw+NE2A-Oymg;+|piMOUIo zJEZRfpKH1>MXTD-XGM0k=^t=v?~3Xns;aNfekv6BI57agpV5F`E5jwPHvh-uZcP$W zfN(x#&4FLNn6aq3tr}8DBn4?sKwWjg7DV|cBh*SJaY{_NjZ!v%p<1R= z^>EU?N%AXx%H-&fYdNtkDz_p=lY|L2on5I4Ccy!#>CUdQNrLVrQRexcY?)Gc_HZHA z#y>v+^urY1-<53nqGG-u- zAsY-uU98?(kNegHl?OvX)zbh`g`_eoHe%buu4Er53|{lF&bZ0^A}uKMVrrg~MS%*f z%%A+kXAafSM!(7u={VGcy?-tFQLp!)+)@uDnDT-77(Keo*K&(#AUGh!j<7lttSp;c z3chC^=4sdbToz2xJ&v&%<&YqbU~!gxHE5KX>p8f{{x@9+`pd`Mk=Jh2{`Ji0;EcD< zC#sMvrWsXK+m6wl5_ZsUmQ$Za{Z2=iNptIsXC`3SD<-2(x*9yR&Q;S&U`Y$13!GMI z4QlOd!yf_w!&aiGNCweN>1GU8Y}w%gmuV=-?UnJ<2AM&gunkaL05`K~bms&Su?YQu zg5&-Ozy;0hn7m6mhRv|!^I|WbeawYwnxjE|dHw7=SD%WVE{9}+u z^jdo^W|rtBG7Berj2;JfnEaf?z(cK`zHN} zZJqRbQf@I4WNL;F8fgDfiGQ{O-_e1UJ0jlt8)KeVmd%TVO_20Krege(8UK!TCOZCG zPv&KWkAH$=fOEfMALsjv!no17CfuBSqJRCo>T*#(cZ_-EK4x9#s@UU5vDExjmN|x8 zE*#Gv6=%_spMW8s5YDbkds~iU&8eD)!=k1m-!ZNj{EEWN%s2FD!#*t z`rsezCwzV*9Zb+aKcyz6#}oL;yZ)fH<%f_FtQ;epz8UnJ>b-}Wy}uXkgLj|V{y$~$ zlfw_Yt|GgHOluJ>_;x=Q+v3NQ)2rXfv-Uru8fhBu;m^DLS#7HNcoX=8lx%s;v{{*| z9Bm)q=_Y0f`goJShnoHxKi@@=FE7W6KKzA!%%1s}&#wyag80NtHf_!hbR9zzfMLVCh$qrD`NB34`kQ1efU)}W2V zy-<;B-$?FxIoD{vN&eC`yG2wa^qu)VIV4;GpUy9#{-dqQ2lKtYuk6+n`T1|%QOS>k-(Sa>cO2O7 zpG93(34WRSBjP^F8`7~=>aS6{-22gs-tubhpLtv7-o}WJkUpov{(E0^%I_3gC#s+4 zTo|HGu;6RuYqyo+A)DcyU#eK_{tn{5qwScX;=S&76F7R=uljvOcM`s~R!n+}FEvw{ zRI6Kg#)e*gnO#?$&PcMwrN9L-@1*3BwdpfkC)KR}8S6}9_(#X+ivmYtA2S0_@71w1 zhXgwn$eYw`{UoQRD-ZxH{s?eWPiEW?C$=epP_G0`DBK42W}4(Wgjou)`UXZ z%`1&nKhl^7o6!1KXgFlI#xSHEE8(5&z!z z!1*F}dVc1y2uT0f+q2n<20;Rm>)UK>V1Kn1$lGIkJu(EZYE6*Eu4j}JX8r_)wtO&_ z<*}aHF3e>N1Z#z)YISx~FGe9V}}2& zb#t)VC^d41f_2pXeh#($vJJ6aYAwa*>JuZCQcZFQLd5C;pgyMP@sWJN-?`2EmFXvg zWrnO}T+61R_LkpH`4ht5L=@Vd*}&#K%uBQE%Vn(%QCnkUOq@MqcY(N>#e%K>*d{Bi z$kHR*kV`WAd9Dt8qg)E1H8iK|6?bpUH(y=Rz2*rJAI4tVX0=5VIV(K-3{N}tj@3w zxm4Fp+qz`as!{d=C*#q7_0sWf!rUBS4JwIWVjEL%{kfyHYQ}$z=;eO1_%7YG5>jvY zzLib|8THoj@x1E)&Hul|ZBz7&&?-l1BrY%jzzZk<0O9|y_tST_cXcv0b*48ovUhc% zbMbI7RoRaFXUWkst2}j`nD!J#4I-|UCb|KIz(s(dUKYnJ+vRFw9G`tne2>$Gk)LaK zCPYtVYBcNPpL{uEUJze)x_PqvVV$K!kX(LBtc~ARb|cccN;UEJ82ZwzU=qm{^~ECFJ9VDt^E;6u5&?Cn`=yauz|2FGGf>ivyiB+2oXxQB{0Qgg2^;E^A24 z%$%XftbsqhBe3%8@es5)Z++uz(xJGbhQNvX7wDd9fSyb$EaEm5FIM~Rbm8jnq-KAM zWtO{GBXDEyHj8B8d4qw28}I0b-|NRBHiW|M->0Nm#YeoUHmC|A3G7HI`+@WK@6u>L zIUL3D8wKfEq2YQ@cX#K{{Y6`9cqJCV9BXQvTqNhCi)Vbp&X{diBK#ovzVP7AYoTkU zhfgi4D0J;wv`p{#nHlJNVC%Z7rkRcu9s;%K1pOW5hY*o!b{zUfqHIcyyfsPAj|`!z ztpvoUi!o7?=t&6pi8As?kT;6SmOs&v@f0z^V1!IRs}kQm?Bc`&$0)4q(1Ko1HV+WP zJC{15n2heN-yTrCxgGExR=-b^eLLZ9DdQ_9JiL~;0^Je+KK2Dm zgCg;8VCCLwZ-J-Xk@Lv<$=F>GvCImov(fp=q;-&Vwzrwfbahji6mOoWo|2y;E1Jd*W zI$0S;znobxfoP_v^#sDJ_NK?$`+tcd{!?2qy#otw*Z=^O$^Zb=|E;!6+zjoEO-=M& z>>VtPo&Qs1+d8(+n`4PTIA2kmwRZVg1j#OHmD>ElbbohWs!cb@s z3?7M7)5pE~{M)?2fD+>`Cv^C7sx3qPGV=2Bz3##2>#|xkHfB32oj%JV@?rdnF7;aL zrQ?&a@5WxqS*ofiZp$aC>lsZ`GFq$WJ&czf8(Hd~FO^h9^!%^nQE-B*E5*9nwu@}~ zC8G80n=Y-yESplTm@@WMi+UxtOHColTIP0g&5f>=${ow8lIQM;wGLUEl|5`MrvEy@ zkAdZ5SL6ouWT%l^GuzwRY__{kyMzgNdR1+4$G_@hxL_0>um zZT_cz$Ps8;AB?}=X0L0S#$?xbP-gL8txr3{lPV(HEYBe4@dkVM&)S{Q+w(`FE^z*R zOQe0nt!7WeUrKweK>zjT^M*P<>~F~Z$0P5)!*_rW>#9EQv+F$tLKmJ+a1LI)z8)EO z;kEP$?NI~c2j05#dw`c{G#|a?k(>{H{bg0{`GG}78~(#Q*e#rmWBoxvMI3kx(Tq%z z-VWOs)HpkWZ)v0ns>xMO%3#Sv{p37`W?5)N6EKF{%+0EtAzphJK;&^*>`5gZ_Z1P5 zM4!%7Ylc`S1!D}I%|rm!da9CYQpHITLROKBf#O1h`iGZ^v{K{a2woK$dz)yAbGWPtm=DHK8A@!;Kx1qt`YQ%MCm zIUf?jIy&L;YyVhYN(REi@8r$$ zESsI&cY@nZa{>}@Jkn>@F_9Sb{(M-o3-yiyd_3##@heF?aJqRr81~ztm~gVPw|9rq z3(ME`<@hW(C%IBb(dgDnQjVcD8TUqR5;hJOE9l*i!7xM;HAv5&1&t!@>fhlnoJ66$ zUE02l;KA~DK)VD#br!JN3@MwR0zuu7WW@d*A_(}Tne1!8|Sy0vO!qtOMgT6pa z7%)z2@IqiBZ28!}!rUpW^1Qh+|GNJ4eQn5CEZBUe{rI!zzFc+Z1ox{yI{yne{alGU zW=ymYKflTii*6A`dgh)B*eRxHiI^JO($;qIEMZtCheB z3S3*6wAWC*4JOA$cKPu5Hn=obz_^q#ZtOEOk)1<=!%<>$6vJ_435cK5{eb|9F=Mse zuucjNAfj_(s)fAbS`sKl;ux8I2K$|J6lG?6;Q}2GD3;q+H3NyAB)k(cn6YGHDIH%~ zLpB8v!#ZC>=1*@w9r9u`?#9yejt2acB9eCWII#j)quel9dx6tPUEq3Aeo&{Os#`Fp zoOHR_)vc7?50N^ZB!h|*B+&&IupnJAZcP}RFR0n~L~HA9ecl#G477A#zgo%>Tut0P zVNf|(P8A7Qvk;S8$=Xh3kkN8b@uNa75!88|RxXEiwA7^myGCD62wD=d>4KwEbrP9{@v9VEB9Q?OoEZr4qK8jtnnNWpV$VSz6HY@uf#^>(g z115{BO3fT2Rr1YQJJq}xsMRxjckM)mt=+117`*|Rnqwhwv1~lZ%~XlJcQg+Vct@Jx zA+3{9&b$dtr6SEXD_9nq6DUWjkRp*>#Ybdz4176!W}M*FkSsy8kFzq6KMw-a`fRjO zkl;uOKeXI}OZbL?TcE0fXeDoIjbHa+I}!6 zR&#z?&7iEH!b&VbI(qv`>&Y(4 zb}S=iwcu-8&@}5>E9hhev*k4SR>FmdT!%+&!MKot#Q|L-3Azq3*((b&@BpVg;@eKokdz>u<;F8I&)dI zCIWm6Jpi(-$>?8^p_jY;L3@mp8?dHcU&4UAi(#Kpv^7HpNB=okS;%<(c_Yr`&QbC( zAQIecncjU&;WpD&-38H08F>l*{^|wKt57J1sBXR{7F(e=dH?&rR_Nx-(&;=oZQ|Gu zqWE&=WM5wv7XanUi{ZNg7QcP|_ys%I+Q0t9yT5>m;rX-t;b#*|?5cV7C z*(38F1p0T-_tFp?9xVN15*uW2*qUgIajughU~GuL=z`{)?3Yxboqb;7NHSsR0@q2D zcvtBWEyVAk$=Zn)NyM0>#4tm4Wj1rvQApj7(})PE9|-ad-C^TaYqbIEi7QMZXDyx!3% z%uIppvA(Y>lCJWz6Y%dj^Y8Dx+b866XnJd0V|9MumucGr+)@YJfk1DVay*%tTTwt} zBe)wf3o`Ha*OtuFQdkPB$M6a6{BQwlosJkcTQELfoYC(adl$<5bd?lhWfL%mOzP>- zHPaqBhxPYRu>BxP05P$&|C$qkAntESI7N~n<36y0nS_Af6%Ax(|1#V%nnjLfXaZjG=9!`*vo_AAaHBtb{R)^utkeXPW5U4WG4>;3AJhDv?1flCpNgy<=D3( zF!~3P^r@_R&U-8%Px4+a*H5a{vCwN|9Tu@I1v(RuQ2z*(nL~L{T;#yhV^f^Lsc0hNflLBQ?Ue-GhB}L$fj}B639SQ7Eq&X=C4g|B!|~%w?_#v5N9fyq6LkX zwvT6Cz$7AsK*-CtKY@rm}EtO)3$EHo~Ejr>#{IUzj4;#I^Sy5mw#Eyi6N7O)#rp#oG35YWfs|) z;(eJU;y8`hSTi9K4w;dE$=#`~jLOryW%e=b(wtv#M8W_SB`4#M3sI!442iO8JDphN zBW~ktQGC4()D8jO-{MT=k=?BA7CVZ06SmllF2wfvh&<%xwlSYZPmClN0!@hOu^n%r z(h)e5ElHn4aFl8hbJPd^_*4K|osTT&am|}}BbE7})=wMk99&fmkjlOnkj{V+cT~4X zu&x`B8zs4pX7#j$%38wxLBtQLHqJ121`HIk7SmGK%_KD6aFI5YFzn$$YI~9@!-I_p%peTU56_xF%#ucv~AbBa>Z+WU})cCYQNDtu<$7IX;GA`=^bJz zkKtsc8WAb5?nyiGl<9KHl?p0tTEG!jYMtSaU{ZKK<#7m{bpm2&y9#PC0>;U{Rn&E+ z%=hV{fM9c9 zWx9T>6KLJ0UwcE0#haZm@A>lRdO*!#w3B!u>+t!gSavqBlgyG7&uxoGXO>DRKaS3pNDcm;abTN7~_l0SGoYL?qk za=8sd=N6e~aWq3vAL^{6D*fwX0YgwS)nmCMJKA>D^h(-yIKSF!$TNc7mOz9-TVxK(;8cQX-$BZ15mkDSPI_8ja3_eS~s1bkG=5G>QmzVHHTV{--AVR*w4=?rJ!Jfx+LQaxYP_)>|zVNT{E;TSCuK0#H?2IPK zjA4I|bz@w;)o9%d7h4jr#c3`8bpuT|cZ4pN$8nqdIV0Xs$ja0K4&(YL95^}Q)hQZj zQA{#`SM93iP>ZB*3-Z|2o8#0B0zBO?XeY>QRXI>sQ%xCui-=7)pq^`kBhHKT@`BKzXV@m6~ymV~JE?op(jG9cUqYy`m zTw_l*)k^P4i|}iQZ|auoCR)Xjz#LTQ&s>>OT|+ZWC5R0Iw~R2$AGQAk%3g>@Xb+bF ziOnFXxlrMpclyMc!+pPjbUjXp>T|OXUP|_4m~7o?e};z9Y693P+_UOd|AyhSFemxs z0mCQ0Ck3~jpNaL#HOPNjE?7>FI459vAyxqjkis0|JC$@snbU&ZuCg-czv1xsTsh15 zHq(fOj(wTuCxUf!nzdYAnI)3{eFrfP7)ptiqz-g$mrZ~PAWr3i&W^t5i81;vPHMW?!v1}vlzCR^PTB$iLAQ0i1r z8`hH4^;JlvE?$7g<4=eA?h!KisOz z`9Z%tYpT7)WPORn5QGSZ>=bkr?r3P=k}esuB`ObVK6eDg6?`7_iKTmqn225)jb&7U z?=jXO2=tBfYW;~tE&LAh_223qd5%KqQ1lzfi5K6a`q}syC~!aCB*Ibf{S?>Dd?

    &goyhxW*?jd+>Z5k?gy-(wJ;Y7jFLs=~_T9cs20sU$5^lXt zJQpVuMJF-fMgnkT(-@O*vA&Fp73w8R-nv9|9UgDLrlT*;+e%HKhkGGv-nh*9d#0i^ z-Q-&9xU6nh!-1a;RY$AY=(DP>PU>fO(j*hbR@b4)%pHV+GcT)1yGWGggf&60?G=WE zR1+^Zyz*&4IvW2419Yu_@_vqFob#FGc$b-5Uv*b+|(l(Kfyu{%| z9}*)<7=k?!w0m&mtZ@H`_vv~1OER(Nx$3sAu}Kayx0tG~yU2}^Q-?YuDByU+AUT9s z)Fmp=FX(|n;v(TsERric*-J6arP~a|21&vUDeaUKfFZ_Ye8O_V_fiZYJWycBa%#nO zAn$Pvrd~wfU>fu=khH$T>Q>*PBNlf2?o4cH5QjZ}z>`ktzMkDTp0dV)O_TgOuqTp& z^4+dJy(Aqm9lDOXeIhLGDF*$HLbKk{iw|fpdQ_GPilQk)n(5 zVbD4d1=tv71S~GD6jJ;^f!BXYhmu-EnQrvJpB@Ud5#4S>@kV~wmZa>U??B5Hjh0wS zQZto^v~`r(_HEQ`ucXu|y+T!#(7tKfMk9}_XI_862&J0BVo~l zj4}91EU=?|#x-tE>@_blgluG*YeXl?7BzLGtq|_82(LpyHn;c8qBWJ6u)-X>39-E2 zL04v@pGO0!UgoLUAqjWoEmG4>GYqbVNdLMSq{a?5dJO@VA@P;#RZAB$CCp;UutFAs zaI!Mca;y)bL3|vA`ce`W2{JIxGMlkhEHAp9`l4^W-Oe;Y3J(=(APpORh>je8>SHlh>=l8mhP4yI(&SWPQZj6Vixhbar`*S7KSdJC<6T`fM1$896!qWzvG zoI~nXJgU9GQyt=cz?+(z4g8=;UJ!$Jm>r*qi?>Xd^uDwg(>ZU67o)c2UCYN`GyIsJ z-leGo7DzKNzk8k>0TgGG44LVzt~q)8-EG!rh92(&^Hn)v=gEI8kgX&B!0#6vJy@ud z_p06vYIkl<@+NRQPBpMW3%-R7(Xco!XcY`5EndP~Zg><}C!h_({^m`{l8v`t&??OR zYTUce(>;!EbZs**a5lHCcQ(=O(#lbPwTNm1lLy9c80tV8#wdsX7<$@b%#=v6l`0pb zjvn2#9oo^V>X#IHaL19ngR!mS89QOw?8%bp#(em(R#uP-j?(9>iw#iun1=}0>o^L%ne$HGg*T9*k%FwF~Db$Y}e>*Y?bSP93MizV6jPq8cH zB)R;Qa4gcNxCHBcXH@_K=(dvcd4WKwit|HxP57-YD8@Z#|2R#X@d-j_OQq3cdOf?% zT)ySppYu7P`Z!vZlYipg$8Ja6QLCSeLT}!}%0t9NfXy!X#=R zS$8j~qG9PQM_qdjMUgK9>A8D=e9!D#JwMcIm<}xAm^)uP$#^8v>#r12_6j+zO1~%`HU; zapf3UUOJ;8N(}q#lOJC!c?m1Ob^fZVq7tUz20Z^v3LLp8T~W{E6=Z5 zU6g~=+c8}G=7F(%s6*gG%_bX7X}@(>59x0{rNucem1zi`#W$My^YE!3W3s-D`-#9((m z&7ABA_gg<6@(T#bu$&IE%{#KXkcuTjlyK(CZd8#D(F^9YYV4AeCy`$LfT$dBRyo!8 zW3e4b9n@#r2R%6sYM(8CP{Z>jbjJ8{TCWVH_QVT(dnlV-pLicH^H6-Wi35*l9PiM& z((zozI>0MG^4gQO-8NI(ZElJEN)H%D4(UM*lH=Z5E7Ux9{jZ8n%mD|`fsQjTfVb@j zwi|2aJ-tSMZnAq^^x}nF*rEK20O9(Z>oOrb*a%jHY)Ek4Hzn-Muk54}lm#^|2GsDYJUKxp^=?#aL$}dQVtUJIXsd+;o$=z7!}M5t5w!VQQI02O}Ak2yS3wh|xw~ z4e%3O1QgjSr!+TE_m(|&^XX&17QPNEP_zaJtxsh`;GRtr+10xToOlf23;BKGuw8za$(uCL_A+lU^DQK~bHMw}h`-rU ztj-z#Li3z@;pFnM{(|g^ilbPde8c}QHJ&1xCA_-Sb6x}i0PutY06_X*=sRNzLpyU* z8+-Hr5O>uY*G?$ysNc6`3O_UqtWCfeYfhmgeFJDJ0OpQK_!TMQ+rDMuVzkdvR{tR2 z1CVEZj(15O{H99fN&!?%C+T*)^Gvt;w_Ay>_=>xDZ7ILF_$js0^pj9CO16Pj)C32s zMy&sC`%G4wl4U2u!7mr(E{ie5Qf1ac)>uX3^td?cSW+BHRT1&~_v>=}a&pn=@$CG( z@$-(3akAR;T77C0CIo*v${1_$3U!W&PGQ~Bo2x0s+1y{Y2@wx*V9=JExReD~89{bQ zR<+SKZO#6X%NA5@W2Tkj*bB;PpDHwHWcrwB$-$*ieL6Ix)03lyjwSVv!o|oHucD-I z+O|1t@j8<B}A@<+~zM{CC*+217Oxf7SYw=kTqriJct;$>$Lz89kdH!_)-?`S|$9sLK0q$7*3KSNAD6{A>*IDF2Nc&*ww<;G47M zWc)p~LiO0^yMy@X=ptMsa1Nh^biVSE_7ew8-8hBiY?0ru^r%CIw*)HJI1~TquA|uw z=$-1W>t!VD!#!oDn&4W?(3qimxwHT)rS2RWI3mSr0JRy7mcErL)4Oh?i*7l!P7^FX zBy!k(eN3A3gn7X-vuAFw#uW&!v=mgl;IpD%w&M-O;2ed-p zpJlkRQh$+BpX+!*7PC!iC}x(i7r#Yf&zp0pos8*yhMnUk_m&b8%OaAyH;);!($D0t<8AKdykU{$^@@@uDin$DATDEVA?5AJfk||8z<*2( zjoR~Htz|qEji~UsfQNuAzWEdv1Onj1VSzhR1m<(B^l1`cl)tLau>x7~V2empc?d(l zstiotQWZe#AL9x^gaiTWGQIDeri3@bQiUtezNFr5EHr;`ex2@evkCyp=$F!r4lLD( z8*58?7KiB2tn-7!omtbRkCXTHQi?WLJM}y0aY$Uv)fPbt{bja+vhj2$i$K}K@}5jZ zG$dZ{kKcx>6>UwqRvQ$P3C?v3ZWMD+Z6+hDiM_W&y;2QQey?zWNJxf5z-h!4^)ISv zq6Gr)-;*N}%kLhF>jE^tr#BKCz!vzYU(C5Fb6YsEBB-s`1c^PiZ)u)@< z2c3c1#LqQkvaMPH%+~usW5dui-z3s3+FXF&-!f1F9fhm zrg8_EI|UJZ1mfrH2AhWZn~e!QQ%0H9!?}v;yoFVX4Q}!CAn)9_K7Z%by*QnCxf9L6 ztH6>F>4I2QmsNbD1oEgAzD&=-b9r_d*iRW#e;SWRiw19d+=BxBd#7IMPFZDG;pREX zO4Q9G0@w|$^P9b=Cx`!xVtU7$-@9n=)wbZ{L(h3}vLiBvrY^61zj~S!x8Lwd$~nw2 z?x;l!@3PKwD0A%YR!#lFer~8>6@Gn9bpEvJkpW4Zpc{=CMi+jCg-jOcC9vKX8zMXL zty%q6pq4O6>tH^@$fdGvK|x;`JDN(K?||WcI17t-jSL+>zO^%LhG#&gd8tUY^6iBc zTH~l6SY$y*h>4iI^8q2TyWf$+#z~k8Q3h+WG7F0yS}}s<8ZR8JbS@e>*XY%#{|bb& zWJH8NEDS(GJhg}fo}S`XhsGo+c(c?wG`QPde9bEIlVR3Ddrgin?*YmU(x1Hk86oaO zP@a{HrDz$@8{CQZL}!rjiO2d#+1}#fotO35M_-v7Jd5j*XPUWxpG!Ab@9P)0OIjFh z&d5P1-RPhqoJUYM&f;KC^wXk-j~Ubh%&v{*K4*F_4bgk9+?)g2*j7v(Z9*qf_N-CnC^r)4aq+s zDAtAr;(?4DWj0ekcH0M1ibW|lkq^>3aN==QQl;dU@E05^}Pav zT?t9SHBC6WVJqPeYFX)BX)DcZug`tTZ0{vCccQ?dqW7)@`ffKC=bREv z!{;4lg;Nv5pE-xTWu&MLcAu$174R}5$eN@jes+Wwr%udlHLPc0#8OaRO8Kjij5ISZ zZNLp>P8E8ecd~;`*7C0>uz$$Zcc9!)Yl2`;3`oI=k27|VyvyLgPNTUAYP39V6(eoFnJ~pTH611Cj za{4omgUkZ?;XFTx?R$ICJMwO$ukNQ;ILoz}y z`EMx4-p<9;&gFm9Nv3g(aDxO0Al`cY&^jVD2@54ssrY}ObpW8@f@ag+eSH#@OtZH$ zt?-R|F11~?1&#JpBz1oaQ?kgWmmS4f{u#5$1`Zk@T09+3Hl>ogJ&8e*X<<60VJ2`Y z)|H*b=%Kvfjuo`(XggJvvz-jL^xyN`4T8EMc=T&ei2ey^5BtA>8ni%`a6bCH_d6f} z!0taCjlH4Ce{fo?vTXb!148$w`V^vqBm!-;N~#n!n!>9}z4yLK=XkN> z<;Dnp^}{XZW=VEX!dSt14DaJkm!~I#d3B71A@3C4dfENp8a#3VBG$$|=^7kD_Y$|j zet7H>A4JJX-#Z5w_tZ-cvs*0?7i|zv0Tvm<*E-V^gcQgVpxjv(3zm#}0GD}O-|CU++WQo{m<)#Vgk_MWrp zwq$5eDHv+~b%ZV1dNi`0rV-|MsGoTX)2WQ#PL|n>x~WZ8GMEHb+gkB29vZr6S`Nej zA0918GL>e+-eJ9cgvZKa)mxAQKVMzCOiQA$H9P8FpWsUzWzYmDT>p(l*V*%s=oL z4F&#y|L^OEI`*2GhXDYn{HJ}z|1YoK%+T?F(7onASsokGPn>?BHUFEb36-BDQ(5%3 z^g^*CS^JSyYi72bScrszkqiG&Z@z9oG09|{x4Wg%(8kyp- zLXoI*8LWbYSw8v2m$VE@jz$B2UfnR;vQA zg_C2PjlDjhp?mp>^AauHbFYHSh?THvK9x^!M;ck$TjU9*q^P0M<@L=X`ZawvHQ+%R zW6Wb? z@Ho~bThP7%2%>f9hZ{4UALkv${%I4VkdzI@kt_SckDusR$XS5Or0S?0jQzpx=4u!20RgQlMYh6KZ_1A1GD5w0ltvIwGV z6Kp^r7ee{p{&R{VQ1Cd+o$Q2Jh89J;eG)?d{Q5n6?{xQ|206GmoS2qjJ<|~#Bg5LS zCIcl(bOnMDz}?^7s4(omHg?b1nD}|S7^dg1hAe0CEvjtlJ5W2CSk_9tUJM2%goEEb zO2x+g7r^zn7iMX9PM+O7n7>>KBSnZU8}5BEjp?hIg18@Z(_DMDka>WZ7&X^YV~1pZ z@!6mmq5s#8*+P-XTuN8YfIitUIY7{?lwt~LMYJo~Mv21u6@gne@aU&oQ8e#(tPH9E zWt_VDxO`=u!te9a`<pF#X zr3nhL?r4_~-yw^^Z8Jt^dIS{`x!Gm}h#x1|gqU!%H`^L}GCh}LaJMHj<--vF7Wdr} z8Q+pF9dmFn1&D#x=rF2)A-9=Y zH9uq(yBmTnTBLms{4vSHp|Ek_2ySK&4%yJry9jnxa&X9zY~7`n8*m@ID8Rd>!oA(HGOU7wM%r|8`~?thUU*W-rcI}CjX)<7KZU8T0-5AUswvulR8v0{=*`7*FjUD`=IEbvLZ z-!hE@y)q?|K^)d)N9U)zas1wpsTNtfM}C$HW!AX~^zA^xknRjbl$CUEj+!MOmsK5X z&k|0mv*&>nff?_08*R@1^j$v1=NxEzYQu2ci10juW16N6G9f`3}zThAH-ufdNv;&*4t_ z%!R7Ro!n{l;A4T8nPLjw#vw19{Tap+U+@vmwg+DXVQNl8j?7W3<99!6;zN%v1=`QF zC&;_YwsMeyGIc#9AP(tixk0A^33!^R=i6{byj>LreVjp6M)8kUShAn6Z8>8|Gm{lK zZ|rm1EEMZKERi+w2bGk*%PF-gOqWQgysjmjiI)D~s^qc3{Z2p2G*xx=Ype4tSQ4_< z6^%g~wm5YF;7qPUFgvd<;7L|GE5T3HB7Iyrt6>IKKOyhUPmVYQ^LvdeP+)x-`$Vt= z*Uo|WB`>BN*P)bfV6CyrRQ#I_ds(t`K3+pZH@@q7%L%@qHP_7A#o+WnPDVff}ZE@I_q1UBo%+oT{~cy|CVyevO#=8?9@sSb~H6^YW@vn7GwT(Lh>>^ihAC8LC%SmX$h z{k0^Nl?Y=EwAR99NOY@}d_`W(Sg_{nI!|ax&)h_<6pkuQ6Ig33wuR8`TRqrt8isOC zI*Asf2-wL&=^_nNglAdIokr)~z|3SKyA;GR%s8k;P}d>;&WQnbBUZz@;n0cje>?NM z85EY>%2qWp@?tHQNZ-JDV!!lN6_1_1#5vVzV(FcU%p0BT z)yd67n|XzpWKm8ggIiv2Rt@xFKLb?S>+z2ic3up|b1}dX^=w*OR`Q19qT60IiChhm zMre9^aA-!(3F6?rpa;1;h9NM@qTbX*bxq~M6QbBLw8-oGDWBJMQmCeyt2}-W3&5L~ zTJP4{p=r(WZ6n4NBRJw*IX)FiE-s4)2imh{p_vs%afJ1a=$ayL=)VYAkHH*8&$aTs ztTUl33i8&Q4l3CBq-B&`j}?ubPh_cFgtgcS44qVrlxdX()#%;xoMDV>3yOg^t>sz% z8g2z`;TqGs52^RLe#8Ox z>~+pVVG1Q0vj-+y=6U}rr-n|`c%^W;Z*`J%)O3LET6wps{z_4PL1TA46kaxf|KON0TwDmY*qZ73>EuZMAQ_7d z_W#_EFPDq%n+8ik{ z1wp@$3$k&RxHsuzonVfDQf9&)QE!`@{-8AQh1w^u^ai7&AS&a|7Mq`j=3o`KmFmtQ zQl0{&n6w@Ofr-m)^x0UNGE$VMU|H}UMp*=nHOpbKW{+fPI8XVotZX(7YII1zw-RHZ zS7k!ax@*pyB0jgdk)5FG=k*=TE=YTKPwaA{NCe)OCl35+dxorMP@!bn%M5b`@pgL&0P@D<(76roS z3K@OZm);qEL<6=WL|wJ1x)vqfY-xpgy26%~6@MhgAMi&52Zn2>d;)iLTQ#a(oCK^8 zgb3?^TOiM6Gu5w)*P9#Z0AcvvwU*s;r%XZHoW!5jY8yU0{K0jm^)7GBu+(Qf=V?x;OG6gJSY3 zyBYvd_i%iBRv~LkR(w7+xDyPls*Vu5lvdA-MK4s7gF#{9k}@u#Nf=47Rq=rk{CyK; z(YwtLus_yf=qYD~_t2C`XmHA4MP{qMOc)&C1G)@Z#p|np1oKS|+mdlL1+#<7nR#U0 z1C`EbDHgZ*1M+EljQ@)I7b$8^j$W`9x2dJFc&eAS_0L6e3f9UBY(Mrm>?{6udhXFu zK1`O$n<30L$QTyOCbRAoW7Ar@oYTy)Ptm-r<(^CN z!>+`1+P2FK?ygn@^-m?s8yat|WohFkoL%sXzuUc5?^+q0i&+v>5r1znK}=@0&%@Vo z&V1Iq@@DG+${#6_F3DZ1EQub*$K3iaf92&}Hj*Ky9D`{P)Gpdy3)kES7~z?|HuM|P zg!7+C_L~grLH)O}H_)emN!NCyS%@&^B1rxZa{N}@AfK=N|92X~*kMYFMEExeG5+_3 zC39037fU;HePUv%%Zpd2L>^4p0cv+`99&8EKYHI9A$S znI=`RaNlc*zqHjN7s*;jjk7ezcx>0klp-ezBkW2=$Z{LxXjCi>MQG@pJpS6E3}MV= zl*};I(=-DEhAcl4DYx|z@m}yOLMiVECXi8hF z3=s8lMD2tpGBt{6lnhPLWcrp?Hxebbx2YM?IqRxW_(OF-eMN$_eE}_ou`aB&@;E+O zjDoWZMn=*JVy{#P^~G^4(qyrPbG(xd9i#%Ou0YEg7)PX_AlDmNWSTJnK0~##9B66f z_7zTqhLtq-?A6PlGszne?oTsBy%^uOG%)``q*8WH4er6I9?b;uXi0SgXu3nl-;(sk zh{pdv?7d}hW8JbPC^IuNGcz+YGc%W&nX$~w%v`q1%*@QpE>oG=Zh!a1biX$}^QM2! zj~mjK6eo@%q_gv6=1N$J+9PV#5D__+1G->^!Vd-9yU(kE11Y4s>b#9K%WfrncRE;H z$wF357JGJ6bUxX8UGNO*6+=t}?Lk*2jao&KXZ)a5Xtaj$3~mdH78~Mt%E^vw zneT4HC2t>ynd#fIalIywO}>fCP8;8BQ>{7{rTY9UP!yFYwx!5Dc=|e~@Dnc(2LSIHPAFPmm5Z3n zwE2E+p``8^I}m811p?k~6+Nx%uN-Lu#7hF{nRmiO+7RR>9Xf?%f_u>oml6ZTRC$xIrYmwpTKX6sO)6q zAR+%57?Ulg38fgOLkw&8_|7xs9P}eb*kCpyE2T7Z8j(d7c#lql(907=OAz!()2#}i zKcD&+3XCJ9H@tr32pSj2_J(MWJYtU|a8kohc2VVq6~CACggK!CTd{S#EW?SMz@rm* z$60DK4ou_|LrGzA8zrhuzh&IXBWOrh!!K(OZirp;LR5h!L|#IsF;x2A6wLd$86=R1 zZchI%9r3oHrTDU{oM}qn!@XnsS zV1qhe$KwGZ_*q4Iqmn|a^c3#h~=AhmKKmQSRq{F2>3$jdk**( zP%8v_CrRyI4*Kj&0n38tR4&9;z9hSd%g_$E%C$+Xh7-hjK3M-}NK6*J;O*LIS)Lvb zA-pL4uE7>+ZpE!VBpumQZ@i!hG*hFyRaJWMt`qtRZ-jT!{+SDv3y6*Vt?R>0!yL$T z1>|IOH*WHgBr?8E}wC234Sr$(dstP@x*QAlM=S}@Tj z+Gn2J|Z7Subb1@|HH! zn?=4L(X!t5-ovE3bJ!({B^VJG{QUjKc~q95@>m5^lMX*B=}m7FeqgcSLdwm1*l7qm>&Ed(@IEqXJQp{el=d3fhZpdatcysZ%tZ zFVg~qe!6Q|q!@y#I22}Nj>esh+M_*E8CpblglzUb??a*QOa=lepCtt(MY07O^*t8N zc(dMU5ML5mcCz_2705V0wBMyOlnW!n9$|!6rj-Mt0Pn2-3i+igrT6F=kG}8@!uhkS zwG*T~1KOJT%5ZBJFpx63BDk>YO>NWxGs%0}d|4 zfiJo5pYe-1D$)rI(>)~`;C%iL?^RGt0(Nj4D29A;!U62uDeAOLv+eV^5u2YlJv#2q zKk7YI+!>ma_3d{I zy1edQFUOXuJR~H>bP%K=aKaq4!V-a$Fej0fkYx9spPxU`F~#bZENBt5k=8k&bX4~x z(Y->ddCLx%D1f@N5nY|^Nv=@TF;5M%no4PnW|6uqk=K1~$mnKVWY;z;KS}7GLa`!Pw$U>m#@AzW$gQq{P$f743F$8A7~x zUr}O03F@v8Twzn?Q-gMRJDBQK-W@=IOnW{enGelv7OR!U+T%%uD&Vvq7^AJ?@_UNM zh?Ah2u_C*MP9N=968(}csG{Vs7Wf0yLaW9VP|TAxn0rH65YjIIirK5g{m|nq`0&Yh z#Lej;?^L7UT~;XNQc+Xr5VqaGlnG}DwS9{i0sKF!sM_8NnoJ${ySWL71@Vj z8w69N?`_hb1b#TjGi<-sbJB!07t>SeBVI@!)4(v}wbaR;oIx(7tc8jkvHxioiXlL$`YyChDb8D#Fn!pyiCFQ`7WjO!lL z-}9vUxEb<>LLB!f5(s#G+vQ`Sum^>4@^p0|{+`Ic1QN{LJ}n#E3d6fO^MDCJ4|b=` z+6a(<&pf$9?i9-~nKE`wmGORYDE$6$xCATXfPL0;dxggjt`) z5=*WOlGT}^>Du3cm>H)yq$H4Y6Z#*hvbXT)u`Wm!s}F4rETQtW3|6WNK^OqWscf8> z2`-8h33RIU^Qnbq~k=`NMf8iE;~2*&Z}^z@T;PTq&b zjB)QAqzyB`pAWlt0p8En+!{NQXsSMG?;P32o}b9MzNhs#)kk>*Jk*;l)g8kU6fJi5 z!44V&1XeAbU;qgH?egiLKaKcSJ8)6;3HXJfjCzfWS68Mp5XXyhY*Y%~rqE~D8WmoNRr!B(&nMZ5BWpy(wD0A`YOKwSLLdbsH zv1Q1>(j9~{iA2j*MHjFQ=g6V)2};E@L~R&H^?(`&cz1OAK`g5I4qQ8PLcQ)p)U=M< z+Xa9q&;AoxO1hLq#-}9)YELQDS<)=b0KUn%zm42;tKmKhiDMs4nH&HWo@LNFZhI8% zWo^cxQPrO2ok;`f7;{TqyEJ`MjH$mxnd1J6;@o*adJ?wr1cI&XE*NBZIE5M51g4Kb z(Cz2^@V6iz#tiapuNxO6pr+RGS4Bc^xFYXCe(V%oCS|hRDz>I&Gbz;jFkJt1f=y#T z$5ZGQE-^lL8zDJjuoEJd8n|;N7aooq>e3?x!aOdSQC@+X|31 z7`)C(m7(k@a1=+lWKeeX1mdBtL}#$srb#Z?`0dVqTnSHEudsiS=YMWQ*_&$`J~f7R zyV4`?32R1wx4ab*O#mKRo)%oO`2JzbWLoL!ZU-r>gMWw$b!n zZR@{}opwF}oy&ZAsP1ZEBR+Z-jZ;0;8|{v~sZY#g8n;a?PR#m&GI#Ru2za|czTVrq zJbYeeJbV8tif|yg>J++>d3M|HMXn_pd0e{0zgp67&0$JfR7wt%yl~~b_*nY;;)Iqt zj31+1w2D&BamcgDIA}okP;(Pq*LcQ7{5=DC<;ONJJ5Hz6*GnX5F1CaAaAA9TZ{8b>d6W z4d(0X!kM)m4VqR{7H$JH3j;CSMWjJ)0I#Q@2X4)CLAOg3xD&MglY3Ki^YIg(ZSrbA zj5FINWviuAKtN;`-adGp4Py)W&$jH}r{2?`{#QC$6${keh3+AP9TK0JHjl-_{XF!$ z_B@5)xpD2Zk>{|?o~S*#IutfYK3`zmYPj@aIi8DR+M`Mp@g)=CQt$dN)02*m;E@kD zo2y&<7Y|#YJTJCrxZH@?UVB1A;ehs24DS9wGX=N-%x2J*Ypr|6^0+J0_|LEksj!12 z#cG2WV?NbulG<$`_1W+gTIXSOPdn;lGAoUPa!=UWmf8qNJfRbCx3s*yK}d2$aW(Yc zV#1iLCDu}6MSMSUI=lYTebZHfMf$X?%m(o$Kd?2Gw;V9Z-6b(`>>YCEYBO?jhAvcZZAr zv6WhQ8CHDC4=W-oWs%h&T1kpFg891h6rXH2bi;_M8Pgj$cHrzj)0JQ)-BTyOqg-%T zYJ3!uq^(3&Z6czgb>q7J3lWs3t7hk}Kot|K67H($8&#EA)BLS&+wBwiggKGcqvu zd>gLrLfTyyQDZXL6vklOS!LQ zz5X|1`H*y^^l>4rM&zh-OX_wMj}Ujq(dL>5Yqd|}Fk4~q6LB~~_PjT^p^+3OIRbzJ z3|@9@ipvvVygClN0T|fkmkFUROw|RUd}7rOpdN2!gBjV{JT+&hKrDkciej66gjJg> zWv4A{@)C&+S72Z4b+ua8Z~vAtCo=&~^HV#dbQnge0Dbd;!+%41fB;P2_|`UX1$ z-{B!TZSl_}GcPb?S2iI`-aM{Y9K9Dq|8G+Dy|V^W-$$Wl7xuMy?=#|NJ5yeVZO!ll zrm&Lw#|Ux^F(O}8`ajq%q3h`$`(jEVxET$`C{ZG8YQw|*Z{IUqbF%aDB%FCwDt^D1 zgZGtaCRAJhY`lr`igl5!kx__*dcISf0 zpV3y|d%OA!n@4B$;sCV{kqFxCkU8=i=<6ey@KN^4s>P3JiTnUKG-1@ERlto`tyuG0 zBRo%SpGHhCD;NHpg`00aVCBxx+FHa$)R{292r}|6Y(ICvVAuQdFrADYuxYEQVn{qZ zz)cD0K_tu5IZt-$Mom5C@R{Lim6(sV%tPxokoI{%jnc)qIN}~VPQu}lKW8Sy6z5^@ z?y=lq;)^(vL*avLPPvdV5Ax1=wVKMlFmjx4;a4Tyk=k0dOjB#DT`Rd4B1s66iNmc? z)I1GBEe*{h=^k@vbnRGB0x2K8$7{bMd&O#jrE8{A{Ha7cNsU@PPfwgE8UI1q$qF|T z_e=RC`i7IVnPI4XY}dZ}tgGQ`L&omVXWW+f%3rpm;fk_aM|=wnf)mLZx~7MTT6*^P z<>N&C3)vdF7{>N1Qr1FFD%fzfYL^YGCEME_gKR;@mhhKMQ~W_x+MAm<1j9zrhhKbh zZn0Uuor0jwTFZp)uQrec_RnZ;eA|bBRwcxtdBHF3nx`iiGAXAa$HdiQmP3TIBF$%c z73(g|lQx@dc4QFGn2iySpxgU1hlS%Y(xHkKqd%Z$l%PR)o%Ja5rN4O5@`B$(!MJ^Y ztb+Hu;I5z`>b=U+Fu_y7@N@veWYR`WDgT}R z#U!Tl!bu_UmpSq9g^Sl`gR*pw3Vh#^k(Z0N_#pi%_})9SXq?47G=@fNmpWmdO<>oI zelbRdFUv3jd$3UWr&1q$u8x6{T+=FqTU71Nu6UB$eeA4qzT2vr_hYfO|M4Vif3>9t zx}IPDTSwg-T-WR;TD$*5jyObFl8`FjX{Fz1ab;bb+ry1V|=R}0Gf0c_(O-+Nx*dXLjd0C`w zCOwnzx0tSdou6;_^n&Mh7yv!29I(_Dg~)c)n@AN9^tzgAGNj>9qjZkp{Jj(!0dNfV z=Uk|B3aG2=e#xKb7enjG4s3`U)7B^LR3CDncNhnXsh?yG>RnxS3>g@s3I6|yP3xa3 zuM$a5gNbid?)k0CN&cHs9jhPTVQE|2f68^fd+)xLIjZkTU8}u}dX|1tHCT|V;JT?@ zV+~v;xL%+k65Sj+N(QRrcnkMM&$YgC(-9{VvjAtqgw=6HKuV&JVmrP;FjSjdV)C9ZoD{s68v)%0%|qq*yGy<{Sp@`cI+dG~E>x$@%>K+~{L)W*9Z0^Rm2>tE3*^wb zdkZ>~%!l>0A`@0X}7*V9u1VmrBN90JGhMk$;6cn@SI6f>=vWzaNmkx&zLeheZB z^qXrnrO70COS7&bw!mn~=y6IXABV5rXx`A)=i$2y4GnbHkcHE*Bw>E(tK03Q6g3!L z-yl7*O^E;eRa7=#(DH+onvJ07y};$y;o(WZ`+M;2&Cl{53AVpO8^5mH+7lRCMB_<( zT=idJ5x#ZHaK}MbIW0t;eb5t*lh55|atES8LY?`ZM?b;5LkW<&IwbQ8B4HxUp>@e&*YItxF zcq`a?F|}-b#SCZB06Ds2>!=SYJIZ9Z0NbiuL7h2x=(232q?ubbHA)U?^rCica`gj?5ac6heZkpq=afz>Fa+v z2aJVVdXi-i4C(b34&n%Ai+0MNHi05BtI(L!;6C+g-kb6f!=;HdfGq1!eo-HwpR+@F z-2<0t3w}Y1oyFpWPDxkDVeb$+al;diY-^KIxRekn&SCSw!;WuPJtSsO3%I<{(rWbk z*jqhD|3O10-Ls{-8y-1|EM)=4y&~QGf#o%bK7TCZGeBX+0#7vD|0*myz{|<65FtX! z7t@K9XnIwQ?kTmZZxQ$9jD+uw%GA>}sEtkwqRRzHkvt-Q>f{xJ#7s#8BU7-755 zsh$Cm@_bMm^=Nw0V3g>FeX%lD8L>;Ni9};#on-_i@0@b5wwR1~z*$5+1W`@adKLQZ zdsk}e!^S3l3I2gEKJXKJKY-%OFnzTBkgmUuAc@ErWp?6~|E5X-${96Rz)(i zU4n@e15SGvhC@a#PNE1UI4?{td4so$;@H?0;r|BUG+F|8(m4G!Ay8Iw=vqLIeC z@3eSSu=*}C$iywyEcYCD z9&7F>7Fro~8)JtS{H7{QUoz`NU;<}69f-znRN&uSLM0(XPbb04%a_a#p-bZ(>=b-P z#eDEo4R=i>I9OpNaJ^5hDgKTGV{m7pQ(|CNhvsrvP>Lkyv%}}zLITbTGU4H)(C2J!$qkp6*G(kpTfDtj|`V)=Du|PQm`n!D*5jb#Ng{;+~5Xaj}m&?t} zJ~7N-=2%HI6hP2*@M6AP<<=cSwhZZB725vkm+VD#Y6H$LbOa|RIMqVVsG0=Rk;n_; z1@hAs-YH1(x)3q}Ruh31J@=7zC6-ZXe3wl%e@+BnB&l4Z`nNEX(7=+8v1ygX!O$eM zQy!Xbd)C7S>TOg3Y@`s6?Dyo)ApypSmOq7%&1BQbUa(5wQ%z4ZQn{?Fz!2r)$E%S8 zCCV09n-aq0y?t7!ql9f^S*)Ty@%kwT6iTl0{-j-7&3Bg`R4{7zV?CU7pzfH z=x49ibGK{&0!zJ99zHfkYc!2;;_mvO9I;ye070ka#zY>P*@;8FMC5s6Or4OpljPC< zJ?H)7SF{j*y%lyRLz?^Z9_CQhn~sO~I55CO@q>J3dEcH7Xigf+5tTySZXi|#y>p=n z1F@Bw1m?Kh<(rajM&tNYG|8Owp`$d}`;iy5vw#l5DafAP`R@m5_Ml|8^2CnudLvZ! z4p6n1_}g#zOFoBuC3xBqZ(F`d%G0EY4_%OVFDpg4kUZy-xTI?b*SP;B>uZ=yG86dUkA1D2fz|FFU*Q_zQ3;~aL zkMfNf23B%KIx9|k$D&%_tivil(nwetmaEHCo#FhwwC|+jUS~lNvc~xcN@s^!JmJq6 z*xnrxG)MN&8}nl$Y77RK9;R&8wR?qMrG7gA?^W|4|*e@*0bw5n@=v2!rDmiFUGYQPc9dDnxXf*fcU~NI1 zbB~#uHjLcZG3{e1ZB>Tr1MfznUt~-!GL@)d14Ob_Y1g}{j6i7%J4MmKd6ZEmf&*k_ zQowOB5`f6s5L(uS$lup)nd;_bVu0ZqIORg!f|B2&8E8i?dVOqK%0lU39L^N)ZHuaB zniI-qE<{U-!$Fx+s9QCZq&rUY8{ygX>G{|O%|LB{CI~DF?V`!*2%thrNNMLx331>b zN01}AEgNeUv*AJJVaEZ|%74eL3gdspVEM>p&yo6ulls9mc-%#(Z))Wh3yQ)wGRi{t zd%A98LSbuUD;YgztSO}zN9ZOK%P5x(E{Cdm8|56qTS`sY_EF9}31y1G)cf${vke{k zq8w>ZSi!O@AcNHXDXu>A(uZJA(`plE(6uDdLr=|-LGfi&wSBgd3oxDP4 zmnkgAxi@;_bggDQ8)z@n?Re>(AF{rkM1E3+Cgrdd?dVt`H@^?8hLNx7XV|m2SMVr2 zEiyMg+mu>(o8p!$P;AT}C%gtF0wj%r2GO+!z!&u0)kksaS|zq>L2yY>b#uk-&okm1 zP>WSQ$!391m?)#F9tO`vj}%UWoUyQ%J#hh3y4h}4q#J($^N3~u>^iQ$5$Y6c!Fhp>{G5@o^mF6VKnHTxlF3N; z#`sjC&K*Ml)6DnoJop4|nLjF18Oq}k=Sa8TqQc{|I_wK$&A%lJ3MVkggvd5QXrror zZBfm%1sS@6E$e`59Z}zyVBlab48>%2S?#yt+Yp$5t75B%NNXt&2C(V5O_{%8&IKsr z6-4!z_iV(uWqvE~xre{HhToO}IFO^?F_8s2^ZLO^>xty~;7|^=Ra|fD$-py|d`vI~ zp0HhTm1msJ{#&P=m&9&D+w#xI^~;BWJ;x6ljE@iaFm%U)GZlbcnXsz{2f*FFj^Hju z__=g;Z*$bmlVGf?`Bv{!;@61j#`QqihCc*nwcpo%GVmc0h%`+?$lg~oG`uTT~I%jug4I?VnX>)I!!$L{!|uDUhUJRvV zdUFVC@`0S;x1zOf=5B*Vd?B@d*kv0Qe8atcpiiJ`YJCSTZ-BVB#8;OGWoaA4`HS+} zy%yIE6fiAX#;y<<*=o0HG%w2VxCSsVU0$QO@$rjsE6H`kqrl0 z!*&UB1nLN)PUI1oY3(bOD5aDCQ`^czw?#lV{x^*gYiVwJlj^?MC*<6z1TMBE1BXhy z4W{Hcp=ZiljqaPdJ_#qBN)KF+0W+Q!mbK}Y<=kns8>W;y75XDM=3aWAIsxn@FVqjo zUgZ4_zJ}~-2jPZWgKrKaHgP9KS7udvhamwp*l$`5DoHd(Km9N>x*uY`t2?8LJTXMb zsTB+*OeiRxe_28YqETa-5(~EiN%P>)u@i3e6ByXu#B1MNK$T@2ryCwA()oT0T(8@> zR3XKe#Lb|Y!kSC^J*J3v1 z;asj4Hn_GWi*EXMhB}BItNaxY4<(B_O@0Vs<|HwWq9^9PkaLg0Mek<7M&r~W)U&nL z?9ZWb>6ZOiSEdTSh=%^aS2(*fAQ_|+RnkDCIcKkv1K9Kz(IvASjCYWjkh}_ptKJRY zsbeBn(5xp5O6q$%CG7iKPKsVwzVFU0?R6~%{75a{Mq%Wp;*?iT9plD*CKK7s4SVW? z({xGwvS<8PUZq$xBO$Lk`>$o!$%OIKK#_5YEN+{2;`CXJL!aA{u$9=w*qj5Zm1Soh z?kF;&?*R9h3@;o*_N}C>GEXRq!pI&6ulEazun-warFpsrgs+wv$zBtBiBzEEcxozP z)gPG@cqTm*5yPAMw;9M&(=IcDI>|Hi92a!x%5mcOmHz}C{|O_4@?!GSeD6)c_ags) z+8-Aq7b_D6BWoiMx__EQzhR?=mAUhi%ddJ_R6|6==fU2;Utj)xvzaCc9DwAd!fHJ!-ttK&O)4J0?a&8C!r+N9%}IvW}!zPDj!G$6_yAM z#a}z8Ok-{OofPuVqU_twchK8c%KCLku7*ZOR4ybt3ZCJ3!K}Q@f8=4@pFw5xYFjhL z>|;Rs!SBUQHm#{gco%p+dECW3qHf6|){XWxu2Zo?2Y#J6>+L zpT%W#Q?oU}se|UJ39Gk`Vg{Xptw5-`qF=#Uw?fqOIdLZYIiYl`C*W*h~8~FaBANy_%gb#b}~AmtEvw z8C3Foka*(#V~Qo7m;O#rX~t?}Q)SNv9qYE&}_2%ka3q zfa_$s2O&F%<0NEw+Stdvs@~9Acw}pk%U!OtnY$g24KX90(Hn<^VS)%Ea4ifjyf!Bb zk|hjp0Ml@?PTsYhS>mo87h=RHwEG~`hc*nHM!`-KgU}HYQ_tvkDl=6wxfJVEG%Pt>BPzrG;|bThyqOk1x@wTt+Bf7RrC zo(OFFK7G7h&+5ZQD28IC$?s=HM|0FYm}^gR58g<{c-hZ%I zTZK~>WWGg9FgWLT_7BJmmM{krs3&t~e2*IaC8Civb$vb6tM5`7e<6^cVSm4B(q7r- z1_PDEXK5VHJvRroHj1Ye9S(BtW@*y)l#aNJx4ZVPZ}$qp)0Y9-@OrGl>BNpP$vxc| zQo;r)(|rOQoRePgW+@ZOlCRShoNqe@`2oBRKb|@aj<<3U@W#LCrWmRov@Je6Vq$LR zg)RSd#d!7EvLcjp<&kL`9!0^xo`QU3_rQ;SPtr3fpDY0j?A^yrhGOENvtfR%?}yt* zGA$+JNi_#?;3TpuAl+onIsx|HF~!JLJU!DOis ztjK02yg7W0i=)9EGu5yS;??*W1jO+|IiGPmzrm<64vKYBzcd911vu1|3Wdbxku-4gei;4x;Q@fzM=7ZI+hY<-M-ubZCIizvfbcmUh$oHgwN+S`2z!><_K zin?zsaWMplF!=nzNYDW&%HP%9-94||{G65LrCndma2h^m1a#sPgb#1;Q@i?4m^54@ z1kuF-iuW&!DQ;l`eV>`fLbO`Jh2w|-&|@k#{y1^2{_g5kAj40QfRb4BpBwi}H9CR9sFscLveU!^H5%^V!ZqFSO+_Hb^7aP824O1VTIe$b zET-e%2>_+jIRzKR>fz0q-mzOB$R5!Np<99eD<^M}wUt(P+kP~!-LW}^Lz(Pc&*GVFDWYHt+MsRYvze_oTrHD!JDhw%Afgk& zj5pqqBrsrJ-2(Xa3-@z8i_$t^>Vx2dwE0UqK51en?*MH0({W7^Rww5D1tqE4IGZY> z0*)laP2A17*#>l#?Uf}8LrE>8bmfWBZHE?mTDwlvnkBJ=vI2cq;!Nr78H8|x& zYLklX_z1>jig}#)GZ}C}xMTiUdbwXgb5z*;t7U(a7<;x?tH3Jhg6w2yd!MMat`KK~ z5?F(_N-=cKp>PEhAv4B>1j?A(Yy^CY{E*m(5RCNTu2Q7sEZ%zTpL5+)275x72C9Ob z2q}}T!DM;7JMBMOl8g{wNju~zij=eu_MR|=W{cg4g(%8Xgs`{lcPGrP%^rh{In11JK(t0ZjRfegd8wl?F1OA@ec@|&~7Sq{r zdqODbO@%C257CQAtV}mNSpt*DX;o%@H|*4P#HFG+UE5p_)uwVNCH%y4$vCjMwT9M6 z*%B+9QJ6g?ZM(aAUbPu^>o827s~C+6S=&fTaZ!$3Ec0@S83E^xo9ENnU+82Jfx*Q5 zGC5(HtesltJqv$+t;-A!VpZcb)@R0YsmejHjG+v3uA8H}l>HnIp3()oLp)D~EZ{D% zoFs$gn<~-*`{eu{i`lP>1-_(sC>R-bk?^pG0K$I+Y?!Mys%mZx`$1;oE-aLKH?KOs zma|KS?||mO^n(pa`xm-13D#*V=O5e_dxx-PS#LwkiqjBP&G(!;zx2%~jLdp;%c(-f4wz=dWOh<03zXvp zOU|WtC6U*g7z!$mq^Q@?=|WiuPOw=UGi< z{kZLC!i!ze{d_YSIZ*22BMuVZESd>YJ&Nx{CO#OMV`-pU^jn&6$d2N@6gYpcce9<@ z>w&xkq(~y0^&&ADZXkf-WjM`bT4*`bmUOS^_DHE_`{L4()lyS}I`BnYEgmZ+_EM0c z>pV1FE3zg(A$ctaDf&Y(KyHKcz4c|K(_0c9@yV-=u&T?daS7K)@_7R$-E;(H$C2=b z2ClJtX`(8HaD^(AWoDUz(siyvM3Y*Z@_2sh62xGbSAe%R;3G6S6HRB?VCR60o7PVs z0s0IZX@m)*4JlsU8ZuGU#C$ z+eD|9{d70taNZnk1iC+MxxkUJ_^esz_fxMrw=>IGN0!ojPjIy@CQvYbGUGohb27M8 zs+f)kPj$vyF&*k|-{xItlS;$|MdBx024ehiFU5LxkRH@~Y z6R4Lz)7Torc{#~BW}qX3)k!5%_F*35*{yHajcH9QSqR}bdIe{LYLmJ>xA^XQ-LuU# zxNWyZKF}vo0O6*5_3aGmMbR-s{%ebEQ^NZR{=YEEKR)lO>(!qIH~^sfdr|$rJ0T|a zc6Q$sFLcIERu+~n^bS8PfU4nCZX*@tCE%d3p})TfCnYJW{QdqNyaOOYf`7jf+Ve28Z_!=>-F!Ngcj_;xAJo z=MNGfz_TAOFAZ@oL^lA)V*v&d&?}eK%ZMBPh44o>Ll-C@1gO^~fl~%D02LtU6)Ri{ z5EKUr$Vn$x0px=M3@1#C4*-gE07DA@;{`xK&U1!05I`rH2n{Gd0YHdi79k3-d9?eua2m%0mE`D>Lv<$zn19-p# z>^=+0u0SpIVe(y{qRkr-1e*c*XX_ew4*y&mv4Z%vwY8(e!&T`)VI8A!4c||TeuG|( z58D?%-uI`6-R^zTKxVx_3DD=={>gjU65N?Ys1W1L<2Z@;-!NbAcnidXQpRnX)Tjqa z$d0k1nK5_FCB%`!iBv;m^IxX>JG5^g{5x!bdTV|l?*zJXJrj^e845&Am)`v+0N}R6 zzHgBd5+uMfcyGr2>ssJLBA*NpU@n#D2mt7dlG3Y={}LGa_M%1e1F7qUaPIpsX?wwO z`@lB)VBQVcLxf2O`-KsOp-lsDoeZg}gM=x=rt0yi4H*^%@mPD6EW?r<;OYC-TH*N| z5MGTSbNeVe1ED~LMj=s*h}I*3jH5J3Vj(dyLZ6BHBtem*NeRayP?d;f64)e}l%rG$ z)g}e%t7yAof1sRfsIOhi?GciZRH%;A>xHFv!*UgnQ`d(S+h<} z*nBbU1t+srmI&){{30Way{DKEEQ7X;^uq(f4V<)4NhT8w#MQ7>!`Sue)gT>_4<<7W z?x4tlV!cr8pzgwiM)*RKnzEYmtCHn}xTLH|>rfm}AR$Wq6vRp8GEL;|NH2q!#>`CN znv%36%gAvk42a$lB7+>H$dMw<6m9XqQVqqb^T_k!^FNeH&S)#LzPay6lE&6eZ5koC zl4%KPMneyC4hIfd4si~NZgrq@j0GKwpH=oLmB+26v5pXrSdYNX(wT+SiZZ{OCJI!M z*y6N>YYJMF>y*OFQCM9wxu$~ArDRHK3R4!aEio*gtm;l(+L5+W@a5BHHRdPIBu>*# zLGImQ;lj{@MTc~}|62_rZ zq*KUKaHttnD=JGXw<}@P?$jn!<0|x3d~3zkSX2%wpcYfB)GBin&sA_#EGl@)!z(Sy z?NtSe{j@$J!L*N*Jm*GS8!|t{0K>}72{1+@K_y+7WChriNqdhMk z!^p;sl8D}bph$^Pic#8LNJ85jZ_$$YPjQ48t6|*((gV9;yWt{oaB?DY8M!}lSDDIk zZKYJDJhD8pBU6)8g;UEJY*~CvE6gWZCs`|5JHHLJJ+x_C%3E5^q#6&kOSP_AI)8*} zb7_@ms%h)}60garAzSXK=_o@gYm-lsugYDv7^uH#8EdKdja7fN3cgX>V%OqpWMPb9 z@q4It;3}yr$+zA2=La7IT=*J%6`nSppL3HFDz`fKcII^Um4NTied-y5h2{+6%qq)4 z3(5r5pjD1a)`EluhDDom@Lfc7q)gBReHuS~7S8b2XxE{%%Y&odp4a?CPS&ri{>$=> zx-0f8tE=$Myh+wPs$4A==h=Z#kzJ*uIRTiw6h@n!YCi=Lg+Pnu6A}g!h6Ih8mHOW$ z?v+nQ*BJbd{7!iX{0F=o+sxZq29q4Uzej&hC=A%=8iekM_>+8@etCV(0m=f%fGj{F z1BC*0fZv|Ie298t1-LiN9vjy#g<|sCgwTZIf@*?}`f7+QM=#=Qh@MW;rxT{F;6oMt zPM2pPjE;?>2mD|uLc>Ef!USR+A~dn*n9|tX=Qc&0cSK)BQ$@G4ESMCy>gc4|sQD}D zt$Eiu+F9ELhUv|N$88e9t2e zEkm!MN2cXY6_kyT=S==9gh*YE9z%AE;!YL|?b(mlZ$Zp3$37RaDxyn`FMlgZE$5YZ zkL8Al1MJ8#o~{}vANqs zwOMH6=g^Pn-wgOE5MwsK;;X^5=%8DKOHWm`DOac=8ZFE;%zLIH$GK9ga%rY6Cx4A^ z#B*d{=Y3rw_apBmwv>d)Xwczl9(Pq+b2aihdTu};pn=hZse@PZwnnv3+btB2t6MMl-4ZeXOL`L9xL zdK}?|+mH8TY$i8VR1LP7&X~`epB-=05RX)EgCZS6EG8=Icu5l}5wy;przWxtx=YBpMV{xNrj8%mm(EwF=hEj2(r(j|dhC2kKK}HcW7Ta8_YdQD zk9n{#>EElrUG}+ey5n6J?I0)|`Vjh4ep$XWJgCf;y(M8KC4GT{q~E=pu+VLMdzj$H zG7_SIuYVr-U1dq%N1z-ewVVL}7?gh&5Fj%f^ZOu#iR zO{Z&Nu`>hvv;%SxuV?QwH#c9c!_v^gTGm&_-jV!PaeMk)x5Dpbn$SZC(CZ?J!RsKn z;#~WD-y2cbYHx7yX8Y|bHKi#}B*iDFj%?nITZ@~==eE+j{mBIW)*kN*V#d$2I`+6? z?sjqZoV z`L>#7vRmq`RAnnSM??%B;?UMpMheRS#6YmPXT_PUN>o_&Mnz>K-{j)%WK;AS8fqoj z(87J<2s~GiU|JPcU`R|vSx4(_r?JAJoe$wF5jE0wrdYT#V#mXXX!w+NAewIcp}-1? zaxC;({hB3n*Vp~i#plI8F*KZ_Ih-feh$00@&~l2_(1UWZoamxC6_st8hx0DCAEW1X zFzxqeCe7nM&Gz^EY(D0c5n{2ULMAAp357vK(6ygF2PjM99N|Aau>}o2#wiKTFmZ7V z{3@BXn%hFL6O28fJ>eh`#UO!fT6CwgdzjrGJ7AuGW< zo=XpP5Nvi8AjPSYCsn5&R%x+i5%8*fdi?1MhrdzS^nU)`+;(!=RuG6}uYg~ZeRcYA zwrAIM#IyGB`N}4A4)vL`p`Ur-^*AteCb3hQU3%xSu=9K7rl$#ybEom4w~anZ0vce; zeUtYp0HC^3C95K;k@yyVfRvcLXsxhe(0|2!6wqw$U*DMT^?Ol%pY^{L)BH#9`TxRw zCbss@X8(eHuW(@hhJF8regB4i|Au}4hJF8regB4i|Au}4hJF8reg8j(eaI*nMEhBz zTL1tE?SH^NvqS^Q99PT{fthRH9p~6wmMPC$go%`v(r7hiIX5MwvE?Z7MX_r7sJSV0 zR4kl{nrO@kZE-eZe?g^5;!N%w=CZ6z{8Ie5?&~-2ydUuNf1DqV z0#Dv}4W6MruLSP@2Yc@jEK1aEYi`@NeYS1ewr$(Cb+&EWwr$(Ccb!{^s(5kVt5!u+ z<8Q21HnK4qIlf$Tjsf0n^9mHx@O{?mrizk&!S7s%uc+^ntqF5IG^bZc46VoWuT=?x ze|z#925%S4%}@7H@v;WFHHDPbOo(QtnnH^A$Qv4gBzXMRtJa6x#oOxhrbU1gewk3N z8Q$SRp9a=Wnw51GUAwG<~*1fjcp#w=O^V_-N1h{AT?wEWXz1YVEMF{lchp<>Bidl(6;hh^` z84rDj*JO!H=}ox~`M4lF-LZUHurQS2;2|@vMSjI6Ith%!0KjM{08`)L%q1Ts`D-n{ z>EzGPvqR&kT2T{oNkurKy3F>#(#m!6vu>X81LMD>9`67iMZb$G534ZwmoE*lO)E~m z@3T%DB_rtV?bFl0)fJ}b$&=`RgL*$lQl;p=Esjlpeez_%bEKa-}Q8rO< zm(dsO{W1PEt8>uP6M62*;XJqc?*0A*dz(<#aNvkatlu^E2C$lW(rL{Nnia>o7198>*B{5wg>~W5PIp|M0#bioepQsj9iVF1LM^*&uefS?Bne7oYw6 z4b1}y!m)h-@csny!C?(z$w|z=rxVa|aa){Z{?L9i68%7)0!>`q@gv4E5Apm7=DE(2sNC#lQWU3$gel^G#2k~ENF`R znCuUMBI_z{I*?6t#<-?Daidcmu85hC7!T9<0%KW9F1QD_g2oVGY%E}46+$OG)F2EI z0+}exm@p+yCaq5<9STVz;!e#P<-Ovor(K@iE1#7BLy`{hEhDZkkyg%=i;~A*R8#?pjntJ%GW!|(Scvlh5cjTaIRyWPgH zeBRs8qc}XpR8-O3%k7#J0#O+0<2mlQU9-C$*MDE=`giB+iR9T0&-knF>24*Ua!0(x zM(x%149wY}nIaNLhTE5mqxWe)M@NY6Ua#%&IG5{n{BSw0vh?II3VE3rIoId2Z-4yO z4*yr*Q>k=+qxD@6hnJJf{i-vmlrOg&&*O2sJm%Ni=Ooeh)6Kw!hAyXu4KIhMhyLSA z>6PWIbM0F3v=<pB#sJ*t{hSm6JUY|Y1#qKY=!SmW5R31+%xGANA0?dryB6XB>G_AIY zthCZj&fm@b_V)ujLtS3=kqaIJ(q4i&2rxM{zP~QAkTU;|?VJA^d^O~>&}oDO0NBF( zU$X1JWC3S8J8LIeOQ&fJ9VKjzhituk_wp%rcB3Ta?Hd6C5uBD(8Xz3ko0drVWNzZk zzZvB|0?TcIZXFZz)HIk(A1jg0z>XF%BX1}`#u0q3^ zVc_@dokfKTcLQ+?_Ze8T4C!Jb=Gg@CcKreQ!D4al16`^pv0*c(MzPT4#4b!2U*B)j zy3bT;yomJ1$Q}dddEyKP$0v`k&G%!9dS=HQ#{6}>clwXBSUI`SQL{oLFSikxr$?kWcBY?OIBj zGu%e&5=JBA0mt^>;{pH)at>lC+Y{r0BSm15Pg1EMcVz_HCcxvmXz6ytBu7z4xFJUE znY4`qC8S<;$Aoc{M9F;8xz9(rhk;t;xMkAa7KrxJNQe%AZ2b62Sz5DfymNKMwmn0t zdpC)cKE0A;<+eO53?{VHK`;a&*Y~>&`RHLRVpciatskA)p^R~ ztcR_E57I!r+rjflb?~E2WZS|QBb1QA6@7f)9ggzEtcdjV+oWvVo#v%Ylp5+K)ZQMj zAqz@H=cYERao%Q`UjDGG#xDJeP!iEZ#beAy)g@5}hDEgN7kpBsI0+|=ct86JwW(xD z1h=2}2V;nnzd+tl07%1;ILb(`y;o0WJV*%;S*I4iVQU5lTvu;{zz1mi-LTL1S(l*| zM#@@lG-V5W9{do0^SFcHs7l?#(=5uZ)hS+@jWb=AAEQz$4GR1eIl3YMjI_|H#X%hWhLe1196)ZijEj+4IuUg~6DaC{|5PmNO z2L@TZzvCo&*&=h3GWpI1y8#3FN{nBt9i@+r)0CKqe@Tf%0mb#Fx77tG@=pXe#ke}B zFLA~drkuo&S(li5%qf#RcN@Lv3;r_PZ99dV`JY}XhDpYtLCYJMFe~HvlE2x(=!W6F zws2NY1FSB|{|bL!fuDv5%K?`SjVZMoTZ*$pbv6$Y>TN>-(g_C(*;70`kb3NcI7#~- z`qR$D_ZrJSBZx#`2Vk{q4XXwSP!~li48~@Q9>U?dT|`BI%2`_atJkXO_sL5e4|2>l zx`+H*+ISbzs}O`9I!8oV5EE&|*6tuervRN?1P99>Htec?^@-0w&`5d2R3&KA>Usi# znhCY%3{-^cX&W)O(X*46bDa|+UhR|9n)Zt(wpyBt*Z0D8BP(W5eDbzBPzF(4kkuEz zU5PGnIt8M%#2p684ppf=X}mWAF3#H@ug%$Th@y}Ez^aE0&e=$@HvXeA2S#hGD%f4N z@TCdAwkz2Bg<)$%yFE}vOK|hLBGLdOnECRjiC5{}o@Ak28Gl0K>bU-trII-F=$BNb zL<>Ot8)VQ%I$z95Opd2(HBzuH^!kUWqQTVd6D zPS!5=x2+9^1Fg@oST(@H@NZWb#tw1k=g>iR4t$(!oB=S^N_M;yA_v?D$ORr2zUEML z_yutiQD8N*iKZ51cI8r)+?Ie(8~*c-N_+x1_#-TcpLGjVotiuK#Ur0zZ7 zNK+%bdI@O)`4eRUl+)q@j-sqyl1B2!)#BLf-8^bkhP_3GjQuGVGB3YvRgw?9QGnn{dZDU>K_bPyTJqHA3qdiYb4D zz%+e*sjYKPSb&u|LR>X=Y{g|)9Fwn~NVC>-3X(H)C+diDk66 zJPllHB<71DM$7g>2$H$|+@(i6vjpT02JjAyPg2g$nvj_lmt-_=?UA%~+uJ+zQ^WmN zK)YzPt{5_qqerbB;Fw#plHYWmrMn@UwFA^qO%sAO;-!gXZOFFGasH!5ks;DXkefIEF_8*56>{L%0cYOGQp4*C03sYdx3fVJ_(dy+yGMdpG2#^2qN9 zE`wlB#mxgdGc3W-6Ze=foD2|0>o6H2tDtP{E!!15r)F^LJ@5e7-V9P-a7F3Nr{~*b zMjPqk#b~2s;X}umYgSbTqwmtmQ7rUutd*QUxkA{ODMgK|WinA~&s{^+`Ea!5@p`}A zR-gW$=*6;7SGrhkT8C9)KuZCijy{BMt6*tWJOp$S5E+dxjA56m?4!Vg=LC{y+Ay}?(9Q)V$b4vnT-z|ph3f6?T>6t^T zgI*$8ALkfKG}-3qe?ynG56@307vr^t>kK+auOl6HacxvN`nI)ma^vR1eVoVq`@nn& zL^}UoeVoq<5c{!c0WdqC{+Vii>`Axtb{zy(;3Eu{vvXH8d-m;JXpm~%MW++qOiQv+ z>7xZ4q?QPLbDVVwp*~1q1=kR2d^>GFQgt*J|4hmBYUV1jIpPn(0~Q1-af5{3h5Nj@ zt~&+Lkxy8iTj+dc0=#kEHqASozj9!iZboGITaZO@ z`2S3N{x8dC>coGb2>>+BvEcxOFV`ej(ssVtmOL6OErpP}II6l~# zuqd9_=LZ;u4j)&=A-*%QKvn3xyAKNrb1Sau^ozm+vH3p3#NW{Qersej1{ZOQ7L`Vj z>3Oap3A<}-w6r&_eF}PwnQU%oy?l?dR_A|PvjucNH<4MCUfcYV*{hbPcuxWKdiO@N zq4ns4Z`myin{1mm$#-7xZM)Ci@ES9<+FCavAkiL}TXBu=U$qtJ9b8IP?vH9?)f#y- zI(Drqozoj@IAl8;XqGNa1iC6d9l`%*3~4ykPSiZ-cD1z~r;U70g!FJq#;IocWO5b0 zN&f4Bc^&qYAW(r^G#)hN?&j$Ic`2dHyunHGb}|^vUXs~?kGn5cpr`Y1xwq7}LA6XyZJT+42wxR6dD zafX(JOv%B>^cclVNjN^3Ok_HX&D1>{cC9zMe!6CZ_Fcndf*JLxQM((}Ew>`!8(*dM z$n0sO38}~05OL|a?wyrfa@r{M9w&oSv*>C%F}YcT@{Cqj%5=ZUx?yt+E@bQUN_P%g zT{I1qS7KyId3eRb--_efm zfh`?eOdLJ_k#JYHm)oXj1Kr23&ZRFn;$by_X;fmCaV}4uKHjJ+S^Bt@;VcPey{HR#( zPiW^nw;sODw2ZEHZq0&6V#@gZ;^Ob?@BLpF*4hQ`M`RUH_>-G6bUsKt+rxvh$etXs z#xnyp+QbUlhQdt+q*=Q!V`W=ai{oAJiE<)nyb24}C@`~5eibgQZFKk2$UQ=x4-piR z<$DLli3gE~7vH40?{?-?(H)|i17y3jm`LNs$kT}5><8JLEU*5uiQb9Z!~+v;GGBVy z0%PZFWx9$A@!AWj`$c`}6g6eN9_|9xuEZVVRlMi2Ha`j?=1UckZ1<#ojWM@Lca?(| zZQDlA<_EGjY`CcW)A>e2M(P2zLMqBXy%1!rg)-X7<9fT^pRn7Z!?t*8%;+X}b)C5R zr=Dq@!Bi!(%3M4aB21{c>*=_3j{~)_3s)5T5#@7`O9#04yRi!o&z?%#eqPU3jJ03J zn$v;$Ic6#u%aC!-{`W^`QRw1KK~CFvGQBo*%QdX!Y)QSfJ@gvgUYI%c(3*{1UQp%eHouD`1pyQwopJc!sl< zK<>w>$`cE-ET_;ag^xd2>d0R_L%Vgs+o%heQbbwyC3!3k4~THl2Mz51;Kv*c2WE_; zOh`v*)Je`((FS<=Y^ zda_QSKJ--S?vgmUO$3cGuc<|H5_xpMuC<&pCQ{oX&`T#T$@dO=z9Vbuhu}UzT`kOX zFU)Pz*|2>tDHtxsjJ(gM{g%_|_WGlj`be~`&x|qk8QB>EeS=E-#Dmr#zVdF#@b0|g ze2adgYj$j2CE<2{706-q*%<_Vbj5O*Y%_Zad-poa@AMT!9}BimNJD5&{>v@z+_L!d zT`A!45iTVsYj6MEoKe!4f7RiN%=6Tz$BBV3sw-&EM`s~MX_tVc8++UBPl*!f8|Q#V z5x_n@L5o(@SX2H%jt&lxEwYGd(8MHY9C6MbKi}3hHcGmm11|Wr=olIc-=LteU5gZU zlE;PyXgV>e2EEClBJ%|Zg@b>2EBcXr%~~z@ni5uM+RKhwEHPeJ5Z6@R5KYBAMwaI% zI8aebg<|zl%%uX8VT)5%1MpHbp7;LV_nsHlq#8D|`8fjJ0bdE}*|zPeOR;eo% zQ6NqcefwvlaXf2=-KhXAsrK-~v?+G?)2t#bXIAzrx~!A{<74;EU3 zoJ=37H_ctmqh1{90!~zt;21PXPOwDgc0t7DP-bS3i9f+#*7JUx9anS~NL2W;txcAx z%7r5u2F7fVd=0^p|65^r=rpm!5rrort?I`+1^N07D(IQj`759-Q7#A|NMSv1TX@#3 z@*#yU`(O-Ot_7c4;u(r$ekz<8Q35N*5Ogy5=a-T0x{EC(27p z)tzA`tGwQ7A`d=MAm6bsy`+N+aG)2(lY#Weu=@dYyodGkJNqi7yV$yW@#-eFu6Llz ztZy|!V_okYcr+npG5LO5|^1i3kSV|vj+ zC1A%Vm09cOR38N!_`B#j6&ZKb)A#{$$3z-DOpIZsrqq82{vex7@UjGTp$p(5wg30g;h%BOg2Sbc!2Fr7aQVkJn2tMcM?5c z6L<^NbS_gOn^VHsQqjMZZHF5%DifumbjM~n^gMs?l%4ILvGjG&e;2_C7Zy=QJi`vv zHf2Avuh}SaBDN?$zltr~%G%c1VTW$JJp3TSTXptERp705ILQ6+=HO=0(lGc%QxM>K zi|i0wg29ECtB-bGLuLAD+(s(q2VV>pZ+sxrQEeh+bMZKwE=$m1+H*@6|5t0#FNB%G z{%YS7xJVIUQ+R*0N#l-^bsWTmXCX7#H(rs-T%dv?XUIdgf%XV7b)P9c^Y?f-vV*HM zTLJu#6t*Jv;5@M-Ks4ee-lCe=O%q%hhd6gh0rf-HP1KWlsA^uDGrP9@5zXduCA@L0 zl$k-8_pN&131(R3x~^XH%|tEX&gIMM>qkZrvEE)MHnpAQvN0VE#O( z4pT0qIDzk;*Y*^;g!J1iTiG{=)G6eJ6!4}y&YaJbF!m4BT#3M4T~m}A*x{c6$y--t zq-+7bAnpNDyGd}ZbO;=9DH@)w;xQ{2Q%wkD>>j1YjY<276c8Yb6%^$G78MZ7zkgin8a zaiy>+4OwMf{bHz_y+HSjW`;3pOIY4ai;b8LdK_9aqbU#0c$VSlWk(x#TA2l+^eN zd$3jBIq#b6)x(D4xYdVodVbF$DmFcG;qbJ7wuJ&IP6z3{_`C(^?2_=C`_?`=v(Y`$ z>HgH*UvJ)a<+V1Dr=x=4BQC$EvfAmY#pn{azs1CsXA^y7(%MEGc0A%vJ*olU@qo9G zx_2u0pAE4jYA={Qauz3y$=L)_ z`RRL!W6r$Aqr7dR2Ue}_C@HQPVL!Y~>qJztdcFHI?y^7`smw%O1N^fEy6MD>oVnSY zv}D@)y8`1uJOq3`$L;~IbAS21ik;XOTEQ(M=j2e4cC4n3N!=nHlCPoJ0xbXpc7{R9 zO_e<`muI2kr?Me`jf8)d+$}%y7%1~4YgG$qdMU4$nU&aQewFWIubOixfS1hI` z^4LLwe#V%z9QY5!@mr`EDB{oqPd&n)g6Owm?*nU zyOvFkY*yRh!TgT%{#~EhjJR;swv?I7nhTk(?G7J)C4jk>jDM~QvNm>@h)ieFCQZP& zo~*um*fC{SWHF{&b^17N=2@;0xM7=Ft5}n_T#t^B&Hvmi6rWw z@-1h^rl8$o8oRzs>Ct!*Lic!n9JS|VVEXKpTTkp%|HekFyt!mG!7YgvP2z+DD*Yk! zt=f$bAS`jy^x6L-VnW?zDDcXgny# z(5)P%Y&;dXlms?$b(Ui+Mz;@~FuW$f-z)xq=~|ozvxBkfnP+X(NcS!bEDfgyV6klp zIEyxhvz`!YX9SU_7A|teH1O1t;aJYb{UO9;1N%gZJmF1=bs=6q+!jkd7LKbJ`eW$( zd)})QZi%8m>Vo}Avmtdrzv;uR=PugEpb7)y;#a}yym!GXX|t?mVD-S^A7{L#Gb!@! zuN*F<9>3R#vdH^9S!yajp~Z~OLs#9M5tDYej51t1i}S=H+qF^7)wY>I{mV%P>k%O1 zS;uhp&e^hba-oe8e2{eLa&%&wdyWi6nHCS8WOir1*jcd?U9{%RtMli_nMy8le8+dn zIwGZfa&(iEACLc%CVk(3hS#mzuDAF&x>enjVN!m+N~6}5ehoc6+};PR-?-C$zPw+a z#$V)kbF!v#z~lmzo-?5I-u}7`?lpLOdT@Hk>gj6d=;rY4g2~A)W7|5GgqQVDpI<6H z^Kupw@!1VlL(#*1SYpAHueUBuN;O^T=<4R^(liE^ZF+jPhs&kX$@1p%csD?ot+CCQ zvwdTlL{3`2ENu>4SWJ${YT-<5BB$qwk)LUH7sX~*O(rQl({l3Mx2Nx#J&~#CJ=a9& z!Sd7=GHsWm-}L7URPD1|VfRQ%5n(179+YBPHRFBmRND3W{`;cUKDZ}Pb3X%VM*}5B zp}+bf1ea^~02b14lI9~iD3&zYQ5}f4tL{!R(N1u`f0519AXm&*TbK5Pq=WHC)^&yS zp9RQCK4+zMu}|cEvK(d^+jKL}lNwYcifhgFBL!8G zDkjU4KWiR&Z^dKu`m{zaN>4K8L7k*WR94Vai;v*%M50E{I#WOM#VL-0?KghXMdXle z5Y7p^-K1sT%{?e|5G-U*Vw{AF3lI&KH+(PmQ5|>out=L+OWL&NtuCw`n#6}hLY{zR ztP|141ec%z8#8M@+MePdrg8=C2no^nz;a1%7o`%qbr{g8RCY$RA5cc`lCCI5OB%4H z%?i#cE!@u;H7d$!Ncg1xiX0LRO=$?Bjh zh>s6r@?~(^z$RhJoiL|gqr}n=KtH$I zMFJMv8M2%-J`hZMkZXSBCGLLwH4=~Jj(IuKP8BX%2aA5rmJFq~m5gfO1i>G-3vPF{ zMCH>w0(rKt4ue_`ArocVu~vO$^dwrsxJ6*PzJCR20kWIUoUirBc+YtNxV>~=WZRR) zUi*V>kxz5l^b!TM7@SRIpU51yexScfV{g;?sSfbr9&l)1E@{{{aBdx}M+#)$b6lC- z5FGDhzn{*Sh}pxmirdq#u!rbiUK-{?2G1pUbt+7>F)EtZ5ear`B=+f=a#~4Z+8M2) z+)BmChRS30vXkb28(yu`m z2{=sbnU#cmI*8CXkb-@wgdnv+ZKMaB->ZDuzvc_^&0x7InXmdhJ1gBRBDIlGaehr5 z&m*|B_{{QT)!LMCGV|I7N2K$3ys)Dm2_yePEZ>5IfEeX*yZ?vdE?8>8ZSxv`8KDrv zuSTPj_lj#yuo>iq#Z4Pd&s%6uujnyH4(y5XXT)vFF46x5JqBhC$mZ5vir4iCN(&6< z_xedxNe@tZ;t5O?9!LFF5U2~{N*p{U1=L+bh^8QT5LC-8lp$GLnvbh20byhWAt2{* z3p$JK1{_vV8P8>T%e(U_BeltL%FuCI9X)1foDe$ia>v$&94ZubRI88$)T;*1)qdqu zuo7dtR^4G-`o^m3Fr*>biBHDpqHvuVM##vT$d;yNH87}d7$&65;-y_YMNhaERt!P9 zM2DDQ4IPY0U27+{=EAwTZU#`;Ghb+7&3PsMbP-cU@{y%iLZ3+V%ot9}*0W`47cAdN zCK^Tsnkd^`fyvRmqscy(wQsAns=f=#CCG<-h?g5d73LX$>p-n7q0eysB%St0^wU(` zt*CY4b2pfB>Y8mLlcV((b>11Q!kapmXO(*!V%|w$54iNc;&bQkx=oiPx)?XFZ#DQ= zVdp{&j9$}?>(+X{!{UmB=#Yrt=!Eq_Bk|KHuRLhIWFSU3$WkV^lC`_ULjG<`CV`{f$4?x_$)ygj_-R_bG8u^05hYk1rI}OM)zd_Y!xQm)TSMsN76>;Pk za3&y=6tV8`rNB|CDKE%^(aLS)MsD0wsT4@{S_3I)3)feVS_sd-!fy=g(1rOr|_Bc z_{mH}ro{}eNh90RfP}%&oOEapF>V@kNyiTjyj)SUUHT7W4rd8xg5=tA=Dn@`lW)es~dL*mkpE0W(mgG{H z&{;=4s-%*h5TgSMbsMBg{y^1{%U)K=)TQDV*2y^p{lg_0d(k z9K5E>WQd$-l@?15joNFW8ly5{AdW|gb&L{VTtRW|PmeJ%=vbt->K3z8+*pk~_` zT|HJ}zwtAWy%f0MCn+q(Z(26h+2MJZ9GI9u?Cu|#*5v#5XwpZ>_YTScsK>Eu#Qp{G zZ@QK`FUadzS>}2cDMI(ZFQwH4PGfH3YG3fo=H8$9TrnBNmeNEp+vd-oAP_H~2d-fh zyzT71_JhA^S60k}n@_qnA*A@B3Ct)sQxosur;6$b1QPYMIJa9uV=XjG7mr+6$bt12 zv>$M+Fbg?pL)X_#@uuF(NQUj(v}@->`Yk6)SBLvu}s9$iXzxRJCb zwG&%_SnXCEc4kpJohpc(Jli6&JW&k_W3L!KX=D+ibh!4z`c$xsQjP<0x2WXov+d z@D36(E6+aV@Ue3(WuE5hg*v!%q^YjRspzI^pg&})@E4C}D6NQr#prQG$TKo8ce|OX zyOaj<1s``MnS{qgAl1VZg@P2Vky7~5uZ%{bqgb;Ubi+@!(dLfOv=>}v3Nt9aN$Qb9 z_Nhux{YcpTn8FgZq@AvD69QrFtDaH{SSmH8+1?t?$Slz5`COx}5i55a?w2c3Hz6o6 zW6@CgmdPL?w#OjLorJle_$&?slrbL1Cs(`Ix0eG6YTb!iU*$9W*Y!fi5m4>R1Pz-M|6WFMxFyGg5hqBrG8%Rb z(CHWornm;ircb61Tr8js;+A=%ZTKp|s}_HdPv6*6gRqie z`PZFuY$~rYfl=YFqO*+ni<3+_@2mBYZ-W{UA16gTgf z7%T>Wm8n7vt{;;g30PN_G4}~U%gtR81?*fM3(hJ^mU%b;!zJy^x4sMN$sYKlJt4mc zaL+GU_2sD+t=6^7G3%oTXjX|;@rSrTm5OqxQV~DXI1mc0gx+^LQmgv(5e@nDR3YYT=fvg0PKg&kas&(CYif6c49$$1 zr*t42IFeq9mFAr1?;aNcN6EH2#}5N5l|0cNPAM0Am&YNUaPnExnw3idKIm*oX={8aP<30&M%%A ziQ!g*W}22A=F;{}H!8C3(6o&JTz8_eLn@3D>pkWAw;jWf=rM^OFu*@(%`-MBecr?G zJPiSn0vF@`Rf=Vu2|2?TVG(zTse?CiXM@KKax$`9UfAPG@R6BVLozHv&o!x)VMX{3 z%-9#xMOdnDnBU*MMRzfo`Q>u%dFJ>5;@I8_5h=h_aDQE|D_r|D2JxT^|A=W+bMYFEMJn zsCdoJl6&dEK*#PX$Vsk#1HcB8lb^e4wJ_6KM>QjJ8JY6rQ^mmjW|L*N z6Sr>43Bp9b!kV^!rW7}G23~OF5cp-Eq$(j%n`qBu2n~9ZVCDQ?Uy+vx10KdXP+N_n z*htlqex{_=IF;&)PLaX70;Xt|oqi-133#$eUFE`#4FS!#v?*LUZbN~ccHQORDdc1s zkan2lK{dVxn4Phq<-Bohm4Tom$b#rQ&|RP*jV+oiDi1C2xD^a3D1}alnlIDOmqQf( zn=w%>kL6))Y4p--f;Kk_|3ps4~giC^yq=HDU#s z9z>)uqhp$b@%ruh{?=qgPuJlFnQ_OWh2}T=GNwGPD-X)ORdk!Q?B|!H? zJZ~kedr*^h#^z$yBYjt}kmdWqq`9H?B4De>h+aE=&ITi%(WyI!$Y?nY%wWw0^jXuk z|K@%u31c6Hti1b{S;KB1TlB~!x7}vG92+T>Pq;Vy(q~;PLrSs40Kn4B6uS<;LNF$R zSqFZ$g8MavHz_=VL^RG5TO9lca`MRfZw}q_(&Fd|)!Lt=_V8ViR2JFECw+ca)K*D5 z(Yo>jE9ZpGLr9XJ24MJVuFn+YqlqP}eTm)(t$MtRZrZw~%I2&4IDe$8+pk zEz)(1|LnPLXI-bjQ82Osii}_fk@Qg&KQhRQF=Zk9C@Y#pbWtP~d{E~P15V;|fgARH z>;vmKp?3n%T+LC2a+gDW_yIDS^rNR}XCJha>2SM#@z{?TM)`l;%nXK6!*@y{YTv!u zGUJ}nZE0qTD?`Buzm_inANT6-)43n?9z#u<`+9Hf@>}tC`51=OeM*d<@mLc*N6W3e z-;#stZj-9rE?o3!F%jqaJt^-sz*Yhg=gv8y!~7Hiro2Ul>uzWneFGh~j3Uq^As2*$ zy+d~E97(*(==w;cHS3-sUrk;y0Xe4=3bH@UZ+Zilh&s5l|9l)Lh11~ktG>`4l z9iv;9Z%BLW@n;Gl3j>b^SN>yUE+JVuAS*`;tguSGKSy}~ilCwDp5?z9 zF{-r9nFz?3XE9X5PSsO=iOk_6d0S-GX~=0KUOW!{$KLFQTS#eJ@==Mr1AE;M5uU2c ze^8XVZyVNb<^XUfvMeHJf@T!{3b5-5n+`*1P0TPAAF_q!V7Pk< z<=Ws5Tl$Sry#j%*_|B~nW$E^CcB|q9(1rEtzr|P8hPRY1s-vI~))$g4|6Z*BdcI#Y z{@WOOIC%E!;pE`UrIY*l%FX>=9VgHAemF11_xl*f{1!4B2g9Pl*^>-Yx@La|Y=8km$AE%lr+RiX(uDLeQ zn5I16lo8^)97*NBS9xCkIYvWF=+uP;)rJDl5?sEJ%9iZnV?vK;G2X=N>mIX^n$^W+8!aFOLL>x_LJN zgAc)Bb1Xcu5mGuYF=@u*Y)*OM3OB*?`PF!;u= z=bjK9>}-OX7<01@wrSLg8-AtpA{koC^W>PK7@vU&!P)5SUTp4wMF1Is z!BkUWBX5y9t!bjE3#AcY$b`F?@+Pv@^@0=OPTwAM@*?|Aw1v6=5|J6kI8JUVy z^8kG|^YhgZH^FojRZ`&)Gb$Q6d4UYNSpO?tG_W)b$14~1N3roGgo$;Qc~mO%f`9-u z`S&WEMnP83OM0?sN{$=tJ`1NbQ>VtCaFG%`VJ$!PlueZy7a%!h*X&`D)@Mceo)9wd z#~YelmJ9C{hIJ`U1!jZFFpHFko14%mLqpYV{Tt!HMzyZe!O@r6$9(nX76GAK*q4ap zkdXYzJZMuGD>gZT??T^a{u6wX(jP$K4% zAsk;SJ~chcy@Y`x!wJ~oe4o6W-PC8V_(=AF5CK%PE!34{t_&)@w|u)323@GA(LXt6uYA?DUC3$QGe(Uv|divECX)QdJ;u?s9K??epbpLT8JXHYQibk zYc5q%kkH$v*`6R&T|HVlN~3ly(<-bdIl9%YV=lrse@l^9I|?PSTJ3?)@`QM2X)Ww0 zQINB8z2HQ=(0a^QUkuC$c^%#G1RH(S|H?nabv)^GwycniyylkP{u{4W!p2!t{MULp zfkQNcz4Os>zd~Z~4GHC*yW}bp`5FkVg;!~pYQF?g%*43-If9PmB!7MeEXsxflK4qh zFFIbmho-Vr^fwW}94WAN{ba>pj}X;So7-#ARgvrnOotN03B4LM@E&RPk((150c2D8v`5+N510jNtp*~3*@6yh1nr6rYuL{M69mGJ zTc7n=eM_mptf1+%Ag4_W8sjB3iMU!@`7<&H*@n-bkd%A)MH#@W^{AnmdRbSb;R8bz zvEd>(aB(kX$uLFE%+lu}rnWw49`L)abUPxnj2CQ+rWaMlB;+QsMdRDOe$U=7L%r{Z zsMV1D8K0#dR0_PTL50hDO+>tH-~;Bt%LD}gPYm8=-ja(CHVfxwbVGzUp6Y&es2;f@rzH$q59(Vca0(#lgj-llIE61;r{yTxG z=7BKVB|PGKt07gzzkgDaAX+AwHKd&@s^lA#uBDgqfu94p1zhPNsi)w4L({&Hv;qrv zR2mAK&etQ1@5Sl-XC)8+!d1>oaB{;(g<9kh1rMFo--}0|2gi|u%QM7y;3f2lvr+We zTg*v@9AGR+gQc?+K{mz$EOhNrnswtI;iqS6%`2{nzZj>kxJCVM0A4_$zbM$uT^D)1 zf%NzRqNXem@Ep6_xEkli?%W<*IcB=*J*!+fOOs!S<42?Pl7gqi;{)oUSLiv z?i%rPgtXw}Vgs=Hon?$p&gnIsu?kT1c?P{#m;BetA1CmdBkdpSs1sPBkLBmTJ@t~YRoO!u%xbwaSURro4X=a>zJ+ibNLT zaL-6~3n{C>s78-$@c6L5#Sp93B(1X{dNC{~DxHmJz-fCnIy~mlJkCCR1kOvVLQC= zuidsqR^bJcUp3HFM4R$_+T%cUNZskvn!ke;nhp4?25WpUHr?gE>gA{PiLaQ53O%}F zq~j?gun0K-(6@92pOzfSs9>1-IZ{E0>Imtn8cXeU{9e0ld35;|UwUeutX8lxhNXmu zn>TVJZ;TBEzxhswkyS`lv57P4Kz>KXGb)>}|v>memp15 z6uxDb&}5Oav6W4!^;$ph_9YnI1TsB1wHBOO3r?+FoNDkxRQF=Myt4;^-7=Qw;fcX` zT8lg>8+=tt$n`yBm4>|gim+@jk}11~j-;>**=$n1^!?OE)#ID~5ww*Fo;~Y&e>#LkvKWl{duOcD`9J;2gc1sri_cr$H>0*= z1gGMYxnP$F<(0*p{ysJ_S$kr#0~3?c#PlJFy(ILRde#aPrb(T`OpJ}44;wRDdUvY$ zSr=NJU=l5xy!P&(E#F7c_kXG($L)^e+`*l)Skk5E?Y!*2?VlYmf(>Eg21u_uj9WV! zw0qYs7_eG<2fn!JymlY49Vawno<$yA2VjIu($@J%o9RFaK%~WL&mLK% zHlx=t*22~knG$M>WT#o{|55wSVzl4v?O!1}Td5UoUybjG%c>=YPh;T{gDoqko`|Bz zXp^+6Nfw7SIe$fg>Ccy=+0!jYEg$)#VZag2|EA-aUzWiQ>^c3*(dC=rHn_YxRez_? z-X-^^>Mzp3Iw#6ztK;gfldKN5?6+~Mev$&(RY0^={5P+wt5sc~rGG^|i((S0>%=)D z?gIzfH`1lQDUaI)qAH%8AMp(xo43i6b1y*r1+JkJU%*CS(GIC?pFg!wYjxcY{w=gq z>;AvsN~c}w{3gGAVuQ$y3!P0K$I&7}*tUgvLe$g3h6={XAdzEyxrmHqVZb>fB^AAb z^`6a3FBW`tWHZ*+{8nzFx^Ff$ZE@QX+OnmlnQ3jgf+R3nA~QkoPk9MTU+AN>V{^?3 zw;b1Ics1csl!)+)#qR$Cq2eZ%1G9&$dCLnc_J=Xl!z%i#L@vyb_KN-U=g|s*#&05j zC8?0Dy22abk}v|n`*qZT6E?(rKv)UCoLsXF+%@uGfVI0Gfehq$?%VDB@VA?0rwUIv z=k!=y2X~5%%~GY;Gb|*2k*r@6g1b~1wD&J%SAgDSgT-RB=81G)>Nx5&bwnbGo8yg+ zb38)qu|=TPQB3_~f_@!5$Q`0EtnC2EI{xwhXklA4Kv)Cqy|}n`joDp$&)X zjH(KCu9poXVqQN94~PjNYEnD6>zIKMLv^EXIN3-XQB)TR5D;jaXg>)W4Ti+m8(d@O z-$d7XPuhK7HLzbX(C)#%WGc5RNlsXv8~Cr_rO)sWLv$pIR7?7KCJV2F?bV>eZvr!V zodlWCz!0DF(2zUEL`vRm_5i&WK(CQGeAC_}8+Vh`Y7yOVX^t|pXhM=LC-D5106MYw?tQ&=#3s z;L7H*F>+*nPM;)B7*z4o6C3anwmpvG=3|(~Irsl1+p=d%s(Z*@B}c8gBJ77<@sq6d z{c!Ibvzzk1Q6>*qhfH&DAKmMqFJ}s(>zEl@r{u_#yh2F})zpV8Yaw{0VLT8A0~SY0 zC!9%}-XlB4po6ULXN@!AzE8j;a}3=lStxmPWVP?Xp0^%AF3*HBy`uGgD-!@WVVL5~ zS44W5 zT)ba(lzkod6N%EUqc*;#F9|X0J#am|4<=<;y8N5q)pEQVucDPJvDU}vbfvCm@Wx$@ z=g6^Eb~C_0TV~46+i9)lt+ZCKa-yns4UxCDIBo#Agt1h7&DpKqRQYDZf%ZWEt)exp zA1pjI9P=#*t;jd$?#iMiahID)(OGjcQwf0p*y934%jBVz_W}KK2XE2WaNG$Lo|Di7Yzf{MZS{{FgzSN8|t6D0CG3i1~l5!Z2<)s3Zg4< ze{bs(ONG!LL1!S)Q{TY16?;TJW=4z7HNFA21v?E5_Z=6yv!DxW$lo57n6f#02i=I5 zNJDis{KCCLtA5|Bs`+}(i3(*eYPA%H1!KOV6*4=s+iaP>N8+7CZUfg*NYiZkcxV7c zj6r%~g~=GADB6cNk$U^VeeR=ygFAVBEmEYTr;7$`14LEb;b6K(&LPM{?Pe8hW0FH% z0jyr>!5<&DZe6>eHr}8Q*P?XRwxe5VxPg>Fa4=3G7;$uo43i zL+{o2P)oA&>CDMDxaNF=bE3`)FlqB9VNs2>HkmQ7>@7}ba_iqnwjwzh2+-)1B{q}Q z)bdKaAV3Kl3m2{wIc*sY%?WE*%yK8sBr$TSn72yGaJdV1hQxZl?iAxjOv!~N1J>@NCm@Wu28JY}-}S~fpj98APtL>o_}S!R`uoY`8Pd`MkOKxA19bBTx(7L|aIRu+t0*)mU!{=yb;2z7;UzKlzv!N6XtawLF1 zF&7e=G1)1NrNajVdJZ29Gr_DmlZ4?zOXTBe>Abg7AQF4u<~33VU2)8rI+Kc7vbe84 zZHP!d=S_0MJafN}koVOHI=aq(J;j|YiQS4gM@}21oO|MU!+!0<{S7Gn+KlSn@4;6v!{Jy zKWD3qt$`bCs=us>zf_ok;*MvC-zD>09Mxw2aYGkUlI zQ5wL}(qYKL$Leep65Thlm0TkeqJOM>yO9K&li&Bu7&;%SMC&s4DUN?{akZ{Z3fU>^ z7&x`!k@kR@yzd9ICpw>*>l+d@;Yp02gta))IcUvu9prK!7L7%+v_1O8LBF$a`}%74 z?Exh3%!3BT0Vkhpby04058*UqM2W_ry?J6GxUp=q2gT!PzJC{Y%!WD}RnH8urM(-h zy?@)`gRQ;UgYZt4TM}l;WsmwibKqCln#=C0RhBDMuGE|OJ}la}dqP*U9>kFihoV{T zNi>v*IY9rcot$Lm!#-ZXe7hI)qm^8D&!Wn8y>YWv?dQH^;YtB@ZWdMA4O!BPX;#m# zxpvM=R$A2yJMb1ZH<32fL+};^hoPFn)JnAWBgdg2EV8JWzLMr8>yrrFiIk}tSB=zl z({3WHi3?~Rr9}DknXvhin;2&Fhd?fLNtGdXFC*t_+Uj`BlhBXzXPsip3Q2qERX3Y%9n zM^?FzTKTdvnv_03bJEkvc&S}DS+`R0F%W4pX|Qs=QTQ}$xYofeu`UQc{$Oof6qMM- z0Xcmr#9Fg9B2~aAs?PR4>3jeXGjfQXTBUooP7XwhGqvbq0u2X^2O!OG#bK;&BxTQm z+@^cga1UDbY_~9}G5oBAz`X7k&U6OPX(%~6Go%Q;Uo|O)I#!f$ZMQ^-jWxnEDXj`? z2^Un;VmO?c*X}wn35nc>39P^ST4u|&vo!Qe@4;#z!#1KO-(= z<98_+8M(#HPyzy|CGG!ALPD28t< zzwr?GRe5j)(3DOb>@36-l|!esBJN1&mjiJb#?9`Dv`ohEG*E@@ng-JUqXvAZ0TWt9 zi+T>USGw(e(+VPFJ^hB2a^AEOizmp$YojVwlKMxhy#!I`CnrHRO`~#}&tPCg*RyEg z7x=+-AkhJBhjQcm^5_4zul+^J~aHqH2RNr^@8PeQ7HwbvR5;qqRoHADt@`?j5FDo3qJS$8kFj zj2p}H929S=wSfUO^jBId+ZG!}(*qs@F5s8Qu}%I*oyh4z8>K`NY+;b;9Pf65Ym*wY zo)88S2l*rjzv2XwcIumDj0I+_!VYRLxXU03GA^+X8Z?Q}|98Wor4%HLe53r*2ZF31 z9R~nXAXLaDS^)`+078G^Xuc|#mde^Y*_y8aOv9W52d$#z(9klEwShYPmlMKGDp!aQ7$;S%q|$Q}iCp!8)#pI^g&r|_2wb4%|4 z^X>Ip$@jl&I2_jcY98!3whhw}AjJ6s)w5g4^?q$s*EQ<0Hh*inqk6`v z&t&K|V`?1ffABfTj>%ggB1$z6!W%q)esa!MsY~=S$Il$bWxN%xdCUa!`Q+MofGm`0 zBnYpO_E&c!r|F_uHRKB2w$8wtsvV@UY@jk{)%_~2h+Cfmu*GHtUVYoG^9sinN4`G~ z&j^$WL-q%tiYjDogbTy!K%uwBA|a97zEYZitSkA^8xD~lAekf5caDJMgdMV=0k@?t z9OeKe4V$n)#l(8OVK&_&aa7Q5uVJ1-=QCQ9{r>G$WnfJg`edb7(d0^9r zA{q{3X}X}M3u(I5aA6we1Ip&vt^D{o0fO{{FB9t%5}%wK*Ci*Go1rK(6<^Rk#{%mw zV#RL$7o|Ln%yauRv9$%e8xscGoR!>#25>eGJyY25mCw%{2zq|z#jezesmgEu^Xd4mIEuypW zEurDXY`liRZ~gab&26Hn9QXIn&vYQlJ3=Re_cRE?;A{Tz@e{q4?X(Jl z@fsLo9qis%?}2rosJDX}o)c!dsDDGdl5i}jbT0BYa3NY*P%y)sN8beMy-l%;5kr|U z`Nq$K&^d(`me3Yxf&9mM7wTP+@~#jznT?ejPNu2owmHrm=V)^lfy9Ob%#F=*LB}Sx zfE%0jp^k1~_wu`Xn>ZR>Ov9(5^cLq1lmGL_I<14w)aFD8+JO~_ECc26P@9t}hi4)( zFB_xs+M=13Wz0THU1;81HkY-mh6CUYF%OtNf&~))kzMZ@+FmhoJ5#R-(;+jDdczmA{R!Jm zH3$;Iqf1RjZ@2pbRit;`3;kJLWSqv>NNy`QH8j}~#)doH5f<&YtOhk+3AV(tB|E~k zt}vMmnjUt@L~pG~yZpJO-}QO%JQZiTuk>mg!-;8SV3|!*ncIzu9b#S++J5N=>MQ>H zg8zOSdDF|w_s`BQFRRPT;_@=Pyj;#a^>ySS2E--2_@*b9mv^)6^k*t~c1AbvW7@Mr zPyG^o-K^LhBDMU9*6bC1I(_={lkMQYXOk%K)OV40c{%ll-wwT}J}vps3-rw-^oFm@ zr9b`X(whZ-)9%msodr)P!R6)0Tl(}D`gBYG%IEaXQ(r`0s4L|U{XeYf+18)_><2XL z+3e{S#nFxLQ1$fZaJJ=7f~VWj(|}5&HT4-9ffRC^FBaS6x9xScT~*txy4fl~-W)Oj{W%z_98sShr z@A>$tk{^2gW#4Omf3cqr|9ckLuYPmC`XWI5dIatG6McAhc`_RYTmSNo{$7sk6U6O6 zc>6HRt5XEDpK=o%0-s>%>698cMdW%Exb&&?kJ0JP<$|NfpSp2KY^?jOh`6Z4ljxM@ zbTXi+JH1wajUK*y6}HCpCj(J|dim9hH*eZ+;g)LOfAiw&*WK4ss2L-Mmx(SZHvld= zx?lXI{RPT&VE!rGT~Ysww{O1>JCpuhpuT(a`X8^p?Y^M7rK81t`kC@ljN*$gzv_%J z^f?fJ2i#8PoUzUYIt!w{KOGW`1wKPxwrw_?exdK_HDbKD@aai6S^+D1AZZG?*?L3i z^-ldQ+Q+8X{aMoy>b?=Q1Wj~%YooJ-<Htf?dAdhlSh5} zsne%+XJp4vNA!iH$Ej?nawi>dyA_k=p^1Q34I($r9N%iG6LsTXA2KzxutywHG5Sa{ zN*pC0>8y_X?>@5m9=nV4vI*^d@Rjc+cI766a|AqfG9TMe;Z#a}NEI9ak%pO6`_sLM z7Q-{u1p>wxA9=&Y&=Wx55q6%+`0m>YdJ}c;Cg>fTUDE2ML`m;?=r7zp4|_`t&rF%d zOkr@{)%YwB-N4OyiD271*_fInydN=k0ijtEbqE^N5aoKmRz%;t-{15^KxyyLvADQS z6JqMYa+n~-j!7=GdRlPYOE@u`A}O+rO+)j^8*N{3@YBlqlzI_Z{%c3fv*^J%oLb)) zkVKXFr*;HQ$yW)(p!4k(KYx~=a}n#>%?GUcQX=O?d+Nm2Yo01FA`M`S6Xr}DJx7oY zqY_CpjhZi&NL2Xwq^Z^hfs}H35g_vrrVo|4^N*YhmBje$!QLF+vFqiQ1YOG8Vv$Ls z$pQw!e(@y9Jz6m)-O;_hoU|9t*gd<<-z4ca7jhDYHY+rCgkOlmHSh})6SIlWpV@ZZ zYY1g_A^NmNeP*C0nen6-wy1e(BRTXDRN3eM*>Rh3v=6R+7eM216Rj)&FzO<_9ra8T z%|VX%LrxG%RNRiZsaWX7@d?=yg&3D$f&~t#{7;iktUH*%^~Ov^+~A3I>H+g(4~q%z zCl!bI2;C^EnG*;&6t^rSXsfJIP#HZq+)McBnUb^nyE3U4dH9X^WQTuMP4~mUOdfXO z15U{g3Ed|lB%i^21}+O9@CAoS4%quJTZEuA_x-1uP2Sa`%ke(Q8c|&b>pIH?=xl&s zLK`%k&UU*!%e$tvtiLs@M(MtHT1zLXDD@i~hjee1rM~^=zK1-UI$|kRF`1^bu;Sh_ zas2nr4(XqthVDW;jVkJZsX%w<7)AMk;wE<|NIDx`cRE876fO0NkQj#JYjv?gDyLG> z%+{_!kIXQ@BANf~)m_~1SF!iPm# zge}D>A3N+1>K*mGr^iG4xJYi(ye{>j)cW?n4;A95-CBhnJ}`^aw-u3x6Ms4~uw?r2 zEb^1-CrF$%{Zt44>y~*2fsFUxk+>Mr7o5$7E;##&Y9#reVCI>>dbAJinEw16SwX5B zZ{jB$g0RtHLwqUS9CF(ebQYKB?0FDAd+M7Gf;&U%C4EbL8W$)`E{C09SV65e*sA;KY#z!{t*J&vCyu?>-hhEKt+I#I0=PZCL0t zcIjYwoi{jXq1z9vvXu?0I)Fs?pG%{yfa4aSYCL*Cl}9M88MtMT&vv1|*U=8t_Rd6; zhxw4z5qHVrr+uREont~{s%7G%6CI3&1Oew%65#~L0wb_^hbVSEQAtfSbtk)1Z^{nC zMfYTcmPAi%f!o0|ElXkhxUm+9G^s_85+`lC4tIfd7(`xJO?=J>r1(kXO*j`uXx>lY z2BU9(rrV6I>GVe7=sjoh^6CD|=O^d#?umIPNuK@EZZ5O4nl=gHO*rdRIc%ksa;n>! zULw$*uJX)UcMZ?EwxM?dLOlQ`-^Y&~+w8=m!hOWU=3_2hu1)IcoGsX5#S*#N20$Os zRaCm{PX*4g#s09NOUH<2IBaruYsG~*!cQVl06gmSKa5w)X@+>nVb69Rmv<>RS{lck zHh~9Wn-}sW_sR=}bVs_E&hf^Y?LHGCG4PV4vRp?qcC^TqLu}#J-5K$rdMy&z^_&4N zhd{IqQ`3?)7OoSOoWmR!qm^XJInlg2)80?X;z%k8ahx{PE1`f1jD2nPrRV1p8o(6Z zW!N1v<+g>;so&mZ-)Rp*zteOuaTFakUJD0PF=?B7%n#G(tqF+8cy)wu)G$8d#H1nWKOXTFJrM=i|0(b`+4Qu{ zxCUW;w^LlOZ@vE6vtHu#+SjF)kkFOJ5m^PdYNG*y;W6Ubl%I3-|7~;agEnNLngsQg zOgh``f5RhI()=LvF;{koSnl+r-Skd#t7YZy5!5gy4HS;!)@w)PH?2<2Q6~AVTcRN> zr`g5XBpX&?#wCf_+pi7Oj&=qfXBX8t8%EE9#HjS`%+dG6{5|&h`5%0~+8YWwYQ8Tb z+t=~QNdZF^Q%%vT=K6!H84e5TP;V{ZAm?sRV;I5ooC+kl0@T65_Z0%=6Zbg6c7A4J@tDW08 z%F!&MCf26@Tn9el*ujm=G>vv{FkdJi_!_g(eic1w@)-@o2}dY4?iORJt2dC#+T*?Bnr5beCsibs~H z`v|&WdFZI_2@t#U=6h4~(ah?q!Nl9M&G#1E9HevYF`>->WZ~eZBDs`)b_R&q)xk;iF;5k=DtLr8+%FP zE~NOtm`4l3SB2U{MaR&U1twh$MR9=O8@N#pVr`U52WHVD?+hmMXl2>gOcDG%IYvJ2 zPNGmMtLBIR?@iMMtQqD*@1xm~2X})us#p@Lh}m0Wh@ZyN^d_=)qHQhLc8ej$x}@^h z5o8lM#z1dujC^f`aI?RR8*%b1wGqN?@L-LFh^jC4Qoi!=tz@Wj)@sKoeE-(+k4&Q4 zxJEd(O3gEd*#TfSEYshAXUbpu%s~REzHp40z}%KD{FE;IJZ8qgG!_IxX2A;OMISvm z$4-&coyGEl(qtBHfGrDnLK)tgv1I&NOjI>U?uGRYZmwI|)2$m^tVgq?(KKR$%r&d1 zT(gQ`=Oe_W;Y+jXCUUsc-;{CcD5JF-nb4ML$&JJZv9ell$+V{=JUAVLN>V7bl*;-U zod}-fSlbk~?psY9L@hFb9)oFg%t2I>_0(ENs3s(xYs;{M@{!NoDmwi~R;8;ar;11q zHklO2FSn2w(k;K<5(o#usY+31H2pdJ)8+JXq-Ia)yRtGp-cJgp@+x{@gB+?;GG?-f z1vxVnSaJNKA&79tt?)d{cEk|Z2gxLf?S@a;cYxQ~!#T99bDoRr7B^Tzp3c3RgU#iXub5uT_IK8Z#P`OXQGU-{? zCh??+R|8Ce2*~DnGWwOz@Ch`(p)tM}!u^=mgM}NgG{=uXO?eu~bLLWb) zq6haDqI4xjv1mpr3Aw_~eFNr!4Z0R*3xbGcBsbEQjsDsrD}FSKYv%_51W}#vm7a+z_;>0*=g?lHCSclbi zJsutG@6&hLVg5&G?tzGet+h`2pl}L_52)Ju6!g0Vgucv?6=x0!C{q7 z=7rSmh2De07^d&rS0|kY5_4X2JBe$vE)@T~ioE#hs@TRwmCjemHZD`zRdKyY^X=6l z-Ol6eHZHe_oW_4vX<2O%@;+Ua+huw^k68+Y%eqLm%RHwQ+=>Eidvjgn^=7+{i+9^~ zf)`nQyUptgzEZ?ET|+FmEfXH$wyf7w@_viZ@Y`GJM((K_;xLYW{X34{%f*nL`B?D< ze;@DkT5Z6AT$aOAwVrBx!P287wOGqSM2R{72 z!r3sG{``FQG`Ku{etMlkEJBs(H>YZi+Y0$lPE&P_o9*$RCYN`^@u|9(cEfUBq?>9h z3Dwa`KxMDmFGbFwyCvc?`2c0_+Qvs0?9Ml@;K_A)S)M+hX^*DYnlUq4ye;ea;Jx;S z-e}R*WeIuH^yg@{jp*8_en)7C*u>MzQ~%faHr^)lbsR|V=*?T1&8HUw2&<+qU%hzy z;&QsZJh?mxw$sbm<;m>f+3q=&ctUgk4KZ^vh;VwYjIi*?2<)Pd%+*KFsq2lHaHdwt zHSKr#YMCah1+mLgrnv*7MNlM#D#)^=gubNLML2zC-^udmB7Er&Y)4#}{_(Z!SbmQg<}YsecHx8!zNyx$H%XDw zj{hAw?diSgcZ5;>cp<()o^}$Nb>ivnX~peXiVT0HW(g04{+Nh7!*A$pZ5|Nlsqy_i z;(IU4qn$*NqEEZ|;hC)1{Ym&ic=rM{eBQPB={<(y?IR}~Wrd+ymRCDE1>Ex*5( zM_^uB+$?UPG*xHq#yT<#UTA{){lKlA`2N`Q zoK8VEtA(yVq}7d%ZR-%7R3D8Md>)VbQ0t3){dJtBn}&8%gR|D2ZK94FXwhN@_Yrb` z3fJghH5one0sl8U&~anbQkaGd2uwn#P7Z@XUD=Hl{80zb9}9us*9DCh_E3`EvA2W( zk~nh+=mUJc;W3QtPGUCPo&A4dj4d+t|L5_FY>&pv!|EBXXj2Y})O27U zEB`XFeRS9*#8h+ zgl;&*#O28c_bxYmkp3Fn*<$T-v}6ma>4%%NGRaa08@KG>n`-LcRwk71b~_cCGuU$x znn82em?77w$>v_%QRik@kOT?zGLrg3+N{!1RE z<($*64+$6D_Cl#Ox!cOMeDGA zN)Yy_wpb>@dM(LA=O4_kAYm0csMx5b9KmWsKya{ss%}D$g3*;m=RtT&7|5lUAUom4 z4~mO*U6Qp$nFD1{+L9owk)*QJI&_H30vqz0Lo!8(8VE7GT0t-~?{3h6o3@C!-j_fQ z$H#L>uJrsZ-3|CgZQUw~ln^CUCsl?fm3?9AC^iP+}qDX@AQ-j%QQ@KJbFcSC#Wj=-1FZKqA(9l7|Ml z=OFwzp%3*=V*RbGxsBEwtU9993du@|lZ=0j-|HX$p)OV-zYu?F?=P?4dJ2*)NewEp z5KqPBeKrq?&beOYSMds09^1T?9+cCZ9a9Y7vit}&dw3*;U*E^!yoBMg>zmM9$?G6|uE2;J&~Cc%?UB+&F2b zx@#56VWfmyx@H3~T-6b0d7^EFh_OgC%rXKqG`oi^dB$^B((I|6TXt(h_(?>)eJ~Hh zQrR65=A8S~mPcSTqv8_e`8E$dx$+Dngh%?)Q!W`DT4G+FzF-R8P`nz3AiZcEO@FLh zXgtl&j4~PM9Z_42jyZNQtX}UuImB3^L2I{!7%Sc4z+~xr@C}0YyD_VCB2yh92BZ~C zbfgzG+dLGJpPD2NRxlC@ZWg`gtn25s8|DU&;49TmdL zB0}^%m<=GPmk7Z))gd;mF-3I%O|%jN2UV2X{?h5lshKpx1?wD>EUbp-X}!Lpjf-IW zaK#O@c{}7(X;vo=vrs#McTnhBgHas>FsAQ_IO{+n58p@k+wBT|?;wzR1<2~YC6Ad) zZ2SYnKOq6jL)4G)5Ja01zY){H;RuWoUmFD5+yUt&eLAd%%jvV36Xtpz&6_Y-9cqqA zgy7)@R(E;|V2Q~NQ%q41c-~CrEup4+=ir}kse>UH&4T{aDWR2Yd9>+-N{?E7|gv7De|u)Jxh*B!l4N>EeDAIe5I`kHEVXIrseP)-Yfce zy@o;a&<3x!)ajK*$q)qe%i!c3B?_dDZ2oV;IJd%HDA)-ZkdpAQuXUMK)JZ0(1zss3 z-`vh>u7lpX=jU8YHJ)blE7qO`E9Ta_~RcyV@oOkd@{r>m82WpY&9hNgzCTZaut{BQ*pjwt7f z4gm09N_UMOARvx+hDbkvz%A}ytlU{+^s*zuIJ$uBv!91$Kh8}%pr@s)4_bfIi>P4_qsiLqcg zK2B-RlogWF6KcAplhJQI(Z)ltFw&Z1)aJ-RfE8etV_R~%iDw#8Q9+}U+-V4YRjKbP z^+lyVBUp}Co16I3pZ*-oo?gNN;#Yc@)3KUX_mMpQ|0=uo?zU|#|5Yqb#)pikhttfS zS-LDw^JtT4lXPRZO~=vBp=H`;LxC&9=&~uAA+7*)-*9TV$6Fj4NL9V+Gcpc?LfjO_pa_x(KE)5%~j;S7~*Xt6BKA zYs!IWyDJ2Co<0m7Xe*AN@2QfEZZ^SLI{Ezf#n{yS+^>5!883oZzoIMsN_+Fkn`gg~ zpjed44Or{xWe;4Mn&{}Y}xy7-Z%&0B>3LX}^6qf_A<`g>J= zrr#BO-hODr095iqcJ2L_b=^@9{;2}L6Wyg6i46f8*O;^(L{93O+uCtL&b`!t@W-sL z%iDp|$ltUDIqNTJ(z5&O3?;f8H1ajk2aoNSb=1e;W}Jd1NPcLB(@+kdg8M4TqVM&X zV6QGRGCt6U4p(mREk0C9U75G8IYVJ|r@zo?QG@@BW`0^jFNfWEzfL zy&IUCRaP*yRS=hS-D4H=2IWz+0Ofb?}|R@?q!=hSRWpG zw0Ok);*z*dX7-?qKY|G;Lka{UwSXs!wi*rLlI@%^)bVF6Q&g;6t75-~0CyBTtch@r ziT$l*_WTyipno=!ZlK2{`XO%J%D{R3I(_Zi3|YtBrz37qb`5oL z^#_Ut>>B=8>?5Z?kiE(9iE*WmmI@p#g?%j3$-FhNTnZSnpOGIFN37qZs!ra zDrn>1b_`#7x-Q~=n85}iKWKs)QHu0;!fU*}@#=&0T}1X2GD9Y+W{i$=p%3ktkD=nWIciQ^2@%5Jx7nzxV>^65=y zeV4l0@30X7Fw~{t1^HQvixoZmqiH=FiN#5$gsIq9C!MZx^()_$xJfs1)0R7><~>g5 zzSeD+vfO>A4xP73WM|ruG|>whtl*qO>5#A5{My=+2c&aBo7hQGE;_g>dx4dq^WZ@65BQ!{?XVPZg9 zPV&zg2vN53A|$sbi7DS?1=y4^kcoLg-P;W(z0>q8T{k7Q(a4t5ytbLEt8=C=Vf(~RTGXc6wQO3a2BQkP09_% zoXQ#@~P zZk(9jg3_zrmJ9RP#i+!oc`S1H=DWOc^YCas2eVEcNczu84-GM#%$4UIt0`ER_((iI z#Vm)Y@XD{PYW9g76lzSR`bvV}Zm-BUE{}mu9X2w(>;1MfV~C&ViLQgB(_>ELfs=&h zyq;ZCM66O`T z`{+m*6NQk=y>YVK*@^dJ{cQ)oh;2*kk*TaIz~?5tfN~s{f@>m=KXBl*(=Jpt#!`uD zv0W|a^wY#AcbqSklZZY7b71)&7!6imKV~B+wEX}n=+HyX#!&P-dBp$QBleoO%|Bc} zt={0kvBv+#p}ypjR`KlbkLI(hu%@f`XYUseW?_lTDtWJ{c&)S-NLiE>JaaPsVc!{Z zF(GJ4;RTcC#a8BYyDE`>N0{ra3GlY9QgxMW>nyF7*V`s5ID?Tm;;gG;dz)n+f(J8V zI&`nSJV(JS?@I5rFrzh~beK!RmM_P}hkywT`8jD$UUtqhpRTN9?3qE=LH|VQ1J5X))qBEJs}F zne87r`v546e2NMZaRgLh7w>2lCx#t0Jo-@57qm8gC4#0@?-hcH1kpfMHC$DW)`gVw z^2Ob*JK^ekRyc8AkI}%eNq-v%Z+QL`=Dwr$(C zZFjG>ZQHhO+qP}2wr$&I{mD)CzRB797wDr;;mW@T8r5bZoSR3ErPX)n_9!|Q$d`*MGiBj%H zTbGE;4GG?YT&zB)jU;FUIW%1X&J2gF$(kl}mH}f!zlKp|kT{@BW*TI^kQ5>O|1`1y%PvKxS^*rD@jA*52)RvF#)kDMViEt~`dA z)&2Tmz0`B6x|-%LK_MUCzd=<$-%g+VuT3)rpIU_%-xpst=sN$vM2)wIA3xeM;oG^n zpVv#!(<3P{y}mrjbM}E%Z5$r!s67ogIrKFAv(mT8aJ9;B8Ad}Oo%vH)u-)Rh5H($@3BU7$0T`lA?|eEtf~QjlCZEj z+Nxh|SgYq@kb;(OhZrcGD2at5V%~OMa!zwr`*ZgE4~6E<>fTXjXbrZUtN!XxH-MCR znCu7xLM)NV4fhH*UF1vb?!2AsY)|*Lmb$6o)>_xA3w=ytS69SqOCbGh?dNs=qYq<_ zs=>rkRP!n_iMn^=ERqE4f%Y>XHGVpD!NF!1S@o!MW4W&sayrYJno7_n`A7|O$o9l2 zYB>t93#>(6F9hi8#<`_ zfa&zw9tW5Gfi9%t^lOQ5h`b zG1K}s^!gvTUbh#3S%YqMzAVI!1M7!;e5R-zKWDRE<3{)4e-wbECL3HY32&7dMSc$!|D-ne3GF)d+JnE;nUQW8oG+(d9=x4GNuo>2R{%bA%Wt&@>03yhpLUl`N8(Mf%(oh9 zeR@xcJg9tRU8L3#h;GX9^V;N&cg40f+eya2o|X;wk7K{4WZ^bQ(DqME9uaq_X@{HR z3n|+-MHF!d&@`Q7SMimzx7GB7a;l^b^wKPv8XCerGpwy;0JnbReTfKKZr_dKDFtU? z{5zfK`HfOgfm&V1R0d2{RQjc>6 zc#3J=bsJo;C2hT3IPx3=kB=9euak+PxOt6qz98-Z!(3o)sRFx74vlGd!W9ul{;ieDzbRLCKphHtq`eO?lz+DcU=k9Xp7KF#upm=hG!H8S_;F2_Bu zRfrE~)e&*4c?Y1cZ-um~%TE7+xcLlteCAQreYw$Ld{0YpN}D%kyw5pZm{nVc1*Cu` z>h`z_3`7a+k$71z>~+_8DsiPB%8fgpstQdOG8$vb`bbbd_qR7ot$G%Z)^+&r&2isa zOQO+M_*VE+c(6&;7m=6fKh1$$634kVEk913b1VnbH9{nkN_kMeH1;p4y0;V z`G^FTx|>*f{f34~3uEk1iU4FGO8vcZfkG&|(liTQ0=3dmfA^4h1zdzHt@HJ0fb{V? z{Dv4c;>HzJ_bTSEP>#xZ$N~h|b(Z%0(xn5o?_d=;-rHfyg|M~P{nU zgn9v&b~I}2@ugU&Oq3C^bK(MbR$iOPk|F^zPLhXPHBBclq*cz);okUQ-6nd-dyZyu%FJ? z{G@IlWF2oQcznr)s48-H>9jvZ+nwB35ygTIAgQ!{MlLJZ;P z8vmjdJ!`y95OusxtlgqTll*3&SOpNvPp&I)*=<1f?QKNeQH^l-Nn!tq1Enl>hhFMq+0eW&JY zz9Gf-D)_WWa-SArS{Hy11$9JFfX8%dB)+?{@mBh5? zx3xb`aDxD*YxyF1)Q&wQd)@IhR_DAza)L1;mO1aw{v#PPR(p$_j*Af*NwJF<;+@*2 zG2eNwsWaqn5sgISCW#Jk6ZrF})jbs6MuPPsgSo4DKz;7pjPrtu^Nw3h9Gjq)^IS&6 zibF5+ry~8ujR2uphA(_Cu>Fs)h9$ExbCHg!%dJ8D6O_05i1jxrpFh0sa%q<>mw zmXgkPazUG|C)H(_`Y9o{#vBqMhZ5O>lB^KrVrEN>XKmS{Lu0LuVeJzcQ}hqaC&eNm zDd%{KORgr0Z}xw(=|?S=X4b$rf`>tdY)RcOK(BPW&B}~no-gIp2?jas#dej~v15p? zHA>M%X7AwfN#b%NYh&2+7rLHv_m4X@l^EyKGgBi`-`n*gM=YT{|LC|lri62v_io1~ z3$M3sV(!e+wT4>GHZ!IxL(=i+U4(3NSK5MLs7E^o^$RfLEB%)In)8vn&s#!>^qqZ# zEhIV{<-7!QWopDePZJYC-bi#yq?1?W?J~7^k&_$R()~#yBL|*)5CWy&)2E5) zxA<`@4!^)1wyCY6Zz;C{BGFGOH-LxlQ_i=fKX*g-DeDZ-il*DhWDItzZo-aX3giyt zh&{dDKACx5qUy?roLMRi?I3OBdjBkX1YR|Ul%Fv$ghr}G1jK+i4g{L7Zec4$JOf_b z@0t1VF~Ta7bU zc|jlv%=0=cGlm)@zAiqV?rQ@rbW@fpqTZA>9chfn2B*hTlbdkU7_;rzyid6B60Xu{ zIhu8sbZ-AE3NA@-L*dxt7-H8CzvmNb(@We*T!2UM05lM_@g|AbSKzmd6~0wH3^d43 zLQeLR8NWkuP$Y6~%Dwwn*TMUX9)k8FJh-9%{jw&)4)WUV1J$r90$i;osz?fdFo(hT z>QK60`CSc^-^%IP0aRniDyWP-Q+Ao38n0h2$s!%^X8#*t1hm2>{S$SIIJQ?#!vq5B zL+nxH?K06UJ zq5f!tlej1ILLx_-hbGW!j%rapZ##P@_ja%(3e!T)%^e9KFrcREm$SD*5=?+Hyd^*P zeNVKqf`;;PgBM9Ida36pwZtoClfUSsQ-$_U6i#6yPAG@(X3 zEjv(D$}ByPHi5(hJJnvp6S~&mBXHCm*7Pdg6i5EyBt9LKb>4$VXRjNp#(f7A#w^M< zbit?^AzwuW(MNhMvwaMYk;7e;w=Bo4(;J-C z-f+CZE?av2z-(fzt_pSR4h3rV1}4T*OQHx7ju>v71F24J>f$y@pbIaGZJzuO zwn<{-ZV&EE1&{}fSfnrL1uB9h&hz&M_{rWg6UOM>D&#p##&B{J$oHwIx zt=EonCu%sM>&5CYZZ*g*4S8bYk1&pFaIL(|E+V@)^ffnIt$2pz_vwxiiB8H16e#N9 z1(@Qn0zc)_e1IL<-q-GmGpQzYA;qE2qUUd67Vha6?auaHvm>60@?Ctcyn(*{+Kyok zY54ukPIz@SOB1gnl9V!0%to4qM;Y=W>b>-7jOPFAQNI}y3&;jRuE-xR z_W^#;VHZg|nq;Wx)NQ7*)DqILVLG;GRAKzH4Y1EIhiuYdkYwpU}WLmg4EK$nd)xcXwr%Zhz(8ncx!bFb+h zTOYYHp8#yS%&y_OFD-ij`w|H8lWz>9Nt}->_pRSJ%4wAK0LB>Yx-ySwftc>G4-8LE zXIN1di~sS#AMtV5PxE>RJX!uqkcZ1WnPoJiFr@=Q?8O~C&>vyAZ8|$asx>Gf-nJ+l zNcG@&=+OB|AFL|!(a3>)p44T1)l!`~rVJ(P9D;$44@pn-77X%hBp;9{v9(X=>#-LW z))9w27M=Y4Ir$ygSGlq<-97xgNz-b{#5qd&AQbLCeUv6k&cblV+{AH}JV(05)tHDn zh==e{WfQ7y=oVxYyo3Q6t!>Pt!3B3a+oID|1r4-ic3cw)K?jmNS#K^-&FD{xC+nZs4y`$Q zN!(;1~$7v1(>RD_3GTNjs7DmFKBE!B(XS!T>)FD=^jR zCpQ&{A!r4P(3*w|^H*@eJ?zmbbwqTk9^oVf?uD(44%~~pIP`s$`wgumDB*&+ec1J} zKHY4|Z_oNZQ%o{@P`LN&+=RhvSVU%z7j0@!wtMW}o^e^z_pE3Y=KK|=*S}~2|9U+> z6?Y@rn?%UCArOLb@(X|vn;OT}{>~h>CZ+g?`PQE1GDg4^L1}YlHaf)fYBcm7v(tn^ ztU6!z6|M0}D`R|KUaf5c+P+pNT0=^IG24H*Mte8VT|ZB(5)Gfo4x)aZw~Fi~J|lb{qy!!Um$-cX|Oa4gzSOLB5V%7i2! z$L758?C{dZw$4xdtOwZ2@6%iY+UizBl6*1AYe{*C!&!P%J#LF zhAl!ARXLnX)t4-9$SWcmdYsaa{jMM3Zrq1qUAc3&?%cmj18G+xJaXj6bm4TEG*58t zR%KQUr#ULZb(1>TXIo~l*b1x+&hxWJ^RiXwsr6Gm{5(o$Hv^T*!E z4r`@HJS{1-=Bd#N;@R6TJEGj{!%Kpvm3%jjHv(;@W9+466NF5Jc;*}`(4t;$OW_LjQ)@81vHXM;Ww{0e-5U4IEkgj3KgaJSAsqjKY~nViY+ z$!i2y$UJ}-9GIVN7&iLvFYEyS0?NyQ9~KwlI%SC zjRIC_eSDo;u|=tG8k-!>CVchf7u*1JbD zWyaEE>c(Hm;Z_zD=vO)0X~|jgJmt#3^p|?0kIko==_#HO(7hXXc7z70X z00062S|C%neop#dp#=b7?4J?*GkXsfOIijx4mw79V>=^yJtt=aXA2{G8w*<(Cpvpu zvrCver_u<8Br3u2C`iUFU0|K-# zf-x~oXA%IQ8q?fAIvaQOk5G?|%DXo?p(#i-4zLstvh%XCGXD(jX(wl@CuSBZJtIoF8BiIe6IJ3Jw5+@r0@Se)gFDl>hDYW(2zWQ6n?5*?eZraY~@Hf8B4KN$Qlyz|XVfXV;iPjiuj zl>Kj@{3j%C4GOpV{(p*MKm&n1`WQmQ z)CAkeoO}8q91fSg{y?FqY{Bi#ZQbwBjcsGn-3qOicJTif6gVsioMwNXMTgGM#>U3* zb*bK3`+j~S(YWYIULn+n}pne3NzQvn2fz(R@O6x7vcZr1B_A(tE`B0GlTz6SXXuP~D z|GXH9VjK*!65@1yXIdsMmyA`m2O{iVoU#YFh3f4XmH*knW15b)i^s7^H-po*FG)kv zhI|D=*MhHd{M;}qE_}mV^>9pWDrk3nG^Q#e2PwlnM^?(06-3m_MTok%&-4bp`4p%K#XyMK5btuTY zGDvz|uzs0&RH2wts7;nhPA?Pc=t`eIu)Y2KO!6Rd^&mDmqOtZ(+C8Q<&OnI@;LdfX zae`?z9pxuwcZerzjx#NB+|DkmeEOZqAO@8;vdS?ww&g02Br6A^o>C?BZ^2f`Mt!7K zqII@Kx!B}};P)OQENiJXN5{Ez;Y`T1B^$1&AbBKd8fFK3_=>XFf^mapI^{)ul|$r8 zeCNG|)-9*)Cg4nVK>eAL9VyGZdG@(L%0T4~+qyl=b^DxSc+B^ZAf^fhc3u#G%i@Cb zUoOc@RL#E|5xIwdLt?k;{__D+Uu=|Z0kY^V`gGg1U&sR&Ee70WKxnkft z=`pa?+cO{?vZfA5~BR>^|@>Zxke_<%Q{|}OBm&`oF{&yn!v7B3ttRxc=PeQOA!4Pp`3eG5MBSAeKJO~ zDU7k%7C!BCil25yw&aXwFTME8C|+Nia!xt|J)?G8?w4KQz+12l5|qyO=Q zt@NM-uu_QLC+a77XItj^lqVq>X!Fi$%6b^~2hcqT&c;;X4u8R~sNbW(JZfF8sP@yj zmVtY#IM>?nzPp^Uo|g+R4trq;Wmysu+5(1C;%(GQ*rV9ID9sLv!3E8@S-4ee!R27I zdDfu5rJZzU<;s**Pb9$rZjdvVMvUDpv8?dy_i6yRN2hT1Xa3<|=ui!4u!^V=t`l-BY6_8jXj zVG>K_T_Ohg*fWAw@Ffl>0E#wewdIY>_gohVBeKS%kYGl1c9yo2o0&w#`8cHlQ@&*J(r&vJ&!^IfGf2Edc)}2?+>#@m_>(0L z%SfGVr$?8^6T_O-Yr)K2YRVN8ShlkRVv=9z|GGORi)IQJSXoyn0RsSN!vg>i|8Ga7 znT?69v(tah%A7ws@|z4OKD%|{&FmyXAEMAafHr_JiXsc{uGyQzgAK%+lr^Q_uN~K1 z+1h2&W)*gPr@lvTTbm3<7Ngg5AO@ozcg*3CKJqCvqK;CeX%|dyo%Lahd#h=8kFLk1 zJ;3!HPO3d9N?tZ|i*aHLqRyx^#SCQ|)tuU3(+IOPPWG-yTa!HC27yUUP@oho5u~a> zsU^zQK(w6Ffptq3#&dAoic*za58IGV9hwE@0;j*>-a` zAJqV&FtHMkhnKK@X9Nv0_$;;k=Y&sPyQ z0VJ(4;gPE4wTeLSLO~BHR~L0{ysCTEETR#*G8Y%2x$uc+2YD6z)EVgMlDA$v(b?Ff zysT+*a7C`yS!oU{^McCf`hEO(e+ud%71qv>JOFq%KnAran+ew&0tN45M^-lhM?73=o&P4x8s}*G72@J>Mntfudz&5BaQFC zAIGCx)baRIJYdco)O0K`60jyM7mJ7lvGlWQLsv%w4;J%LM7QQQXs=3*S9%x5P*JxB z%JJc`u)B^sjzz47l`sVmB@+XnE#@oiUAWeAhZ<@mO5AcGS38iVmay?mGa-hiu@?FZ z;N;X$pmu=~q0wD;aA5}mWTgdrD0CrLTK3^iza$I=l%>Ln`}7W{P)c=0ZyA=T2PSp6 za2&kxRGsgFD8vkTx{t*_$9i_n2ifPao)jZ`Il1Slt7YC&*Tqk3c=Vb*xAv2z14XjU zuKQo>g%nocJ_uln*R∋rNOuEO6VHO7BMHsD|#E7n^Z^*ofDVk%Mh^J}kYBxmM1fXwTxafXpEh!GgJ$mv58-+BP8n;tY&BW7EY|96Xe_X?AL9m1 zHlV!5Orn3CXf&wrhFo(EiH*J}gdy?WpzrPe|0zk~K6Yw~3INb7`oBJBoJlVrvB+7L?7>U6UC8A9aR@h4dw%| zoBqCD-F~E<2=3{5t<0^epPzP?caxX*n{j{gj^Z^Vg^#pXG|^D{(3_ls+Wc8~6ToK6 zJC5S$G5bZ`0y8r}JVmXl;;1mxZS+v9w;WjSmP+iL-VGQjt8F_R- zN<)3}3t<}t!a*WIUV*3D!B@(%9rQECmLI~<+$#_REhJ^2y|?T3M{^$!)d(aujR2D@ zCzN1Wq(W4uUMAW()>2D*f|KV3MZB6W)Pw;ROhL5;zUyu_i?#EZoI<+asuZlBY=r$` zMj2-l1}3O7M9(m1C^v1uF>;r0q_*}K$KSz+yCPk8OC^B|uopqc0<&uvnEu6fJAps` zDzMEzUb^j%15EpBR|#hDx;=2}KdjDuflL317NeP7L_#5|Ia<@gk{13m)6lk#D}U5( zgn{0O8{@|6LnpS$A-Hd6`IqY7*b=}tQhfe#N2UNFI{=cQhw2rPXkHIqKQ8gr9_!C1 zb+6{pI)1;a1JC#K+N%#*+P}Gms`~q^o|(0~Wo5hhh8j#FYxF)j$j$xSf@=qB9TqBU zG#4m64-aVIPJ|s>{lBT&*@dmNiuTJybUxnix7Fb^Mg!el0V={3*;BxeYICReJsx>u zq8cI4OYs4K-N+EP5-kF#r~Zrb?w+c&Yt*g$Ye*uf-IY-)@WN+-`0Bs^dZc|mk4u6R zmJW@5HRdKO?_9&Yf`EDJ@Bo0cp8rm}jn|&tS6vz9J|1vTaHdb-20<)XOYR3q&aCU! zla7O(Ett*idhp&&I^x2+ckpt>`xsR{9yoF0D&&=x?fH&OL5Wx|Ii8~P$Opy?|JJwl zwDI0Z;<+n1LUR0|9j*CyI?ri3JK?z`O&7Uj>%yv-1sPYMIi5TD2g&))37_%IS3%8C z_8H;7EpL8q4fRxT+ju{I8QNLytDt;XKG>c=u{4I!K5kQ%(K(LEdMfrQ;>KeQ4Xv2~ zvFQYX|5Kje<{=m`ol*yS3AE&uUb(je&f=f9ax2X(lQB{HVZ-(QXl}FN7`>;*KIEsS zwzpbe@A_Q+8?c!#q&W;R*^Ds19gN4;-Pa*61rR z-g+i2-^l7?u&-LTh^ydAK{H>hT(dkS$d+HJmKwn|^`}d8ov^uQmzaf=;Opo04Msh5 zv@y0%^59L9g)8dKXEW?qFS2z86=XuqZYhd)gt{d)s%BO8yq^CzgO^!OLgEVhk}5cv zq}J5c>0}bl>{2B(wQ2L}Y`PQFVwhquMtO}zaPVi=RM35v&GC4#bwq7!CBgw=Kf>ar} zz)v?_ev|$(E_|&8&cTUF%^sjVu_+3~T!m+$e~^`Kqd)=*ifFwd`XfeTkp4tibz~I? zo_;o(2BR?|y|6pN%Xw;QR=2Foz3_AinzEEMi*7f`vtNHR1~oMc z@KNeWBrPxkAHBejb+apovNR1kuZUoW? z+12Gua6`{+m=OCV5=1~sb^xgMiH2gp!mj`kWeo!i06;mgSpF>#hWuL8Eqc<+rh6*uNi=N13L3z2(sKxN@dT>cT5yuFrjKbrY(zH>_DQkPy zR3lVnXYdQUmk38D7@GsM%cy5iDaaJk$;L~a`!l=9J6;w?yq0GyA*%2A=v$khQqeN8J+}V^SUls9sUDPpJImSm>NZ)}Q_{x5XdGGpfZU1DITF`QY zX*d=Y(V6W5gHW)VM%cl$tWQvN)asMSXZw&+5zCKt=K5I^+)xCZmyMu+LYnFY(*vd+ z)RV`$s-*GD2t!6nNT?w6o&Y^GbxUA4cr16k?ANfcfq+!wiKD zCQmH0(`|t)3=nO131ZfoQ!2?lF% z@7buniJIaBiAeks<*J%wy_Dly2XI$pVg(MZr}*oQJ^fVn!KB7!MjSvWi@lBrAUT>~%R7hEq7eFGprC$)qre&g@n3qkVhkzxF(^{QEmd^5ALd0T{Q z0J6T_e1uMEe7C4FaIV&hq&;GW05u$Ntaw4=p{hssiKfm6^SgMTzAh_DG-V`aY2i_< zZuTTH$Ux|Z^Y8^GV51I!NnZCZ=>Clr3siDLt^BM48%#t}{e4U)noY5qp%0eN5f}T&7B5`OKGv>gSqQdhyDt;;cdf^91 zhTEv=hcw5${i%LQK<+cBb9EQNAqXHTVT&X$Vp)43W(pByPf>QN34({L6+iR1_9$e4)rWWrs zw_%(myn`4FR_ekIjOR72rG!W~qgf)9v`=&dL0?L&hg(Y#NA z@RkGbf`&)jw8PD)@ItKVj-5o|GegO`iIHv(ND9PT&SYl}l%%99!Z}_?`ePGA2qNo! zB9MlC!}Fn|I~r^aBgAqt4EYG<6;y*ofoZAGpJ9GhC(B$ z?Vc*OFg1`P<~t!bSvLH|bIT!=Yj`0lu9*75%Z}u{V(do-Z%$o0;*GXRmLiX6syAEY zGu^pfn~7jE{KxJ9uQkO+_Ycg`Nm%+0)lGkS0Zp+4ZWp7xn^$2e_pSB5Y>JL#wU zZOd5QE(dNFiGz=Sk2sUHwR~1cJ@DFPHe;J6cj67W>Q4ZI@YJ@UNaET7SV3mfCtmfQ zY+6yc@HSAZS`rqmMP02QVwI&Q}FLZ0ZTC(N^z$h-NbEl-z#;lk;kUmS;uxo zFhJR@r~g$(Sv$X`B#f7=4S{`GQc&WbhuTh!d)hUmG{82%gcMG%AxE~4&3E*bCXX3} z&9Fu=;ZY(zC7Jp7szNreVQx~GHopX^lfgtldr3tX#W{CG{jdz58p6UEQ(7fdoh}rn zjmYtjE@e@bj+j!Trk*O4Q_qh|(~r!ES!B+s5tVM@9!HIo3Vk%RFs0U54OYoA@yJbK z47Si8jvwJ1+GC}-!vSUc0|O7D24T|uZ?zq!ynbvah{KTv94CG-aXc~_W8X2%U0?|! zsu4uk(}~=JD8nIYt{f?~mW?;b%95fUgE6z49jSM0fg1sJw&&{7bF(m8(}v;`%a??6 zm-s_+$*qN-ft)PF(Xv=Px-7=(j1i+Mnx*6oZuNn9Qv>_8>*}=Oc>-f#DOgq=qj3g9 z-N2+(XiXO^{=0rfS9=P$?g`YBI@=@(I&{x<4Q=$|oicDQ0XFPQ@~ysCvc68WFXAp2y+)_I}xWdcr*@KHhNX&&1n_OE|+2 za!&3Rv3Fw|OW3R}2rwaCY%6seXdUOJ?S^0^0>&A7t{&? zDv7neYk-SF{7+8&I6#|xzHjl=vSe=OpQGqlc))V=+M`JS_1t2l3RO2R7ZT`-|7yK1LJpYhzG$vyfrXG zFx5_zA4`<%nn%RTA3zVePl(}J z-)p2M{{+Z-22!;ijTFMn5$Nq58W%S3Ni{QdbfM)fifm^zS*gp|4pjsR_Q=2Ov>1TwbAHy1coKk_9%}&m_D9}mzKdexXL*)<106ag03u->a6uMsp@nO z0=Xqk@n0gZ6!HvLa9u&l^Vz;(I?DxtNF%@cvzER6776!*_iS``Mu{tVo@RRT=v{Dw zlDaMdO>3Q0N^Nc4PS7A7UI;huo3`4KwU@yYJR_}&%FCAgpTOSbIVj!wo(jyrER8~N z+?_c3dYcw^R@<8ke_g}2vCG{D7oCxe^}g0o3YTsrf$oFCLT4AAV=SC5in~QLd79}AQqoP!z)^Gs{1Vie==jhKVzPc<&^2;(9lyL zQ1i=uY^YeuGY04(d*RNdBDwQbX>1;r%2PJFsZ*mfE%dpghGD=~!NkB;xpj&OfOBs{aJ`^!!-7F{6K)0IQ41;>qlst3g5{PS^Vz87>X;t@0&sdD_RAC=He+@BKHT z8WUPY zfNGv_Xyb~`%s}Uszr&gq=`M&8x+*P)!}LNTdIBOZ!Se^1?*B#sch5Zf)Hmf5q{t-< zS#7XMvckJ9F0)bjV^l@~1x6*TXS9XRLGYr(F&NO;qe6HJX|Dgs-~nV&BfvhJCtjZo z+5kW{zQm<+EL*2Qe{^Q9Ju=Hd?KrvX700}QsPwn2k zjbvg}`~@IR&;ZHiRAH=;B%B}M_?<%$z!zr%q;wh+d}98!gPi7~x_AL=P8CE7A8X+Y z0d~S{IRm-=e9BJs?Ln?_o}R)Wk@<_pwJTX>G;-ev;d4>`s9~i&fwF|mzp&&@r*vF`PLp2VVtV|e%DN0giLNGh%ew#|x#jLwKKFZs zO0Rpwv62)-vF{Gs(Ph`@#<|=2k7-OaE~9%wn2wLpyMdeL2(bj;G*AOlI5mN@oU+@U z1a6?{RehyY)NAned9nnl5x>YRF80_P*P_!eYln1mxU^CoU}BBHYR$DhN+=YMP`qx} z+OU3tj|fmc@bO`&!q3g~?vMY|H&wZzfC4*D?FF#YE2BL~m;sD*%+F zZx2AeZSmP>ml7OY@36@HJr72hQYq!~F15Iv@@O_Isv#;$Np?$Z#`eBy%+}uN*gfu+ zD^vE{;mwr8>)?x0vbM!NSNRN{&=ko*A7$Q#=ucZGg5*JJe4r22BnQpo7^OiroUoyLqqJmCPLaz%F?Gbu zJ$B0Q@JkKNMRC^-R1nFo@d;Vswqh04&u64xLrHNh706W-#AQOe>dt#Bu$V<F638!8r`=$o)m@f0|#c)2Q$J2gg(kjIj?<+?XRq|Kj`BJrEJ zC@GSk#UL&Q6+DQOVex6FI=4z$4w*nR#auEGPPcc z+HHqQfR)g0ySDF!t$?DYR+!5)3gSoH34=|?9Q1ILgA40BaT#dT6*~`IluX^$JoZzp z7j>eltA_Qqn99|JgaTJ&=FohYOj0eFCg|8Ix89~{>VieuDPZOVw69naF7yeAY*7As z6Ra7ySO%WDl6LH~LR9({>-78_{0U zb*<*5YV>(BiHdK(Xz}0ipK6-*@5vN4+#ph_#d%J#*5#}?r>yl*Ja3TQ(Uj!A1#^&G zK%RJrUx(e-A*FhFa6oTj9rnOCfFyAj#m)-t)x%QrJ~QMh8;B&Wy6;MLX}b6_`$kh# z_tOphm~M9-vhiZeXk%igb}_KkNWl?8Dq=uzaL+`wzl)LM0AsL%7o~PWVfXt0p0usG z_$)A_;Ks$p&s;U*ug%%r#GF1H@$f2Cu8ph#wfT0%qcx?;zUe9$(i8A^%~Mq_SgRyO zBgJ&daq-yhfUY~x05j2Ddk|GP;9Kqm>_}9`sn4i&8&{_$?s#!#<)^l|vHL z3nU<#B*kRzOuq$+Ds+6@*zWuwHQ!NPv?Os*fn`wz>|k^IlSo-`S%J4q3*UTL6)Td= z8+N`HZ5jVWc33>bg)JiGBH{!&-fyeg`gXhDQ#zS!Iv0g^f;u1+C~<=|yHYW1*6=IM z;o|ZZrd<9=@qeDo4DAGDD}~w|i_V)(TgNHwyGUuaK@Yp&gx{;nkrunei6um~W!IOe zBMPR&Cp%7!;E<5Xc?-VnVnAmCY!;+{Ho~m5P}`mHQ-GyWQ|DQTCp#R;i3t!0p6g_J z<_`?*4|wGpCCKz6;twmaG}f9AX-oWV3&PJ{O2u2El4=SNqOdA!x5b>>W1rd03m+21 zho?emWJ-ANvkd3;LGpJ7Mu9H`2M!=5f2Gh7&RX>PC*>cZkHq2e~bMmn7#D-ag`we}UAT4|N6c9uSJkF7OJ_ zt24oGZB91;x5Q<#$t-#5b+8o?s+}q+uy9e`1AXPWy0o4qEoEb}uSasF@ojByCRs`& zgYRlcG!Ft>J~&9Nl)w=xMv6Rz72{z@>Y_B3$BAoHoTb0SpaDz1XmgTa-8Oig=ky=L zir8f=4TF38b2#5IHr1n$4yyY$?W+tJrbvjAIZb+Hn^WBDaM}5^YtL!^gC>+Rx<}d< zJjQF^-)9RGHlXYNbY>^3>BX|_({dcqz{S%VTjM2B8jRJKDPr#egFgcZwU(%9Pdy9pAHk zAC6=xxkrjk2SQ-pWMPFAm$g5caC41Vma^8Q)v9{{96NN=*l$qt; zo!iTSb2Dkeb*jNEnmCxl*`WK%RAiYTp#UEQO@lbJ}Sf)3Wo;aupSf|DkkGt30 zfA|ZKG6ZL#8+f48yhZQy$RR=6iS71`;vODc zPwr^_FeMwYpNcnHU-vJ4=G0jxeScR`nKxsWJUpKRwspW3xkdGb_cK1d95grc3m$fycXrV<0n68}P_uEr2A#Tb#x znFfMWF|NO_;TR`0XUv{5O2@g3TINsAMlv5on?TjiDjwLjMY4)FU`m>cmI^A z79f@SgTV1c^gdn<7or$QsjjFR2Cx$7uwF+oaT@}=itWfmm)6h8Z=Y4IOpzk~fCFe1 zr+Qw5e(uB2STvh5X*Bni(mLy~qzs)xnD#!JYLQxnQVBRNKet!C%eKaC2s>hQtBghDUrC5jd<9!AS_43#96h>Q5t6 zEifRR0BbEPg{#@&t_4n{;$cGkKOAl)tHz0iSmm$FL5WJm>uGdOal%q^Bz7!J4=Q3d zc5BHqO(OC?0o#IWfg^yjBoBE3wCq3taC(l!#)6!^j%~V4gaAdhz&-;$1~h@B+)#&2 z?bb)hF6u$*mef3l@|IoMBvGd*eUsE63#(F0G)d3`Kb zP|vrEP%ACX4e5_k?M*D@1r%HgfpT$>&T3f`7!FJ0ZE<*Q>!v^)eTKyd zuEcGi{V728rd++ak+z8|;*Ca;zztk>0;>~g{TN}Qt*&IUT;YP8WCp3ZfaSCR_EWVV z4988btFVtrkbi~l5p3>vSBQ2AK&MwP56ss?0{ERujCQBWgbjMwXP=Qc8pIUd>yM`2 zcY|#r4$wk9A6h4LS7-3_(}gpIXAfustvG>iQ5BnageFTCrU%<^?ja9vF)1d59o`Va znIAfD6W-J|L(OFuvzuZ91Yoe9{+V#&1$7{^7N-l4bASM69yHm+j51%tJ??Rx3iv#dkW$|7pU+6{p#IH?duyV0_ z;sBvj+jj)O8RTv+_n8?Qagj|R)ls&qgWKywvIQ*&YTLmyajNOlSvC^3Tx)?g0wT63 zIQ*Cg0wq%j_k#+*GkM@((o1z3 z8#i;dOqx$7;9%M5m~}m|Ba#eD%P6*~p`mg8g+R-&N-xz30**SHV@eqz+J!OIHHi~| zi9}iOT{FTw4`c~pBZaOOi1Xx~-UlBuq^axow*TajC)b`bq;UOMKByEKRcT`74WdA< zT;z7-+ck4Pjxd`-IF+eFcT{{dIWlbte!L6Zt%2>&TL)yyJFr3vGaC{EdWjrAOUP=8 zE{le$Tx(4SWpGe{^c4Q5;mMVyauQH`C9-Y|f*B2**_U}Jv=m)7n$7+Q+d5E=-3lN+ zv)_IIk=%HRb>T!&(XQFmX}rSJBUew%v;T(n<l4mqFLX%JE97s~QCZzB;&Tge zf7~cI(GY)zjq~(fsOXn{Gni2 zQEY9+I&~UPxITIL^18fPs*h$%KVq%~+@JrCssBI8~tbDmp`tvfv4hX*&SfM?(Ep0@4O%6WR;tInePN^L$okP~C$WKDI?LvqW~)lJ1+ z{|}r<3bxP&kyZD6!Hnd=>rwrOh=ck<#n?~vXti4z4QL;J`RQ|WL9)_ zXPG>}p!VnU^%h!QADW=V2xFd3U;r3HUxf}WF3_g-WAD)ON2t)oMh5^_J%_Tk+grDM zG`cFbowQR=jYJMx8~UNn!jRKQ5boi3-NirX7yX6W<$bEZn{`;e+?sUvZ+X7C!_E2B zb`u+)&ANHQG0?)mQpn(5Rg$F6YZ|@UEXP^Hnk>uWaK($@u;Q;i@yFYs_X2iQ&=+%8#j&x3{BltgS%!&I+^+ zDOa++2s-r4sys|cU;Zi9_J49;(&xl-h!59f$0;vU1^g7ZOmsOb|APp^9~p!X>9e(@ zmohwQl%gw2FO`YRN}WCuQU)u_0)MHgaU}0mNnPTpeoh|bFc;Hfogj{@37uOy z7`?kC57(XqLmIn?8Ze8#xPLLTm`&9HK-Z+WeRS6B>YNx(#htCABtfBk?wlQ9>ezi; z@4u*{S^Uf&9S&Jd>JVaJEVF>vVpsNyt`(gQELi z5NYCtRZ>lVgJt!*aQ}-gkNy82fntiFI$z7QaRr6ydj7uyCCc|3sOB%d6%H2A!Pa)Q zNMD;Xqn%ofrQ)Lk{pj4;2`I3#L%*NG)+0D)1_yFB^fVb)iZYgz7+Otp2?=+9v&Dq=1kWyJI-!P&`7h}Kt5Y1 zj&?em#+2Fkx8AqnIi7%<$*hkhtMP4X;~Zw1>l*FB$&5J(FGGCsEB$P@=jilMl!69_ zCY8FLb--(hj7yjNp1N+iCj~T(E_f9E{~6ooKW(_r7<^eVEC7IP!~e1k=VoH)#bAcL8SiDAj*?h<~e!dRRpD7O~%x>cw>=W1= z2akP0_H(F^M2NgN4Saa?&-9uo3Y$$w^|W@YkPA0MR>PLtt}cM7#xjEk#;DJp+9NFb zFL}{cc)%c+Z-;Pq3wGGq88b1PxQAWCk|C7H+yD%NIio)v5oJ9jh8aB7@CFC4S_#{ z;cBw#s4K9cV?vP;3iHec{#$UWi*&!Bqj=3kpt)UDN@Kv87%`b7GaHOf8lVb0#pA7Z zdru*y10XJ{4xl*>Z=#nh_Tb{QM~w&#BfOm26o-xoq-?;uxG_gv>J8M4^0bjly&pp{ z)~mrzDiPpFf)DCO{s!W;E-@U5lyH!I*TVz$&#xUWUAs6ouY97?x@t0lv_`LT;0dVQ z3{6~K?jA4$J3$Vf=FGm{)|icYv?_crfJF?*V*?C=qLorsxP(q?clwz7%k=7Hya6#0dt1`cioSan2du#`FQnvj^ z?bbK2!L8y>-}nIXgpAAvQx)D$a!kpnblU`-Ga4KUKfAnhgkd?+a=@&|xs~?oUR!Z& zpEv|J>j|bR+c?TD8c8ylI}4i-4^1F%9E<$E-FBG>$0@tUbK&?T{98vp;W2q=e8jDK z;`LslGYHqvvyuP)s^vcZz3(8O1es+5d~m9j^qAK2ol%F()$>oPN4RM9-BEe@i~34D zp>6P#K=7mP`%3aS+AZ_)QfHBq^`M$QouQIkslHsBiuG55Ii%LgT{4^MoE>0-p8*m9 z)4eTUg0?N7xS0fzO19TOIr3771m-(1_#s*(b6~8+$knkQ$R9OT zhd-7G7=HNdqpA+czw-)qc-Ny>|0rQ3Wet@}-|6)Mqzj#h_rPP|3B>=Hiod#hELWk5 z{OCF`ze9$tC)3Wp8{;P(5}(4ebC2K7d3d{_cFWNTj65eprXx-xJlh}9?*U?3XU#hU{#|M&}L&Mv(<7%aO%5m~qSA-{J* z6QJ!=mvCD+0w~Z9KlY{CEeIsd5Ne^aJd$VFO8m$UoGyh^tpHRPhDKgME+Y$dMgF3A zlzjFpC6}{CGoTt4u3e9xYT@TV1!!Cb880L=Ik7Xtc?Ghoy>X}X&{XLnbta z%ZPLVIYkNke8iqTBi6CpP%}UBg}31oalRWu7m34=mor1Xq>>!M<*MNB0q{r7Yfnie zm@)~5YhO4!s#L=TVKi7)KXsGg->z0+NPpJ)^E||uF88ACS$gtfDtdxI7Qw>jJkZAh zzItacAHr|YKI_VE6%ChCG`tW3fLt2d*YDM|^!(i(*9e9kd&kh|N17 z>G0gxhC;GX6<#X42zvEF*pM9Hyt@P{@*+rsb6RI?(JFU>1W6jx*@&ds{~mCoX-sJ`YeJU@VehnX0*!qepb00h^Zddo@*CEDS< zEW~TTvaW8wTrExdCt&4!^Xf2ELbN<>p^mcGD9|5wUqoqPDYX-(gyd1IpNou~qYGx+ z1I!QLIg|v_^b9zhx#BcTMuQPZ6Se7$)~obz&DNq0d*pWQYk>qVF9JO_)RWS!7&v&B z4UeSFCliO5g{8!O}}BzC36^Roro<^uw09ESc2JQTsBc&dGt zxR7G>I%M<`_kQJ~sOnW80p)X66=1ZYO0EKne2`hn4M{Ii2&xu}J!{7-5p|Jb?jy#- zxG798^LUtNsau4MhfYeOX6pVUvH#k(C5HP8KHqj3t>a^zehLjszBV1@2K(o_%skqt- zRK9i3A1>2IaY8W~b`6-ON+u|K;IK3GWMI_@dl=`wdugoJ)A0g-heA>b8C5q$d;}X8 z0)+nT7GZFo?pR{d?{IkfPamf+vX<4TkQ|=-l)Z}8=w<0ek(gDX>a|0sT!oI9as98AIT6N?ku>#9PRaOYyB!n<#m-WQ{#MADuVC5G??l*j{&)gfQRIBmS^Vp2W} zS~mN#Ky63SoSVs)-b!gFTni7)4NC-#v7v4|KX-v3UG%dC58dlvh*sG>`VW?d56H?|l`S@~{SK5pL@R%BeJpJ1O! zsM}V{TV7yjV~dD5+l`WtRZzfeb1k8%7O!n`a3Oe zaSYiI#&G&cJ-QD?7FZg#gOLU@r|DLEgL`au<= zKDF29Dg?zkkaSc?dXDnX#HHnAOjmCVMGDbTrENU28>2+WtVMKuW7bVhr({2yY`1 zB;8$e<)vQ3W}3k&5c}A&W>R8hR-mkT{a(tZ0C!)?*PK5~FOb^?e8hy$URC{5BANGU zIvYEL18A#L62~Mv)32&C1W@59uVO~ zoB@inPOrKVUx0w>O1blHccXdyZ48>2>kBZfEIaDG*QsR8SzE?=n;0i!N!}Z2V3U3# z;7SBCNRcH4wA;~}Q_Ao`7B*ERI6=h`kq}!;z^fRXbj|~ao1yfriM8~$qG$LofP49g zkpcpnDF6$O;6K2U*G1tE7#*=qzRsgjKesMQ99?SHMv5J^_qvSnu-3^*+ymC?GvAsz zNdQFg46p$uIw2rmzKqa}t7a{EDc<+wr5J)&PRvnC&tkTo z&D^q6kz6m97n@T$C(D67PD_8Rc4hTQwNkRJx$0`66d{E#O8Gl(=Mt?s(`-$e-Be?A z@pRu1_`>H0E_7^k>|F3KU~R*-w~8*;TWMT~t}U0Hf*Ve+@qFuZN*`gi*86eVmEI%= zX7T6EHq|u{-bS=sx3FAm3*tb!V9K3}@`wb)qt#mlo2nODrVh$Dh>QtUjN`n#sY7pW zqeZ|g`(L3@5|om_aeo9uM~nX{Q5|?y7HYnW4HB7Q>OEMf6{(lcaxKR`I=+SesSv?% z6ExUm;x0DnWCcNISL zW7`iv1N>ssH=@o#)yCuEjtLNpE-%72(=w@NwIjaq5$PuB1tEevIuQHOa0Prd0mb_U z!1HLEgdh|q#=06G{|8Z6y=bh-F?;Ch7?G#H6&5*d#CzGdM;cRWs=B&KVcoJH-aF1Q7*L8oBIX zqJc$P(dxS%B4&hiK3%qq^d(awq_DUIdTB@h_G{LO<`RI)cWMfJbmBfv@4e4|&B^~g zaJ1w`gkg^|GAL?F_8rDiB@~miMtWFny z!XQN)RNGCT2a_lS1tF1Hz<%4$G&^_~f~17TpnM6$_Y1LNksaGzjef<4zM09Op1cR^UqMm{1@l5$ zEMvG66|`87amon!_kIA8-vUZ1CN+HObqRGWW(k<}%;8=`(^`K4(IW}j>5Q;vYgjHx zvMvf45xp5IwZ2LkYE^E5&{R<2+Km;3B_db4Vok>E;Xf#Vs|qUTk}>G*p&j^Nc2=$0y;n^r)k64A?Sc5lq{)DSuq zzDf1k+7ki=6!}B7%|yR4iO8Xebw@ZG%;sqfm&z_j>&tY>(a|98_YP+M9%7Cj)ti*jZgi&GeFioh(C zAUGWG)~P&zYMb)D1Phq^0v!W0NJYD;NtR*zyNFH2*L%AD z#$6dM*q_E|Y*^V62)LC$UUuTt3Lp~PqQVqx4vdN;^aTshwRksc#h2ZNFhJrB*ld`C z2^94RnsvRgFMSe!BL7aj!qQ2{)xZQe!*Y@i9m-5HbI#qNkgbShbEyBOpff;m6TF|ks81EgoFI6(p1Z(@$6@Yj~=b$dZzQP1UzyO<~FI2cU%k+fu zg#kgmP|v>tmF@^}{hkdVfLz%hPuC&m$9_n1Qr=S?te}EBH?89dlA=&qeSyQAD&>Us zix1lgxgjVorS#c@-f*{Pbx3c@dQ2t@Qt)|rOJu>MXJnU3 z$Vt<83u|4y;Ff(YpH%9+SaBYMk^_pxH#sX%`hzx7<&Sqiz2{v(vWKLkuh#D~0Z{!% z9@S#j8GE;jdW#FS7J<_efhFJ0ng9R$KiP?0VVL$xAYcwnM9aF|b_ z;jA!_esxLG2al+KMgrzz$Ix3%FJHFx9=33sg@J99@e_fNEe0e8Gujw&1w$BT9mhZk zpD5lq{7^M`;o$auZhMKn)7$HTwS67GhlewZ7F;2bfCO{jQ=-v|wx=zf{=wN<#G_g) z^+Zq(z`l-(O(!Mi3%BtjVqXm=dkw4qmmsv(IsFR6{TTqQO*Ed2J0&r8U8S6qx&H}- zLZBjoH(MAGo?}~;h&<~Jy&WWL4^I*UplAKBe=XxWA125n%~m=tWQl;T_s8n~e%ReN z7wkahb!u-fZvk;a)L1UGKEE<25Z13Sjfu2c9nx_jE$fbi?;wL!70dgi z>V$Y|qaQ1BCO5Oo^X>kP4*f=#x5xec@@z=SKB(r>*Jk-RQ&`5kghYNK&czJGWL7NR z-C(U6&u<(q?;v?!_c-}WF#{IL@R`JpnOb^buQIosS(3howK)MJ-4A6z1EwkK#c*H* z4tU>X3sd(wi+w5tdk8&wap?ni)4MRSck8$3p2V+X+_3HK?D6d?koSHj;EZW?ZcN5% zu-_H_fOfnxCUHY^^&$P~pzHG}JU`?3sK~ItV*`m9PNO-~v9*PtyZrN@tHcYk07pp| zm{+Cy1*t_t0>Qx1Flaupy-?!WeA(&!adP!V?tN+I?_E7aYzO25mLQa)+D2)izWH#9Ak$Mr$#qEyrWY*4 ze~fwSHq2Q|%zGvDRTC!^7Kk^_%tQ#;8b4GAOfOk%SP$WF%R~x zVg;Q}_scP6ZnYGc4x0!zJ0?+H!WS*)ThX8qB98}PSYS@=AgL#Y#lj9UGfT^%NrMSU z1~fW6j9s+EuBa4O&C|b($YRoKcv=K?aODVrR$x=v#qO5u|Xv`3eN3TwtTb-al37 z%&})tL@uzb*!C}D=g(0zag1=!Ph)1o1tq&{T6CpDM=1PwTKv8^TDQ6)!nSk;F z=g=wg0_YFBhj!X(r<=8XbSS{y9^C!z3f^b-uCa41t$pT+kSj?fB6U){Kwx}iCqjC% z3dne;!Mm3gFTbzv4*YEP2lo4o9JiwxuG^$quBnf&7OvwajXF6EA7Bmn{3Q%Z!H$JU zKbfHiZjHOcudy)dzUHf3BN0vm@cem$6LFN;%qaQG$7b0USR~uiFpZQ>RDj?!+>vpu zZOcZUYxjsua~`bfsn(YYSi2fyItv164@-w{g@ zF6({b?yF<)IZhN>;QvAtMmuO{RCx%?0S9sOaMZfj_jx*uc)fPItsnG_{TzTIZh zMR?v``|F)!E51y`N2HhJw}D5y2ah&9?~aCv=@JrPWM6uu4)T&+r?7~X?6-yX(@d)G zskvB>h+8i2PigbfmO+F)J?r{+A^}m0>v}BmbUoiCsP#HDhFO zy~}66G)}zx#j`ds=Pd@RqY|5#}YK?o+OXk^^H8*R_FA-xO|9JJq(_vuY&X}n!o{rfV8{Q{&Q^CieY^iGb~T(Of9sx}>s$+}R) zCuoYmkDKOHu{&7(vb_ zwTSJq=Acgc^1a0>(?Ve{3?34WA7PAK`~C;$zm0YqezkOD3+ss0fdBwbAOHZE|JRd3 zO9NMf|EMw=SX&!e8~l!VooGxQ4Qxzk&7Ezmi;`uWGUyRPZoW~mv(|w}@dvMr>6sv1 z7d-Fx!X-F)T#Q#SNb7GQ*8ckGjgggY6%M+}!c27EG-%R5{FM6PiuQ;CzvAiW_MSdp z?u$VT<~^E`7dGNQt-y1)tObfQq&ZD7je!&eQ2~)GdHM=kD4^K>3U#s+&B4k%NxLf> zYsA>*BO@_?5|0E>QHxqf?6Q;TJMq(VTp=d15{o1&ho4X=FhoO2OSKo&yX zurY0_I1io2*THJhX;wKUkQV8vofN(B@f%rxQp?$uEHOVwhMjW43OpnexiLyOlYza@h5vSiy~gi41&M&b+g z$)r4kSx>kDnJ5}sgJ0XPMvCCp#=ck+KAw5gx=+FZbvLg9y4wy{>mOkxrCmd3TXDUG z=I=09P~{_CPE|N_sr%^z>v3HAYtenlFKTu%9baATA4BI~YeYSX=YapL(GFCW>ASmm zjMS}qg(p={zeBmimyKvm4Q)STuX7HHn{Y-pe0}8vHyszT;cySJ^Os>al7zurv3v50AT(9l&Q9M|1DRelXUFX=}|&%KB#1Unsi8xiA|89 z;@AD#RlfL1uNo`HNJ<8)E(k`Lr+RW=Eug8G+5}TixNq*l9r_RMWHLXqWp~9KT_6`a z8YXfW5Mt-`zEbBM7`HzWUbARPBE}M|bj8Uivb2#2+Lk8>i&RwJRCeeV1Ib#LS7wNE z^C=`BnX=+fDY&1xTZb*j(IB$T(5Qg}$A-dk8--)2t$LjjOH+qoj#;?t(UVary*&g} z(#=obcRr_cb`Pmi=_I-X8H#7I`AC;GOFPuLz2&S5PPn99UK&hmth1xYZ zp>h}G7 zd99JZJzcU`h?qL8bsE=BI!7%#E@%szs}R$eL|iu}JHSR?C$RfWLTc(P;vJaqskFjf z+!RLBA5Cz%=b2RZ32v|c?mVnO?IKYNd81_vGXV&X9CN;(( zDS5@MhwnI~ct^oolRlm~Qo4`Bjq@GQ9G)DfDpzLbF_zpVq^xHzvNCu<>jcOIENs!! zFw=m4huh|DTbD(<3N@=<@qBNh|>X z!1TY*OCvkmUu9ccXBt-nYZnuz|FeKzY5O_tvm$@<}B?>1cdIl7$mg)1}xu2`7+Vov)dxSpo8X+p`6%J;YIXAr1qS9A#+PLS{zuGSB=C@Fl9u? zuW~<3mwG*j<*`CDJ|UtisnX8atqg4!7!?kRZueyW#qQjBYWf}-FSpuAW!=)6KMC~8=LWBD-i#-t{@hFE$#9^aMU-tPx zrr<0**WzWQ*}+)T=cb>T&+;-qY=g%4dQ!@E4)I_d2yfH6gN9`hSc-FuVK(tM-g}q+ z{pJ=^=>f4eLT-SCuccY90n=9PV;uMxxpCKx`UyG1*u=>#6McRnSt?B0$xH)WQ%|?s zVe>===Xob(!D}RE$&*jOYpY!pyh@z7mJz`osrkhJj*fjMN|)(29Nn4unnv~b2Q7ZS zZDsAzTnvh}KKnO^q~Fq$;ZPs}T0D)9SQNR>_YSb9D+C$(wi}0qo8jRh?^LYidNAky zYqZ5cbF5{p^S|Y$Vaw0cQQV-njJVQ(EyZx@`fFKC&et-=btFA}`|XW_)eT|(M}K$k zLpc*hHa^2~q4v{Ku*er*EJmRNUMJ8p0%)LP2myy;ZJI|vw8tP?E{)%HZXG8>StNOF zA7%qCqps#eBPS`-wVdPn?n^(rgUSrSVC$`$z#Q7T`0kyySVG^qe+GTB-x%1pA( zgMLu^jktRq89226hEAjAESgyP=i(E6!ZKm?q_q)bU@M|@buOWq5g=?S7`?6XcAO0=3Aiv7|VN}DC%tbQT=beq9nI{5GXZ>GUDVv zX2$Gf`L@!9vC){}hgV8*yR#;T#5EAsI0BETHj(Zkhi}eTdya8O1WE}nK>qHBHtASnyyPD!)ZUj!Y9V0NTW&JYs)xZ7bG<`^u<7#- zYqlO?^So!9JVq|(2qUFba!KL!8NHF3J&Yb1U zIB2bht1J#)toUoosr?|45c~IoftwV9bZdXj!K8q+w{p0i>b%4XaF6BOAuM?9CM#=$ z$(F&Y|jkGi|E>Gn`vBtP&$RHTyh`HrtzexjdprX5 zYuBCSharYQA4zWX{Y4vta0RxLNrOel=;ijt=s(f_;a8vG;BJ@lQF5 z)wsU2^UtqM8zNSE>h?3kakt$C0)Y24+B}=WMRt?6M^;7+WP77`0Vl_igT0aXoJq6d zqvE1rlJA;h_n59EoV%7*@Jt|31Ci48Y#s)fs_Ef2BTFZ8Dv%f`w;vmoouGE+pbn)q zl*3R3l~i@rG$kCbDMBbB4&UmYt$;}Ce6)@RO#3IMZ7$V3_#zPpeLB8I?ihC$aV@TT zr5q+=z@!BAII6MMT&P+-?Ww4eISnttux&#AE^$u_vW*lgPtnOx)uRAK#}BR3 zHV?*yJ8BItMD)AI07WEn<|k2n3LOovCj(nxwBRXf16m65Ns03EQdIi12-u(6+ZGL^ zfN;F_(#MZ}*}W?fWVTp=>T$uv8b6LBc~ot&_PE7}=EMJNlk&TpF1SZWMiEq|3?`%V zRXYhXON=*qJ%PRJTD}4zE$Xyz^shL_3n`CCcPUi7va_Nw|KR{}%afL3Asx)0e&QsY z6m{EEzC^*;?n0eES>*s@|9O|q;>;j8jkyoSAL()egYhug2abP2>_(7)O=lL5s8GB#tq)u5#HE#{7Ro$aWJFui$~NdVBfh2U-8wC1koOrEz!< zLct$D)h7dm04OUN>5V>OaT^ID70wi83ABp*x$;l%tZ>Phcb#J4i9BVM?JUU{DQRyH zkQkab#zl}SDxlD!hwGdg=LYRQ0yUF0>Ys1*X;qHJMO%{>QI5SdsDD%`PW=Fs(XAWk zT`fZ-XgH{;vd|N65WAXdXU<+y1f|ON)54*Sa0p>oHJ_c#@B+=liH|#W;_p2*H?|6e zH9!}8c6LQu+uc1$>^fjv78Fpx66kcnen$|7LzW*LcWF=);VK4yy2CHq^j=JThcbD? z_N3>Si*kMlAJ%J9#cY3VE`#Naa#OP3`9n%y&}NKtiFfcOZlBdd?9S_8D8KZ^|6vkE z>n#povVx@ro-hr@no&SmD;OUf=u1S+Z-4Opj~CGehFgxBUs2b?UzxA}7sn`LlV7N@ zk%6;`F^#o_lk@*|j8faOTNg+0JE=)+hbM)91`~8DL2=rzSjkl658pV?&Ig6xHLa`K z0*#RU+Ob=z+f zQtgzB5SNQy$}qa_85ekuHQn}M{V#d`DI8 z7^7IT{3r&&vE2&^qEK?6`g4;n6Po^(NjF0kt(c0Y+5b5%9`e2G6`JjW*Z;%|yaN&4 zn(4asaj?V|2cSpRru-wD5c3*hsn|$;oI#lqOF|a{Q;B#WI@Wy93?UWyM&zC*0?h9iM&t{L*l zG%=Q)=}I)t&+;YQ!Il<{LLn`zZuNUWGpbB8+VRG;MA4ch&s4fkdFyhY@P4r~TRxRN zfXDPDeFk-DPruJ{HGvE^r8zCSxUxR@WmjRiyqTjnWG_Vkag~sm<}7|M%A@a}#TOn*SIf z{HFnRrD0>Y!H)Eut;bODQ46UrC6jT5>*)Yzn_%m-LMCuvTbBc+iKS(2tRN=o>J$2P z!zQF?*F6731IJEX3z#5&IBCyLHv?1la8rFvYsR;uH?Xnx;b#AH_tcRg1u?Afcs68$ zqBB*!IjHe^iDIgGD4MKzL=>5Su>la+NFAxB(70yZ|2uv^GhzNfC_4JX70Flx6C#lY zf-ZKl6Fp)dnc-C@j1g<$Z$W_k=7^*}jY1hQi2OUnw^T3$dZ6(}w2Cjqh*>FxrGYKk z%Na1snek-@^h_&4)i!{c(d%#M>E!&_TZ&Av|Dq2%nUU3N4VKg}b!X`631~@0J&E_? ztBtWOi1c9tV<*&R)dT>Odzj=Lpdwr8yruz3rY(7aAK-q$sj#3AcyNNcazQ4j4 z5e-SO)OxD;*LAU)1rpCa)kMVipHq_cSx$Hp^>{V)b|p3}?J9~GKc6GuNLmCaHt%7u zU>f=BldZ#N7qgL}kPKW}VtfCMoIYc~-5KQPM*Y1UA+rq?3AQSg@2%t#8sxVnr5{HG zkyXuTm7H9gB_48H_bb=g^Dc8T7?Q;xH!*rVa&!4S{fecLeCP=x^1n5qJm~LC(JKa^F_cKsho86P z4-}COp^bMS(f@Lrv^A}OxK`qvM>;$DPl6YV3w^HWT%2`HnqmfTWz*|54x8g|_Dhy3 zRS!ox-wb?aJO}J}s$(286ky}*9@3*J_xsU>PfLqEc@=ezDhK}Qfy7_1k84VYC{gaC~q*H$Y%Y) zY_2$ke7(E6Bw}*pU;fq!)5fqF>o=uIn^%hFcc0D@nTQi88XHPMgE|Seibaa_%+i^H z!5dGi#K7+0g}j}XJ9(j_KNAizJtAJbKU-qdfFVQ@;x$#7m%v6a(92&|z@kzE$qjE) z*=e>i*q5XbrTpv7#z1^Pajw)7|$;Ox2{-91k@(HA=#Gtp9P2A6*}M)_RE8+ z=jXi_v!n|iK`taijiU%8w=xnqear)jGR{IE8FNiT9!iV~nqU%oN8WgOTtJ_aejziu zv6>fbakk0~_w7A;L8pW?D_Knu`+P5le=6kr{p}Gpul4EOOvnLl3_+?)1 zvbx57aMq}9t<*4}S)W{|E=#&?mv~f5o{sbDomFH8=+G^9ZbOp-`0fnYo}iXM~drzY87wN0$D6c>o$AwsjQ zE7j=(&1f1gbx;`N0XrM#BwehPxMo*vO)t1+yyP6Q&1sZV_-E&sObc2fVYN*q(Nz66 z)DrK8GqUdypW`OcbmJA1s8s_M4%I`;OSflpgQ%*gO0Gv_(b}rSc5!LCec0x%Z>=op zS=2E35?)DGkyvSnH28D-(2H8BX<|`tF}$lD4m>L&Bug4UxG8eX{jp`tc$jd* zRpF36Gfo6ev+B|v84wqHjk;OCSAMuGif1mtfuA@jo(R>68j_7kFfj3un{vUyU0ykT zH#4W1AOa}3njFTZt5ByBsS@$;E$S227A<4b;gEirii*592EmL1w-hJA>OWXyRJ?AR z18tnRHLlv(8S6P&$2>nyD_j&!EiLtv+bSj(@)TDz9Jqa-{4IiFKK$DpR1+Pb?tiPX zgB|_2v^mA4fE7T(xn{4Rwqd$Y9)Sj#KzNWocuEVBr7@OvH?`+m9cxde%Ra!uf z{xu~9aQpB=Ut&rH-X)^THUW9e`JE5_lcm)i#DeCszIYah{WJ~w1Pt==+VOU8)|&6@ zfKqM4n^wXcS3@DU1-R%}(iP{<#w=A$g=y>+;yi->Sj!rm@oz3!vD!jRHjmD)d3BY} z(*u>}JVP^GI;*9-^n_50s@+pflxwGEDr<0pgUWR3);5geuHcqZOBUU(o3mT%VUVj> zjJ}Pe3kI_MTCkU8F0E!<2*R)C5F^!pJCjO-V-~I?kZJ$sMfq*2vWUSqks1GMJHZJ*L9xw0B>qymIZG6i|rV(G}+ zKw&RYP0yh~Sest%IEgw#_vVFq3;ENT;g{FR7kgRzZ>L(!0~Ex!Nnfq1nGQgh)-JkEfm2;uRV&&2Rn!HQo?-UoNS|aogRKE_Wua8TTnO2HS zlhzN_q@^9fYwkj{82jsWzr%9;D9JF^Pyq~nZe?JZx{@wYT2T^4f8&Fx1ts)q2XkSm z(5BBl*TS=>L(>eB2Qo8Wty3IbV1)ywi^boI1xM~y+~yqb;xR9x#5 z@hGZe}Nt);g(3TG8?L&iwAe-F}sXEaZY(JYiDh<1jd`>Hg%5(P{Reb0O!1@2yr@rA~cJ zxyce|r88PGCj0NwUXivn))0C6mJQAtFU3}WuD$0SZiPJW{8W-y1_G)KUVfDownBCB z+kghV=yiUsGfM!kD3j-j@v-5?T6tL=dAOC&z=32qd6+nR6g*1=n0)8gL!d)?E}n_u z>Nbn`U&ME70I_V{WbJYJOPW8bb&eBfsl-TRl%&n6aFnq|WY*WjO__7=Kvq)A0=fLh z7e{5zu=`8it@kX=tGg9y#8_S1Owu*C=iYkv`EuUcRjFC2f6dRo*`vqk z?mc6dWS3!3)tiZ^TXTPgm|0tlT(I$^s3B*;E9h=3Ue4yfMqh~eNTkzsMZ#ideJJ`L z;)l=MRxYnSzLIUO!tAQX#=#ihTmLKuo2WS678)gpPHV*!1!>ynCkm7%@4b6X+SGoPm znN-pL=S0?5j;=lF%7zH#xcx$bq!EyMmJLn=vl%w@W~zUROo~Q2qS!=4>r3yg*m#T8 zBu(9YpmhB0O}A%}^9xrZ9NQuvuFZbxxvlHn-P7Xs2bNyU>6eesp?l(a(MFpcnCYF& z0|@0(j;rUa->}AMYL043v1U$ZdY0#6tWFi$c`(|t7~2v7R_+_lsJ|yQ-Wztmy;*-B z{;>hC0Xa0G2<}Q>u6*7R=FisCJtW-+dMJD(`e>*$J%B+rdN)t^`}c=Wm@QrzeRg)P z+J+0EQudt@*@ir|yg4m0VM1|>~W zLtU;bwnX;gu)Mxu9#;v^(k`daJznOi^Xd=G5V^cNEG{2kM3|VAR6Awx{>=`F#c}3% zf7gTP+Jm?iktMM-wQD$Reh$l^>DgP>pX4hF7n7fGf`aMT8mAM;qbd9zQVdq*isF`T zeCSb|l0$rC=p#D0?u3+=V)SmqFh=wMji1}ui|F~}32t)n*kIPnW(u=}3rL%^+?K=~ zu7x>!pnW3(Dh|w-&2as`NxT!P?xEsPOMp+Td|#XI4ZaruL$q530(eyrBdFManG_pT&llp z2eRlckjBx>zb>||b@<4?R*?Zy0Q*!WgWG-=85`fP60sfj+4i0-!hox+S+vBR{&pMR z$Zy<&QK(6p?skvI+B7rGG_D?ht%&&zGgyhW+%d=>S8Op95{Op21FJ({7i~P`!W9d` zbfI5@d%+~MP!(1E=%3IZHm_WFAjjd-?BN-ZcOG`k{U$ZBql{tMa>*Rb3ET!wfPiT{ z1mtnG@94L*FGPxMd<2XymwsaAk+X3+ql%W>l`$wkY|wvf5$bEX8S<4M&|5)#1?dM* zbvXt)?02G`L0WQMD~DkduPn21K5|U29^+dq@~PH*D#V{g&Re)+{TIBdL0~}2q*&nm zJ?GbAKl8xh{+dM4C30~I6V`$OZ_Ws*yf-5&MTHS<6dJ}9doBIOoX47P8RT zc}4-$EzzL04|9|BeN(yBu}QLIm0-K`CMegh*=QSbdy&7yLRSOYguGZ1xq{RT?@a5*CCSmHdV z(r@ot>dm$OBE7ZsbE$OkI^%86*)Pt7;o(Y`*5mK(PLbQMSl6=G^NX97U~II*UKvgF z)~^r8&}k|Pu{(!6iu-VyATNu%<@%X zpO+Lkk^uY2)aAz9vTTPtRW~1ur2An|=SPPkhI-ybd)SEMoIy$m2F`002HOrE7T#+L z?M{t7SG&ExZ&{)RUx@gyFy|nliz*=ELU&l}>d zE%lZ6Vh6u>Q!smSRyFnYTru;voF8ByVgOMAY&yTxLQ=5N3fTj~$ReSj3OEwec*!B# z^AI5$8;c}(RA2!H)-b;?=_q8A=Kd)vZm1L`j8An2XIt6>G7Uc~=+;%ya@ugS0%*^o8taZb&Q%iloAN;ODS9!P;6RDPN!Q8U^w_YLgvqz*#Ld5LmYX0^A4tK zwf>9`0IM||EUBuIoBjKkZK5GkYj<|A`KXT~KqCqwloc-qBTs(9gfgglW9LANJi|s$ zIDg4iodo;kf|wIE3F?bx7EXL*pewNxz>z5~I~Hllst_j_m1K>(`j#0nnWg};;}U2) z5D?R|NUY(+u~wxL2oX)b0H%;)7){2|Gv%FOVF#uJ7@~uXcJ)YzK807*QLPrEZ0AzP zAJ!N<)YIHt0#>R>D=%-}h)2xTCsTC$jPwrlDd(6p0d2=|Ts#r@C-)tsa77ITbbTCCUHq@B18;}0SLrrT*#8i_wI z3n`WkHJW%fMz&9}DQFpPs4yg>zSXPFP8|3s@f47O{E`77zPoo?upiU zTjt*%(bvTAI>aurWZK2T7UCn1Q&FZPv7$}8&bD-cd|*6J9B&?hpm1nxtEpoR41WhZ zhuW~5rUeGe+sXHc5u_TJ2@poQdxERNlh!dt9{+5md&*4Y+S`SaF~(rBODOVZ%Y~?% z=6%hy$mh<09ik>TNkEMYgZ>uRCBcQz z&xa^IRE80iY^K}oP&F`yI`zaU*$3r*p@ z%cS3<5T6{4o95KTAM!+EM0fF`Q@MWSA$7!$&s4KS9{^KW2wY4i<9Qs~K_Q8nppjr-7N0Hy8UvnGVm#@P?S6pK zQ@EKLe5`*iis!P3P}%U;3XI#&_1H9f0)5@g8rY4+vM>YyHmR`%@!lre4KobY7BjCw z42eLa$mR?vvdcBP&s|CliB45lr|!J5gihuvtk@Y=x5EN@e60?JZ*!s~*?lV>2+FZ& zk(YONqHUrPI%{usu;((TJ#Y;Hg%u?@)A_J>5~%SY>g>ebEY>%=x=C+d3>7XYnkC!D z!6l<=jaAQuXk+_S?u&GUQkdk$#e2yF^wv)x%3|Mmwlm@vBGS~~5S+`~Fkpv^5q!>#` z)S{{1Y%;`97cJoNR&Dp2%M@V!`ybMx${^)E6;G;WI(TqMBAFcXfGIC(Z`8y(ODlQ? z5E;CQ=aphn^RLoK-Ix~_2AyA_F@<+G^hayhVr09JYFDb-OPy;~9yQ3=>aZ_t7M3y- z*L8*!nfhi?>jJiG!%)Uv1peyzTD6S$#y_QKx>`mr9v;4*mwPT)WHWdDyzwSKCiQ+= z?X2#96#L*jhcw)}>s&55YR3Ilc~n|1=%M-eb5zd}Tm}E-pwit26R=sUT(gV+u1{%< zX^W#az0kvCS@VTtSjqo4L(5mX=DTv-dHt`gvD-9}kK*H{Ou{>Bwugf1-%TlpbcQ3;1U`RwK9{SmtGTQgnp;@kYp9Zo>l|Y7l!9|XP@XhUL!rc3R5)%c0{+f z4<(IwH8NTw3QxuOAR$95gX84T*2YvdqT{Q{O^<1>vw@BI2`-`|6p!nPNlYW;NfL<6 z@n*P5ka9cV7)U-6u`!zmVLaf^0Vp@95;{p@hz`IiP=1UFwLi#?Ep&_&GLYiVyuy(t za2`N*CA}JC2~qBBLR zt5<#C`N13JA*~10dkoyeFh*&1o;KwxWnqc=0a4m+nCPi01`_?B`{aq;Rr z)qGiHYZOImxPX>jK^y*j?)TfAPc;yUv2vw;#*$PIwNIHlt5Tu7WL~e|$j(P30{lhJ zmmn-_Nd+hx)4El5I-h?GreZs0@UK=$g*wXS|;rv!{ZA8V+u<}aNQ|C=XT~u_i zw%JnOLPg!OXif?P^=|q8r(M!L1<2S?OdU*NAE=qsviRzeu8Ak_sILf&k<6uemXj(mTGicv= zK!+&ARZW4IPi~FtRX1GQi)j!F^UCR&moPP0N3!Ui zb52Mb$A}`U(qmY3$;;okZ04W=+%`Bx^|QFhlkmdU>ysm`Rzdr3Ff+7!F(7Gj`(WrS z;H9N#Twu6)Lg#sz**2{U_;9)~8 zXN!vowkG1qOTJfGR(d>oQ}9D;oiHG$%e)RHGyvM)_52$Zm_ zs32aOve_rO4hX6^{S%F=`~t#V!ma3k5z9FCjgv!KoYoc!U0%YA!xk9nF4e^JJ0t4e zGUW2C`yi-Y|2hL(ojG^Lg4^?Km0a`c>IwxYEOdrX=?szP1TVZKp2^E5UGBH%R3L-* z^zSO2p#MyNYQj!7BsH7}WKUdj7b+$=EyGG~)S-;#T9LiDH9dL5eHgtrsOXkH=*uw! z*50qDA;*@)8PIUWjiqtVp+D4{LZ#l9FJ#HCOrEAA+rE!4dVVKvI{bz7Sf)BfHs)LN zsLFmcI<7(Sn6!zm8yfZ|7@c4GIqz8{k>gTQ*fqcO#okrn)QMY8TwWPq6Ao8o*B znJcs~q$^rBg6CInXGO!3Y{^HP8-ZSsjzriQN01=iJ08?K%Od85mOM3ym-XIFUD`N%<46Htu&jxS-WRl`dHOB<9A>!TP5V3 z2ev($8xNB$RaJ1nJD0Am)+#MQOcs}aq2~_#u91jbMFf-o$uZUu6iSwuEah-mur5qL z-oM7Z09r5ZD_uZ7dGj0Ww;~Z8?a)>jFvmA_(S1yO`bce>;|DH0YyW^_h z?Zj+BuKlWe`JF&dm0m~Av5;0|DavRq(7hb=v;zh?SmRl3{5?$tqXTog0?m(-w(|@R zam8l-8e>u+-oe_`NyM7cX*g}2$z|5yr-%*uc9j-89|OvMT^HKSTlYt7Ixcn*8kBsx zth@Y@A?D^$>aJc{?Z(T}vpe08vC9L&+KPF0FYhTu2cwOq`(e+znIp5B1u1X)Zq5JY!7;Hgm zcpR>ARpt35stGYl7!1(&oUxCkOQh%McWKdfAIGQOLBM*rQJP+%HxeT(K^dPdNjAII zL~=Tl+pXWt1c+<~4r$H2CYm`$z?LZZ=BETVbG|5w43pw;Zf6=|23C1L!ZMeijF`<^3Hp z*{~rx%>`SYhAtI01hPl~G{z8kL7l`2HjH}%a*`)YRbQWJpqB9REMFNOv&XNFPN9Bn zG-ph>(`cQZN<&cSxiAERJ__*-0;oG--0C#ykRUico>{7*aC&)sobcBJ28nGLGX!uo zFp)^J1P-L~JIKA7@LY+X;WAp{wPvacrOZYxQG7E|0aO7_5Z`Qm%WizZh2?wKtb}=@ zqyVauMvQ$bS>-^*+lyr>3}_<^YsGQ3-W{k8pZbU7EQLX4zftR$OeF$o4xDlgM~pN* zGmYU3)Z>uu(;zkq!(`mxwX}teQC|u53xNZS5#2rKK$0A$XWi3Dk-1^7G!XCH#O5g4 zQmCa8L`L7(y(-SF3WN=RK#Dq9qs=$ljX3*Bx36N1SfVofFSU8bIASk>eD|}iuA?}y zfrIB$a4v|A{AP-Xf;m@k!*xcK!0Q1wgF(#Gk#V(f`WbWjkPHt~(UO?SNDt&YHm!{c z{r566R=sVI!i#$^XPguoOuRBu(;o zKZtN6o*GGxXRE`A0?jlKM%|I>2xXuT1|ry5!X&8<3i-9v5ptmqHX_~@@olQYCi4B8 zuH&{UTV2HF#joPFYPGqDU9Was#BSDea1uIOY(tNFV1C})Uw@vq!8}`zcfLA6=0TXu zpkJh?&70G%EYwWKxvMJ##yDfDR)r)E4B-3jqRD1%@;U?dn}{oUkL{?Zn%raupw)?8;ThlD4MTis48*>*oUSj_d>^DQUYsxGxs zU08M&)KDFvY%4@B)Ls-=SPOgJRQyqP*8O1mg3HvUX~c88EcX2$3UryVadzd0s8l~h zW&e+eO8;+2-Id)pUa0ntZ>zix3_3<5Lo!xns z;XL=CRERwM{5vbly396hL-ba-Dv%q{$$G)FwAl*t4^lZS^|qihE`*1W&@3oK@+o;rJSpVNO~w(nKz`iwHXD}mlnfjusoJZu8( z>}k;W`19cku=)KLWp2t$5w$gjZ-JNuifOxm}!^$#3#AHh;kx9Om)!F;{N5T7I4T0{Bxd?{=DX zed`nY=#;zRLGGC$QZDnF4QGo53{i02dvKgaX5xkCoTTI7g^ZiCH|;5H{6dbPA+$eH zJzV@fv0Doe?>cc4vLX(3Q$TbxB@X$?3%vHcsC{^%=xDHm65Le@PR&sV?$=CZE2@yA z*uo5Cf}jimIe+_ z-vmTifr<-Nf+u@WNYz18p~QH7J>yhGVf69%DB-Q26f;{eXO!@Da8W1sZFZyy`^SCS z2)92>J;;Kpv(#OZcOHp~`8^lYt_E>l^;QY3aS4KvKtrW#FET(z4w5=sqL@I(qYfie zUFrg5Odp5R1Vdr?_11U(-anywlw?uzi@KG=MnRZ)(wy9K+E_23#Jt9(m7MLWI0`vpY1*CQt`BWbIxi&b9l0YoZODeyytf+g2VDfEtnM-P2 zd>*%6S{YfiM-ZK=g#ynibvg#cGApx)=x1NSX#~TPob8KIMHx?J;4k0lVw%nJSe0t< zDoX~+)aPSsC#(54p_Vc?r!IikBB|sPz?;!^%jUQ?fnb>VIkGTsgbY)nrgY{~Q$=062?Vpe zH~15n@~wn3%mwY(^gflD!#==@=L(M4nIz&G}g~PPB6i&;zTaTbOR>78uu%+v|Le^+=~~3x0(j=yiX7IwwW# zNe_0huo0=TI=JvCLNn#8SOX>!eMVw^BOj=zJLLksaWJ6mX6u(8{%l*{47IYfgbB-0 zL64s~P65Zbp?e8`J*q!3+wJL5&^e9`ZY_een{qb`_kGvTs@u9^T`f0L zWpEC0A?ibBf~nU{ZLqBLq;<(%-D;Jci!I4NOK`WJ=~<0Xxy9vqI>bb!^5x%q9dpH% zyy*&d+F5ewdc^6Vu+qo|BH~7VKfL6KXAYRh=BZjbRYDXS%L0`ND#1oC z4KE6Z#wNK?v9!5XXdgZ}!eux1CpyZIXDC$S#9zgLlz3NvHzdGLv*(1so_%D)r}K@^ zgv#>IP`U6zIw6l67L5M@oqam*_6Fa<7jlVdHSeghCr8 zA58#GpgUVs1P8RMEWHz@t>DGjM^6t`*P2=mw4XQ{a!&^8Rr2DKO{<+gR7^mPmG83Z z5?lq{*jM|F48>@mMpQMxscnaT??giWdeH+|m_uW)yxixb2d}doJm(3cUkkTg3Ad{1 z`chqkLFvapB%r6TP$ij?2BldYL@gQwXTMWJH7CF;kFOn0cECHcMPnupo+T6Sr; zq~s1_u{!X98j_d>6>g7{QL%(J^&SS;S|1l2Zoa%zWrnq+$bHY@igmT)1{)6+1Qk z&GI!y(nXOSp4JiNfZPt;@DDS};C9b?q(Pu#qEmrX+LIOosPxxKUkUAIAUX6i7ENn} zED(7cc%daz1#ays+q`)AXlX>@!T31jVq#4Iz$tbAb;XMV;7eQK`DUF(+{2hc#W$oK-)|7P@2v?2=x$ZH;r&s~y447G^?mFyJ6Wagfg6B#Dp;df$P3zlFb(M5u#4Bu#NZ7U-rLL2h=~ zf^E>aK@p0qr2t#+ew`|4W#CH@x;C|wENEo`b_}1nS%+zHW|m&d?9NNSU#K@k(mLdLGt%D%q1Y z4zh?`Zs(^b(1g+Pr`BEBY>~eu=$A3dC762yzY8K5_TueRMBuR=^Qhh7ShTTWo2V(} z&6<~Q24OUhN^8bSreh0i!@pr}wy7F$EMxN)Wz6YkWq zBYIQHagL3}8}fV&_^*T)VRv6NSo5a$_z)6vTi*{?UfP3M$h zjKqZCdOu2<|1ESVd?5RG>ekRI_YnV3mk{kgtxy^NyF%5p#pFQtIaSS)DW-8xOaD!K zU4o&qpFoQ8i&aYF8i_FqViEc<-a4=B!gu$uewu{dx@AEkQ#lTiJLh(*=g`s0@o&Yt zb&&6~h8@aIQ@*RJtu9-iz4!N{{NQV*zG5%?3oI49Lqdew2sKIgZ7U}%I}$}B)9%ka z|IWdf;pYyV?N@jf>8Q^@+J8}3h12)AV=g}ux4m?)gFT-Rs@Z_>hQPjw0D)!A98VRtxgcW- zM78tQC3Au9toN51XRd-cK8YeGJ3@wo-jIU0A_lv}Qnk|dMY#i2_#)UIGXyq4Dg+9W zL@|&4G)eqNHUZF95;QM{ssK2Eg9%7C9*AhNL>6TK9g zY~};|t^4SML%)YLwlJwWM4r3WTWIzDzo9EdFPauil4+0b&;p<2dHN4@VeFHCpqq!M zvlTMe9aOg&I5S-OK3wfx=@`Jy zlZSLZA|h~VkTq%2r9pH&%Kwz8rjDjw$dkrc6C>~UGIP&@`8wkiEE0m)HkMmS5oROj zV#^tsdf*}@?HSt%TVbs-Zh@rY$0-IjQs&o-eEH%h2LntIwRiS{#e9-WzTG;k3A84I z+~*%+K1M`zz^w}P$PS#LtjTH!ApBVVL_~A_7wafMsFW%N7#5Cw~ zA06T$==l{jW+s#IZL1S`!)q+OiSqJAFt{daw7^jPH9~M0QlJ?Mp$yDH1N{98J})xi z4#to(!GUYwn_5^IHl|o7x`BE(MTnXP*D(B~Wi7fv_*WykL3%e$h?>R6&y%$pbp6o| ziXe4oZiH&4@#`b15ZY88)7sB|s*1{J-RokG(!>hNpsvHQZJJuao&JKuWi9G6I zhw@W>(z{P$45Or2j9h_=?O!S7k90YBL+{MR$`fDqpGnE9SV+f#)qRs-8~5B^D4ZF? ziAQ3LuxH8k%I42L>7O(?Z0d}a2Y4c@hf)m)jrq=CY}J__m#Eik`z~VM;Z`wvGHz8f zRTH&%!p+4!=^IOvmpw+P5M4uO=UA?T#hbj1=omw;-Cm9v3*9dA|D@)g-z$8+2^f-|Nzz+P`Pd z8mzN?R9>$KWSUPWtZlbGIttBWd;VCneq(g1ykq@4c?*7LFLV5C_ksIyr~k9w0Moz7 zt6}}m9PrEAGa}E6m%S#zB|GzY81zuP^HqPQ18TIn4cK>MN($CB`@#p%)`D?e!3> zMEVDge30UL>Y_A2(6Nti(Z9wB}M+D*)`EVX&u5M4D zupP5=J`@NG=8`O3kKCg8D~^3K>NTG&rYasC0#FF}cK;y)A*kqMMx#HeF^iKky;A2#`Jhe-D_@?2(O!KyVr|^^*|_F4 zf1m4pb9Yx-guwPeh$Y~Y%Wn}H4iQ$Xh>h?JbdbxQqL33sf|DqRHykC0~)f@y2p#KR;2-P#AO8tl>vt zmN}TtY}g)m`%7OM6FBn}*|Jcxa;#T0uq^~hmdhtL6^gc69{;cEqjCP(co|uw4)B8l z^g8%j4Mk8?%zl!?Rain%bAJr0nf~h$%u-(sObmpC>%vdFz;Ra;aE`x48GbJahFl>s zqxlwHrt#=vA4Ri;a10|A2F0t@CJsN&Aj7HAyD@24s}@zn8$n>O5f6Ay$<={Dd0nM| z{;9S*3zD~bFe;M1v(PW17HkozRfF-sxnJI=my_SEK)gGqA!)CnMq0d&G(izNjsld? zLo6#b;jrhB50ltQc#t0Qe2R87+GC+4tA2{j-C_Y$os`=G2lY|!7t#etMYwNREL2f^ zCbRjpP#52T4q~tNf>`HKo|52fr%0m`k#*LfLUgLeVVRPls%00|vRg4rj3kSWWptuK zG+*;bs6zBKOtphXO+tZ(N=?GR2BoUF{dH1hQQu$1ngl{motgvzPx+bz1IyIPqCFpK zRq^>RCEiU1Cq7dpr9mgI?E>ND%?z4P1Da$%cg+F>;={WU9mGVt7MZ;bY}lwz&NY36 z*evomAhpXYw(lK2-irutS`V!@g=`8f);v%&Y&2`Ik)S^YmOwqQzyqc3&T;DpwbXM; zXJH`X6y~sTo)l~#gAUu7M$1AI7yFRA&|du6jIC*md=B2F7;=a)2NxalW8n?zIr2Bc zH_qcILzI#Xl^N2^gL)AWSSVWZck}rDG_Q8-klxL;AfrV3VV}@jMA+PnCGaBH9!Vke z5_W~_vFPyFgK;YO;pK;~-bn?tujK4elx*_4+L!$LR_T3Ks+@tJD#z5+OI;&~hW-Et zTT(QtTVLUtm42q4Z67Lw4(52O0Y9Svm;B^aZcTo)!tj5D_QuE%qP#~$iOc%p*mIa` z6&&??{AQ5JM8Jui_^qB-q3AoMAJ3^6m`-;a6%kjNpI9B(Rn(OJ@H=la0J0M(Zrc#W z#MWQ$s9xMvtevGDV5}$s&U;3_Hb1dmx1u*ui&r;JWUaey1ACGcA%z{~4FMxGn&f%? zhNIYo0}HR>E%X`^aQ1y6_i=FB{8I<52ipxbL;nIrKB8DzD!~V|v^CVOE_w*i3`jmd z;ls6<4JW#bjafa58m1+wKGIU zYD7~-8}@5A!^J_Z+E^&TuOXmPJ#1!yT)!YWvQa6t({Z7>ot90N)|KY3H@sRA+NR;W zkk!3}p8GGy=-prW4&1n`u)^{~E5|WO>Gt>VW{$W@SI>2Xj9ptN6WcU5&srX6J5zxz zDM+QC?ZjOlU25G1Aj9psyy_I+h%(cZ+gP={DoQm6>x5-Dsg`(n7OI9%>~$0oD+8i4 zVcI+K5vkpuPOKxQ{CI zT{R!*qZDc>kzTRqCGF8w5{ARVqw$Vk!ri!0_W3_1@->*s(P4gs(8GV+eq#QALg=6N z(`>DtvYxY_coYuVwZf#Jmw3iNeWIFq9a&(@SsDzD`Ty-cDLg1!(_fJYLldWr-EO(h z?Au>USFS?;?JzHt?RN&--eTia{m1|0!)9Z!0ikJ>e~y_ThbK;us%Iy}K6F{-pAjlQ zf<7EJ*r$oS= z_8Ps3_5h1$0T(QzPB z%U@nKwVN!LTbg3yztu5N-n$7kUGmUO8g69lv^~X$qm7mM#l1;PoplNxeT>oI75FMB ze2IqvOmg2HSXJOvDXfsqj3U2%0UR#ruZ4w{lF#4?vex>7F29TDlgf_~V>pB%2SAjP zjLKC{aaFotujITXMwmVO#6wPHF(iIb&Q&c~Qnd_~o(}$Y6WhQr(&H{KQ36)h=9l|i zxtun;JL*H(yN(gW*|xv7Gm{!~$5v^eKmKQ{QBn9_N-Pr^B&R4&1wIjER0}X!N*>gD zqof}4N4Xt6y>aTp`=T8#6FGp2*fKbOM3^XK@o?4#)UQ#gziQoIN!#ex@Mz@`YG+(Q zk51_m{U$qB>h%Nd^4u$gRh}~(qSB3xM2kV!>Lk!_nzT6DvffFngmpF3QeX%yvW-&cQy(E03gMUt>Ks{+u~r97kg?SzlEB@Y zGQ=q6M*$h-bW?>G)m;C)?}84;O|`%Ne01tY$ZDQ!hr)!3r}ba!=_+O`J&e?PnqNeV z3zG?G*ei9DA|W1o19~_B_>d?i20z=j^A|PWwQAd#Cf6C>8+7ULK{H1W z>nPd^w(D(JrF#23RKU*j35)${40}{^M5!m3N#pZY`g+0l;F=Y=F4^ho_#d>)ELDi7 zfDqHlr^t+4D5~Wg1kTL7S7lL6-1@I-mM`_6*(|z$S3IMQuDnSe@VOp-Q*pW8s3?f4hz9fwr|>a1&a&nF;Ek6~&jsLT>{qy={M!ts>Va zPECkG!k~}dx&iA@qEJ%HphKIw^Drg(1`Ni-oy_DCqn-eM5zOdxQM@5r6S?u;JPLn)h-{r68iBH-;gw?=@zzh4xk`HAe(;NL*ss-o0T)T3wCN`E1h5 zjN$J^V%3}(db#H>VTM`;`V%%m6*IM6SWEk9^N$L~_pI|P!5mb;+nc~g3j|F!nGg(2 zZF^M0>nGlJz~3+6@1PRuU<^qU9H0fd>4ueMVUAfd3^jPDLf1F7L=vQ|+As_uerPfb zF>X@*-*%JSPrJ#7JVedqHu7hH_|*`$0Lf$y3tK~}boqIpc4c8|*<6`KK*ipun^Xa@ zb`?S`^Dw-z7q%z3*-~63j*0>2TpKCs2qSblRRa5~dQsuLk_c@Z3AY>uCL+qCh}m7V zLZ|ji1KEk3eri{{sT^cIhq9??kN%Tt$Jw1QQGTFFv3te2a@B%O1E@I`Y{cQ#o$Leg z=UP`2`@(8gJ|P;-83Q+y(=$crc2D#=wYpq$QMn>N2bLMENvDjGmgCxB9XFAyl1$>= zSe|Y4CDyOivC+Jvudi)yq0~*Ir$V=ZvuSGSHtNdd6-6+LS~?D=D)X^?somGN9v(l! zT9}jE2Q(M{-6xm8KeBHfmF=R(S)M*5-$a4RAgzRZsDz(96kq9yW>j zu9DA)9YUL|^9hMc_<+fQ(-c{u%Bj4$NX(OE+gTqyRL8Fp&)l??dlj|4@%7&(>AXr) zxTBS#Rf6oAYPqZlvRLDIRz&T-IU8G4+Rg!aj;VYK*V(22#9prUjuUP;Uwp>%dJiMS zC3&|QdA1q9W^K}0A)bJ=nbY~dugJ1I&Sn3d^sU9L z=zTwv7Yls_46#Ws~-VWkuBe_VCP_xDTKTCANWjjOlNQg1Pq}d7e1XNA$dq z1|z`tL>TVR*@t))%1eZatUY998Xy4&{0uTTmbl``qkem1j6Gx!`)Uc`&n0I>xQAiE zWsIa+e&$WL-$%FGINdwKb_+OBa~Kfw{a8(}kDG<7ws5<7CZ&1meb07ugr? zi1D(c{v6vJxQOBM0ii`F!g>>e1d_z!M$?J~%U^*P{a7qjX{2yriESqk9MR$E@Ps_X zr4-;258|Nt@VJB^;BEK0RCeQW%8 zf*)mwjqp@w-o}%q6PcAh$D))K_#*59QQ=a`ET=f{Joymsq=^FF1S$L)RFEPJ=|2_xEJ@i5Aiu6UBIBNdxcywa-iA zmle?xZa(pmEA@2P7FNBLd##ApAZslpTfM0W#~}Z$9~BHBr?xR**f$_d)qjY4;`-aw zJvd6v4<#{ti0c0LqD6=^DtKOC0Ymqy9wCa(uZ`%tg5XZxV}StO$r6_s(RwT*a4k6k zQxOfOFVeu1!vJXUQc=7>7Nn#WRv^DB2`kPgufj?Czg3n4=SoAj@-8g&VsX&Q!R(PUZ zs*!zz&s?+=aDt@+jE!MeLywY%-JwD{@mQw8xAy-8=^;Va7wh*UF(qo9MYCDx%&1?m zPgIRT9dH{oxHPxJSDysA23Yp;%NPVaq$CUv$*jGL&m*7KK;wX?>^H{k&pNU|amHxJ zQXJSftM{3|yRvDs&T6Lri zeSg0OJYBdjMg}%3P43lRqBU^X+eK*Lw6_ec9nF67ju=86RXeplhp1$&tZXW*z6A(`8I#zFE*K%Y2qZ5c`HN|J=GjDWk=2Ns0 zmQEn#6BS@zYRZZN#{j~Q_9Y@zuYAJZNx7RO$^zm5F-;*_$-mDW`y86aZ@&B^Q%s%U zuqmhrUa{KlbLi%2_cMbu;o4Q}-e!NhnM|=6(T|~4_`cgTCDh%wX5FZDXd#=pxz@jHBNAt!A-7d z+K|=*>}E817^w33=$Js0s^E&yhy{P52!`&0h?geR{wgnWDf=FQ%|A#FLf!CVEISpV z-NtBsE|_1M*zHXS(LXsHk!jzX2m=#C>=XvbMLmE!E|^_qEAh%>zuE`jl=*{SlfzBB z!d)TSQ$3?~y(QiHIu?0rq0r(X=T3Zj+*C7YNN1&^)ZEt>I!f85l_dRqB)j39J zjN@$J`naIWRHXj59dD!^R(twXLfq z6JPAfdAu~d!~|G1TL_laa&<;UNBeswry9vo!9>WI|+Mb!&bN$yXM)v<{G5)2x{MA~tGi%-H zQzIm}Dox4(k;sOA!_IPjq9m^U#@_#=4m5u8&~qJ6>Hz7^-oX7HC&vT4dgn&>HZ6+u z$PQ4ry%g?*$oJWRzdfgBXOeNy5$efj1T_Yv_$u7TP$L+#pv6Zeq&W($0|ahmNvE8I z6NmF!Iqs5dgnhW_STYF7SG$a!j}&aH-u@^`9oqqcmwz0kDC_m6PeB7flBez z3s8DHM^A!Y(uQgnlEhXaWzED&WG8v{Yr^MqPY4YwA8ql#CT`J8>K9j&ueq=5aE2NF8I5i^Nyb?cxxi^yS~(Pmy9eKGQm$ zQ2pz$0RKc7leazy{;fJq6O5@P5Gq+279ff34=)j__`dj*j|1Et-7&m^q{#$BM zlvO<&$2>Uvn)cj^|6-tBYX&teIl1iU)ip;_b%`neKILo01k{oJSK)RQ2+ozSk(coz zOsq|i-Sw-EQR&GyJe2731w?`MZzC~WkOdl%b%PiRu>@?6PGr@xi%KNz!%iEW$ol&Z z3Xv~s*i3NPc%8p<1Yye3hGAGylNL-}=M$j(UEKlP);a*MTme<@CfE>y^py{wK7KFa z*r4gqb0cWi#|sD*jy!8@*L_*i%;4tyYqjA1t8kOrf8_GFa^i7#TIsXW?nrTF)yKTT za-!cJ_dqjZKPO3)M3MJ&cpbI4h&@$QVf(9X+oR$G+N73CL?ALkPlCM0WLv=oq7Au& z3X|-LW|<<~F%IrPtynA@ z-+qZGek2v)dfjk96f9yX*a)qL~mII3`cp>pF;>voF9Pv+lTyKe;{ztD6 zIMd0v7!WI4kpJ5XlH=d8a_4Wa(D%4zwH?qa{2MEu6ene0;@NP4v>7fpk^eVVZa(<+ zB$Hx8ykxYsYzC!+X#MHlTB?3Zl=+nH*tz?I<@~$BZ6~;|bgj$E-vs&YvOrW~!|EKI z^|%vAS4bFflmLp7{MSN5ucid19&-Jfb=FicGO5!ZZ(OkU9oDV_N!q^jj%CuBoi(ka{BaN2=k;)wD>w2 z4ADYkLZ_M$mTKj$k5a)yfj5Xq!#dKVv}T5VX^jH5MkG>C!U<&vXf#M{PP8M0+;NF~ z5?gJ{KVYey24$R(UbQ}u7D&PlyE$2PcmdTQlhn^#eJ~R|*)plFLT^t03|T)_Uvvm; z3!1)!Rwj))H?!aE`fAg^uanC87-j8E;Y=Nxl+np35|Pi1pguH+NMteQu5PKK?xgwE z*}L`4g>mf(df`E`?f+^bu&a>LNan<92g{H_#kjqkRaDsDduKa5Zu0z~b^X3|W!6k( zhF+9aV`BZJECEP?3tGij(jqH033cSnnTBtzb^OT?%LUqz?1qdGHoRs#5jbK(miu^Q z$x82_8mq_Td85n(iAbi;S5?Zo_cWp;PQZkY&snyyKT7sRbWwJKAj zGpuo!U{#Is#254-<`r$FUstzcKsL5e){`NPhD$?JL(!f!_#%&T@^E~MjbiXbddIsZ9RErncSVYB_XV(R)P9>SeoIL4kcx)r$#rQi- zFr=1ns^rDrSlNFaGCOd^Np{lfdx}GzmLYBPnNK?dgz|0wAkY;kfy^mn9-vCI37C-i z@NzFn?}CUK70nm=-GO$(Pb1$XF;dP}?!^&JDXb&>*)8FUV2#!bK=eo}%GuAfrr=DDV63 z;qM3VCSr&YPI!o~o1@tjW6D1ssb|3K{($7}da}Uf$qzCOwA@99wGV8JL6cQwKD=(R zC)$gZM^FhgMUTCOal1OC$Ot9dmB0e3PX&B!4j48_Hc_#wvFGr+(B5#inia19l>00Q?5mClZ!3>~0PP<-V$#S5M#yqDjw-4^l$&5~@W-P!`P#J0=I_{n`N_o_6yoOZI1>fjYl~ z4h8mWpFA$rD~8IZ4bbTJrDEKYq;8xkw|XvBQ~+iqoe7LwhHGJ^i{Rs`ai!+%`-N`d zH-b@)6p|FLS>4=X0KLl{<68rQN==r17|lcADW;1QW)rv1M586MQd+N za%hkAzcux^vmj8&0an~ai1OO;zgFC4lq7I~6*tdJT3dy+k$y3j?wV+^OH839lq(fp zE!FJ_5dA0wh{b|_{xbFmr6E9X)};J9BH~%(36k+Lzs9Kr zI?#MA4H)EF;}BN5%M@*}Azot(!}&w5SVN3-M>Hxq^Q+xw>2Nn-R|uH((}} z-s1Ue#Z7Rhxr!<8wYO1wGxi-tNHghgi|-~CCx(195xx|FOc5pecPjrX0BR#(LzPdA zBWUG69*%e-@O%MnKJsUo14O-)+WE7XH5QrR+x}9`Iz&Q{JjIX5?WPr+=Jb~`FSs$e zkf$J>>H-ot-hTBunIZiayBv|azE=x2RWcb*I!s~MPvaSY6{3Lx<^dqe9V#KzbV3wa zF;v#kt+L%C4jmbZC``DWLz+*d9Y9$AYlVP=0)qRQ<@k3G{GE}2fc>e8FfALZxUdIWPg z6v@I883-@cjIk%0hA9Ai#2#!yzTGC^<3w!Ux66W95PqbW6IrVl*nq9m=yDOZ*5tMg zo3fgLtqcF$f~`yMasoIZ-T|*{4nkHBwmS&fI^eIiOdH-^zzeWVZx}dOluMEyf2%b# z{@|}F6{ElqU$!D5KlPOBwU#5_{)2OID`Q$1g$mc{bF$4Mi0pkPr%ZJ=D((X*>=}d> zM1EERBNb)%%Nc}UtsU=46G#MA43+?viC6E#HNxhGBj!s7B~a}7>8akRpw8_pywqMH zUro%@n-xooede~dP;>O6U*STv@Yrn5xB8D2an*Z_j-Cg|1;}@eZ5{I1N-*v3FE&c5d7PW zpmA=8C64elt;g6_3MZjOk*ZeI`5Ir}fW4=ME$vT>A}|yBBPr+vRx$97pxFAIIs-9}jz5GE3bypzq{m zVJ4G`2`|1a$3K~q9skTft5qboaNT0`JO*O>QJG0j$zvcdMj9WmKtGaKL-V#{#c4rR zM*^}wNJ%?vI9XIfDM@4hRF)#Zc(~6UHg_~ZgeGV&!O_1Ue~Y#pz3TD<`H)$ieSDOoc0$Mj z^jjrJG+1jg(nq!Al9?Qb=21R6I`dMkltwN`4a<#m-*eM|H!+?T6uUQrubCP)n?Q3ZJ6W)abr6;>nUG24X0B_NN#1I#ncvoaYT6%yw zvJXC!Mv*b1BVBT#sFczt$cU4)#sqIdoZ7r(Lad(!BQoi&@Vz=AZ5vRv8ZrXn-jgyL z$CN+j()f!+u?4r(aPzy|tPOc;%|Vi8@prao&MQSx=V^;l$0^xa5NW4w%&o%Bx)xdY zP=FU9WN&?Sk!#pbDzPon-wd;7_zF(iFTT_cnVvh64$Y|RWF*U5K12?rCBY+ZfmB;5 z95UXhp)Xh68{X*{!EW(QY0f<7sMzDv5JTQj#&4k#3zGn{r$}8*W>nWcit$$EFb4Sv z184S#63SuGULxVdfJ*G0(`hDzc3O5#B)oRof)vtua?)R>!5E%OZ*<48O|_}m~Ew>Z+Sh8eZ)$p6Y2wkQ#*qW>KQ|aw{$LFXG68 z_!4RLNwz0&X9cjUmAacXtf3}jNEd2-w$~r-usn*Omq&|z_(=OlCkLbg(qr$8xRI|F z`wC0e$3*qNP7a7HHGD=zvouB6(H%??(c8j~(Hznv-3<`c+QM>a_hjh`6r1DDy(Ny8 zT3+KOtGWPBD}sC^P8T(Rj}7dA8{j`i;%I{aKY{aEhmYj`>ixrfqVwGV-Vn)j5f69c zN6FGuam|wgZuvsF6o`()!LI8HSwPNGy`u%2-W1Rt3JTD@$fRWZnwmYmyj^@<@j%ot zXV-I}6;*bD*KgCRVr$29V8zDVy5a{Gt)C;ytTJJ70`m~2&cn6x+xdqS$D{Vhv2(l-GuQN<<@FA8TLZ?AwE;Z0SfUEY#T97 z4FoP*W}Wg+$aJ8mIa&q-Z?`^}$OT$x)|xH!l9pjbYHu$9Qj9nj{A7jbv&;frS_6l= zK?Uc`Ts1!GZBK4WAcn#JibJlMzOggFwRin_C5_U>QY>ip$`}ov} zBu~Wk)6*rE6pvD9EeVe*JABdZiVQa$?aVmv1HZw|Pv$Y9lvS7W=bDbHZ3&&0xhn3i z9+TsoIG%f^*&h9+kv#56>-EMjWVBb$eRm!TUc994hQRqh-%GifZE zYq`FB(?g!>$N&G6?r$7A!|i~iTlK$9x|#k5)$l(JhaX$%)c+*iUX({<9}*dG-{~9@ z+Q`N9zuN{~#)BlC{=4B&I<=)`x|cp!|8ZIEs`g$b`l-{g1N8^fc{cTTaU|ID)RhY~ zC*5!?Ic?O2pjJ<*#|SElJyN8En4}EGaCS~0z7%DTfw|p}XPcmVSP$so0ElAvPNbnh+usy?SFjECGvgu=(mxdq*G=|El z0SdVY0}NVhJp&6j&%Ss81Cr4&!59MQbK}0bFmh_aHnkBbn7{2=)z1$hx4t2ppKqxY zC-6V6_wqd)GJBRdfkDkD!w?XXNG{_^2Z?4>Wm1UyLm6_fsec6n065tI01ljp0gn0%ia@hmaY;|z17KR)V9mzk^B)wYiXoA`3AeyKLs83IDn`KaL5}lRq-K418 zJQVG^v>8Y}uX3!cVSyRb$UcIUv1UNly${H`!7KjBx`8qN>Cak>pQTCHMNBr6-%tz6bED?nxZzZ19gai^_kpRA@nXcoiR6o+Q zp41akBe}H}RfNzEVU)>&GK`zmemXjK7B)GuN&nzQ0OR8P-`?zEXIQa7&)mF9)@fmY zU2g|mq2;Q!UZ}}LL@WxtXyP4Ie`2LK{*fcnhRUT0$@%y1MNyS~am+xD-sA2U6{n#ECvX zytObz79uzp!%|QcCOmro;%{P50UD)L)j@zOrJjfn6M$@pU-?v{E-)OgoNE%kE8Sit z^icP5mB3r^$4vl#vg0O+y{N)X2z{D{8#BTNd})7u8OPHO|Fo4N{_Fr=0B!oh!L!I= zwp7qo;hR1-U0#mBGz{oANJwfw0G;&QF0Ng>&{A9>K5$c{q{TdzAkrLuMnIS+7yx>* zD8f3~QZ~SCCn4h+NCl&-0#ez}il3sXEA@FI`S}@7zeAAfXG$1iVc`b8_6|^4c2PC1 zUq^|{Rt0z5y1WQ@6!lDd?yqYtGrhb{^xQ)mM7j}IupPRX_SUa=G8?1-{+G|pEwt{H z9&vz%=i|4(XpiS<`cU52NS&y3ceeJn99_-LK|N)HnzK7&q_4^`IyE*fNB;qNnlOwx z1w!UdLw)WatRlbvSU_Js!<_U2NX3`P{>xPSf8A;F8vu72U|F3@kyTu+?9X{TQcl5y zW@QA&!5>@Xjdkqdkw#h_nZM@BzCKO6tVZMvM-?=ao8!NkeNIkJGt+lGIX+an%sKx# zZQkv2@&((fNNiXj!zi}#+q7DDov@G5&Qqws>1moVo8Lf(smLIU6NdpI!BL6Lk`#tWngiBdNN6JR z1OCnFMWlEzQoB@+a^D=wa+%ud*!1fV?Kip^i$9u+!Sgp)UHq#ao=xtbt~35y+3Pt> zB;JU#VhcR5yl=0gs(RGoMpO_%#xmz2p~W!vg+FG0EW%P+F1t=`0drRM!rS@J<=NUZ z!c*WYP{APFV}OEw(paZD0gFY!orRdJLB}O;N=zSjBu zMPtEAiheNE|)PX*7r8UvMup=htrZGHB z3Zao3dE}E9aWZa`^wOhl$U!tDdvrn-Ud-Q@M8k75(^xbowg8v;^@nkTZG?))l zHpEy%it%zfU@m#S`R6!!pY{%E+0(Vpa(44^cFSCsr{x3L1NU<#APwO%iPFuR2OF=G zUx1A)bOMnYBSR|2yHDGFu1*Rz2>j-t%6JS)9t+cN0-L>a@ zTclj0pJZ7Cr;EmVRK$bDd^WDh4D2j8(O5qhU_8Bp+@HY$VVR0zR48qg{Ra`&5E)sX z+Ba6uiRx04z^}r43^ZH*O>%Z6zSEeP62yt!ByNr}OTiyhVXvY8+R7W;$_%xp*rmq8 znSL(izfhfRli;}SJU=>PR_;6jTC!iJ^(UuVPH9S6qv#|Q%7na1#;_k`V_Az zORagJA-dJ8#^vW;R-GU8a-Oa^eQ+h&woDdO#9B3-LN3(7tD=$9W6rI+TT8k{61kDWLP^8hl#uTIwqLdrV7 zRAEI^dcadtm!rgZl@grfSQl03uM6E&Gp+Hf5kTKyQuf|-D}ZZ#3eBf2hAf<8Jvlhe z8K1SY3P*UZd{dfnd>^7^xivRHa8O@Hi0NT~&{k?>U-G0P8^4tp zDP(mBuAu_z_cENeSmGAbE`s9Ou1d0|uoZTTDT?9v!S-@CLmMQBe>Z|1w8RPJ+BdgT zcg|i?geHIKbZf>@qH5qZRXGph<%U*C^&Iq(a$kz(lvjB$)L=;hV}v-=&82oA$@sz2 zJSpPk6_NqR&3lD~wl`ToNX;>=M977cyIrX}HaK|IBx)v%eF4>23;C9O=7qcqzDyKJ z3*K-~^{{gM!@pfwmh5(4LBo=~AIoitc_)xtt9bw2xdjb3PV%R$4NgUN_L=tFH*9#( zW~a-_GEWRSAg!67tte#LPdp`SRo}#Yly7RT1Vo{7YhvtFQ=|c;m8q${PP$)udIdUY z;iJ(&Q!{H0HEzT^;Q3t_?I(uv!;8U`2HoD@3CP`3Q$c}Ex{LBgZcYXsUz_C3#tGT; zg0f@w!yt8s!q9rqak)zjQ9Pbs6V>#NVupD_*stwPa^pSDVCk0IjAd{%u{^`%J%~;C zEGEfh1x%@~C{2m6#&Jd2INO#K-lId8n{oynH70D3Y|T05^J^k#wC&O=&X;xuM)#Ks zZSyqUV(TeQ?$UL;kz})FFPktU)mcwg^;o=W=ZoaBL^B zgMKE@Uxwyexu`zVghRdJo?l}l-4CAG>41^eggSZ+^2oQDmpQUv+N>_1G%EOPZxQK0G*?e9rkNM@WQhoi$VJg$|{F0o?6?M@8sPsg?k% zgU}73Zaj^-pV28EU=cEql**bOyBAibBK&+o_(YaN&BZLvOaHE#B|j91B()~ug$&el zAj8a(M!7bSh+m11>!#5CZ4OLPKb3IA#QmTTtB~FENcywF$*% zwzvF&c9Blz6PIOd_&tHz zp|^S~J?CdNI?$W{Y=0M2bok&MQyv|;ofAfk&N#o5+Jv^k7*3mA1V>#g0*$W&;Ds+e`pW5fr@s!)hJ`cIHMGcJxiqA&Oqh@{h1_}9E1 zQsneeq5a%KROb*e49F)-Jz$H`A`uP{k+4~~U;YhlShh|l%DLEbo~{J#ug4A&9#_JT zLnFaE0h|SDAP>>@3r6C6$FSGYM~syF~73`aZ6{dgXv_iZ+`B zw*CVv0mBRrchD$++hXdRp~uIce~-~|BHZ_P;$vraXDJTbIQmALDLZW$WTr;#Tv^F+ zZxT%@TM3r80Xmzw&4yV_AuIBS@2$A?lPR=A0m71u2#QFPj0oX4e0mr~EmnFMb1=t@ zNT+gGOL~|Z)ZC=VM==6=de}ZOsD+6zQAKkDVY~+STAYjHc!)^ICowjXMsqG~xD5Ab zh)8o&Jz=~9J>VBEJ7!*5##`{%2$&25ZwxlV*;$qBK6SAnwrre;{s^Wxx^(rJPb*#^ z7ZZPyn>)at%zJ^kZ-l0=MdbWj9$ivv%w1DAo`K(x_DF5t@Rz%S=p|qNV!G1Q& zcA9)~)s%eVwci-53w!|yEm^eUSAPowHy8PhG6RI4@D4@0dy*i$|5J`J>CZGTwmtQF zf)D)WPlv(J60pgrIr|3(z+A8BgVD6I&%qRg{43))!YH}iYnnQh)=Ir>Y} za_K_)$DFQmSBzu9=WS|r+qsj_$SN7+spr|PL0Wy71U>SjU0{~e4`qUkP z+c_y4?JLaHlPdp6b0)(l4sC}DmBb+$B zVJy+tTqkTmKy+JcJf}`5mIMO!k}{lxED3x<|C>2I{^zSbOQCk2#V9v@cB1a?2S)8~ zbn%Z8zAW9|v5J-8`xP_v*%MED&jkL35Mv5rP1{v92Z0_GovlgK5oM&Xv9y%RZ?YP{ z8vxMi6dF>?WBJPk=tEVMVYglj<@z9P5#=a~WVF#C5Kz)jA{{*lXt69fsg(Uz>bOK0 zk|QZqH5Y1^YRHC}0MI`WxuM8y)pEfVURCu(#NaQG;PxntxfEx(CKHezhVS6 z+<7@yRp7#bg5(o?LL5Vw8T1w7yk$*+9?H$B=r1;qRhu>?`R1LgaN#x5^;tFLYrOub za-LVr-hx;x!fe|Q&|jvUBSVSxsR z%qR$J7f_Sct_^gT;Arq*tE=o`!~(@5B51WRVX9b$^rt6i)ki-nlLiW32lC#I0oAG@ z7anrAy)v8nVe^%2mV*LKd=({j#ZgsWg;?=FO*>xi&Baza)LF=q?r~P`euO9)pS2Fc zHZ;5B8?5L8@6f%?w8ixjKas+4fKrIUH*|_M`HDqx$Fr+awmt1BQF2H8E+v`-@j_^v zpoF=#7zvq(*fhF_;f3sInohUlQD*@|w}5u;u|t&yg7NclzpeRu!VTe2dqL_2=?E*? zm5XhUeqvvYbxI)u7-^)4!n~}g9kW@pq5AGc8f+4zupDVbUa`2V^%ahnZz)f*|?>V0*}6!`ouc6TNuc1(dY>z$AJvV$b z7h}|g4N#+!VFPN^$&g>2fTI;qqZ$KhRKWR~v~J5ra=1JZ^6OyRR9MlK-?*kZX)6{R zA)BE~%x))yi%ZA$%@SW0ZSF*wR7lamxQ&90ME44@m3~Cn$_?Jyi7b7lV9@`VM*UkpBa~E+Q%~O^p@4K=X@6<9WS|VM z#I$msX~Mas%Jiwz)6ZMRQAE%~0)**0rrd|#^= zo>f{kKIf$QX~zRGqvzgdOjv@>o9o85`X^NIp76{R6mqb$8wD>Q!4_mr{x_y3@8 z4SAWHU;qw|Jb)MdzpBXoZBWzzz_u~2JM8KX5&D!GeX5tsmjPVpN&pvn*#ElF75}=>$)#w~hMxbK z_0VDdWgZ3|Fo6n5;*$UrR(X&mMmx7WonRhoLS)l!V3wM71H+RBNOZf>QSM3OvaaLW zyyq-Q%u*11a~R1QOhn_4ISBw4Iso?ZL5D;a_b(UvoE!K*E_4BYCdEfdNbrBbKET8% z_mPi{z&8%?LnedT(q3_vh;>h|% z{GCa*Bi<=o*z+m`dU+T!#NE=FV8mW<%sUinM&#tNSR|xj(Xfj7hHi4_9iV^WPsIg zopMN|`bAkntYSq6j0_`d#o8g-W^m29D?6@!7$RgXV{r(8eWbLla9zX*Y9^Ck(H2}7* zG^MiY!DA_nk0OC^3Y%vfN`nxQq*u6G;g0_dZsJvKYxY-P2~?sf47lyqekLStNUwlgBqTg%v=KAhc7PtU#5 zt1W28oOw8MQR!%Q2G}J(xEXQFWZR}Z`X|bjJkGwIHQ|e~s6d0u5yQ9Xt_)Q+niHVw zC~-;(pa;0FAbkbzeX^%LpMGO!;!U1o66 zx?{a_Vl=ShuCqvF`_^&T(}Mly0^Hl zK?Scp-w$oCr)<+C(q)`#S|8?>?Al=7?6jRN##cJ30T;USN)ugsL0A+z9h@xOiyq^1{ zgOxrz^3KwR2 zA10%(FI^_GB@1uPHF9gcXSy1~e0NW@vGYF{aj%r&*x`IDmS#cIn{R*SX~IGTfP1Xs zvrrA9;|K@f4!wlG1R2QTEN150xd-4{7M=ywerp%wesp`h%=~1}gpg|=K*&bGzU&s= z@ax?1yZ61OwRCAWck?oncOBHZiXV_~QraQphokXmcVHmGA;yoDDBKfJLp4^)Mb2fX zDzLOMYiWL0g_&^rybeRGQGk4zB<_nt&=vG%3yH}3_??0ag34n2Yo~M(0J12+Xjpbr~p|4HnJ{mg$(Uz>jfikcrcyV|yJyXDjo?8*sb5YA{yHN9U3&c_; zhEbmXO+(USS0DzwfnYMrivK+0&LU)c=KLBJTDj%7W0ge3l^dRUp)YGT~Zlc)8+(!@f1(TW$sXXvCFzJ zvhaMrkXnbz1q_y8V?gkmQ2stN&vy-Tn^*VwfeWyc_3F4vV}m{t)?2EVs~9WDWpzY> z&=9gK0wgk|GuS*A4bF`L*&UPb;M%X%Y;Z#Iv4~VzZQo;VUq`%e~AUwFQ6c4kupwJTJsEdGge-^aqpv3#NDkY%w5&UvA}B+Y*yxDb`1 zM{+EV9YwUXYXZSc&tvZIBZNsK{lbs<0`exGp$~PjTKPtANo#p7zh=J!`kWeiZHymg z#72Oi_G`@7Sl_6xzYkTdGmk=Th2_TqS2Wa9XP8VtX5)-8!{nFevr@Kcn7ol&fHvQS2P11L%p zl#u@n(2ku<8 zy~$Jbx{SlY#^>l&o^Mp}>GD~y zf`g^J`@(5mQJA;_z1u=T8}}ZmI1uz=-9JKV79~}JZrjtcSLh4BTMrXePz8YCh0BM< zyv+)lJB|Oq_bq}Gd4J@lq+OO*Y#)!n1KLT!u|mERf_L6}(EcF__kcFekR%I%+pmvW zB;v4W)@#I8m1qe}Q^vGgeY%?$D2;LpP3vrQuB#=2Sw__IE;(NtmDA z5jslXhvJDFjdPS=b+2t~Y;4Xi!Uo}nLi($@x$Z2r?C!uNMW~mbJGhCMj}lgS5DKc~ zT8;Of6D{)p*30^Ow28MAPB_%+1fS zt=j}yX-JyVxx5>COG&kNO=z;GY*F9iXI{~K)Vy&(lRBj|XlFo98ZiL0+^aPLSv#SGJM@Fnhz>hh{wsdZMY4Imiu->3X}EXZnCzOG|^qU zAe2N?VQAjb*q|JP6bDY*oF4_?$vnlG`ViP@)3%qyvN$PQ&;H71P==)uYz1!~Szm06 zcmgpYk^X7uo+cCo# zzV*Dph%xm14bGe?1S>bCw{|1S(Nf6{E<`a_pWkd8Dw9g3(d&e9sSdK#RFg^tG;VKD z?hk@+dqVN@FipGvUY;Ao>1pp7E)f&fJ@@tLVG(PfN`s`T!Wwh3^0N5+sgj3OiaL-V zrQAAv^cAI|yTQ2Cbd_kKjN%p0ZP~q=YBH0G^Wm^E43jKPIu)$Mz|RE*GI_IbQsE5f zS@JU4~wrrYgi{|5WS4(0o`Ac6v&^O=?76}og))+3fW5K8K6 zWWXd~@Bpd)Cf!qH8BFtbi%9BAQV7yn?_`&Dn4)9;xtm72mumJ1>bWJ6{QgyN1ZL13z2Arml0tYX1U2ZM zk5WG?r;ZZcH5Hrj=q2ZWAzAYs;9Li|3VFTEk>(^mk%7*tYmL!W1I4EEaKyZ~S+2@IeDGla{28^|0m!SiPILdxloQ8l zt1JSs0To{KpIh^8y~$LLsMOEIg~SpZk;;|o@xay$YpQn0QI~6=h zTbFH=8m2bmC$=CZr-Gdp6=`kIfpz{LXYUlGOW18|mTj-HZQHhO+qP}nw#`+xt5(^z zUDfsPv%5RazUdPkkr$cqU4NP5ony@L?DHlWZX@kjBu0rd{Jpl8woMWoQu^x=HV{*E z;<_luV{A_1bSc7~BtRuV{^xJ^mn3!ccREdry5xP`Xynw+$3H*Z$uB76zu@32<}FWQ^`h|xK%_m zK0gHZ>b?k~RA@NIk0(fk3S^)nZe3TD>V;N08Qz3qp!glSeToET$X>0HbMWTT2es82 z-HsXEwktr4*kO*w00(ZBei~WYQ=#qXSJ+VgH^rIARv?-25p)?dql;r4`&V+@;Bj=K zQK_{Yu(YD2eq;we^+BkCWC_&jE#tM(l(eHvX&a|{uoWAbscH%m&nZ75l@1vlDN!you?yvouO|3q!WG7C_&(8pA&1Yzc z$PYL?fp`|Z-k|u(jQ)>%oL~suj@}@8w_GG2C;`|`Wj!H!(TYHL!qXc96cJHWOq!?V z>{DQ&FsvW&|2-ZH&!DN!|Lb7qv;6;*@c+GztM$L&JpXA&&6o0*n5eN>U8i+)P(qKm zTCqdDEGcS);{KEV)7c&~)NDQj4I8Y>i`nv1XUz(23NA1L zX0QYqW|)i}Dv|1dG(`Apsg*+38<%OkG6EdvjfY7LM#7Gx%#$cN6ur+07AE(xaW+@G_eFQVp;`{lg{HE^-=?U=dm>+Jws zaruOJ%|W|jhBrmXrwdgRVGvOY+=NQ>L3!MBvkLr*a?i%9zP6hiym1Q><{p7)VwJck z55d=XggpWVJRvcG-thX_jZZM7bJh_rlu$R&G~I;yOa&bjop$$gH`T_4d=LzNt6 zwKh){i4n9w_#V*>iIvgBcm7yZc?+2f&kytopHoqc_F~F^!Jmyml`4lfSNVO)YHxpx zz*0f`_l=41rix+6wyi)vJ5fU;>*e9$T~WD%dW2PVU3-U~{ybc7sip^_j!Atvz&6rK1$NPi8QMt)3hWyW%3 zx^q|D!oVJqSxYWrRJ~;geuwT-JYY!aunYYmX2@vk!;`$}}4pN?q4#enEI_nzY6=HH9KLKo{qb32|f(qTN^N zcd^xSkDgv&m&Q$&sO#1aT534u`&j%2m}Ks7z#cs0nQWE4J|T38_KuUW7O9CMeKDQp zKr1Uz-Zg8~Ql*2QsMTz)V;&bGsgNZs*Y)B%27PLUHqmeAlRSVrT}-EK3C_J$+MOCINAsdQ-(s* zYw;N0Ne?Ppy0bgSh^6?+T&ugJ&y1Z|}dwr%e zaO)MwVbP2;5H8-vG1|8gUzFP~KIpL74QF%xX)IVXnWntumDa2aP0lmc5VY?^yq@5v zVwMR(b#$ik@+;&;1HWMi1EvD`#e5i~g|nKTzW}D_(oFl;=)xwRoA`Q`54J3IK;!V0ioWJL8Ub8}?D-#=rMW(Z!Ok5X8n-tr6a&T52dBa1Hd9su;A@X7Z^N)9C^V~ds0 zqUr3DBPhz}7Rn#eTMYb7cI1-a_fGn!Zloue1k1kGYlHMy!tZ)8!WHm;L*MC zg{phEQn;~w&w9$G{kXz|cjL*o{q&8u^`uKG7g6h#C6RVM($*x2s{P*T)*D4xL=jW) zHjDoxr`%H0_djA~avW*mD8D8~A?p9nkMjS$C$8|zZu;%WP?K)^C5W_L@Gms4HfBRe ztP6v**UPxbjn#v-YPhPk1{7BQe2rLn1@u1`O1{h~sRDIBvoYoVnKjEAiHg$YdhPOq z#0%Phd*sV;tKHab5BBD@Kz2mbJ`GU1X$J!=iwHMPCS@i)Y`pMiN@Da98?$V=S0&S( zC?jqsV&aIClgKXpuSOx$lDogZjY6tEoF==S{|e|47LCujs1?Hh6cqsa>O&qVHzJ5y zy$Yr=OOMLyF#fWemb%%t?`|KT5O<~MwfaG^guY;6cY3*ezOD~pXu$!O;;hKly~b`HrwmhW(eIL3||*^Vl&oKmRY zfOZ6_e9R()O2JDq4V~ilBk8hQg1thtgLDMlTVN08e=3DsL3SJ!o{W==%y2~=7}Q{y zD3Fm)eBeHH35el;D}@?Tiba^BQ>U@A*ZJkxkq|@!V-2SeO$~Br*`wV%k)pj|3;WA| z7sXAM+u&gWS9Ab+s31IjPFt^MwcK2lPhe2{*-HiW<$r;BX&BI1HNmuE$?y$3{{iL& zQ^wB>N!=lsT%$FM|Hu3`VV(|xr2H@Qd-NBW7Y-fadzDqmjMk9zUyVZ0L|cNt?4~S_ zJ*e@OGCgt8(f_cU)Td68>;A)TYO3>k7)SVJH*HLe96u%LXQe6jBr=&MgZ=xjJ@Llq zriMxuHc|BGDUtauqK=~*jw2hi(H5@Ij{j;DiXP60`Zd45!|Svt#|c$`&2MWG9codJ z1$fZ6gqUuG%{JD+Y?^{WVl?51Sxh}HRv;bfTpvK5I=_~f@1t`^9Y4$IB!~+=wx2mx zennmraca+LT?cLB{jer3>VGl@a;?^d2qC>f2kbYdhV5F+ao+!SH;Hq|kxAKW*h5=! z*P1Gvo{7JI?QbcugbH|2!~(Z82BqnTCl_^^jJpKFa&tu;1LuX58>2jvHl1bNnO3p@$~yU;S?|o3-}KF&y?%?+5{Ft?T#hDj(a>2rF60+Me&1 zpyzxusg@OUeS#BCo4z-FI~U7Yh2GV>F8**heO`&+IFc|OI%ckDjnvLSrex-a+}+XF z6Zem@?dQJKWe3w$a|>R`zC)2NOxx@=~w!w7rpQz$C`Dqs4Nt|7qjcumu31`~T)R+j}ru(g_+knXvwMZdd2Guv{|r zx3K(lvL0YL&0YX61sWhy)MY;sbmN zLYmH*aP&pGP1nwCvYI=haLu8oSDm_fSZf-7#HC0tPgv=t{QBK@jD@Rl(ySy&&yr>3 zD9XxdN~0zeOF6Wol#Q3hDNa(l-J&WGuYDAlX`dB~b`9^i#^M95YV#gnc=3||BdYc1 z=yVHr5Z#E;jK;QM@WuLPCoXOqr_X0}9XJT*8m~JARz&vjzR8K{@Fq_7{9fzKc!DU& z<2X4l)uvh5H1$SgGfH++xyjXDoy%KJyCe1`1$2{#CTxD;XfODoe5&vW29@ZWrc%p5 z-89KRPIfX=RSw!>KfwIDnLB?&xVF}lNmIkV8324_nhpxGKMqb zvP4#fNlh(WM#4VwyQMi)a|`JBUF za*N`uxiqlkdr;sEnz--sS?LQZ>;klzG9@N;%R}g+CMyp$dL=QI$}Ee6U#9n_3;a9w zz4^jbvGRH#xo%ciH*HIm_)f4#qeV!13i+-!Dyw}*0W+umgGN7%<4k6hZWU+E;q`Im zUsEe0aJa%+cx9WMUYlC>z3XbQ3zZVOI~JvOUbnNl=FaG|;D-YG972_G)>>fSNmdEQ z6_rKjErQ@BXPkSS0l{huffNqhqD1;4ds9z|2YPHfZQhH&JO`quZ)=zrS^u<>a?C&L z0;{TBHH^WG;0_aY9T5AY2VoaB73(gVS}Lt=7SdOzFOGM15jq&deUgfswv%gX+ zyC#L)3%fu1s^ICcuWdb33CaHwP`lfT_}+>Ppva4D66t$F8b zcck3*go{nDl&a+o#2xKuGd$DxWeYnVUp_dpI4jc`MAu6eyXrTvKf9^HhkdoNL8|oY z@BoYM+>A7`{xs|Oghx$>ib;?6V;wmziTP+zL{@)1h@&9?X?M0`ViqTZ#um5gxqBha zwI_VzFXc`j3#~R73#eka8@3A1Q6$)NV~O)B7zJPLV_+3 z3j4@M&ERx8OXOM)zlUEL`MnD%nzOe=>>Ajw9azn7cur0!K!nbBw( zLD`8?A0#bPNkV0IE|kQ)-$nrQ82QRU1S*#^x3HeLI-%P$yu2)-gEB55FcvU2Y$Jl4 z`-rTQs>8MU)7Gvw|C_sHd#L)OuetnKrleo zCB`cKFo8SgOd8CI!&UBII2xRC<)N@4(s2^{econxPYg*hgO9Qyfhi5zs5@&kHXTU? zF>5%`ZGkb-6ZY(HN^?&_yRss=*={xjHL+j+M|muIs7;+x5fc#pWY@wFN(k3+Gr!c@ zV4y^hbgXU*9EeHp~Km>H3P=1nVp@qEJOb7^QOhuZN(W+|5mV4yLo`&eZhPp8Zg3e120F=x`@oLm=Npg7zKtO2UiXHrk zz6_Lmer0mXCk`Z0=8U9Q9Xv*?2;P31%MLLI%MAQHgrXBWH#wlWa7Je+A)-E7{6SZI zoe6-H^!T!DIY{*wFI1@g2ZUE6z%ik~b&>+4cd>9Ok6e#t+qi&w{Y6*;dsWg;Dl+TL0g&P{tbD{nY-P6wOJp>4 z5F&UAoY67Z1zNN$kt6g#1r7@LQ^F@BDxu+__>f5`2iTh-xVL!9EGPp>U*K7sM&Pjk zGW0Qz1qw_!00eI$N8WSmT(V{``$C|Z-`0QF4tS8%&cf66Cnv+GFR*24UE7SiCQ z!FAfgP|=7YW4$6p9^@P~Rot3XCP)LLusybAtl& zmG=I`gM8f}lgk>^V5~(zXruPR@5$zbN5io@T!fcn0RI~k3)hJE0LHaWh=LO#!T5Ja zDMaw6U|(2s4|@b*u3B~-VgHqeC(;D;OsL*bTg@E^}v|#>dNrFV@4gZr+*N zDh|#PfA7vIrl*&@iD%ndhsD_zyb2gp^#+r=SlDv9Lc&&1YC?u<+a_;t);WVFE6Yf zi$^{+fo5T4@Te z=5ARwp6f)6d5;2F{>^pp<-xRA-%I>eY>o^dkwAL)j7A+Q1+t}j^`aL=veA=&Nx+aX zPA9bO?d%u3l^NyG46I)+hhig<7x7H8d1V(C$kh!mhgxUhT7Ofy<5pThq-;DEA|w3LFrHO&O?O%->HlbsK$6 zG6?lZrtvodjCqKx@XYhC){dIkTPmlBNCSNpfuQ$D$5A_Z;)1M%`3c5}<*5FUjxmcqLtQa(cyT3^cJ=lrX&}9+=&C(0T>c@cRGv0YMR%iZ zMAkhQY@{;Ho_xp!+K-vkwVJ-gHS?EzeLbmbN=D9FEOdJDeH*vkuYWNgQH@12x^U!UUb-GLDDMFD+F zN23MvZk5@0REZ?UO^>4f>632tkJ8?n-Y0^30*G8Ydd4cU_)V@kM1GJVC7{=V$! z6Q}lkJ)2$5->s5o#@O%wB6iV<v_=ipcf-t_aY|v zxcL0X@1zbTN@CnhssEtZ{G;skfmO~Gg}`eA%hUt0X^9El_Bdymd;K?*Bb9$29tJhX zP~r@0yHU^;49=0c*=_Yrt}!ua;(F7KT|ZPayfM#KLDZL;R(o6{eoL`(-x?`L!;dOy z>oRbnd%r>3#U9c_$W>a3MTjdUU^=?SA?gU(FXQ!n(?m_^TEI~|Ia-=^EdtBnIPyg> z>h5{@)=U$pdSZ@Mw9d_)>?JvjE5R^PdUB^*_!&EQtII`JE5kV?YIDNMKX(nz@<+(* zK)_GNx+%JCa6H!e(Da|rxMzQn$8H44{26uR3)M*}MPfBu*+1g($jo_=MdyxG$qjFd zlSP~l*7w31m)@AEP)UxQJ2A($?vdKVYnn~@EcX|dXNuG}Q@Dei!K(90_S~bHVYp?s zGP~zP-Th>Qp__l&)iUsDm*PgKItQ5PK{W;wd$fN_r3bu2*G6+&Er2Bo#12(Ibu;a_ zsb?S06)K6iIK@NhLAnLm8G#!N42~z=aE7`looH$k0~qg(yWgC}M}3m96e6{3LeQL! zSpp%+51V^H@pXB6`WKT#DsHZJK%?}yyev&YKh2(%VL-O_{nMR7)HeDCpP^Ne`r6RD z)194N+XiQR)tlE0vV?wJoK}uvZQ!0bvC1Zrj35b5;<0(x_YC$^;y6R!Bt(;uBQ&o` zVpieLdZL3;8SubUd$>P7-ZSORo>Sz!UL>>k!)89hIGk^Y zbfO@bxBC9RZuC-=g#^%9y%0hN(wLbCl~Z*Jp3OzW`<%}Ng?TVx{bpUyXr@7RiQkwC_K_D+q6Wb3if*prBZaew{m zHRtmQcn=C(l>sJcHbrutEYNRid+B@R`G-~M0iRuKa14+4h7TfxLyOVd9avGMj$?h{ zEuWACHH^0o{X>p2fjE%lG*%o>h!(mwW@ zs^BHlDWH2Yyd|0h|Cq*}*6=_L7U!w0Z5FZ7Cs-w^EO=?4nws!#sgz5(7r1GGY&L^% zlID|a3#Swz1Yz}n?qfC31Cs@k_H2=qin?PFL{_8)@GJphRn1Gr_;q4Ss;B)7a@U&+ zy~Q0zX`x}LL08+D`8^%OZfD2wP@-m|u9?Db8~kyM$q1tsG39F18*--wj+mu-^d>u9 zvj^IT1Qf$9Y{#;1Jc|dq?$x{XdmM;b-TgTa$l|d-Xhk zIspw9!|U<@4OZiCe?hylv2&qwm-=|u+FDHO%zJ?;a!c`p|6JGF3iCyITl||Rk`Qh6YAfLe>^0w$A^n zL|EI#>7W(yyH+pY-W1bogmBjEWZi+<#W}IoVKkKojV{Ogo}ZYQKr8`9K(nm%?Pd$K z-%T>ry5fZSB;(Z_;UjQsxJimSW0DWn_TzchP3XSJ$4Taz*QdSr9J$CwVUnwzWT_~f_8QaKU3qmhb0buyW5F?H)3_A5Vwxui>>9WA@hZuC_8WN%<+GIG2y52hydt%@_8E z11YJPR$6rG(xEcw6$b5ia|E(mqx5-(c%hF+osTWArI~|oGbDyOqj2{l1_m=~R`G7A z0D4nB!%4)y_z&SST%i`4Z^TO5@%x-37-0!HC2Cxdo2Z{e+2LUh2{^TyF_=2d5>fr| zd5J`H_T--Z!WyJ6I?qS0o{u=_Huy$}?xo0LkRR9S&YZhm5_e?9h}BadMx@Z) zbCM}CMxXd6KP!#Y7Q9jxfNH++I-Q>lAMkVb)sM%4?b>q@;JCnf)q%u>N#yEKXc=i} zvS7TRj*kvxUj#;7Zgxo2pQ@CmaI;a7%j-8>*}6C|@w%DdGg| zW=63iGfuo#@lQMj@sFGg87I~G4mN25l6W9ohfMY&ooaAgX z)r;YUHEDE!aP;Irn5Ohpi+CfMJaZUSDq=H~l5N;`n6gq2nHUUVZ#LV&tJYb7t(|&G z{u#WY2!k!ElVf>8{cg#%pXsiLov$|gf$&2k)rtt~Q+Y%c6emXhBPc_wb&vo++ zDSfUYA<-^p4;CB^QIQwnkAG*jI}px;bcNZyqtk~*LH3}q`!S`^<*k-MG8gfd&5Mv4xuXxoPhiHx7U>c$|7YC$h)F5l{cPkc<`iLGG-UOw*nj~-a zIfx>14sh{-+oS?4gVMV7ugPv*hqh7TX*`U+fs0Vw-ci_cidVDffswE_FOO#6e2~d6 zq=~{ZO(To3DjlGuKuisPES;bAH~jFy_8R9AXd`h7&O3Q(DV}GSnfSCkLm@n+zPCJ_ z_Z0!*H@3xV2;|z0`6c&c8atsh!~G%IKcwF_$|Yv&BC>UTtP~Ha!$4wfPyG4}aVu4#lg!YGJw-pgQhE`Cqa`96qGL~^yq9k)13S4B389cJV zEha-SEl}ULzO?Rf+$iwF3GFWZ9j+=cmNpWRN;=`S6qy>up8lc-^c0^O(~-l_0MvR* z))%l{&0dSRnN6G=uuVP6C-$ z2x`)euiDif=Y!+QqBMXxVRVggz~A_(cBd`+#1aM+ORIVUAWAatQM|PQwyz3sRfXyz zM30qqN<0+6@xk(~C;v#Et*z$^U)r_!=~{>fSXRoTiXU-$9>p#h=kFXTdk>g`m%Jnz z*uB!{(7K`w!x%{zzz64mh95oog}_B87Jl69+^)Wk=qL0;(|MF!f*|n+a)1)sNW79l zHhI+NZ!bUWu2~!rTuPUkhe|~jYOIJD@L~bH+`Yy5yszxIfl7PAfk$l4wwSjj5KRt1 z6&Z%VH%dG9V?;3{tFp>1K5yNE&LU-oCf)*!WB+_{DiU@HOgXs!6(~}*QCq3(%L1hK z3MzwozXz{*^Z|tRhvJ;PHj24;i$063ik$^q&2!1R9ab*({rmg0{%ga_*}Cs3EZEE7 z9R}{m1hAy-gaSo;r&^@~3k~<1t(_)ae+rJR3Y%I{*!Pt=^W}zGTM_ujxoLV`{oZkE zRmZ%-ZI1oT{SL#fSBvM=w$?$`o;RU{qgaGP)j(4vpQG!6Gm zGJa9{D|7&`2OpW@q>&f^(C8CkDMB*45Df=R;p?jim!ltN9(n)0hPCcMUG9Zo3!$sV zxP&V}g>No!C;l2`xF7E=5Incq1pASkigKMbR@V=_VaeJx8Isyl5Akts#;R*6EXxL2 z-VKUrAKi&^i-=Jb&|^$}RBA~2s{2x{7&tz?tlrFF?-DZTgxe3~to zD?GawogY;>1wP{&<9_7M3?3ck1%#({0nZIijaETRfsqY9k1v3(8PZpFfKqz?E- z$F`vCSI7f8&%)i_Thrvn5|?07cy0!sU(r|K*^nWcpsbpmFS6m`LGt2O#UEH>M_{8l zDOU>hno<0JzpGiZIN?hI^EnwEiezI%ry_IFm1*|{@`9LN3@|~Xz$NbFc|>>A|LL=- zKH!fDNvH@UGkJ=bEjE68{(-YmJdXh;;RuRf*W{4!@65qIhWLT~U@6FDuk3UIM@0=2 zI%hw!AOC4d+EdIFItL92y-dFSbU`al@!I!#%?R-_|md3A1yQjOfd<$U6{ zb8GQGjn5nonCR*vPyPM5j^yZ$mUHhj=pslAr%s^^nGl}UO_iF;V#PUwjSXxW8g73$ zW}umg9b>(x>Y|4gY|)LL^$~Pgi(m5YyF)w1iNG-it89q_(I z`LMXO`?}7~TrBPn5sw^5Nhq^xQB+EUwYEH;EuSKtfgoSuie50i74DgpWZ~v(Qt>L? z1hZ>vRj-Y;uFa8|%C+6W#R%@dMc+{#8P@w(&0{;}^_g0AFG)?yOO;!FIiH@Gc==0p z79b%bJ`4C=c`vL=Y$8hP^S@Fxl&Huu1bsqcXDro6FU#<2QAcf86jzXW-}bs}=IefDP{iYQDt85);oiWXtuY26Xk)cW~vlxz+P3(0cAj;=7%!I3y zLX~-htVGEI=oUPy3Q^Uf3dW_`VB3Z2f{tczSgWrJQJK4FU9$2@G`ViLPt;8H^O|k6 z0KXJ4_QTZ9@I|}OW|Y82 z)aM3U{Q_5evO~ThVZ5lHOqepKFs&iT{NbEuy!nZXmSJq`dDYU{%+-+^P4(uF#S=+2 zErQn?*}}d$K#dD{p!F>stQ#UBqT9KakKK5$Ae%>xZG#MbZdo^z3kM%OCV_+M37@rXnVSQ3~D>&O%zZ{avod;mEZ#CIKkt-3ujhj`HapwNeXEr|J}i*??CQ+EW;M1p9~CXRU(t0n zs7Gz^wS7MJUspjFYM%oxaNvDKcC{@FJG@=r&udRrZ|&T zKXTk#EB~|D#XMEvga{G}>zn3FbF*zW)mmK2ck0 zFTo)MT8yT+8|CEu-xXAm9&Vr6EyVDAhj6 zuAAwJJU^D}5m@~?ct-MZu&bTF`Wk*xLDzsCeOt9yb?~&6>tk<>7m!Y00e5KO;a8 zi>rkeaTx8dza#xg79k!8dN*$#+q+M8$ppWgC;nG}nX~oB7LO;}hnxR)0JgMTjNADh z66ed&_xbMgk*E{9^YrlaRT&ydIk$gz#jdHs&Z(ybB z^^;(Pm4*-p^%{riFwybs9^N$xGn$C2JE<4*0?Q zY%-gFG_ih8s(9()3Gk9-!N?updmLd3AA_vE2`jlXX97IfQln?J9_q9n%cz2-Gc%HV zr26yeV%f8uzhilWQmpsH9XZHkfX%UWX1>B-(`;hb#e}2)xd83;&Xn)`f{hNR`CTSQ z@V4@ux35B{wBF@Z5U??5hLe0ew3j>NauOEp;5<=w5C6NRjd)VW`t0JE`vAWpha!5G zCdg>uBC_}L$H?Lm4*S)iC?wnw<~^X57V65C>`)3({Pt39W5M@7epk{QwDzg)zR$}M z13(D)u)I0a_Ka;N(dt)PJeB?I{G=ApjI{O(BnIx@V*GeI9NE(oB;{u@6l61vB{At< zx@ZZ=W%a^>;Te~I4#&i5Dpj= zj?V?ShBCsdeMcdG959cl=i5qi5WW%wsh56Q8Sz+i|lah}HHhdW$yeKkS(HoIWP;KVOVl>3{7%_q-(48ZfT zFN%&&38~gI&CG#(c6UuluKsYKSaQtUQhxnh!?XMa@WE*T>&YX=iiV~Z90wRSZKajR zTaRHK)JL)dsGC7avl|V1$1#Q(;$yiYwHi3LfwI@yp;-Q+BCdG zb&2=oM6+p&NvH$^D49j0l5Z4cW=b{P+)~xCz^sd+288q|IFmqC!br+So+zM1d4GZ! znp60hpULTlH|82{)+slCwj3M?mz%xxayU_zjv4(sl)N*V&gCkaBc~L8&EN&YbGY=a zWT>%WOOu$^1C`n8(_D$^73DHYPg(YrqABmZMSnAW2TF?7{cAZk9Mamw1m7H3^T8R! zc}8wHWt1wauu8Y3qD2C{@sA!6iV9ttvSP|i@0IChErHTs#GE0{gA3B6m;?xj^Tu&w z%#8JISo&&K>alD`Odi5EFD)T#b)v#9gGI~jtG;xkQ^@DKQ9Y(9Ok zIz$lQ>`qHT+q8dC&atBh^F+YQEwiMLH&?(V&kDIt;kUXzYPs5l20ya0X14I4&kV7S zH7cvB;U;}N?JW~)Gb5xs0zn)oU!oo1lb{Uh@4O3*O-t3yhmgc|Xf7ZY;GDM^o_#1^ z8p__8PS=(4_IdyTGORSv6cynTtP@INp!?ql{wBcpg?ooJVw-ecng8`TSKzvcJX3kI{h1E7ZKjHvzoeNMdJ5DUtwukUw zHl_@N@(Xb5I%CKKkUlD~TEB1iAn___tEn zEhcU35lIgt5mOE`xoc&yK%BphuBI$ADpn~*?861JUU8^aB3Nhxf0Bm0Cbvyqla@H< z?}cNlJ5%I*?y*I*RU*IbTkte%{U>tK_jfYwHOJir(THFmQHdFg;!1-@4LVcEDi)0$ z)9CiaePD%ujfZj?G}G)+$``pS(i~Ey%2*gqYJCPxbk0jlMP3C>=W94l_hdZZXY)TzVVdt&P)hkqvRb*t)gPf; zz?E99%VK?IDd3Fs+T1WJII=9=c%Sv6c<^zPwGlJarD7O)D-_34$f7+)Xaq zV^S+#;ubGDXPCc`MqTA`tf1691Zpo`+xcOiwnkc)^0vB)YstS-UEeJ6&|X)o z09iJxc|uv^I#IM+{C!}|J>Y(DX{qW{;e_!oHLerk$sB4ylQ*~#O;?Yyv=V;~2eTE< zpc`3sR63wM-nuB?WR|bFCVObdoH3X6GD>I_*kjyFI5Tl9l{-Z#vKjYgm-8Z9&9`j) zBkf+&?0^r`0BON~|BVzocYQH{%mM)b z5KaDnuDyg!ObuMDodxXetu2fU{$K5->Gi98U~U8f6r^KzH6Q5e3WU3m1L`y>FAr0( z@t%Wi4kZSo6SycsSsT4q)vlG#+IfFoo$W*~s2DIfaZG*UZaWR;WE3SyZ=qKrSPMar zCxh-lI%9#)v7q-rQf!sMh%dJfcjxbGh}$Tx9HBT*`*+;f9|wo;>-|d{ZCD+49$t>c zNMU>oo0rqzsf^Ph254@=vN`9+EKMg-(17?U^T~UKDo|Y%D9p%Ik`iWFS*)@f5<#?J z_;~r1#&&gviLzuU=pjl`QGq_F(SyxE3qdlP$=00gF;%+c&QzAW`Ub#VaBo@Jb^0Em?Wat zIK$sEcw%-YELhIGad2SE4ek~A$dkCt19B!#Oi11kh38Z7sAswGU*!dk{pp{?AT(QG zr#jCa;DAKKA#+(u6rnJK+4bl`_0>Ojwb?=c3UIxP_MYRX<9fZ&ww^=|z%9xy_95($ zM?FVfaUj@;kz^cvV9^|4>KIPEAnP%R7uen)n)wDYsxU{`uI2=OW)Zu<@=zK&Pb}Wv z?_pC_$#OzH91gPzh@O$?OR5L16bl_hqMPIAV7Sk-3uJNoW-XeSLbDpa+`C)^6&+-q z=q58tBGH-Lki-@Fd^)igO}UyshUFxiVu{}V+rkrnomBr9(=Fko&z{+6g#0XW? z#J1{IA&s6@G|0J#zF60`Z)s8z3WPx15{3XbiC!s80C~Ge47n5Ffby+Zhlj3Y@%2)y zc`LCxlzBjzoMD2wqwkrQG=lPKT|SN>^gdScd)6gd%K-H?RSMV^P!tH_(^NHaAN&Da9N z%x$>byIb4m{sJ{T&R7IZXJZ0HW0AI4h zyZ8t_l*LG!3iLgRN`0@cnpFbR(x0n+yS?;LzQfuZrD#LFxNo1OYM$7;CsoW0r&-CY`+RD^k|{&jk~$Wyz~LU zWt*%0Ux&~>e++x$lMC1Zow|@S+ScVZtD|SKzUJsE^1edGZVDX5kfm8%YpU8*ZC8QR z+8Mt_ zEC0f#ma`_;Mf4+*UZzpzcx~e+oHKf_U*#3ERoJf^F{3&+m`h6xsLG5zs+UzuYn3J) zyScg%GBk(t7c>P^78>-uQZZoGMlmfh?srpkz<-66tW0 zND=;LLIc#vOpF#?1d^mY$vFZNgySad%@_Z%%JhX}b5(Kkx((2e-BJUcoZ5!3WmFu- zvinIBKR1e(uh#$&wq`p(WSL`OS0{&mv9~e^fT|tRekD5@8WE@rF{4Dw%`p9dgQcGW zel60_XsjBlDl1!W;h+!&?R$)!?3A=VxivEUTPzh0E?ZBqh$F_0s zRCI5RkDi;L%lCI&(xn10IB(A-`H4-iNB%_9-s#8}`>{3f!Q-xD%`7>zv~~wPZlW-A z+J8wL?@yHGl2phZct;a0*uKmmc#{7)xpPAw>z>+f7pa!DXXGr5PRu9>9lAL%JetC8 z)+=LzioSw;F6K4UCEK|N4_+h)&i!N^#{Inv!&sj7ZS&Fz8aIuI;QNun9vk4_STp#-q?yw;i67Th4 z{B3Zx2`6!F1LDAOOFZSwgWGh@=qq^W47dnsVa}TyVCo|d;#v~OA>MT2uYo)L%aIE{ zChpD`>OQ89x}A3edrwLzTVu%u;_#%E+NztNM<4mHRDDCV=4BBw7rK^P=U{Ik)YTuC zuvvk&Uo93UKXrL1>2@u!Y+>?TvJtis-TV9v-S&#aaS_g46eJle@UZ3FWa~MAt~exV z*sykZBqwC!X~3&En<(&kZLx`?2?+DmkFM%{o|m8v1-^fA8jihtNPYk7`j*l|K7l-- zb5^qE@6$#5&-G~HU;;=(ac>brN3~vR$#tx^v&09}YW4F!Xt>~+SlG2tze+>`06^sb za1!+2YDCPy*~HDjL&?O^)x=TS!pYgh*2M9@UF{1eOb(>~REgy%Z|zeuA@e16-D9?f zJFxaP#Z1O`t;^9e5b=-GXJSDjU)%n?Sn(V~GWo+oZa4m4oP9%dW>L3oY$p}l#uwYF zIH}mSZQHhO+qRulY?~Fl{NujPy{&g&<1}{noH5Q`bFPU>6hv(2%H_|CP}0MPp9X@+ zuN&9YQ>E`?>@DvvJ8x(Is*r<^i^BnXuFn`Bdy8IPp-(8tESL$9OYWJY>fow+q`mZm zDHCQ-7lTH#!^fNK`n1>5m|0AU77Ct9W$y3td4X0Vmr!vqR_r;#e}n{(g(adO>xR;G zrd(fQASn8RJS=I246n$ZQ=(7eDE64xc=u z_YiUMP?t++^GP3Gei<@L99#n5%0>4SL&#H85X-U@_x(`H#grEqqMzc^_DL3abf`Sw zwC<0mh8H)4wsNxp4u=XckbG*Wm#n6XQ1HZT2c!+NcEZ%eVpU?AmFMsb@#y&=lf7=7efxjyzn zmpc5 zxP26u{?;V-VNvsk|5V<(meNM7G7+6%nF_23s}r4%tKsq?zF8{+U>J$~3z7ZvDzqOA z!Fq;!A@C`;}oASF0=t3o@GM{Prm1Qx&k(cfg~hD(huzogRKcD=d{D zP)dQaokBZlGT_sa&JH2WEoZRho_ui1DsS9c?DDbSx_*)`-Lfq-Q8!3kMn)rm?Qdrg z&R$MXphz+I!i5wF*8HPBO&T0{+?W9Z(|k~zK9Cfyhdt+%XDq`c7|0Y}*5@x3VLzNS z3&36T!d|1#JY5sWdO-mDnUmux(8X6KWLT^YrzV@DwIz`AqSpUNg zCH$~MlcU=f^TjP!7G%ct$+5vqnz5&Z`sX=LJio#fZ`3y~sZ4~1w<;yeNJ##qs@_?> zVWmkV7wDL`Xyr&YeK}YxzBL+^OknYVL-UiUhYC@V%<+S;!OGimG0uBf*HbIG*PH}E z9Z1xW9Qw-3HV>Quog-C?HZ?(%wG>srSfCjDL&EvcS}0Ymm{~nX$#XN;{V_u5=F)DN z8B~E^$*t>SLjN5+Ft+HimEv|n{~(bbcb=WJEVe+QMu{=It5J7vK&fXS9T=}AiIZ+s zrc#EoI4lTnenA{BE5fK&{^mHOmlWm!wvAlm2%S3Xh=#Pk&I;Bc*#Mj;C*(^twxN_4 zJ(EDK8lzpE~Ngi8h0g8Ujqjm}QbKBes|Z znSNvkXI0cKt!I8`ZrC|*e2+-y8^zso4~LVqX+))<%5bR*BIU*nbP4_mx?CP<+cfde zB(tbz{ch8~^uo>&Sfihz2&^S#RN!B{0)r$xjADDw?~oHuV6aG_1OSvLmMCpSY4Xt# zduo}w9|M}l(cbrA1u@oMd$8yLrS_kAm{MsvnK_@yU ziF_kT_bR2_l5VBy81x`@@yUPTFZfl_gU-!W9oMZXFVK@(&B~%l(y^&%GAVQuhfB9t3_%=+=b0yjzUh})AUE! zhI~ChNVdeP(v4E>AmpcZJ>h>c3|;`u-oB3YerBl&&?~6O@M4)z=!=(%HCQDEsOd)E z*^{994o$HDyx+(gV6bmD-4g+yaE~D8?CwnD2buSiWlerHu`wnHTx_&mc2pV`j7xvV1 zpKkb}W?K2`&tTrMVwhCpULR_hmR20tW48rd-qFZ-9SN;=RJvXv&lFl>0OZ`;xzp3x zp<{~&b0i>55y0Olz4<4B0{nX_v}_vaANecli6RZ4>MJy zJGDuhhgOfn>g12P?w1aisfRKh_Q}@u-?$vmi__?ry zG%qslpZ4D`hM!g=FMMMnFU$|!1YK97=W171b!%&F|Gj&dlWQkj9T@e}dS*UerS;Ah ze)T9+KKuT6Vdrf}9Yrky5Ky*15D?@4<3Rj>C*l8^eYGuZH#yKhuXX$F{nOFHs!PtC zFsv_waY`5A?uGVUm~uhX3#?^~=SdXpyE`9tUheZ-jV76vCj>!A>#6wY5pQ_Airk75 zb3;~33;Atk=1S*}z&qYP_Vz!}eqtT&ZVr?NF=ImR2QsfKNm%Y>V!5V>a?Y?v)|GLZ zO{3_xu|^!)FX>}DhjOtCH4R6OuW<&+g7#~sFI{?_@0CgeYi{pHvJ95a1VCm3wH7m< zH`Wd~P1c%}M|SE$y%Y6uqE%1j9{QWYgAwrSo+ZxS{#_N^`|z%s?p`+_H3u>D6I?AV=1~_S}ljtJN5(2 zYF2v=#=E$LogJkwe|;3D+|a5uaBXX+Z(KVvYC!p9L37!K8#|nKc2(CLtiNtP9qwX; zy5wO#>q~e{A>J}DDAa~>i8IXmiYk0HGJ2n?W5~lDfBbYy=`Z*&0KT>WaKLFqnW1v{ z;d~o2!r!h)SF$jd@EF<5{6XjhBS1c)MO9)Rnh#3z{vMoN4U&+>JM)W;az$vIINPE}3k zC5@sXKo;@~AWfAXjy_BjG`{0d(Krj%D6-ZPSJ{1EE+69Es zE7^UqVHe+FO6KzWy=>kq>03iwzQsyl^x-H+CdoHm_1;U@oMUa6cI_t;3$^TFi9IRu z*5rTu24B*PPa|YP>S?j{6FdR2WbOHoY&x%N`_Jq4`I{9}jKAp^sG3@s?fXQQPDqsB zvwUe7I0ml^K;J7Jd;@C3r^I&edlj!xI)nr3=dugxsj?Povm_fuHU`6-kC`UnV^%HB_f_P-X{HSe~sjyFl0|Efx3w zfWtCz6x{iefx`|neNsV`B#FJwLssUK@oK8xM&xO&1CZklUp7iEB8;SO7HQ++`DRw{ z#)8=MA5C^UP%;D9il6@w82gKZJ3kNrxG`Oo%sn11*dtjExxLObr_XiK7yF1Eao_%*3+ z_rGQ$UA2s9VRTC0sok&h{Wj~(Y4!SF;CKkMVjrHOk$n!t@v+Mz_l)_oVR#tb zXgjNADm9M6cF?%&tF2eVt)il_@qD!E#9i1Wb_El%4sUPrza->(O;rc?$Jw)d64wvL z-yaR^TthFN8`| zXK;WJUfmS~+=}pQL%c)NbAlVK@;_8<4z;n}v?HgAhYb&^flXxK9rwb}0=X+ko?hkj zOva^H9Nm&|mReG2BTtl+m5q>XzK2YPWBLh!1V3EkRe|30$T{k-_+FFYaA|*1y@;GPnDNr^i15CQ zo+`#h6}>vEEs7k`u$fAE za)FhU@rZAJQ13{mO_r3TX-KR3jp6To*vW1V(@FaC+o(INMXtI>voD^|PHQ)4h~!Sl zNfJ-L0kyX(R{LG7=pGEGsorsf5^+!+2+DXYt+-Jp+)RHTq7jAz&VfCN_1qU(U%(Z2 ztDL97y?weFGjJ~245^;58kLqech!6i=+`a~HkOxq*%Ko}+c!{2W9X#g1RPqyT4Dq(( z;R{zR*4FQ7$_=R=enXtyMAFv)$RLKhYg=lfPZGM@FUlQ}sw+w!P*BYL{M{%{llS=muIepHYw_{pTyDBTa zEDmUnhE3af#S_#g?bqzsZCQqB+ERPk^4yb;C3B8;uFbhV5az&Z*XB>|tvU>nK2MDX zwXHoqW)R`M`rK)|IG>espl{X$o(p=0(V5zG9NB1-PbvxaVs*@5m0&)`dFFtuOwT<^ zIV!+4cMv?G#o2_Y?7YtW)t?dmB6kCMk}fF)fK1Olacc!|X}M8Ls4hNb+=AQw61x#F zpzdY4;ZrZPJ*5S42~ItNi|^_Yr`Eg-L$u2D+{4$YNM_+zonBp2a^CQ)vKgmP*}L~Z z3-jPx+V5tE;Mtn^d~M3eEZiGcJS9v5apXXF;Mn;R z_&hW)R;>=nY%Dgp^xzci31KwvtQ#hA;Q4ci;=xm4pHGU{Qcm(AT2>iy-+4@>xX)fR zdJot+t+Pimn1BNPWrrG;0w!$H>uK%?_Cr;#qp44`oVK7UUlxl3!OQ8Z@t2)`I@i&@ z>17-4=Gd>4a;2JdNz>B2q5nG2$1<8WY=}{-7x<(@)O2>M$(PBFW5#bUvIPVipC?(G zb6Me}yM1jO0cT{fDU=A)50HRG@`NX-9X{O^0`{}7(fKnF2=Q{FAx0uU@UP6?9vvbyp)C4B$lZ09x*H>dZJASiLH+QYZUd z#-be>MgYIfQGA>|tX%P)LeK%BLx{8+*e#&&0nXo`{L(SZ77z#W&2(^C>^VDGRY;ZlL4B+c1!9-10lI zptW$^UxEL9oYAHf?cc#5kDZKWjubFSBY&CffeW)>hEjd7fexN45CEz*pU0Ae7EXju z9?N`Oa&&Ehu<^nJ_^j*xc@1UZLp2DxE)%W+yco}&S}<_9vSZ8^7x#?hv)BeI=bNYd z>VWI5>N4=B7Vd1gO!u+zQgN|y@}xpj&VcE$vY(Ui(nwu0^=oD?WQVG+!mV$@7kg*Y z+nC-G#q#&N_=#>7I)A?M3v@>e6rj;AXze!)m4Sb7Tys z?*#}Oev{tzco>~2-|?`4Po6EUGj7Xr_pVfR!DNS&+tIVw(b4=G-Z{hC^n%hQVz!3i zh+>?T>ZrH806lebX2v2_nvV2x0+R)a64}8vA(&TBzMn70rzq8r8!*ei0kBn=!XE^Q zQ0m^FaBi3yHqCDEu8%@KEBi)Vh*5x!ust)h6*2jl5|Lw@aYYo=+nl@d{_KKT9t zy*ZzupCHdQNOkKqMJb~owO6Hoz1|C-JyeEtFV_=|&O)oy2vOaq{dp(T&)#k2JWz^| zPeJxc#uu4W=4w<@j!hK}O0Ar23a}aJuVWEqKg77W(_{cfvmZ5dHF1hif_G3IDY*h{ zz{+P~uf#WT}P>caetRPkQJOb!hZ> z5hUs>H!{{R!;6G2Tp4mVbp9T=UucxmE1eCl<7UuYR>7n2H8VX~Tj;p#WiAtWzwnsn zPFGNarKYc6Q|5y`ri^oKa)*Ifr^2#7es}B7DW?g+kYKS=ssk_`2HVHFpx)%iS@9%> zsL$iLVO;-`Ixk&Alz4eD7P3bO$aEuWZaax}5b_voY5s%R9!T}I z)csiKL0m1Ie=>gO^=)r`j)pcg(s|9$cBhKfF&ZsrzpD5f>$YA)`}WbZb@Fn>*K6C_ zf3q2VrWdQzUK!-$b8-Ek>FV6;-0I{(n>H|r)n)cu#j$~j{5j(Lq-D|eli#64Z6P=-l>|!_zkZg)8h4k#dHO!Ys=J32_+`9OE@CEa^(AzEeLcm~C z!#!rfHFHCO_ag$3ITq`J2{^bT1k5tFF%kjRr2&O2(>zVB*zHKb^ifKot#B zx=wta^|OgE&1QrOqfxOtEV9s+A(&n*~Ag zox-y)!y1M_&}NARt((TA!MOo;g%;@^0zEz-Lix^p1U|2MJGj5E_CNZ)DjuPr;w}y- zLGFSQxj*0(nLJ=`AAlOgG1*%}(BcAzNsALz=P9_%O-jhA{p|%FrNtGiBq+H+E@9dv z_Ut9phxE0L6{8#1*m*$u;G^Ay4(a=At*DM*t&9Pl|4NzruPN5>rzw3r2CPgX@eG=8 zmdlJrfEj@^^X>iBLtnq{K!M$zhU@?|#Xq{sgD?&6}w?9A`b@wotboa z%O4<#ta`a!f(SClm^yyyC1E~3O%!Zhs7qjZYI$eFx108wNWRvT^9kEEE$w2U`%wwU zXeg?{6Q}a0xS5lBVPDB9%7l!v02oE7&#C|#0R>x+vLC8xY>au~967i3gn?YDOdQj# zdwG1zpCnCOHrz0sObSn8pWis08XIvxEgGs#s`;CbagA|4>xnLOpI(HF_F+wI$x?mo z<%lsH|Dh(YuE0r$87ed&h%Mc5NAKpErH3bKmR)f;Msjh=2H!KyS(eg$Q zVKj8$YGJP*O@u&JlMa)4n>%(RWqugKm;3Gx4oV#~3?{c|^9#o(oFJYIIFOMLT5Ei( zbd#LIyV(`svZy3`@%T>k&OSrOtA16@}eo1Cnst2u^V=MPuGs$YXi zJURad^f2&cdCS=Ol&znen-~5mJM%#YQ9I{lR>4xI3PT-5UuE3+C78WwV}J=OXT&xt zHVGnjLR}_7fO3~}k&x9>35&_e0E$_##DAZ<84f!Nx)2wVo~i>sn+!HqU))}=b1%_> zV95Q%FE$L9C*NTRnW}yf?*qDua#d6M0>+=v2nlxgfXF303??B&z@RxXwPhgnWjs@1mf&f)v+nlNpcuz?PBxwEBmz zSbU|Ej0X9|S9wmI`=R-THw~a(?lyQ$SRxB4ppl>jX;u7Eln&Pu$a4#V@&Cshq`h_$ zU-6Z1rXMes>}J$U=OuC^%M~jad&wf{8C)wM;Rail562=wZGbTum)}&eCKHibRRDaZysK`V_-gjkZLOAU*XfltV)Y%!sI|KQ zFM5pt8V4VY5AKs@RD#I)nLaow+J>VArgVYIm};K1m={Al1qX5|T(BDoklfV?$Lrqg z+q0|lV0;b>r2X5RWieHL^Eb`FlP;86Ud`HHm^_uwx3|SLReianMjN9xA~`oMJP<$! z3=za1eHr6%_ZilU(Hq{lucL6Rn5=PVI-xp6*8w|xRta47M=7Gpx15fWHXyyQ{k>N5#iqd3RXe-(9MWYL_>T#S zK?z66tSK627j

    Py)-jQfBSA-*!T3XQ%q79SaDktIcjL`soIB6DvTzc&J20J2s9C zIL zie&Ua6e(~eo-x!_P4?Fe8#hvsm}eRwDicyeuNSD|zc$+OGd zr%wB?i7k}Wf6Nf>BYC73BglxkQ%rsAM0?ng9At?_a~iucI&hyzJGQm&y-uy(A{rkV zt$_2&S_wks5s@kmJ*^fCf<9?mUm=y2g&JIAd4IGp_fq0*9 z+(?+AdGST*CLS9g9;9nD;~Q*@7P3+oest+84v2gQt28rgp!W zHM|su6m72QD;SWch!w^j3#*IRg_F+q64W6#Jfil(O5=LNJ}~!y$ z*8-yoQ{a;`OW|_!tJAZsv8Kr zRj6|*oCGX-qVbQ*ev1qFXt-Kqsc|HW$E`3EttC%?@kBm<6FymE&>aQvcLpQc9IzH( zCY?L*EUM(W9u9Z&06v@`Ud)D`WX@BeEdSJGL^Sx%OR#;@sqC8Av;&^uJ@(>A^cn?W z(TTuDFtEhGp8RupYdak78#XBUd61rr!P%!#-o#wfTE^fXdM~s%R((=4(~H!N)}SWQ z+h4`r&4CZ*&2i=Q;F@it=3r~}CoLNTi~IQB#`?y`U;f&Bp~mz_hW?9B_=`c5-do)O z$kEK;Q@Db^ff=wl`-!D%3SU%wZ$md5rpe9p600@Kh^RH~p(&@%n+Y^KH;vxxTRoXJ zrVe2S>|5LQH)V#oJA-L1OewM`>^J&aY2#-jofD)r1@7js>Kf(W4`qzU_$S+0Og(i{dzHsB>Q|(rsw6Am^^@?`dRCx{{7>q zC5GtolXwAXs;0U;s-tt22q@Y)I1=jl@JSMpK^j?PERzVcM{kT~ZEOABq|WIp*sTGG zj71HNnu=ubuo>7V3k0Re8DIx%m(BGLa+wNgELUp(Nm`#p?Lr4Zd(Lnc4D=IsO10Ufh_<(8OyBwU**k zP;$ryeuZm2J5h~Wlo(j#;uPvarYgUbR%riB&JdtdR9 z#zC-0bwRGt{BX)I4CNGt7U`SjuG8m#Tbm!sX~mIrkVKG>n(nd9?`G?jRZo|!HjOHA z|6r?MFf?u$nzo-{tCezZ;kV4I)3r<6Y^8oDem>s^Q!ZnpddPmU-S7#`*;Wc1STk<_@8)Wi{icVT^wyY1H$jPu z$hL=~_WS0u^JcsEGY3xG@c#JePIF%2G>HD{6KWT~YB>x7=(vak28Y6xr76sGYu;ut z1X~7h*B$4f-x?02F>0$}+arFcHTv&2$hk`?uHngD(0P(bG z)2fx*zL?Fouk(UUTrFysD`8FZ$6Z0d2Aw5Z7ARaGYTg)1YmW44Uvk*@M1c;I7wz4Q zH(RnpUkd5#s0N!c4+ORdA4vAONp$8SdGQ+ek;*=t8~h8%*n0Qq9)Vw?2yTYOcX!#> z)`=_29nT-k>;B`k@apf24+xkzh;=CPrv6Gb>^(?$)rYZw4d}Q3e4h70{{)b@Wem!7 z1pxs2rFQi2o&-x*CqD9Pmf(H$ygZHoh!Azv08Z_IfxgOfH!LCDbZbYWA3li&kEB3h zbl|rTe(l=N26Lf8Y~RlT$mwQ^Lkb!u9oT7}V|{RX30e)xskqZ3i&i_BgoVRZ4b#~H zxgWy=?Eu`PL9P2=bz>d5uD)8++WqwmjEXfb*7SBV<P%}=y}KJD%vYk z-+r0OU(niZ4ZsFd5UuCyfb^+7cd!DcJT*=1o}sLD-e5_jpRCM`>4}@9z zW*&Ee)vo!SI^|2O+Mqk8Ptw@bwu3FJ^9qmYrr>qkfxq#-lv3vd;CXin6*`mlVj<)E zy_phNys@Akp1x|g=`=r(q&@=zzBURTeaA!V<8!!{nr(~V!9M$IDZ`@S^+zC#b!CP4 z!!i-w^z)y0L8iQ`Pi8;KKtE$S%^zN?Q@+TOuQeH`#@Lby{~d;4igq^%dqrKUK21ftaaUi<|zx=#%?`O=#$xZ%iWGv2 zyUk1n{EWX8;yL>KywI!TfzP$jIicGx2tg1Z@e#>N++@vX5hwltw95Jlo7KB*;C{h{ z+OLpsmPaZw;hhALaJ5bw{kGE4CJB!dgHo|_!54N@6zIi*rY`;1ohVuhe4(r61MQd8 zs()w!?t*`c@PYNp9u>q_Po9?-r3RL2eJ+{#0Vu_<$1Z^f)50WhYpV;C11@`0fH?&t zqtY_CB+OE}U!2X=@80$kr>`n9xxoRmyJ6i$B=!K@S#M;9=k#3&NE`J+hEm{tEm<+; zC@sScp;Xm)?XF6O`d;^2pyOg^qNer{D0CJln3Ub`206y$L0p&+`HkD$oNm#47Lu_@wa zL(qcE2$oiP*i{1&Ss)`hM~pV-Ma7#<3rSx_>**u6m3?^cZfrdKWQ7d5{4ce>tT}e# z<#4xS<5C=(XP;4TntLTEXUl#|(=|{J)F5d*1jG1dd_llbP&Z!(z>TX=Bq&{-&Gfm)JQpTUJ-{m zaWr4K%&@g|wBR5QGVj(;_Ks5Rr8V~j&9SA-<+@pPv~?_l_F7M9>nn9fbLklpM23TB z)fHCc01GM}@5NBi7o^$m-W8(o&CYA>?AB@`ctHf~**_ z^n3v@b#+R;g7I*Jr>2{wUG z&qX1}y$c4HJhYT%11YwbB3c`Qp!S``n|!9!$k@jS^Ss&hGqXYrDWPlb$Uw-^GRUNbsp@)& z%hKaadj!E+gaz3AB@vu#EzVcnQ`_#Sa3=z+`#rt<8>GU&jSogpq_#k;pk~PZL0!{5 z#E)L2zJ%bfY_B1h^j3;A$$ed4;)_2XR+n#wkAS^HISvMhyJ12b?xLxAC(pqvRN%)ypp>zlzdvPowLNi7RidAa9cYrhLZho~QoZnF__bBMK~kdLC;l#? zAS*T7nx=1&x;B87fV0jfyTEt05HkM4aEOcHRE+JjvHay&u8p|P7%8C7tPse}1I36ytR zj){?fIYUL+rj2Z2-Y>XRi|MIQbuf-Y3>T>ctt17ZEUu*Z0ADviUAq3&Ll!WgP1loj zHUy~TWQd(CL9>j*2llP>Q+VFOpK;MqB} zfbd5&yr5{3f?DsKE*t8k4f#qD+{kf}B1EBG?UY6Lg#7A^g7v;qH|4nKjwMcib2^!S z$HDY1%N(J23NOD@r9uN4@RRzo>1nRk88Aykt;ag^QrzjsW?rd^b$N8!#VeeGVqx+U zIS$sm&f>(uj02B#st{Ps_k*qgK3}<+X~B8kyYH7&$S62-3!m5vGZK+p$L+A-*9KKR zlnJ*3s(xn=B%;X-?~_~Ab<2Ento=Rm9QSt)I=3v~;`%duAu(qjT&l+%#MgpZmQ{bI zshgn1rIW%lYZn1@(#tvxPtZ75KdbX1VI0tBqRl;{P2xnx?tq zT7G2wAkK#tXnqc<+iJQ7A$vJ~O?KkXF(3JdoNt$=lEd-$VND)~S(Omh`4iKv@n_FP z&c8;i(_1ovuki*z)+-3C7KN7i1D8;sWcFY{co#}@SuRDy>)9WBQu^`)g2gjtg@mLK zo&paMWJB&C?|Phc-LMVz#lGIoJvtkZ%jQV!iu0B1hG^4Xjc7biNqn0UDc7q$9wq5t*HAj#6U;C! z5IEU+Sl}Z)GA1BB6(gPiKhKLjJkX8Iuq$a)c{0UXz2-fII34{%x4@u_mYdf>fD6oK_!b;F5{Vay|{XIrl>zX?It zYJEvRZhAb1AHJQ0c5H*SD zBk>lTvt}etp0i3e;T{Rp7qAMb=#Kxghel_)LATZxB)M7g^7F21mVCD|VQ7Mn5W08! zm*)tP$nZRyz|xRRZf^WJmCHG)cAGh)fwj#6hNydF+>a#V-Q#2+q?-z*cd$my3C?;| zLNR!Db!L-;AiisFb=rCsD^#PD`Wp$7GREJ!?!uR{M&bsG-&-oYZxL&{SPu476Jbx5 zuN(aSC-d!E6jj&Z{EDf8f%ED4UvY94nxZdHBHrLUNSo4Jiu6`5(LopJ_vSfjuQq9) z0Nj-chVrnys52D|762lM%k2VMoGBz`HFHeLko;bhHDzdCaZo~B-d#^q@vkmqBit55 ze*%iAYr%=+esOb|C8fYox1*LoRw!(M>05?VK9Wk1tOVkZjymIB%{muQ=sAlhDL}E* z4yo~&sSomQtA-yulo28UOpVo1$QT&{z+LhSKA`^g@011KQ&Kg}D5b4Hak|8yQ7;WaPB%g9> zXa#PlWR*^;qoR~`=2$F>Lui4dE~W`D-(JV~Ru$8-Mcc1>b9hqHaOYWdFNu?yQ=vr6>WFPQ`#;6cEB zm93s8cJJskEiRoVj=>i7HX}aWDOfE@U-Fjc6~1!UAg@cSR9Jg}(US%Q3L%>!2G)lv zdVclek9T9BMaD;4Xt=MQil)EH!S+_ApZRKf!rFf0YuY^9@hviYf$F{zv#Ff+-(8w= zITH#|NiyOz+7>Iti`?g>VaxZl+C#1#{wtyt4GeCtql>!Ww%w@hGmC)G0s9S94CnEP zjb6&|8b-iAQI`oAM(;~rKz^pXv(WZwG{qJz?hD(MSC6gIy-MK38{QEO<)nS^xl$XK zd(>YqOF_{OW_mu#5l*~T(~8pd7vw(^ip&q9yY% zSAylyyL+#ZCm=0w#Sl`+vLEE!%CpaJQfD1Z#%Ff<&|!J764O(z!+>Ku-ssB)8x>)c zA71XRxiE_Kzaci?k*e{tyQ<-~05u%JY46@OtZ~X#ND-xwqZX(RX)@>iOP9fcF0rZ; zFadd{epCG;%nHV~n=w|a1K%JP@f^K|O<5(}Dh+cD10`v4w_R76(`OSqLt+$hMXlB# zP2gGlTK9-hv2n=Ix!v`HO-sE@6~xS{1Ol<_(@!xT%%(;3C{h!LMMH=pI569k@Ft@}Gklt~$ln1Q7-y-+p)k<6(SxEb&o=$%Vz_l7L<2w+JX^59&0Oz?5+F_@`nE`o~}z_vH5^Hr$0x# zSRGYBr=x!B7q94`P7V&ku69J{I1>|iD;o-B8(43%0FcIbj(+EuFZay~9WbU_^}Ztg z#Sul0w7^7_YS}2mep>}r&Li6w%QuMsjT_%BX`a+(XI6cFX?G+a*sFqcBS1M%2O^>n z(8weO_b`m@Iq)(&x8_k$8o~kx=bwo~Iu1>0TTAC`Xdvoo9lrZGk+Q{r+^->G7XswE zj(6^9bhh=zR+TuHFV`c^7+%zF&mdln5ox(}eCzojG-g^L&$`{8zIJ;wRU`avGxa## zz6e;jo!~c7Y2F*AYX6XcIQg6p%X%L5P6$I@tw@}2^W_~mC!ilan4CQ<_6oftFMXA*DU8w!sVyv?x@pk7Hl>U(pudfen7pRS;homd(40ZM*XUHE zdEu&*ir?t|!$N8(jTZkT_@q?N-fH>`aynzsVg74)rg*JTo|(j*PHELjZXit|$#Jm9 zTbJta+eW1#<^26Hw$a~Z18FL%vB6d;!^Io%Mq< zZu_3P_)~-)_IfoN3)wc>9WAt6)dj;|9MH#K)3*GFL{qE5u!6Gm#%{iF3bIBm!V(^j z^T(9SScwWk_N=YZ{L|2zN2?+I#qh?+wnpsU(G(;TPszJotg|RT+lR%9vCanDoO(kd zDl^!^c3+>34WraPFZ`^;43VW`^=;2 z^r_GHLna^N_5@kx09d)~I3_`B{<~_Oi;qrONSm|!93F*JCp&=!0f~ar8c`BSrs5&? zpx%WrhXi?o5t`RT+OSz3GX_eGGVGX3-?h-#Qhq$3FS}AlOnJnZoxwf+Sy^8$Zg<4N zc-YZUaS9Cu6jGwA&xrjLBh~`kVJ%4Fc`^khZs~}O4|kL?QwFe=+lspYfOw2qAc^4 z^YadaxuJTZW6u;V|kFc=# z#)p6(0Tcj%$p+3)}76}fBdGe-QG25o9m0+=C{|puf)x{ zTLI9m5HVOs*l??ne~zRfM*MKvtvIfnBLYjJFll_u|*pliVDBGYPe)EC8&xVJ+M@<(IAJ(I9Q)tyAm%CPr zCNaR*N$ds3K=;#FVB+v7-Vc@F%vCzTM0YZv&;NpLTD+4?YB_!XZ46en3fJwn$sc`t zWjXnqMBRa0h&04JE|$fqD&|fGNQdw2YtmkKRA)9y33~{4(pvBx&JR62%agpz1qq!& za$=k!>6%~-p*#_yrHy4w3J?fELP!=LE}3lkYYxw4$jzqf2a(#jVU?Gx__%tQIsHS| zr?)_h7EC6(fx9WAIU+#$Q^NJ(1SBZ*iafGdvAPc)3oXtvZRgF{21AvYIUUx2qeTL< z%zzqq+*r@nlwFJ55TRnIRE8hAVb99t3S_HVr`Vg|M4Kg!-Z&W5gdM7)r&nG3=@v8s z-UtGo^o%6jpSn#KTe93uv!S*8TU;)!tvqt$bZ4k55i*8={`X;eIAdz7= zxO*ksR`DRU75Fb-NkmsOtvZ(mu|js~uvtr&u=wCNucv$rOH+GQNRL4L)SA4Q^ zWF(zx&qMoaARipqKgC-GoA`J1L`tHmd5OT_fpnqaL<1y=d``(GkbgiGA=^MINi{Ip zVbR((65eoo zx)%uQ%m)kUuYLqC9^~)rUTeyIcfAYSEt6IPlpbY7w5!RjD&4?HZ?E8w&~6nJ!t;2F z(wgEc8rg(7M$L+s41+xuT3NF7%6{4DGPfp~X$iMMBCi(ZV+ntaNOC zTf%ayZ?%#kvhs>du=nrA9=J6bWk?d?Bh)694y1k~xjP+WJ{#Kg0N7!jB9Px)HH`!D zxfe42fJ3R3^Di2r#OIRKVcE<(spF%mAFkf7BSn*+z*TKxCDdUHE3wfrinL= znPZa6A9e&eNN|wAt%eh@vSot6vM;hWT8KtH3eXHir)XNdG6*cw;dpo%Q{i;?NUERtIX<}20I;{Ijs-6T|m(WliE=lRH7Y5r;n@1zw40_;S z`YKBq7h$QhG;OuLY)5FOx_#7GxA%S&$ZymW7e~ZKQ{wbWyhX3J&e4xM*28P-6xm`UG?g)o7%Qz?I)uwm7h+dBwWhm zW-3R;+Dh5J>9|{xl0ZnC=$)tIsQ`R?F?c-=7P#_ zWKqWQFa~%yZ6GkA)#dE4p3ZP#Y-`pg-`N;XCO_Tqh+#KeQ^#mvzFyNAh+b~t)D=Fb zO(_A6T~E);%G%0auz{JyyHEqOBJ7NgpXfD~zSS@+?_E(n?>vUro+M~LA?khyUmwi6 z-q=mTKT*Z~8>Nw4tA75%@U`AlH-UND^^g{G?(Ly?crKjYjAE)g`z=c2>n?ORjb5;? zo9gbNQ?}JxR#RV!U9!_z)syviozCh!M?KZM==i$2{0~~AT4X$TzzmQt)rL0SkF3+uNIjF6I_|`L$c*th%7i3}^#4U&{Ye=bv+hh5r@S>Vt6 z{#unwr?$Q3=4-)XiQtPgxbHHAlY7ZPxVenQ4HE;N3=0;azK5c(-PyXd695JkEJVDy zKg`^Iwh@ES2TkC(hR4Y57P9}ebNuR~FKIh3;pfk?l9*W$+&M{S=Jf4*J54m({$2?B zN^ywc^rD(5u6hdq7QhDGp_1ANL9ip}&j7?H-E1{t zGS8C`9EKWZs%=dq;~zP|7SU&da!JcOA8}K^j$HQV+>vlG>MmQcfHK1&e6}0ANcdty zuVfj~EIr}e_ptMQE}OCf5Ceu64+L?(JwdQymAg3wf71$%%)x*&FeA5f-A^f!_W(XjIHc=|p5YmYiHC#b1|aKm)c2^B{%0a4O;djN<5@}H0}SfO3Oyd~)m zQNiAaKZn~946>++RG*}!wsxbQ-s0ToYqHP0p;2{YV@Y#m+Sg0tX-QgEu#(^vuBW`A%mXd_sA@$z}hS0AU zAO9fT5C^#qKvrYRG?-(fNo0%<_@|MSs6aU4Y#}?uPd@X37|__vWU~Rf`6dVHiW52I zga%Ni%ml+Ojph~bU8_HQHed)5hY-|edZ>1ba76BBbcFrZD|q9?d?S##x41RjL zi;ePdrwm*?^kmwmj8i@2kaLL#{_^ZvwJV;pk{^O~QP?#eDq(Q6^(^4QS45euAhC4B z@))MHubAE=5)lX_=^N~F(agZ~&jcO=?(+c!v;nPsCm4ioELXPCPVBU6kUQPxI-`|e zwKh=0p7s`edXQPYRfDLb#jD7ESdH-h#ilaV@z(E;Qh{hJVLZmnzXg@oMLl=sv%oO3 z;kaU_;vR$W0m4b%f0v|Uw7v>9SRpe>0!OU}{1RZ9O>g0yt>@uTLuw*v(VF0ayt%-> z{xn@lGH4GHcj6rfX1*&t0<;Ad7J$qG@WIm<2Y@8X1cLx_l8{vB3V3K5&zTp^Wed)2O8&eIs=qJ{sH-Jjm!P zCJcb9qmMwLq*V?eHO^3F#qxw;=flH~j{$5JqHvhT=_V3HhDe}zaRG;o!nx(+ewIzw zw+EkPtl*(w=!*BiUj%#0xMh%<%m%^#x`l8hoOv&WkN~c3^;e)GnIv#I-B^Q@!9mAf z_RsG-gWmP+?6lndi+d%~+M(~qbIb>={6d&Gl>s5oUSY3qWvatRI%!2Ng>$LXYq~Hj zj==X4u}86QtLrz?4aXcusJ>)+fc9fDtZCb%uazLec|g;kmzk4ASTZOlhTWnfK>xctY2W{;6APLzqEyR+kJ8@7KP zowaH}xH&RXVKfnKL|>XpbyQE;Q@m^S3jVsc8N8<}JBN2@97=g!NZERf_r=Zu)Yx=} zPVhB#G)+J&74q5ODWS2LWPh%iJiwd1MTI`hk*(zM2g6i57AZqgh!>vPxK;{8NH}gt zw>NU(giCy`-i%>AYigiOZ=I02i-3m3q$Pc$3$`q5BD1d0MS4oLyb_)L31tc<8+zAO z3RaDvl8+%fkvgO7q#O7(P$o_4107Ug87L%N2~a}+JSA0V4}@kCGg zyOxQ`#}|G$nMNG2I6A+s&AeIX0beiLME%5#@O1GKC2pBH8Jowuwg4b<5_8hglwv(> zf=rna>~a+=)wVXdqqqGS*K}IBL#or%-p}OdBOb-GM2?I=S%a3Zb#VNp(q*)&^^x`I zSaSvMtmQ48wNUzCfQil=Deyihs|H7fGW+d@MK`@5d+&sJ8HzUfv!dNC!7!ik*Fx(6{93r1g+m1hYIBS%$1; z!3g&P>fm*9-Q2nDJ1g``l;LknE5%N3O@LnvN|6yP`-jR|U9&RTNR1 z_pZQjh!nEgx=wC=b!|X>r4oJ(sBFRt3I3l%Pq5-wmUV`9Ig2^7`gP!`n~V-}bE zf|*Ca&yF1`|Izfa%zpLZ&xi`qB+e5H!NT88Ar{>~^G8fpRqzky6&C$-@<`#Ip;O=2 zr^5eED@|KX8bgUn!$ocq|40ArY)*Jj`w)B;`zal7skkgqpz}1)=|xFU#|1-AMemWk z^r44$bOGFYazEEBM{#hw;rxlTHpzC>@90-cQI7GlV$(>{-g+J1TWrhR%9tbND6gc% z_B-} zRRrq(n5-}@D_qZxsB%YEqyhakf{u7mq;E_7(FtGC^qN=;TMk;t592OMs6BQ+wbwv# zM|7R%mHvMN{K_1bPW@j}QuUXVWce>iNjXy&cY7ymRToQ}{{i`0+P_pJn%`WV0c$@S z;HZ*aj&5-)^w{jBOv0LA^>m?pD9r*Rd2@N93(wB>_X}_LKdY$-<~ue}B!UQ$yX=R% zEy{*BVf*R3-|lv}Yli=n#LNh?gx_ev z#V$uv!Vz)b<#CMi#B&H8lD=c{fFMFId;^~Vi4+mP++6Sfdn^M6VgGiC_2Lj2!=K;h zpZyOP`Syn{l#h##hl??gBmt%ELxT>ZuP-H-jhDH3HQNv~pCf%g6*+(dJcK5dBb8P> zf&D)Tp!g^(QU~6{jJBLYbd+ey(Ir7gOn^cWeUkJjgM!MjyvAY(;X~lqT>4@EQ5OlY;^kyQ+;kNaC-X28}<&h_D1hIMa)X zKC%8h0ySt>IJo&g7D`V^AUJ9#fn=x5u7-lBj|ZRC0Pq&vj6F}G~HHv4+`}=9Tg9X8^_lv@;jaKJSFqB zvH^!tkN_qWV5B(%htH4uYvg##0QU+ps{A7qIu8Ap#5l(NAosU(;qUJj@EiX*o})(b zE2!(=Mo=e>CJTm!ywE~zSIf>SZg!iUv1{Y$Z74+0&IdTYv@gby;>`qfW~&V?P1%21VBXZY!MTE|a>Gh;7KZ z95S7;(GU$rjul3&f`yW9W9hMJ&Q|xb$%N%DdD-g8dcV(thZ--)r2mHDNEF!)pK~~@T=TE-uy3f^ zzl(e#q?YVIO1#k{WGoti1m-XiKYFL9{Cnui0eKP_ zO&I1b)SkR?iHXm$^VLDn@iwcxS<1=QA?cz(s0yx_i4n!r1ml(f>ziLF{+ zG=d##dXL^0(zaSQQA-btTRoRBVdblL+~un3S*rX&9ZiW2zA59`t&^0$spO8CJI%X= zMn4HDUc#+RfLLXREgjpfw)r0O#y0UsXykV&&O|{R!^_o%UFxWpwdrGa<7Q;{4KD*h zw`h9t2)qXsYRlNseFyU>9r~5B>ZJC}a242i@_`6dB(LWZh!;UUpwI+lOuWpm8Oh>Rz1dZ&(6tOe8Aah`IX%*qQ>NyA$sts_yRb;zrI~vaHwJ%bQdViZ> zzOpuJ`P&7DHmj;;XY5nbcSAi>>sCr%m)X3$jF`)EBWz{@()ks$I9f&|LKxN{`2>v3 z$~Y{hIgavvQ7$D2)*(qaQ6XC@`3auO??KbU4p87-GbN-s`9*@YkpUm|Pt*%VIPt6K zEB4sadvC#aw%M}=($di6vbkbE?UWoA?{t6jS#>xR|Qo)mMuBn>KJSM<-=W2v{M#8 z04pUNW!5Q*U^7dRwH~e&S}pU-AwQP$+IK@Y&LdWB^()Y_s5>3NsUs@Ba|FJHo0w|! z$By{RdTp1^>NG^ifiXjG^Q7AH)C;;`)Fx&TyJsa}aZk#Gg5ZQ+1|*H&ro2@mg#|WO z>wR;OsV?g!GJC$d91zY=IgPNdUXDWR>v_r>5D!czb9J`C6><}e6IrtiLAv;%9oL-N z-CWzkIj&&{=IQW+@eMwXUqJwM6nk%M0CI9&OyGRzZCY^X2M1so+XH(JH_k=8utH=DP4P+E+k-xf%4L3tBvwIArtd zdv<5g4spMI@e~^LSufBKV8e8cvJu)zes-iBDQD;$w+$=U?>G0PEZrikvBE9UvpxRP z=0fK4^peh%YzGe=?iL}_Sa38?Zx2jUvQAnnDsaS>{`WKIFo4DL1&gsq z9U&Kx#@f$HFTmE%+(@r8^2Xa-$y|r@UjIEu^4;xqL7nVGwv~3CvTV}Lcj


    2uWRuxjMdJnwrQ-IfY?fTc3>eGk+uDM;!X${M#htumrbN5Bh(Fsj! z>5k1iQrpSY#(7%%r|{NW{|mVKvAsLB{;zRaicQ;FVU)+~ToaS~19Qw2=6EwAqYata zp4?1tCL)99^+zbojOJgPGqe4vncn-Bj4h*%a{ZE5Pc6VE)a80n_xq8v_s#$3ib_nu z4U-E501)t7aQEL@q8xr}xed)tY0aDrZB74Q$WoH3>s-iy5_y zUbtynhA&sLy7^YU{rOV6Cz$kFnW~JtB{^Wj_C0i z)TFXTar4`(?VF2SKj9lm4!s;6@%&NY;-kXZY2&J3fI=iktxFq=vXtZXVawA=vV+gb zJf8x|1q?5?1zt>%2nFKUu&8MRw`z$wg9CX^0_eq|(h2SiMZmE5z5|WQLzP5IIl2@H zkJo0?MAZbH9f})1ZrU`6ST*jDWL9>}7R52M6c0~1!Z)o}f{ZR>>7VA@loWasW=e;x zSZRTZoB%m(F@K32kovZwK6f)KhZ(&z6Eb5rSSR=7qf8X1duj)r%iMi}SiF&c%Sv=C zo%XG+sToOC{;8K6=8QPO(~eGqN`|lYD9wlsrI-u>va>rd-Glp}PT^JLXmcIjUZA}T z0!HwJDs5|HAF2%)e-Y(l_Dr#8_7|n}x>l~MnE)sL1uitl5_tuqxAHfCoCrktH2j@I z9=k}4IzHZ6U={uYr#T|;xHWde?gjAa*gHPI;#HZSQBy0#N6w@kyes80 zD_k1HV7?<4$^aX-xsVX<;ITWvm{Qb-Q~$n$lKm-eWF{SzVO% zD=}$CIu1*6ip!j`^DWlERyNs}j!sQmZq9}24OkX;n!S6Lyt}S$n``HZQr|28X`|rU zp$cxvTt-irjV-`Jo#}&O&sUVcL z^86o|uBq$1*^cVFTG!vWRLg6ekXlomGpsb0nzLW6y{!9s+5I>SmqzRB*cy{T3 z(*YnMm3lNDxp?Ka!rUsk=eMVagQJ6B&aboc2=;TS-;f7>v(xw;7S&egJ#7s(Kws8F z(2k?C5!>p&WduB-jC#Q#?TG}UzKZuSs&z^qXuy{B)}+jz({tfLL>kesmw@~!SuEs4OT?DlH0Lu}^y`jRCPdX~ zO8rs)c~$;cCmfnG(5aV3Y)m@6e6^?FnA6MVGgof|c5~X|YKe8JMqZdXy_z!*@U_d9 z4L+ecMqaohdcD&~FVQOQJWSAQa$V%{lrxq(c+8bdaN#LAjZiQqY#w{$2`fa?cR`>! zZfHO$iaq@hq&>=lJFGq2R$|V8)`=dX$vn)1GR?jabw;VOedH}I5|Wje78joq`t%s*!d@eqf8dNZZRtwxYV1+ghC%p7JYm&IIMWxX;3D4_;HKJrW+;U#&s77*(QE7|^9vnne z(1)f8eXB4s-*3F_qJi&ASz0}5bM}TseIi)p+AuLK6>XO7zsr*)0*W1O|B`3NOd^Y;ARi5I-qP;i4Om<8LU$^jivqtQv;+Cu(DrX;$ltO_9PHNJ?MWln|M{^x=I!{q-U@$y05MLIL7HSoY z!?1h%H{m;J!#faoIl_879o>A>idp1o#l(!ag@s81HTSeUfdyv1K2)S-q#0CICDICw zTNO_cv5X3!PU{vLAJ@nEmI!l9IkyH^>`NS4#y=xM#T>auZV}A9V$1OyT{7l26dIXl z9w^F)j7GEk$jTB`ozV=lwswI`oB;u#yc~32H~mWvqhuCb=NoWo(y6`pWV&~^*oSw4 z)OX!IFakxYL35EjxVEP-202ZLATm4TMv42n(PEAkj252&wnVT^5*P&%>EbMln-XS8 zU|k%2JzhJdfH6>)-aIe?@VQ0-(?%xOUsZT=$XDjy>j;vD&o{D@evDa(MnC_UQ`)L+ zrtCk+^I|tgy?|7tpYLzS4|ZdP=zdw{bUOyVX{Z^kF$wrq4`g^Rh5Sm$vVDn((@0!_ ze8`GJ2KNFdQuq$b# zh{4`bkk{+gD^K9mC^LiktM&|_-bO>IPq1swL)>!@By{lKaYrVqpW04? z*38WYS7v<0^{|@t{-}-~c?QTR20P!~UuL`E4#4Y$shW9VJt@q<2ezk0AW#~xpT^8B zW%pBwH=&AId|02oj3=MGnF{1h*hML-Is3FWipq3{)e16(4UqGwiIk^bFp!);t1%$y z5F@;VG}wNli)3JbLu5q%GGMdSmyY$D&^2*qtH!0z@Anp*j&_rx*GSwXmb;VjhPn?< zBCKTPLMGIT*iv>*U%3wkkL)^rS7<)^2W_(6bR+5iaJN0C zF^w0qI&s;Gls{cMZ}uW|6fSM_Q`iGmoG598GF{xahi)3EVFb$neWy7r?{5{c)7xqG(+cWefH?3T;j1 zJ{#>enLMN%BZtn7Bi1IIbAuhsu0_Q6S+#i8s7u7MMYAE;J>LK;D@ZcN!5Ki6qkHo~ zCZRJ8#U&mA;G3M^UB?$PqMWwoTFCkfrFG*>N;?Y zt1oeESg?sv6+Ru6lGz!fOxRU$r0gs@=^m#^6!gcWkuYo6XcN4Tq-PgI1((s0)kt!p zrJ`+XOe%TJdp8^*X;ZpQeo#1s_oQIM+kI?^OirsaU%UNZIlVW%9avD7`;cykWqK`dR<0Zdo-XO*$2 zXf)qMJ2Tnt@c#p)wj;Mst3Uw&>~H`8SpR=QDHkWx|Jfa`esRTo(uOB+nOPJS1OFWHU2i^6oTRxY1KX>8bg^$ zhP<}~F-2sF5Q}EsoQYT%vG5Qx1xltU^rg!Eea*);aE}Z_UNJ26-z8Aa_jBjl`O_O{ zrzgbxy5->()}oB_MkM;oT=`6d(m!0_VIGAR$CcEX!R+ zudUGxZ#P2*Cp2DS$&MR}o;{Tq%4N94C=3wRuR_Q20T}e#5}^qrT@`bmwj#MHkrRl4u$Xy9bV5eY+i2Q2QQ`Lzb}qk(WN0H5AwM75 zvd3e{d*35Axq1GG^0#4;Ky$oc7cNG0%JND8&OUM_{Wu0Wp$J;E_DB&wPLrk;@BTM~ z1?G@YnFPSFOF=(0BdP^~$d1Cru3Up(w*G*+? zS@{4VV5A}v!@5jC{U#Mr<2C|E#L?!cVbus^W{I*>^ye8Pc7|5Ds2Jd)%+JXpXc;=x z^L8QXv^=rtDp;Rd?5-}H$CzRW?Z#54Vx1j++H%r53Up+k9Ax{e6^OR$LOxI;f_Wg*2De=CzPm zii$0g))VExi#R1Umf^WU(adpjC>X z^-qmiXJSSsvPA=ZN4u_`!BW^aH5-t3Wm@-8EZ^iMC=d6cijHo*Ex0�zO8VA+a%M zi1zdkzGzb&RkbTpopZpJ;cS}zcxC`NA32P3>QBYmf2n~9;TCjlqc)ZUrnr5RXcxnY zP|2Q`rJEII4N!Q@Q2;3uNNeLAGxMO$BBR6qV&W-C0ZpioR$|3Wy^E>Uv3nt+lzPlD zf2xsutZgN6@<>(9HHXRgO_*3LWWITuOuRiKN~ZkpMP!NTQINICQ1lJTK|>^`6A^7d z9PMv2LQ{3VHFT?;5@mP<_ZWcHkHUjdY8C8oBjra(Cf@)w)?l=j<0U;-COz6GVUhXl zW8CWiA4HiiW{D8$ZOv>LtdF70(r~(C*4b1p0PZ(Vrd?)cPp0lah7c4%6lL7dS_+Q1 ztcK3!5i^{8QOB}Pv2%3yt1s5jXSFrJPfhLwNNq%6(X(u=YzvOf6$j<@8C#%;u3NONT5kt*)-Ow+Wawrg?gHbk>0 zZ&J=xK28re>d^4ea#uN$G#`aWr-_YFaHG}{m@QqZq$(% zAVazCXAmVN#k@*EWvMss1I^_|xQh1e3ilo#7(IUq2>D@ID5<(Bxi-nALCwT{h}r2e^f-#pu&`Z;!VQ$OHb zdoqGQ3g+3^o}~xr%-qM-2d&9l4)059_elm9|MdU&BF$iqyA}rq05JHQsPo_6y8iDX z?dtr0e*PNT_8Sr?ey8f$t|hiapxh*u?kPZ3Ad6W@0+3f$d6ffvf3Oc5%tjHK&;56u z&7uPUEU|3ys8A^fayeh7*uyG*R<2v$e;(=fe1Fc7yz*x0`J4Nzzy@~5Y6vV|Hti0F z`Ktx>BUHx7QY$0^_X-aK*xAUqVFTA~PUd2LE5rnUxG=ILosdYBtR9JxJ!ho8+N@By zI4&X(g<7PbWq>0j#9`r;OEHSVp$6U43z5KxF{byc(jz58>)Az1Bw7D{@_!D|lh-{5 z=)E7}Y+wv7SaWl(3EL$SK{@UVJC6){C;LbOJ=$Gc*I!n(>%mtLGEM_Ezbtk z4r{kQ{2EAaZbOyU9_-!I2Z4j90nsOqCBmrYW~w+J*J?dv;-x^35n=Ud;|dOVJ}bv# zmcj1Z`#Vz_>JhN0$7O~pvGgwm3xBL^@xNQDKPQ5T=6s+m`S!yG%WVPgcQk74QwZt{ zV6tf+lPB*PBguoI!y}&9PGS;nUNPs_UJ5v7_L(9_MO5 z=6#{R?BKb~^XBGi(YxrfNB{;!EMyTAHN|Bp8KVA{7;zz-L)Xw#!Qm(qq>q;zhxEoY zokLk>plXo|E_s*`LuXsdF5*7;!s>@MEQWq=sHj$)P7I0}wMVy5rnpGG)n=!>{3}7 z5;-KsRL(G(Xbbo)U8TL<<4d&}lpCmzbwba1c8lH&bn+3&Zv;1i#Tz_FLdzKK`prNB zuRNRWcC@wLL?PFU>J`c6ikZ4^+}C*?Og&~ITS=$Qw|JHL6QFV=H^fci4Oz&9D9Sho zFm|o0Ouy#7L60!s37z@@fZ=6s(d#Lj zeC3p_Gm}IO%Xk#WWK4e=NsRV)m?UOOWGNtTC-r)Mula@+=8M)a5wL4ZPSZ}|^*T)G z?mC|7Qj)HE&p>4laFN1;L%EV!MjjbqNP!|tZH||!$dP$sb#;IZHb^L2Z7$tc0I{?S zl+VM33JFJIEG*%@6aV}%gFOdU;pqtJI6wr4@gytsRVSX#l8`IISPT|fEZ;*}<#I&@ z)aT4fx4@ic3U`_3rwzBoiR59t8UL4&zMEf2nTj1szO@|BC(_2<|HUZ!mrjUh0w-U| zSKz4FHucJmdlHe93`ap2k?19gxfX8{B8%Im$=l|@UFh|Qi{`ObtV+YH{2mHPUH|W= za)D8X^?WE>c{PQS)lSmU72xs_LWPkK5i`=!9(8Ca_3{zbNcmtCke4%a>ELuS&Vr;T zfW_b`Rpy}E;6XA?gYkFh&D6}cVtZq+GCo0@s306L3-hqc5a}v;R6nmXo4flUd}}{u zjfM>*q}3GJZ}A{=Vqg|Bhk!&$L>S`_9cJorfU;f2snpTfoO+&dc1@=`4et>`LVvQw z9vWj&q6$d94YMbCisS1|u3(YEK1SbZe^S$OR-R9HJ2YdLeuy=8R}Q75Kt%e0JKM=* z5wJETkpCoNOU^W=(^ywG)vhG_haK@w5I7+Rju+ZwDyb z{p0I;`8~dykr2L^(W&u?s^6CPnJquxZ!qM~;ioJ4?S&`i=HFXhyUjRbL$E{a zR#!<;_~)*F$LQVPiN&6{bOosW)_Y^5*5sLQm|!nIkI6 zpR_LMEY<+gQ=FW8DNq6xUsB57u8THGYPVt6Ewc*3L2(k;TLpfn52ofATc&D3F!RHUyzmN+ zXue;V{Secp7jwqjxC2jrG{YfTT@%AI4#aOJbpD1E7m)68nLOI?DX}0n)PZ*}?>`}l zDK9dtr)!O9^c z0ZWD9v3~&RxX=me<(yGH)nCHw75G7H1%U&QENI>C~4~ zjX-;8_w=oNa{)r1ZhoVsgRjG1!2u?T)>46MvF~r5lIRiSv+X-PPsbGZ@$WZQ0n(~O z>KV70Fk3~^1gj>~PVV$<;QS?C)!IbKS|~Bj_JDJuTXHwB}i-71#}LmV859jdv)h%9w&>|H7*W z{abi#yjcnhZZ^1MwvDH8MUuyiHVB#y^O~hF|E95>gVpgp-^+lsrm7#UKFpQ@Wn}Hq zaCS!lTqh}*233T2qx@H=18zH#of35B3E@*d=io&9Gf$`+HkQ7?{Qyf_ir(5R!K7nV z+V~yaCIKcvM!Vg7e(f7*_AXgEf|tdfZ&e?B+{ZAR3ryEeJlli2KqRg(KXCG;=!4 zl|+0D&}Q_h22(i=q2zg0Fw`R6RY<^bHiEkgAi6;rlmq~y8OIN!1-^g^@aOlBW{Me^trTF$x%@Y7-Q&YTM%NHN1M)Xr8c>FoxJJtS4uk`w%LVk zD4z|nPFn(|JwEGff^w*2iKr_&U8eLlwNocYx^HL8&SIqbv;aBfv0rkVZf=F-DG0xR zw;7-C137Yl*llhfgj2ydSwywWb{XrFK_tMsQwwm7WMT<8dNrybxJ{tT4+ug?QHNzxCX!)g zGV^guuwgg~o8TX!%s-!) z&X}2Q^#&V=$p0X)YBbjK9`n+hjggl&hd=vS3Slsr@<4=)W^uiu{$Uf8>r{GT2Ig^GYeqlrK5f%g{Jd2$j*@=dTy> zx<>@?Ah&fty63donX7>rGi5+;OU=`N8F>8MWEim=={-XESQvjKHBOMp*i0(PGe~UQ zyEB@aVGh!o8ysyE=ChT=`a#RPq6Qf78tAl$1&VLPE*_ywK zb_$1KcbVY9rL2TjXC-~FbF-}p``Ew$$G9IcB734BnOyQ1zpM|_^GL$JUJN>vseWLB z(J}<+{|hxt6xY95#G_A;|pUw8fXmlBhb%QDz4i+OK1{ZW{uJ+Y>s9 z*Gy4m&nAcr4V5Y2EN=$QEcISacg%)?#H>m-k?vD%^wz<12|67qCwa!v+qWL#Ga1&u z{NCa=f1B5$`TeuqX-P-HYKXq5Yrs_>d7I_g>CA!&2QD-p`mK&gTn|8e8*p0MLYgo) zH3p{tXve;X(7t)3sj+2aRm*;D0jz-Pz;eCX$=SEgk#}&YasAM2{)cv+kb@=**N)p= zCuFbOMy>tIvJbu%h_o=NQL~UAWBb{T-h19LBuw=*1%CV^?D~fI-619rGbc3`1)hCi zn1zQXD`>~H^Jo2yHk~!X+_Y(Q8{I$kh@p;7Z+ zFM2dpU{NLH3~+6L?wa8G<*lbc zE`9v&B&qWt`po!9Q?vuy8FYK=r=7Ca&tKff0e-wrhO#XW@{Td+iM0}khC0$N#mS() z>}X2sFVqr+yJ&i<5E<2;TAVcvzl+~%^_+Vg$qlZ!z~|+ojLtX7?aiYNJQ&c-8i?E- z_1)7*a>J1^G$Q;gcSpMGWJ9Kz3HDT_$IVDbB)A9TcTDTm8d(LsI^VP0*zL#e5A-#u zPBL$Fi9ssqnBJa~BlMTt8oRs+Ev~MrFi8A$=X<2ZS-$k2%e0q+#IoDXw689s-e);l zjkvWwleb(kY3VnoA}#Jt>L4|Y6Naw4-vq@G`T9+=W$G$izf11bFU^NVpYB0q=UVMIMqaJsVKLwGFh1&J^-=z4vu9Wp>NA@YgWmTIt zY>8BZW^8hdK{d3hT!x9}4pRNGcHs9bz@OfOWEoftXQS&$5hBS9*i#nD45m^Z_;RuU z%zb|GyV~q2g46v{sC#`w15?!8Z%gw+x#NyxF5go>&Wr_mB$IE3Qj+c2&&%tz-pX%f ziRC}^HN~0`9Z$B8^`DYEgRZ%<_cJ6vIP=WB_?t;aE|nqXP*Yw()Tb`m(WCaAar%#_ z#<3bv2hQUEB-s@9tV--42A4!U`JHGYmtatPx6`n_C(8-H0+SRYmZ&-=b}0mtfs@jKtM^++>JrNzrOgEyDFZ-!Lp3PcxmD*z*`YVF>1R zai}CLQ5JNbAaVw}A*?!w!izGyn26?W#rN)4RZA@+;(O(Jgp1UM=CLxb_3WHS7en%dAHFs}#u} zS7X}-3R}h~uqCJdK9}3e!8LMhJSwe-g|Snc(`{9{)ypm}TKH8lX#w`onsPdMG%@M? zK6$b_In*&}1Af(rpJUpydR5toDYH-UqU_z#OP$3u27LK?KzPb#7%(|i&5I@BF;g;P zuQzvtY%5N>qwUWKXX>LGUTXi7y7JS$G(7nOZ zvFe}ct^29(q+^>V9?&u(>=M%UeR~)>VqKy)qBu~&S@e-Eik1moF+tY z+0MK_@70GB5PcMCf@y0zJ~8O_A&izyAt|`F}3|B2`@llQ`ohx$1Gi|D_ZW~bWYG@C0MV_d3ufoMk>L*c$+h58mlSeQ81i=W zn4aXQ7#s{=@{n9vfE=75B4XeA*}nvdO*63h2v+=n{sO(hk++aYBLmX19KUVt7r9WU zsX@0V&b#Ah5ph-Ld47k=AJT(jNc0Ub2~dpT!1IegegH_wJt-1UfSyl4<&iiF!H`}# z1W;TVRRai`aXv&MNHC2E%F1*9_Pv;cL(y%;V<04sywD#Jw8stj;I>B@5EqWY3Gy3X zCl*B#y^JF+y6>=)A+(muji`j%M6BMEX4 zf7TRTZ^iX)9G*w@S0zxh>M>rKEP80haq2A4lxB9yg#I!qIa0}uPE}?}kk}1Mlgj@* zJsN}*ph;Dlnn>Ha!4bi5vo!2NZ5>#VE^p^h)Aa*FA*D_5+__!;4v$&Iy}^bJ&}!Mk z($XOJ6Yv<&QWZb!2)i6qrsTI!Q%2!?UP?%RymD7rq(Q04w@eZ-pTQ9lzxH=V>Tu-J za6P%ozN=kS<5zt3inlm3d^NqAYV|V4qt-a1`XbHygJ@zDReM^Bwsgb_jFnEwX3dH> zxsaKB)F+G&)BkV*HY^?QWBa<;Tq+0aiM`ry{vZRmIsK}Jk&26$tPpIghqL=CeT$J} zO*#wmQrk(Di@N2t1KbXjcdmq_H51e`oAn}ubEEoO^`)5Cah>|XX*4OAUo^t)+8Tys z6thm}9Df)C9WKrPFizGJ0g23|ue6Swq9tLdvLVL{j2=e~mA`{qN`uCYkcAD=v*uR& zKdFb0Vc!gZ-fMUN`2fSSf6={Oz}^wkqOk`-;Ng$evpuC8GDwP*xZ=P9l z?v9(W7wn&-QnK{0YDUbUC{mXIuw$`|fOXT#FDc6dCm}>d=JmRF(*zqdFbGy*9W@Fl zj6DN|%gi26Yt-E{r-;X~=@gQYQixjzv+E}d2;VKjM#Y;wfqHFuzUKf@1{RFd_v`%ClqgWj( zaD@MJqG6>ft`_{r+Qw*Ug+o!OJdm6MHQY^@+DZ0PL*_}zxKq$7von2QD2^z zZDp(;rfV&lrDoAok?YYBOIzS9nq*5bn@Yfqdg4PPyVV1iUV%F(cE2^4qD+W;3c-H3 zF1m{w?`S|^fE1iiBh;~}*{?ugY2@xb(F@$cNvUjPPc1?lKl0#J_7_d90fSADoq<$v}MRODcp@Q76~r1D1RGk!!Pn znyB+ra4tT5K{yW+&4D`vmYGPUy@zr$f&vJBX>p_g`&t31|| z_PShVq)=f^th{c|uJkcp%R?7re59KH)YTv*3aw|g=DbZxkE<}$p0ylaWmgRY-9$Cq z78leqXEmb)^;_e)j@iYPyF;YeJc;i$E!lN)@$mbT;Oc4pLMN|J6paQ-%c&7geWO58 zOxn2v*PP__h;IR`rF7@ja}AS-vCR~Bs9QDuwSm{@Rp4(2J6y1Rt%hQ+ho=g)UU%G=cmR zamCNgVe1I`z?bQPYQrlfo$yYT;y~V|W3_%8h7n+aG-W_h4wBi;3vwIn`elO2=k_p? zI4+l|JAzTBvQJOIT(+-w_vesgL}Ux7t8G%rl|^m75%!~rhlTd?@Bcl)6TIkC_=YPH z&=0Euit9W|t*4k)g-kv(_#U#$!Sh^@(5l9`xu^3M*<{+}Pe92J+N#?2Irqxi1>GCJ z9(v5Hlcb8~j~BKw2VefGmk{sd@4n{I?@_{joz7qW62oGl=dV&6>keQIOrQ#rEKpjY zjnE$6HXV3fQ4Eb=wla~B*yB1=MFXZNRvkvO*-wYYf2{!Zy#~&$Y{*vubhdp%4kYl& zo`7Y>EDiDvn>AVW21dD5OU2Qyu8vwbl*svndQDweK=<8nGiw4!`1K&uUMx4=wf9O5 z{TpUM<7b#RzMq8}_YM7}H!*+-YBe&Pr`a%;PFI|MNEZ*AIs>8EW@5Dn-pxkM?1AsG zdvST*vuq=B^*7P-o^c-*7a=rLmvt(iUQx*E(E?UUCF-r)$8W`C+n%P+YTRPY9kU0=T`p%J=vnbSv4V|($k)|1= zLO2fQ7DM>EY3{_47riCU<+!314WH?4Z!m#{e6VYwxiCFK^Y^pYTXHu>Bf?kX9N*+OkB&frDt_=!gizhZdKv0;;CtW35(e%?UE~OQ59HRT0;V{5OT>6C4 zP{hfhcdNrDkgSIR(HT#bP@C*BXa89d1tFRaeE9Xn5pA#eBeU(_Mj-3mpd}0cA!)!o zhSsQtsckKau=(@(Lf2~9SDqt0tX}^gv8kZ(e3R+B5e*I zr_HSzNo?aaRY*ly5Z6LzY9$Jr#|(ZN-emiI#(b6d^mqtk$shOMNA;RQQQx>Fw@pg> z^-wiYUaX)M%7epNDUW^Q$zD5^M~CGD!&2yq)}>jPoEEvLD1m(jjs%oLS;yI##4lTE>2UOGRBPy+w;*@|D)!-F|Yz4QcObXt?RG@~NrYSGE@X0(=wktf;OmYFhtT%;nTJ~h2(0z8=+pT*lTQG)e zp$`&*M_B^DuwL$qC~oRuS*!qH$HP`T9TZT&7sbYuo!NH^{nzp8y=XHgDW2^hc4Y4k z!$ZjUfrg?=8ZReZhq0s@(!Ci)8_sL^P6>q~IR#x#m$F`)-|vDXG5}L80MBSzYB3Ia zk&d?E=wspbeZKr5teUNuewx3uKAC)dT~CKGLJ5&KN4R~m|N4N7C78`$D7TsFD?t;9 z!33hXm-%JpURk)ea+--eA;>GK>@=RL3qbt{x(I7oFR zOZ|lH+%ngaZl4VSq2Znq5)s4KmW~usNFat(KDQkLn>aC)AhXom#D-uuXUUGCDl$tQ z+8Nmt{Y9=BHdZe%7KUV%W4P9Xr~&l6kB%M$XIQ1e6&8|Ti%Qi`gPkl0(qR=i8Z(D- z!&(_xtQ(K7Cx(GB0u`AG-fCBpca_-$B|UY4-{(}@hsOzg#q{O%uHNI*qb$sPRCoAjO|1b$Ljg2A_4gTsnlizLIb8@G z1RJzhMS^o3nYBDwZiyt=U*?y8E0w$WNX-Fj3+%>(4}~a$9x#FjAq$XH^iRFX zzM5xXZ+onz$(#%>mK4u{fLFt3IabY@gbCJ7eirU47n9YjOB7YF?oW}tTJBNX>#j24 zMr)UvwbbV-GEr1t`6gN%=*GFDw0E=Fip7zWBrz}6dp@T{QQV^b7Z^LKX#ZRQ0x@>I}3;$+Ft5Jrq+`mEG5js8ocv0QG6^-I-*W@GoX;IE%9Cy#(Z55)QLfim1 zoynoGN@$1grun(h6WxjWm#tEDGyw`t>1APz@Rz3N8cUTHtC7>K2}|Heamsw6jKA4R z)um<&ApGu2ny*Aw8Qs_IH1(HIsKi%DnLn=eHO(45W(&>mjlW-s2A`ihq}5-KL&yfG ztTwIw&~7`)$CpE#={S+^Je%iq=r}>A_nIN~NpnE&RRe(^CP8#@n$Gi88d|Mc} zt0Ro0*#TUl6%e;}2ew-ye?*uNUD|=V!{)m24n|eMed)<~>gY(C518Ujcb28tJCgHRjzj0+f3F z>h67i2fDd|?p%8Yy)(SU?PAj9>*-;+{aWr--^Nl^BeW=YRUf=4ucw@dcI%PrLRepW z`CsZf9Loy<_79FZ|KQkvTi*T;j!Eb_>;1P#hDs9ZYTd+~^X+K=0IB7u<=EIf-!#z$cVH|OE*lo2T0g2pq~klN3augU}v4@ z8|S@MZ$+MCSOG1exph9SH#W@(^{np$^KBP1CnQ;)5dH&Ygj4lFyMphg{64C(9iorbflIjI1y#ZH(LVe5r!iKa zTyZOX^JAnRCX0E{PD2$Jb@*X26VlB`$^)PWs_=CV++a!(g4TRZLxFWnAKX9ebfHOx zo*z@d)8ZI-n@F`IZ8+1$M@Wkg>#@isyQaiP z)H%tq3&o`%#DkK~dl#8&xe+o*B3y0m7K9kRvyJt0uZdLg7G6*j!=Ahhiz#<2p8dXo z;;{PblV3o*2|1s|s?8$4%^X@y8jATIV`L0CNyyVF+AYeR5=w{iN;S zI!t$=kqL*7!=6b@c2-0pj2^1|=GgW9TxIL55vMgc`GMc@demVgBn%zov7C?kACtO| zx9sM0!}+)cKq+puh>sU&aDpt7FI$eqWk8mSo6bV<0Wbv<%_Gao(Tvs{IN>`NPlzf zUvh3^0yAmEFoqDvPNCX(bp#cbSD?*^tlK#PvOZ|f{~n%f-dpt(0-nzC8q<3hlVrNk z&g{u%5LSm|hpuIbC&oT6<~}SnJjh-Kte&_qLqQby9#_xo_hro=N^{I5bBA4l#qJtW zHejuU?}zPDm+trn`v-$wRZyQ&i$I^I=+hi`G!r_Uuq{ll|rRU&vL={t_zTF4KU;+;5ekw+pZsdT^S^LvF1t@!*l^hi#9 zEBRy8rvH>Fbcx6NT$PUD~-r%OT2=tAV6`2)uMxr+~zYw3xW#h`q=rg{47}5rI*RbIA)O76|ZbjII|< zc{CD5oRWy3H`u=>7X5v2TY!L|B3>8nY(c&%Ly%(llEk99$ObWfJ7@~4HV6Wv0PV7j z2tfamGe#u^VG+iSYsiX9_Am?V;Mc0%qXDa|@dlj^Zd*VfP>!DcvBAOqNwZ|}L3ukw z@p|CBbjQWEH@6FCOX>ULTf5u)yJUuJklXTpVsr2it-Gz^8zl>Jnl`F>0I^$x*svEB zi8sk*Tzp)bexCN^&z`5o`>aiGF=t=5%n@uQ@h3Ld;lGh9x`^Tjxje+IWAu2^0j8e` zNW5^9XA?x%V1fT3E{Zfof{jI~V0ehuP|u3Er!X?-e~3$SkZANhrxrMhcV9b`;*YUZ z2^bw?(;bQHc_?4Hf`c9TtOS(ihClaG9nEfP%2I+Y*H}SPS$3XGf8nut=R= zR|9mqgj(5CKRqDyg(tA?J<LLC|k)~;z*M4%coHBr=&PrFNN#5}P3<$zd7|^8J|u#c>OJ42yUn^(Sdn+RTx+KqKu= zp)th<_Hk|0;wgyUVf~le@I4UjorF2=T25HUuh(xw>=j=DNxW_#x2KZ^h(2Ae5lI1Y!aL}umirjgWi;+c`f%AVc zmq=_p|%-2k{h> z7>kDnmSTkBObK|%Y6P!Q} z6~^L5GSD7sMXW{Zj5h4$DuUP=K7jvIr+vNTNvHimuCpKB;{I=S+8^ZlPd?gzMO}ia z*vhd4`)y0-jsmzHg6_7KKRFTq8H9a^=?mhB=}HS>k(s$DeV)%LEm^-mgh<~^ety2; zcsbY5%{`{Xq@bIOnzK#Yh?+z%*m1t~*LC^$Y6c5RgGLG9uuZ$I2rP?*z6{!tC4_Oa z*AqLRwEmtJiwrOz4<(Ye)Ok_1xOfN=0d^WY#M(hzH0|}x3;@E}W}!tI$0YXm`t|pQ zgaih9zn~ArYY2WdhO6 z9rqd^$Bf@U+_$m1WTeMAELG_8>mx%{zihCqpvEdII;@vs&|tpNGuDET9tspvcp=^s z3p^xh03OMJZ@^q0RsZ`l?*4w0Xhl;ll7;z;ZD4{J$I0$sv9Ty z5~tvv;I8Br{~04nI4)~ZD@L8xER7ZfpF@a~|4^^YpVnxN`C+VO+xA3s9Gg!BSdEvuoVNu>G;MxPbyow!NXP4K_6^nc ziN$?9PKF&1WS9GPhFJmwtJbt1vOxxy9rFccw5TWJ4E%BHDJsG}Cw8MlWb28Sr`cw{ zVm+f5zzy&ihX!Rx>jNNITnl|a1kkvJAzK7zUI;EZp;4pxl{*SMs4PuI7YBoTmeWX$ z#B0j0gDgA6ma{Mtde%a2aoG&15W{ab-aZcU=d0($cGOCsxqtRog?xCMn?Y1$SgV!H3_+lAEZcV z+GYPr_*!nXjQ$UKMPdW4_Rc!i&zVc0&O{Lg4Horr-G;tNL4*5Yk$pI`vQM5R%~%A^$%Zo^F2Vs>H6f*OL?=tE_wLxXC88FF)n7ud zXMqcOf3kJmx=4DMR4)${LpbLV|PsaJc@U+lvKyIQ2Q9_s%CNZBHh*FP%f! z3L__Swx}m1;bo!ldL-dS`II}n*T9PY2fv(0^Z!=OA1eRg*Hze6G{MA#yKz%B$`Em1 zORhIK7Bm)(sg4f64x_q256Xf5@SliPPe9Geq+s1<<{TH!qab$Lsi8ABLf#I`=l9YD z=Z#V|r7B(1ca%4A6v%Hlb_^hd?p{P||m3j}7bN6FJ(xIcgV&%S%tb)RGn zCblezz(VlSWXOu~mG=e`W}ouwq#*`hcYDiE;G)a0>4xj90dxl(<2ZtI z_~hj~sC|lVbK@4kFLr&u-rcAZMU*Q>#}>hLjHJqjLdRe$KY+cwsHMRG|3zO{w2v|7 z;w>cYv*7(p#HA#zB-NNJXUU~bSS`)6tmurA?5(;?M1=nGWrRfisg2O0BoFyQ>7rA9 zRge;Fus4cobu9>?W9MY6!W=D%pUyx>X<3F+t&U6r^Ha4U$hAI%2bAZ%Jk%g;!?Q}A zZ-Keay#<)ozYvLU_%TlU=qxen>X^2jk0i#jV9mY#U(>8!U!MT^Jef5n;U<|(kn|4ul>5(^T759F3* zVpO1E4#jw2l(N9*Kv<0)S$!_+>~kvg2}t02#ID8$m>s4W7mygoi|;VT)Cb-5gx{MD z@d@<|5aFJ*CUsSPl>wKK^Rw(xi6lU<#d}kxw!)KoDU{@sY0Dj=unFl!9w!4(40U}+B;PK+N!L-|ppj%| z$cBx{*^yh1lAg~0LRy+fyoPb!3GfUGyh2|zG=3G3{CBgM-(k#!q?Y?6wkNs%ii}u) zyV;|(@a7{LwmW5OkJekerz7X=j71-w+_Me9Eo-PH&(>86nX`O&67?Ys)(2TDt0TuF zD+uYMldAHjolDO5Q7W=+{T;^6$@-29NbD&s7&hLrQP*lKa|JsB+rR^bFyrCOi|6m>!wSp0T^>Z5n53wAc)8sUHqQ{(h~&`ZtH2!AfTRzZ{0#`VWVN zjsJ_oL=?F&-Tk6(8D%%L5rUfHQ;1xRce%quKjLT&P4& zxg;WuZqF+5UN`h=6-kYmCJ4RwH-|-n;7VWoo5SGz-~Pp6WU1jC{||>f|BJ(@|KYHt z9rS3YhXL{rw-Jtp6OJs07B9Qk`4NsY1tyv7^V1W1NkU0>C0r7&C*Kn$Wei-EP2oW3 zAp?aihQtD~7{-Wbca-OktLv7$k&RBLFV72QT(ZuLkMEO4{Ud}jT!fUDDX^gnjJGiP z$JP6A(iwa@o|;tN>X*8c_O6OGavbz*6&xV}p@aDp_=h5+9bSe86<1Jxh3=l9+b=XA zZy)Hr4i9L;8Kj-L2neu;_YR1g8=}f1t-$elHTms@b8Y2nsV?V9jihs{#pUI1fd9#1 zrIb@?m#_c;g5LjqYJypV z4mo4Fd|+veN%6W7Ak||i6hg_%JGy-C*!NE!#3zoV6zLr`F)_RYgpSGn+2)@o0{3} z-sj!ZEi6aJMM1#Idx*d6p$OjDcD$*?=S821zUWAD>oNyk0t7LBjQEJHZ?@lMZ%in@ z6U=f?m(MJZ>p-nr^>--6Uk(urS}vq$h%guwTC!P2A|0X$I6>ej3QT91g~oXivYx&% zr$|ocG*}aHZE(K^Umt;t_5=z}oC7sYmUc-ZS`eRWgezCSsI(6$kqJky5YINe>yOJ| zt5Xq*(<0riA zZ(v%FJY5%C!!AKjx&VeAY%u&xeTPspd<1plo?O_v?(i!H(1Q61L2AT#(JFfdoMtBJ zWb3{c46~D7yGX*4BZQH394C~U@LE_Ry()2v7p~_+7_6#FF98Y`8IuTB(-W7+ctONda}a*cTajf8iF8_ubxGKo0Ht>By_(p*r0>u#l)L z48(U0O5w7dBLIaH5!PDaLrAOqU_&G+>ed{sJbF7W9nv*vZ^o`#xE=NCM-RN#-ZQh3 zTyPL==E43XSMm0m71yymu6_qy8NpCw!nFdPp(y8PWem;XtB|CFKm5InkRsE+X3Vb( zeq|edM6`fEMwYJ4K>!K(i_e#lI2o&m+_6J@5~VUkx2TtG z5rF(1MNFlR5YFI5uuD1g;US~fA4uv+d*y|i+Yex*8+YLdbLsZM1bL%GAA3M-wel1| zNK7bIeBve`ftB8GYwl)*p%ZQ2Cm;k)dX3u3c@M$q*}~@HEP-&t6mtRhGa9?q-9GDW zS~aN<%-||c-8Co2(*__k0QiY{^SJEUP?4`etQTKYb+>2?WIalRqB?F;^6U3`rr@I+yIUDUzcp7PPzZw)mnLwMSu z4e08yiX)wilAdUbV{rGk3}f67oW(v{E9?Bg7>B)KYJtteey*cKu%ycy-f0|&c0wA0 z1!}rsrZaZV2##==nIqH-`8Ru^A+q=`+GjW|0V2n&yLno$-ztB6cW=D;;-hF_JK~d7 ze9j#cuFsA5aUUeWmsiFlX+&PJVPsCr$2ug;1KVXj#1BEHYp36tGl@JX&w!~10c4tB*C*S zB7()Rx$VNBKjCXHQf(9m!4 zi%T6O1MU<$pYDdWzdKH(v30n%javxMjCyyiYFh| zl=&0y%Yw3rYOuyI?;9hd#e_^$N0U`MnlIz-6=irfg*U|JOE zE5v?1+|69JW8wPWQ0v<-E5sXg19gT>kl@k~);3V9XFtY&kaO$=4yQDF zK1C@EQVpo4(`r>G_EHtJAzfWLxe}2*xK{QfuVzkuM@&je*9Pw|7sF63qgxX0(G$+0 zrIS-xV=S>_Lj+~`wPA3L#1TR8Ujim{X%l2%SRg3=o+GZXdRWQv$d_z%8)FGH5-l$A2X#6p5slDbz>@YE8Ng91pnu zoU>iSN7k)^N)w5rbv|3#!SP-Bl3v&X#@F=P>3PdX7sFTYu7-=edZV|_O4W{XWEJx( zKQ)0EVa51zi^+BRmXOCO^b(i&X1Yw2sH}0w!1mHCkFg`OyY*)hia)DPChym6hrHe^)Et6*kPanLAiIb?yTZ`=&r=}$BDspr3& zSICTdR_u`j7d7LP(r~pv+TzuLR~5BSy|HL5;13jhiv4POy3<^6K4>bF+ENgE4SG;5$_=0pjN$w; zc>v`RS~+HUm$H()2IJpLylRvE`}d(tCI;`FtyQ`M}l9G*5Sg`>Rblcjf z_Rt*|WKoNS-bnau<`3C!?0RBQ5YNFBMd?GYzR<`s*mD!BO`{#JS(?lKa2G;WOGw=k zwwAt1q4~VjB(m4hG*p{9SmUHZVyJ+(Mv@VOO;hZ|Q3uqZA^quigBX@oFxYw$k-f^; z$wQ2Q_l;bpuS>q0v+x-t*2y83PJ<>M%mxs7#%z6}vMQu zN1f33>8GUP=kU0s!dLb`f7Go4+UgE<_UdXOM(x0#TlfP!a}+L_L)xpRovt>jqX@gl z<(QQ;uMQQ5uoP>J|Jny$|4vd1e6Ht&=(dL$97a$C-F3{3 zvNNw&HT04x)>+fuZHNk5-*p8KwzA=V45}H`QMe3BS5?H$$M`v$4xz&dFTrk-8}eEX zi|D$Vv90u{zW$k+65~Wm_lBzUp+5f{TJwf(?&+f{=8wEFKE6~Rh1a?h(QBdCQy{Q^ zQ1$NfTnw^|ST{nEEqhH;^wIfWBk1 zSy)ghcqJ9fWqJ818Kr5c;k>=o;HwB=>$E7B^|TvlPo2u>P(BS(E!KAiWtTehtNd!G zUJm50+lKnE&Yey^_M|TIbo$epE@ztubs6@UapDZU)DiFOv?MZRTCZCBFj!7g-zyblSMm$~)CaYRf#?I8lWn)J?zXsBMcchy}~s z@d*E)d+^`==N@f4rRE|q;}u!#kcX;2Z|?-OzFIZW#US>$%*%zI1`%1G`_@O%*{hQ4 zcbQsQ6}ibQq@mhJ+>tx%5c*fC26g$M4Vh3ytkL)c7GJVrsaf2a_D@C;odVl3M0ep+ zC0u`?E6Z>pMI~b3-O@G33(20j5=8eTc7_w5F@Pd*5{p8V?xBt5GXF@J z@(HbZ3q%@^afy=gi}2^d3hk-U21rIBPC)@a;JikuRVj(qC5|$Qv15zGbVMU18pXiU*qIbQC7_i=e3k$&@l?g#6Mj>uOscd|RCJaBK4Ao< zUX!pBi4-dS3M0WOT{`$0s6!aDCbBXlTQzRkU$fN9kxmE2Vp8!Ex>rxCRGQ+g$E-GWj>Tx3(3-&Z!v2+f(?FxBCn$dOZhkH2 z8^-B2NvJ(m$=>UiTTVSbUS-~}Rkv0R{0PE*s_(dRb`=z}gGCfMNJ&*a`xg26Clla5 z-jfawML7?W+iUKr%UiDLIb8E;Mn2NRgo&^Y54C!v($HveShwBnlCT1%H0|q_{a4nr z^~Lx9{C%9cOj(@$$xmKj008*)Uykt=9PQ1lP5!@z6Ai0>%9g*fb^56IE&wbjZMHc7 zX*hXub`aYqPd<(&{b@N_;8)(V`|_Hz%dFm5f~UJ$_;onUcIXG88d`Sna7}9{6%<;H zW74L%J^`IEggoNqSgbEXk2{V*-c0XV;k5Wttu6X&`07gzvUI^fa6%A1AihA9m&r)F z`WxAYkT9%{fCsB;7QM5V=U&!|Umq@iF6|Tk-O=^+0MZ}Zg+o+pKkq)~^dUzjg;!h4 zQ>e=?Vp?W~u8{x{LP8xF5q$4Z9HepYG_GJc5+YK7i&$y`+zjo<^%kQ@PcE#^$y%A= zl`v$6=*dr=Egl*QN8%wBQWdKtB0xFJu;rSjWsVwaHT&^brRtY}Nvq9Z4}bjC)THRt z9)I4=hyby}wK3EmRf6=gRjoJaYa`C;py zE~UuJ$^WfXct_sPSUCwMJZ*AE1e#Pj^d>&_c`$01xQ-JAP%V(G{nnTNNYD3sHWfJf z!}kCY^p78{vL#-D++=d*pz}anf}c#Z=6ZA5N3P$7x1-!Qvi&>!Y4_)a(>tSEIW0I$ ztSIA2E;`!Q0ca^PqiQHRF%G;Q*I&BAFdQ0kGXtFa8h*u@pMp-ngI-fa$wZ#GMh5Wf z^i~=@HaLAcU9F4!Xe@_xT@e2b-w4G~uW=ntxnSr_5_K(aJEBXkAXvGR`!$#mYK?2i zipSi4%Mw=7!z|I9f%6TbKyRm`dydLv{8BYD>~>7h*iSN@qa4H%yP>HFzt(~#U>o!h zdBXJf$D-6rYV8FVq(px!Q0-L30IyOUzfH>SjCM=~DhL#CxYnh8Vp=;SWU(jpG@Tr6 zhRleLdzDW%^|8hXO&-+pz6o$O)DBjMrwElWWC_6-5HYtmo2mNH5rQEliHT<*=S~owy|qj~>s5YpnP@i461@ zs%iPxEY~diRgwPusH>S?8IIrmsD@c6FKOT;WHs+INW`zZXspdJ zJo&$gM;YQXbdyNavp_$+Fg}sdYB-9^-Bn+(VfD`XX0b zE*1A?@pAVI4IdkAR`6h@8$Uz&HcG&W^q|3IbyX)L$wqZ16w=h<6Y)YtxEOmDk@RU6 zrC0Vuk&~pq8z{!P&a?WZKhq$DKm{alpdVKi99toK97FHk?H^ec7|f1`0|B-5U$SaF znv?zCub+)FB?$nUU>ty$kyTgH&dzxTyfGM9?5Tuq->QYO$mNs4w@D$ zxr@;k?DPR7pgJatDE(##PxfgdwoHtXv^o&>o?7Jz$#>EE6`ppTip<-wE6rO-<%p}?bF3~)W;tTj}YlD(TNmqERN z@1F}&H_Z`XGA=GxBRfzsiX^KcZdgMSYldCU+1jUUk&^5FpFV^#mUlvN492#X8X$&d z#j2)*`k_Kn7by9Vi+zYsMQiEHLEYe=?&o;f*mF`91fY~(_^~u{& z<}sokZCr~3S~$%YKIj}s;p!F zyd-Bk@S7tg(HlOR_owKs4n|Q5XNy^u$9-BI0?KQB4!tgp_c#kH_D?*CEucA5z&?2= z65H?Ydd@nWl*Ub&=*!aDyCJ)lF5J%!-%SJNZOYY-{8}b>oA2TJn2_`n{$5{D2e;R%GM~?*^Hi^G%aT?ZS2|JzCl#}C6|)w-4*=y05+O^1e|hRw)-mrk#ubO9Wa-~CgN*Gi}Iht|0x*Ec$kPu-x0isOE~sGYLo>_z8PFzNDB z!m6~2Z05Hr6|9b>UuXOgw2eb}@JjRaAF0dE1Re+2q^7O=JLhp_rF1;LBDCN5Ue5EU z&Q@e5CFit;)08%UX&#rQUE35s+b2kiH=U!simu{tL%v1Gq#o~vZH-2re&Rrgnru9L zwEyyWetx<=y?wObw10N&V?Q@NpAeIQWGHzL6ysmLVP>R#M$)e_AH9!Q-rzTM`Aj}l z#;nyxz5Ra6p7QnnF1-3oX#RaKzA+rB5|@-vsF`FWmF_f|@^Jo*+Zg@zzczi0)#A^0 zfdK%h;Q#>G|4TqJu+(#Kp#4e3q!-Y0Fk)r?_l#Xd!(m?KmPJ`SZfK4GLlK2*TKuWfUmi2p5}Gir+1*B1uqjLDo+sTqeoq-+wdkKL?S3(H zC7c*u*V|i9z4w^!_xU5LyyqR@RlA*laff9Rj`2mB49>r9dowJubQqh}tNfRQW5+Nf zuGci_g(AXe6fg!ko;aGGuy;ZLt3^D6ci$WJ4Eb&y(cDw9!r1RxQ9gVrC8`;N|6+CD zlEySZE~ZeRrUi@KI@o@#y1RfmB2Ph|9)P5$*nUkYU2H84k#2Gc&2rgjvI%m`0deM_ zQjHeRB5P8Pu2j+Ur%-oVHl-n#HldfY7kc{Tp(ohElc|+6F6EgdzD5=AbTUVWlXwAz z5j`YoLn}H6UHNIBfGoZ!HH69IO_E9r(nDZ$HQW1n8Pf)D2duGTsbRjeMXWp&kui-u zR*NJ6w_G?$2AlJ#Fa@=bl5H*$GaUPqME$+6RrkrD$kC-Jlg>*9?dA?(0O_#<^U)}6 zv!|nM^_AD4nap=QH_Y$Up?%PZV$4uE#>rPow@S9Fzx9)w)H{+7V^&grBno3Ao^GUwK|qP*Il$Kt3QZqMQKt6kjWo zNWD8soe*8GS%=37wIe4sYbVb%2gp5TcT@!%Z)5 zf|h#~^X(8ii;C+L>9MG&(y6wRyi=PdmvJ=pRH8xCtOj5I>z$iA&y%nE%XUw9oq-^kk}GCm^TtsyHSew z48N*JpdrUQRXi+_c56Y1(}BZyG5sjgJE;9qp ztP;B3M}t+_*;T@yJDwiHV(ASf)(jt`8Qq|@Q@EieDBGS?+ee> z?8X}lp+s*V6@`703S3N-rA)TrLeixAyYJ9X-j%XX;Q#qwb-PLe!}}Ab%Z?2I!2kdI zvmvNwX{rCy%Oq%NW@P=JBf6h#-3?Zhe|jX9avgldCaczFRjH83yI{g_NzQ}0K@){G z%uVG8C_9E{zqpit^8}`#pPnzA4UOR0 zYes%^o$7g_Wv@jG+PJ_6`5Q7RQO-aNyma0Wm{cl$fgSvW>rTOO(TT8i4jZuOg{0ci zm8FypLN>Smg+KsJ_ZQHhOJ007$?R0EA>DadIbkwoaG57sEYc9BqfpT;$6)LB*D-p^h*{W6^Nb6BzSpWikKOFPzJ>|7~SzojHXKjeS~IVeuCfFt)5#ddhVU414Z zq?fGYA*Z~=sXB`vrYj#3vds!9%`?D^Dsx-Vp3J%PA`b2d&#?H86kTtfnAv_yF;G4Y!bS^7s}mLSUl;F* z-|Du#Z&g~(%Z9?thNPk=e=UGJ>x6cWi;b+)!OY?{#=+z@?edddu)N^`^ED1nl1?d3 z5{1hLzd7ncqD1x~uej;ZgbmIibJc_Mdl2H-8+VO>xK`wB6aGjRkXNX!(uZZk6#N;7 zu`v9HKSMZ`nv#V%Hli)`u@W~?Z9o)OD_wu0lhr*<)H)&`4^D0|QHo_@O{U)NhX&sk z0^k5pjP>@idaI#YOuj>+v>MOp>Z7eE-G0Mikx(Dt2TLCNF>E3{w6*wW47%2)DiSjI zNc@33Ouh5Yen8mLY|afN*k^EGdACgTp3>NR6*UR2dEu%VMjcvuQc}d@-7Fn?8a(#l z!X3+`OqHifr?}O5@%nuxKf5Z=d32Sn8 zvG|1{_r7Aw(z_O}+ZD(bBv%&xC4FfidCbO(#r>552ZkW9w~`;Qa%=pR83+&_Gv$7~ zIsUTTI6=Qwc+(JgSOf!ReF!$FUu`lU;kvyA9k@uG9e#@nPP&It^bVvkOI=r_-^Q$X zS-)$e8x8YRWjD*)mk1}z&Aypm>TV2_dg&DoR4=@p{DKvo9@g%+r0?IB1gojK@#uBU z%H+ z($GM41P*wap$`CJ%}FMedQd;_>ycqJ{Z-K0WCkQA?hR?VAL<8)!M=? zhMGMfqb2JKKN5Z)XPWyK{lWo&tfNSKxXX-zMW2SKVIc0b62fbouoEuW;oq+=BvQ&6!cWDK0Edv z7FDsnwSKQHqh89kP@0KwPd4YmQa;myQq7jis56;!$$SE%i{h2B%AHp)FB6%bxLQ+n-pM1HKoM~2YfjgInLPmOY_}++yY1^1fq=g za984+j8;5iRr6g*(bM-_{yp#D6PDOz`Au@8&e{vJ+hzR;SO{ZT^bCwYk=ut#J|RXl z6inkPi$To{HoohzcW1Zm;3Q)zFcjWQq)f{J%fLK=&4nTj5rW0v{f32n%D6edZaV67 z_seCck})p=`{)*Jiz>iHbAn3+&#}Yslx23gyA^X)^;4y8f~J^bmnBZ;_G6_sbN^CRJgb7lo6sq)F_H2d3jiV) zv|l)157!`Zwdp|*5T@p*>ik`L@p-J`=G$+xMP0qaNSm{a{-j?P%!sSY9&c@J-W)+Q zCU}0BcG+?Lp+WT#LBm+}@(2Cmm~L%Wr~W8(;X4F>U35M3mANZMOX8(=?O^3*J$o1( z6}kpotq^p}=fA+afsRw9mjF{+2Qal_|NGy`-p&p{ko?y)yT%S+`#X}~ab5Z=zBK*? ze8`bJ)e)-SeWnTwrQ@}W!h8^xG`? z*cPIAHD3z`SX^&RKf9eB*S{?8Tjii8$5{tP>}~_Ze_7m>bMM{hrWt!oz^9bPu&4z{P4+3xX4$_-X(vq^F+0&8Yr z&`bcE9&1s5SB<3Q4~Ami-a1prsNiPlv1PE-vL63sb>j*!R0#278dZ%eMP`xSnmY`7 z*@dD>eIfp~y3ndDe_7qbzpd_1fYl|H7Y10}51UJnw;3KZEh{)g2y`^)Nf2LPPDh9M^q@Ak{?<1JM}&_4QY%?iz;O> zIm^gUBPfU5w^{~|7&`)DOrHP?#*oiTgUikE2OSM3;((u#8Oo`LzCHP}HIo^4!x57d z+6Jv;oJa>FV?WL?9v9mwraA;XQjJDkKBE?i;s@nkD*oJQCn5 zloAg4R~*09@0)G9uo=Sj*-?Vw;~K+0Gl$fS$K%%U+m*55-|k4?>*L~#@Pgstla|!C zt=AFOjJKhCrcEJ3%lQb&GQE!4j$Qk-iBsprUEaQzGaVt1ibMMdVNn}xPoG5_dV{=6 z36Iuv)%^f-KROk=fD^k~bfg3(ICUJ&MH_f`_3c=+sl;42<5AZ=HluX>>AF!C+%Dk> z<7Q*d-n3)OL^r~K+M}pWUnY5Xi2W5@EU!kv>_9Fd5Dc2B(iuRCz*pEjS)TidxVHz< zTI!c67n9b2)GEx$#=XDtMtD1v0Zp*15TI%pZzIXZ(<0Gpurp$tkU7gfqU_ zBA!UMg&rB^7#-g*CH@1JR zlFG=3idh8kDcHP_X+jGf0@68idr_f_+RUZsHvHetBs40gM*Ew zvC;p~6F|Yt`FFwWMUct6217c1Rh*6oa>~vFI3?7{^Oo}M8OSQ+GSP&xaq+81BB>p1 ztoUsu!=r8Nj%%m=jAzbg@zOoXAA;kC>BR*ycouL+2oFkl#+c+M_+fs#KeXUz!ny0>j<}-VidMzxoo2I#IX&t((G%4?F^^7@dhR772W&_-e4e-SO&irfJBBdjO|FU} zs@o+^sQfPD6@J0sC1F!nYT2kp+A9#n5mo06^vW;mgKMDbidxc}yD4!!6c&SD<4v_< z8`JMh@kmtB8ZZF%rF}@n2|PQ5gsEEar~IWQ%nWF7YCUu1h2)I`A&n(=S;YE!HwjJG zC@1~_#R38PZ*}L7A7>Q|T)=%}ud-B=(fD~PrRs-M&pu>SJkV7$2lj z4l*u!|FbC6?N(|pnSk$ zdT13SYS42lKzXL>OA5*`baGe^=}ZfQWQprQwatDux-HU1uq)0wb@#Jb8Lg16#JmX9 z%$G}FG(E(iU`q=m-e}qfbvIVDvrGS1DTPEfe{(j)2nB4VXitRkhR*ycz02OPjK%}Hwgfj$c#~dfE501d-4xCkuY*GbvN=zh7HMY!bRHt&sy5@?^>FKYTWGtO#WvY z2-^27;_FVHUw8ug+;v{TcJH${$o#_`tBmhHj=sy_vlPFtlj(`N+m6xYgNxXQlF+GF z+bJn2em7hPa?E%5NQ`~|358A(m|%>&lN6bIr!o+#NIT|oBvvy+vlGs*19zGpczqC( z2rM7%c=B^VQWkTXx<9_!Fv%)as3C;GMV=*bpg&tGw6p;49UXT&uZIr|0O7El{6vkL z-9347FSFmt$?iYYgBz8&L@}2<&TKqs{0=h&)samNet35;GiOY~eeg$mu#`i zt+9c5jRa}!S}LnTd6E*UAC`n)TqyVr&ub)PhyO8tHs+0&0bh_A-%FZ-O9f@53DQf8 zSh2A#vsl`bG0_IqOGxm>o(yA+&6?@{YN}~ovND2{SZ6<8+Eo9{HI|&&ZY5fFOFKY; zKM0gw*{T5*(^*L0e;|G6B2;^3?OX&(g{3^( z%jmi5Nqw&soroQBs6Dbsn01ggPr`p6{c3B(30$kzVu0x~eXS&2O0rm!k~|oJX`tL3 z-VTPWnxHLu*4`N4SUdoZ<;4Kwxb>lu22>TO;ub%cVuC-%c+C4|i6OO6fg7Zs&7NP1Oo^kxW~p`#R-LbC0JNEL@y?0+ zaLe5!TCx;0mE~&yOP{C4$2IN_btvon@!hn0PGasoyRa@UypaM)+F zt*+v_X?CkURv7NNMJr_?)Kh6J>^-Zf1IK zgQd&g-!r~@lci5Ei8wij?*$K?sVNNY4{yAJw4m}gvy3&)6=iQ)mR@nhpFEao6#5Z~ zk4XQ>cAmkrzGOUCZo>Fk_$fWvO!SI`9X157?_?H+FFzBvN3m=Zt*6LZ~j*NBb*r*Waj<4_*{MzBsyV&IrR|~TUFa;-W z^1Y%R&|O7ymz{=A3l~@ZWar1hA|xd3Ux(jQbzSp&*K9K0aFB_Z<_p|IPKD1HEK?6l z=Xo5jVbs>$b?cheg@K~|K*OdS@m7=0NV|cxO`d@t(V{youKH6_%`jaRO}}lk-3iFf zRqw34mDXE2X&FwwTPK#^`O*oU<$RAQdC6oorFn*2R5GMgF~WqS=sFQMd7(647@-q4 z8w_*1!*rJ;7~G;_O~-c@4P>XGbPTJqQ=QcQcGyAFRzE6cu45;bqiRCiu=tRB0Y^QB zFH$XmR%qN&{JpDDWqoY0F>*7Z`n&$AtDTKQK+#5d=$E=7&zb7a0o9VAJ!OUZPMQ+T zgNH&v*nG!+U=F8*#EJDaDqGiUM37wUdriXT_WU2R2&H9WucbafM% zD%yXMl39H~{&SEyx(;HkfYQcfMnn4@A)tqDW{a>dc=U6Gpt2LY{7g2qs2U2c8 zO;wS^M3yr0>r6z2@$)AlV4y1j5oT7axAU_ZPCwv55i=G6kMVO?*Y=1zU3{Fv$E&tY zy`AocXcdyfOJB4VqFvEvOo_;3$%A#jAmMEqK9O5P9Cd^eo7Nae0owhr4CDk)50s&D| z`7UW-U!)D+?!9?pKUir1{f&_e(T4~3Bzmd*FxM##{yLs6Cf59CGgK|+EW@D1?=RpL z{Y@@Wb8i_I zgwhDNnP;uqPQ`_((^tDHYt9&FY{0^zB`bBMNYd;b*_tQ!#Q}|3(q{KZ?4#*-WsS~uqT?;^o`2U(kdvM z4($s)3FJW#-1FO*TS*N$heb3Q@GIMPV39xcuS-DqPwi#u8xUjB?JYS@kLp!ts+gGg1 ze5n=#A9v)DwjNzrf0V$T17eSwss|{aM2$lC&A6b+W}(`B+g zd~JSYtF1Pq{V*pQ_unJAbp;izX6T=BqU?nS7#F%10zc@5QeF_AR4mCY#vHB*_2l>X zgHlUpMcRX*VE1BA=<*W|Y2*cweNQ!n@c%H--I}cwITpWmlmQ8d92n_VI6o1+L!TC0 zvZx>3kmNW+)Ev^RB#t|Z2R32_5`=!997i17dUuNl_kz^Qz`${Sk|7q^iH6=P;7o)pC^4-#IbYh5!PhSD?#6XR-ZV!u@PEvAk-qr;0gyObAeq6k+w^5uK`P$ z%UL*sbeNJ#+r6npXtKIG7o;(bLYdxV?UkKtN{8xKiRxTYD3E zi*l>Tr_Hwp<|*^btoLik_p4oX2W!TLc?=yIbT!IZ{3;8ii^)k-mG(ICu8#Tug1@{6 zwi+tB(O2LJ9j)%B*tnBI)t zEfeGHNj18NUcMd4Fk~;d%7S_^YhypeTTmuMVozS2pzdbnuk$)v>x#Ax1cTn`t*c#? z^T0<}k}NP332oVAm2@mNCP?Ibl~-a+i9%GYby8X$S!V}j#}Pp6_CVUa@lr?j|<@~hbl9mR?5;3 zJ@{OGKxAL;`iXsT>Vhqc2towO*)t--_dPrvy1*jgY$@7J8_>ydJ-Ls9qrCIgWMqx_ z#t+3e0Y99O)I{hb!3DVEjQU`9f|8M-N?OiTpd4tm1m_3;Xop~MN!%70ZoE_qy&)j% z-P_IY^YFS0mE&sM)Vs>-5am*e70XXunQ&61BAz1ra&e<_gTN{~exd3hkJ4Cy?$*Rp z^KBW-k-4xS4dh2SazEQ4_kE0RZ~OH%0aljQh<#!)?R`ovQ)O6+72>HDLQJOL3a8S9 zh=|l%0R|Z6=aCBlUtkNgBiSz!AvJoreiMOi{~i#?y%O zOGEI_2X?oXel(0_(@eQSfxfHSO}SFVOsWNABtxh@*%-1jDJC^l2s)bxp&5`)smS{G zd^;Z>yrRig%APLaoC5)cxVp(Q`dvaGvJsP;^f4Y2kT~k1@$q7b-|*FGQjxeQ) zZtZZ)j(uEGb_FRuZ1}Me+2Ey_T_+PY+TM%~b|wa}MJS4yhIArWtxQkEIA3H1uq}B= zp3+#>`YJapCbHf83;J|a06(&KCoiLtDU*qbd4t7SZfLDh<<7kM{`7{0s^hp#3+BV< z?tX_rHR($k7PQB=ML&1I#Ii##hv6g_>&?D;_WgWGSBg7>=2B3LK5}bFy_Lm!$fzmKd=B*X^PSw_-&yAQHq(*AJ2{NUmKmxOIQgxeFWB2v(sF)wax2Oba zkjWS!(rT&0&;~q}us6ZUu8&6)?W`YS7LzTFq7l*3IC#Dj;-JWi}KD3XucJlvA0jaG|Fy#47?0&2YU{^LK9?;+N0(`x4#~)$~?!7uY9Gi zlB=k^Dg2TDHFL0Vz;y6{j$ooDH-IPAI z#LvZ8uyIH{^aISY4myL;EhrYBkrDHT`ilyPwA`I-NlV(%7om?G%!iVEvY#j7Bg5;Z z;!k=#Uhis9QW%#tw@525@Epu_eGYh2`>QDLI+^b(SoVaevBNyBWIyk=ief?qfirX< zS+2~MH~B>YXXa1h*52hLv$3BuO)izzF6o6^dHh_|y^qjNO@dl13xd{x;MQxTu_7Zb zQKAnG%MtnWU^vs{)i>p`jMO=A%8`X0iAb%aM@`X!=yK(jggdQQBinCg3i+w|{%D&vG-!!3U5~8xvCUVZ+!!)1^0}lgR+0*&1iXGKT1sFN_L->uv zJq*w*+{;%^|DB=NUk`l2zUvjZ%{PL9g;=>K;Wm@p`E1r} zE)ek3BI@z%{A~7|GV(Y%dln$PmpbxPutBF)jq^C$1KR@Uo1;7+tEkAi!A3tk1r(hj4Vd$vOGLs|dpqM`4gp|5_F zz^YAm{aYD!oqb4mCbTz~VO{VBh_xg1AvxvzOz%IJgC7R5(l;Gpms zd?1ZUev0Cw2N5uekF$j_ZVzDe92qeOzhajVFV5ZgJAT+;{G1(`zlW()nnruXl( z?Cxyu2bKSH>*ZR~rAkB1mcW`(H&fz#DDvUzJ;v@?#++=Nsy1kf zO5SFUS;-L?KaPwgRYnqzQUFIwm2Q<|hE~b2L6xqXj$u6I3dmBwnS>?pzZ3x~dP0gB zVeI;-@_tj~Zw@5~XJoljZb@<(TTLl$NMbTjX!j}pXu#?Z!sHNck?t)oqjW_Sbw?}o zRc?wy_e||i(NA=P{VU?!~ zm{y^s>C9#mm$9^TwhP20nD0rDHSI3=T%n0N>RXp3uYO4G1@7q4dsB)Y08@=FQ7wzh z7&MF0;=)_JRHqS@BAXwbbOt4<^u2a&oA-O}bpDAz(E4eI3Uf8J-MB^;wNxUa2*kf| z8OS>s^3p`-HD^wpb>yqAreSSH>)6e~U3}nCe1}_2-LTaxaL0Xw$)TdDr$mn->0`S2 z(OvwhY992(-^)>xv8%)UTQ)K& z&nf9$+#VIi04gRK)4dJFiB7woH0K)nJY^H6kLd?O*oPHyO};Wqa>UhifZc#h_+VUk zY?m{y&!)%y0@frLa)?6jzc5MAU$~|8DaOx9xd;zGZr|F&r1qW^*c4`z^@b&>D5TN4(cdSc z{Mlae2IVjR#jje9Xd)8=LZxR4ARx*Ae=73N$-b7Y!X`KB=S9T0itln;W$R}* zzgV6QfRcNi93S??CuQ8{c@xe#cRe~8CF3@0+teLcm)H_Li{hyo@e*!`=IYNx?BS+*M6-W1<~ulOVt?K|R~bK6*rl`VI5Ird@P^0P6rX^O9jnTztA zhbs#F!Wi?nc1M#!6;Eoy!o-#3-meg}?GOJeZhA1;Wf7{?D^B&Fvps(<4XvF~JU_!` zi~r*1>KT`)0R;j@Wh5rEh@+5W1`0h?j~&mK8^XIkmXD(SM(&l=yk_l5Kz22>G@w0) z7z3fSYJqFXklm1+41D%topD!ep zX3;M%BC)J!^0v;lE)f#+Q24IBv`N;9sc_&~$>!qv*_F;A@?{8LKB8P@pc>;VV`tmWzCc~4C(!uuA8zH$|;Ain2%_23@d?~~Ph&R`w^KF}m2 znq!urE;~3tAHW6^qrM@)1lW&s?|{Wj=N%t6dkx$HQ9gxnCO&KhCkKzz%u zi{uazLF(9QSiC(nHU|vc{GJ|YXA$^Jx`B&BXr&2e%csFHTaGS*LV>nWki!yZb`dTb!PJkiMp*1d zZpIX^3E@{6;~MJMg-8+hqE9$(ez$QRfFzxi8i4!#%=7#-TOe5`Nie)e!D&+{FpJ=& zYx&l5W4yp0ON8i$(rS7~lo$9XHpnaV^As&^Vm(71T`(4S)dCzZ&VPKc97V}!X$1r| zShSf#&RwWjYxHIN2V-;GGDcYnOt58~_AaFf@SUco|aIYcZnIy?@9uvF_(p%zTz=W zTl`UCgx`{y7wq+>ywB`Dm>VyoSX_hk*fEPkB`}giBqNtH@p7mn4r92}86Q2Bc~Wh2 z2$WW5q_IGctMjy&?ln3aM8SNP)2IPLNyl zTM%=`5BVMUq_xsF<;EZSl`V*0 z25bfXIr4pvO}ElH(ZqD~mLg9^$YA{wZ{Qc$%OuvgqBQE_-#O+%`ltq#s*R_58m9@<8WvzyLW&@`iS z)C|}oObgSZv{9ME=E+M4EfJUK9J7}J;Rib8#!My4n}d=sZO|v!K__1P?AJIPX`+A9 z%jv9?_=%^D>E}9p>EaeZ^kTUf=4!Gode-^ZHoKMX$|EeG%`S=jziI7AtIMfdn!5jo zi&wW*+2lk5{JU!%EUSzvI$)IUBE~y{Aar<8aBW*#6ylL+7loSmQc`1E_q{vP4LKp4 zxQ9B&XXDYdmyMkpdVCu*S-(sYGB6~+v_^@P0~XX!2=d2A!d zrU6k=qKf>O)D9@aat9}=340cJW4GX#H?l#pYWxfTo-idkDfew*%?x0=g+EJg-W#GP z9z`62bNX5&{!pCQl$8ox;Agf}r4~pqmrs#OW&}}FsVp@s{N?VG;j!Jf(^s}W{iJ$-HPNm1>{6rvP4x597H$;%mxzErvZou8y(`yeR(okxI4Hi# zb9dhR4oe)P zs%TyJup~9+2mJ`A&q{9yS`z%U&=Xf!Nyx+5x_N)g%7;=IWEF0!47fR|sTgk|VwFYB7QO@E_w(ero6ffZ*c1W(dI8$xyzX7KmBcOC)Nng=r zJ|n zEs6Ayrac2s6|UHbWp@+8!or)IM^*zbZ;vBADPDs$qB{j>p;Jfcs^MwE0ImSQ$RAsb z;SfIXYf8Vz8H}KL6ER|NRXJL#YNzUnF)5r{=8Q}CQ9I_|drvp53uYf(87=7dci!xz z2~M|)FX(h@@_=I1{Hi&%J5Xxyhq~N{mucL}Fi6RYz1$CAmH&!x!IPJ8KS#~ETE_9D z3t!PGP0aA&O3ornle!0aLvz*|5R{aaKDjjPO7wxvB+WSe@wn9~toDevDr3uTX*hq~ zKSxyy+QM2Px#XSi8DwA>uNms;djMx?6s83Ea5x`|;+ON3uTuGwW&vd#?yDbjKar7N zYxXRuC^#8+bA0D6keXi|xp)J7L_RaGsg}84VY)kp2^&U)gmX^}gLqsVE18{~X#v(P z;8bOcOD}l+L{l-o{UAzQhSo84@>6b78s+b9`e98nTW{fvBI(8Q>Z5d9HhE!kS?Ptk zI;f@gkN02E{CjjfMSt>PN*4Fjak?vkMygr?+v(T~QEFu*v^Vve*tN(PPi<_|;?${{ zt~yQE$5!*GcybE?4_i||rq z|Gv!e8UHpiZ1mQyJ-Tq6SLwYaX#)>$vMj)!XgOY5Ol+N;VsUQp&v8kpt1!rKKwR=2 z5SQ@(+l9_Qb|wG!rUeZh`~CkQBFFh<-xAv9x6gcR0fWGZ-PgAAFj6-jm1g!&H3qPk@WGl=GM2%CFE!l|9VV1j)%UpGHKxE^kt zpJl=wP9K-=h(OV+m+jle$roPL35HGapkhA;Me@_NPDzA6Ebh0q$WQJ!?1wp8=LZk_ z5F2I&HS3uEqk}Ck8nH(x7*kw}#-#GH2y&oCYAB1F zEF+Q{&~$3tQ6W?&FTQcHtBgiWNO!kQD-%pU)@Vb42}$NzH;pmTR<{2A@XR;lc+BPoLuhBy8n^7e!^I3Y#jQTEp_Y5^*%P3^Z~+9fwVb z+t;AZ-*@bRpA?d>peK5t%ROCs0uN$6>uvR&}tw z)*q5rLMOrtK)%TU;E?OO8wxx5I(ntYBCDoqY7b@*Fc=Er-Y%%_s`x=_Vn@1f+iLSh z1MMS~UQTR6->G-b@=za2`H^1Z`?_%-Z^GXp5=Vg37Ho_B*7)q-i1;GIwD?=pCzcGP zPq6@52}56%Wa53AwjHh1XuS~!%#C!RUW>-+={#^*JY9F>?R=6`9>YQU<4Q_<6}M9g z{fshHMuuzfhGXre0;F*?*q?uRaD@%>v_CK04m?f01K zGD)Kved z!l~^-4imvr)=&L+NXCzl36oEC%+L3TCy#mDeIYaLJ3i#^W_Ka@$@8uZZ`ys`Hvhn^xwzfmrkayu&Jn?om}?7Z+^MDi zrI%wcFf9TS+xXhfVZuyO|DaU_!4(#_3lNWvXJq&kRwq)iWY0ux=ddu58FNfXstiO! zQs?0INYErJJ< zM8#$vY^p>UL!Fo10Als3-Kd~w%Jx4QdMWbV36;Cup#D`%M^2MnGEEbQUKwW5D|x@K zf)R-q6U;IZ!G9#fv?Mu$i;$&Zt1X$Rr{CN3+ky@)gCmYV_=0>OCfq~Lb$(X)`Wo;( z9FhT)BUic5Wc~nlLP2dPn;}e>Z^3UlXjdpkhsYq`;9f;NM5pk4e~zC@+XN-euwHGo zcsn>x#{<{=Ka~UHM?6^DWRn)oY0bmi$j0*_&S+^Kn$Nu`B*oGn07WOfB;g~RNgGux6@ANs^NQ|6YUE{sqT!x zf#DI0c!a&U3FCT^M6OjocQTiBhg!$FbX~aws0of2NHM*N_@NKt1_~oYpDE+91g}49i|hEh z)S%7Iucf?l$Q)-`S2(V>oLF=laZ-)sBNi4!G=H6JJZyh2UU2)PPxq*8&20TTb2sJzDg zFFqRf4#2*!+J$)GKWr7eavQ?l%CU7X0_6 zQmVWY1Gh}~|e{c$wJq@`U4WS!* z&GxxE3QY>iM|&`l`-u8UQIimHOTD_*t3Nyqu6G#mNca{@0S=Sy@9@WrY1|7{G;&8!}HVc8M9HD zpj@I_w~WIyN?53D${qQ>KQJ953T?u@Nxs$lEVpOw^GNF!DCn?c z`ZF{|d(PoVD5l>(2Fijs;-%OTe-+KPK9XjHkS4QEF#SyLAj^Y}05gyz@4}z|3zx2y z>8}%Tz|r3darnR#frZJ=ey;{ZI0C8>gne=Rcv>2ikHu zS(SVum-r1j8H;4tPQ=B(Aw`n%Zqp8v@Stct`mwhV+SSQ4lV{kCE}eA1*NPYREav2k z=~OGPNy(?cePP1&5#SN6JacQlSz1`#S|^*T`0Zx_|!Hy#H>!Y&g}=Mc(-)q8F2Q1|{>YxdNwy2{+My6BD5t^O`;^|~Sj&F%j$3nCU^ z!j{hg&aN2X?EZaZ>;L6=A)r3oko@n7ITR?lDq1r%dRvu}0AzQKH&pm%AUlZ7Of15c z$}iv9sdSydi0xMCpKURt$3Dm0JAorv^1XN;d?y3PC+B!@JxV$Ez`PX_JPYSSC&bh2HGTEK26Ylc*3s@>IU)F^;(&oUdT;@ws3hwcBh5bdo?S=wphj?MvC;zUE<}3#^Xr*F zyTVgkZqA?$G+m#z+TPBduAnOsu;%Eq5|&~Vy80Knf*>0>+x{OxUFH6W5Mm#b7$DdD z@r)coZ0X50tY&|W--ZD;%@E+nH2gh&yZgGFY0$~c4;a7wd(#X(SS3~AqaX$sm6gwm zO>qjOO5_0`h7+%g)iuo1iY|CEY=I}HS!A5r59(QxByayE&U)(5slP~C1>!XTtWfQ(GYh)0ok`HBhpXvZ{mJH^EO@$%^yjcjexy*KxXj{iF zQAc}x{@1s{ParQ2{}#I|4F$b^Xdv+s(+SPgr{^H#=1Q{(3=w()OaJAR-4sAOy;ixm zqKx#ZRdiLY&=*9T6eY`&@lYe=D#gF$hEu+H0d=*fLfV*K9WXJcS9rf6X&-ydZF7)z zPN)Kz`ez;mkIZ1&oP|ERFPprM%b|0HKs-}o`Dx+=NS$7DmQc!ey1TUI3*jX7nvQ;x zQuG0zYIcb!9KP;-8TqFRd*NYp?nt@}E*AC;I9a*sd9}NNYRkuUJPNnXVTU%zR%Ry# zU}DZG4?(2jA0=YC!fL5bcj=MHrgga0`4k)}Jv*R8q;)5ZufU6%>Ua#VDOtWfhKbks zF$sS)s6;2$l^oKWT_~s0sI9yJp-KcKtkX0J~;JMx?dEaENeG@EFgd9Fet< z{UV*r$9O{6&Xof60?pQ`?G74A_1}0ks4o_*T35MbDa2x4UOeI1$az*v7Bv5^PmK;qfZSXK!sV5&HEvM>E zPWTHv#+x+WG5m9x7`&AGlMUeYo&ZU{#Q)0c{ol`WssESdQwv=FC(Gv;13*9(faw61 zPy4?upI_t;TDlfQa4XUl(O-PLzb2hmeOG!fhW%q9Zisp~!{?71|HSj%_y~XUUVU4L zf3rIv<-8E2wJdPT`F|)o=NQ|ccFVVI+c=HWwr$(CZQHhO+qP}HPun`(J^g#{+D7D z%m>|(Cg&b~;cOA0_)mdfk6X3X{oFRZ1~~WvqG`;wspSSeoF;k1^9q0Y z&y2e@F=f`NL@hjK&7F3F^`(QjkpRC$#+Cv9n*~p%d^?nswEO(nj{_9Kf~RZ;MLbkq zQ$ISet1kY9C=-;kP4gsIe%&z3wFaXD>!IZBa>M91X60q{-JQIl;!jKtgXfn0{U?lD ztBSAOqP!>TNZtOYT@lcW$u{NS4(gd=Cu?{w= z=^Z=2hd(84ggynNrYV%^WY}G+mh7sa#}mVl@2HX-1&;`;FcE0k_NKUUs(DJTz@vER zrcz}$C8I*rLI`V?n>wIz)i&Vc?j>YY-QrmJJ9?U@-cY|WRWHLMf`6@P*v8sd234h> z@kCxB>-@xVYXkLr)3vc*mmjIftl@bBY^`uCV^}P9Y%;H9j2^{GXJW7>XTJz!;6IqB zp-tDw5UW>Ng_qm@{$8*Y<&4-meZ6$^uKW%P%FN~`Ccb+l({IswoaezwkiriMkHZ+7 z;~c6ZNKR#vc4W-M?Q{*6d&YgCV!}47+o0`_W4s2Akx3U@mdq25`0l%=bBuImlq_pz z|3jpMsLi&2(}?FJQa`|GRr_7*Sl<&>zjEX}u`r(3xBr}KP%zQj3{qYjDwBLdrAc10 zcQZxpHzOIePoi~O03Cm_m?eHuh0zWrI(ON#!@jN9XwxX+ePXd2T`Q8>mRJd>Fo_lJ zu#K#>7BP38qb<*hND(PP!6Z5L*p0x>qS_gjcJBS#a=Au!sz`CmZQk>8QnCY!vAymv z>mgdufgu`sye0#3$uYQFB({Dl%?96Xe#H_MJ#>Yq3}$;DNVHxadPGE8zl6?i56*zm zY-5wXH-5u@=%m?w?fT`T+c%P()$x4Y7)Mve0CP=Mw&tp$ky_*^v$CYfS_VEypUYLxxz8S_gSRn}kHIjO2+a zI(O#2y+sugbR7=3BG?;$?%?B{hj)0qc-fVoP8(e2-|HNLxf`kIWlwLf2Y*4>m7~7b zIQw6lye-I|x36%Q&`46G=zp5w95#joX33QwagPM{_u*c)Yi%7oqrxfEZj2(HCUp2i z;n)^FD>g@9In5EZq9|9=<(bO~9c*?88cHK7AQ=VWn@fSpGx22;zDvB5?D14QmdDjG zZP@$Tx7t4MS`)p_!6`aPcsF_awfK7XeqVp?0CIHN6mDN-b9crWP!{Z%{y3bposbYI z9>Q!;^Zpg`PMnuB{4u zVJdw{f2>Vfbaoqy@aYQhX_M?sKs~m4R#s8f8n-A92lRU0H?DGT8#Y1WD0Jc+h;67i ztViWPNjmTZsMe65Bps&Vs=wTP&iqGj0f?JngTBIZybHA`L7h!Y)Wf9-VEhJCO{2|lA4U53&fK&-CvWc-x z^JEjqRVD7UkpECI;oPa%My{I%*xTsQN88`MD5WLtgGbW zBc^1hlim>#>l|#oQ54x@4UA8 zQ;I)!fH4)En|eNeCqW>9i^0n|pVU85I+%xarb98Kw$?Z>%`{QD_aLq~&9EWBzeMk_ zM2s|Wq`4&T^OgQxYn@F)vWs}-alguPnn-n4GUDhoI13FCBa0l-1X14|x8le3W&9sL z{ZNB_pd){*{NUbUfz4hu{9s@>HwoZXlVx-(=D@Ff(-<-j+HsgHhU9}z64nfZ1(>ug zRHmr8NbMv@<;ZyU$oG_K^OKctTzNT4d)VK_SEfRNCjE~aCbo$w%fT{UoD{>;uvVbz z3B!NlS^cI(1>*K!rGyJRD{Y~!n~@MoHS@YRoVpkes6j3>k`3lBQw&JPnOwYg8biT` z!7*gO!o>%sAXcFDJL9+=ODoSPBpvdX^9v-sDyB8eF5FZiRK`VeMpCN_?(uy6K_rRR zcbym?`0;8+kkyM&5P?8XDnuZ(UescR91Vux)^300d*uUQDxPBkGAN9{XJ~Gt-+e<7 z52Yh$Y0c9FXI9GI1VdqiJz%Eb(X6P|$x%lI_XSO!x_e|B#4=y}YTO_k%A~`%`Fq%J0Bl1q}zk6L%(?2sSl2=kllZNNbr_tOGvV>eQjCeZT_86hp>uc*JpZz2@cJZOC8F@*UOi_lUVehN|7u88 zq0$2N+Nb_SQC$-1yQZomsW*K}NE!L@HzwsM2~!+3Qp@|t9iuT(EK~kdv(1s0otL^n z(>=OkkzJ6CwB!0cXogxoc+Bo#vz*V2jM&OB>mSYoyjinvo#%LH86IZy25ef|bMV~T z{fxiN#79|FD84*=8Ms4Spr=wjoY$TQ}pJ z?Pj;HB=-c>^%y1%%2|{Sgk5dEiQ}B6(>ZiT()sM~s@PqVV?Jry0bj#wznt9>sU@Sb zI22l7U1Z>d6ms5gJ@T*<957l$f0hZMhy2JhD+5_3kXaV+*k+9wtd-czE0+g{hE0c( z<%YrJgYJbIXU=A`Ll2S9pdIJQb{k#Y*4o<5Ra+B>8)xS_Q;*&XvgdX-)rHZV*7{U^ zORyTYQJX6i+rxho>!A<=5DtGzGO9l%ng3}^F#k~#{ zBGW}dirn9dHJh3)d{HKs0SKQDTi=reaMJ0y_eG4Mb+{QksffRG z#RENLZt&Po7GWTcr3ywYNBRJOk&0e8CnccQFeCz7O1~|D)~kQf3xph6DMLz-`<YE&?q&66AXfhB}{M3@25RR;L|8@EX1zqmyz z#8ij)C+89W;uc+6G`Zh)@^6}ygE@uMXN`FcSxXaH(jciv=zbg~z6kuIw>AHxw|)F? zy)E`X^|tQ+(A(0g_Ef>YRSc(&Q{79=l8-ZuDbcQY$M(VTL+0Y3rhhn{62h9tKYCj- zEptWLa24J8y8k9w*eW^~iRugFBd@>$KHtrs*^A1LCq-QY1pwj;s$xo7k002WMk@Vb zB%!KoQ?HHSEx|~29PPa9yE@8#%vlLnlWYW=7`HT$PM7$i@*9oTu{3!v&5?{*^)ox zqT{?P%AQi|(ewW#7xga@FQcu-Uy!ynk((GjQW~kX2GvurkWEtzliw^KU7-!>FL9 zJ6^G>3WY+RVo^9oqr$o-l_o!L=pXU_UK>>2t!EQDK?9 zt!Ypo`;r!$Wr4d<%T3P4mKgwRTzoSF;m`cy;nJ%+rl869R8=3pk=} znTQLYgh$>Ra8HEuf4D`;G0;Evc;v#QFMPp4;@y*x%`LY~?9S5bQ7qhl^tMN~2|H9^ zBW~UbwdNval`mWS@6V0qe{YD6n&GMo_tOv^@@KIAzfbxO#*Vhm4u-~#wEuv9O&x5V zZH#D4%}oFyA)lW`#YNcI^%(%{08V~B2>{^x`22Tg!n<(~d{mkz322Q{7s?z@1;;l~>1!N85o*-G)oUhEvT-#LQ3F#7EfF zN66Sq(9lD`z>QDWnODb=Tho?H-CD%VSJ=c`$kb~h}-f3z+X&QbR+JQNmewmv7nHs+7 z+JV{nVTF1j`MSY*y1}_RfjK%sIobhP24RH;;f4C41^QtHdck>mA$bO&1qNXS`XTxH zq4|bkg@)mUT$;ARX1-jSHk|5KBIbTv8rEW#0RjfD{Q559R)J!c{#@!-GLDhl8rHH- zQR0^VVitai?g?_v(Mlda-y%lc_h&NcS^=5bfmuHb{;!4q=Y{YM1P}ng3xN8chkt$( z0Pt@=6T$$K-~hs~z`&r$;P9}ZkhsX`(D0N1^bniGl!%PrtW?6h*u*>&8#Gjes_N>R zI)ujBTGaaHs@C?#jvBMRKD%GP%zh0H_7nCG4v&sb4UP2A&kPzH6KoI|Zmn;wuWfGa z9BytK?w*|OLPNve+ycSE-@-maKis`NK7ajvdfojlKNQEK zO+N(3C5$M7A|uTY4w8gGFp6=cEg*{OtZpEhVFAMsi0k=RQLMlUL=hByA3)$YlmtyV zP@;sudQpAETh^4yx`j}U zGNW5IY(~9ZHmnr&U$!hI>xXw0rS@NTwe09Y(S7jY+V*^H%vI$XZN7|Xi$-Wc<;v)xZ?uCkxs58IL7-Osav`JgD5 zhW^258eOwrs8_YSUmth1w!A3UgSOvapOd*CU}AkhShs!Pa2~gT1Tgs!qGAI`8Mi^dr1GIu#|F?kZiDHf@?lKH2C+77L)fPB z;atZC@!kElR~?;g5#s6i4W1IF*v5}Yn&Ufv$3ULM%IF39G(Aeq3fiIE{o4NM0?^KP>u$V zfiBpESZN(2owLz|S=+ifYHkjJwKi-=+r&1J>>hxLOA$?fTk1__ER}&t)EHMMsXc70 zN|l4B9mAN36sAuHurR63RHbktZ&10KG$p8C8e$Nyk8nIoCIn9zkUtm?M}#wwlUV6O z3#uUjvDA6-Kj_bLVGKSW)A@exY2Fi|cg+>kxWr3lokE>dVlYerqyXMs zSs07MQeFKPABuU54?-bMlXOW1P}xkt#oN;FelX`_D3tdRuuAtW!vFa)ls6dH%4zBW z_lhm0eibs-6^#n-E=v@+pMSgz+$9HC>&|-{A2(J+LM}cov2~H~y6{5w*bM0keW>aBpU3!V7}54=%|@aKI+Y-Uk=}V;P8h@_HMx zTCZ=BmybEZCqnO652G)3h-p_g!Xy=SU|JP`-gyQ}qqPsMv8RUK4dD=jY9Ar(bby#n zcmGR5KY}DQ05Rk?hzJJ}Qk0+_DZ8wJ90(AA^S@=TrpOrqZ0Wy&|MXz_XE6N?J9j3F z-*hx=H1xDawuZF2j!r-HSVLM1eHZ|pNn&-u#n#i_AQ>d17@&`eM7 z_4}I_7&s6PfzeM0BaKAkogMvJn|xn^iR$7o8FJ+X8wj`Lx8;+eaUIydwOwa!0HZki znF8_8(cq_b-v4|mPWlG_Jq;t1IK#FH0HB}aMlbL3#(mmXnK$nf4!+F5cnC z8KK0?&kb*YC^#wmPh#!lw`qeFoU)`u#c*?KRY$%6+X?7U)_C4dNC zbqq`lCJZM&oVZ+!H5@o88oD^7I)ij4ir2*`_j~juK{f?-63@vbThzo;wqiP0Q@>PxAac_`gZck!#mt?o zj1BZ1{#mwZr(;#*m_z0}d2=f))IhLA5xFG1U%zm{C}9JfTp2U!iwv=~p)t0%H_xuG zw;nz}(PzD#Z`fa^xnFWB3reU-Ke<0T0G5%Kzx@UBFN-8?35!Y+cQ1{`tV$(kizO~i zCTxg|#_DGwcLbVs1e$jQ++_K)O7)T}`I0mH*lt1DpFuHS3pj$~Dc) z7R~E|T+jsBqhm#R%%k6VA(GJNQqpHXnhBxp$_YJrH) z6pKlbbrFtslMQ#{g?l$=5_TribV8=C`%P8}7O(0rSjwFn*Dd*!Y#B8S82#xq5BuZL zz9~_BdVVtXwzKg~r1<&mFIxI7uPA>m7ydW*wO6LhM(%GpuO;v`1Kk#6H#R039wshY zKLTndhATNc1ruIcS{7PrR$LNZ4su2^Je8m6ab7&dZC4tSnJ_a(qMsO$az3_R@$~Ot z>9weesppuMhK7#k5&!%%eJxe=9@GTMG6I|AwRAHJ_WpA(xu7($}2nEI%aZH;#H9q z88%UTMAxdY6*+d%UethV!hdUC0(Cc}{~1Z4eJQ3Az5}vDtrUFg_|=G~b^NY2K3ij& ziAD){cw;*w)gQH8hQGYU(QflR?QIHoBiass%445(^Oyk_JZW&Xx{>&OJ1QfAYxPtV zQjB)jhHt~dRMha4KHrY(w1U}a^P%N#LoLepg-~dJU^0|v3h4?yP+dc@*RvkVzhvN7$YtV;>~qp0@hg|=s{n@gI=)92$KoM$ zH#Lke&L{rvIQ(8DacdRZ@LF;%JqG6i&Ty(DRqm{K&{WD*)q@lPmCo{b0_`2)mLB0z z@UsA4TUntnm&Pv6M#?rpw7e$c#8$mfLcyomHdgYFiQXBfIp%&``wPDnn(T+Wd)?^|=CF|f8PMEm6Y+$2>e;sM* z{OS52BJjeM162N}W-8eP&<#}oJ4S_l7F!?qfi=}YrPXyw_MJC?qcO=PXIvPp&w z>d%rrnIp{(vAnnZ4l!Hj0+#@}f^$?n<6^=eYk~X6GhMD7AhJPVsn^UND&Ts64V? z_6&1DyA+=^fBtUmj9_Xl{4~dRR9yUE>h*MUUnm)C5JhjeO*?MM`?J}o`8e-@CM&~# zWg*&n@R^S$`zr+(01H=w>(apgrm(3aK}JBqYB*6emz#rWGAqGR*x}VlmXil$M}`mI zh*#l2j`?a?JiPvs^!~F+GFXHa@IzmIX|QneQQh65W%%ZlJ|rW9`q+N#Bsdj9q4o1= z$ZcRa!Ok6vZ&wN(CqyUdk2UByH0gj)au#Aa{;TjRd(-wVxxZ`Ty)LXCueYOgK&%lhm4Xbf@M+HkhM7+V zi_ToGl2ZJ*L2E&W)Hj8ECDEkoVJtQHpd8L1ubC4g?;cVm^w8&|DUX)&JMmH`f_vQT zo&B+3L!3>A2kBOY!rTsJ=3J65^8`R{^-J9ghI=aBfWx{kLEF#$c&`{ANymRt9} z#NbfYrH$a}XU-=`&E(;pRhFsR^|Mv{_9T#~yq?`~B9{A@6tIuH-*|;#xFVv)c>SQQL_|bYxwux{TUo4{wE-7d{ zg2ThiO*XYbY^gSqgVL;M*RB0NCOO!GmItciR%om6p>a(|D`%Xjw@9>;;>KrTZfM}) zM=OFwJ8gQ)0L2Vlhw1?K$>51fRDjP-8P# zf&QrRy>iX+J>*q9heUrlN4X|6*%`mqpaqtTCv(z2cga=zT41nCoG_>mjixb>3ACNC z_$qNkX?L(zzuA$tWjj2qoOqaJofO>urChZ~6Oz?yxe6%0m)b`3aK2^hZ;EFuf!UJ{f?6VeM5%F9~-5(O6C zV*`(b0QE71MhpnKXbo>@1n=(-Lz*qA7zeM~G{GyfXBZioTp*2@GP4>Gia{P0kQux@ z0a%8=z)mB0_fXJSTPP)BAak*RRd{|$p4@06!bkF+EqVS8%z)LJAhk?D(UxEj7RXy} zka{P$`vPbOR!~U5fDaUi`Dv(HOQ;uVNT=aIHmHCPav)O?(U<=~qSBx~c|JnN7@xnG znh(et1CbWnS!@|ZECg|gaHlAWWH2~{ zkbAIb=%VoIg0O2_C?IITrEEdhATg&Z5z%&`xWYd7Tl~&-eAsQ=or{E2qmojhl98m3 z9k!<#zULzvy$MC`pC_2B3)wzz$-QvETL2KQ0RY-ff75F|%Ko-R)IDX++b!-G4j{KU z{82Y3qk`B@d4$mdG*@YkYaUT9QGTvWy}!~iubvinTnmsmIFL;tcU47hfL2`)qFaqI zZvBg|rImNJu*ktg2RBY6xAWvZLu5SXVoF0Jkhx@YkYo-uWIjrgN(~Z6F=BbT(cgXX zuk1wP>vZa}bTY)h!+Qzj`8m}6iRE*X@}i|#5@hC(q*~6z-bHfr%2TM@GVY0q=81l< z|7E(4btQOJXN$3#bwiBpMM*|Bc6G%d`{Ucl@{!cF} zXOpc2K^qeUAapsE-;xX8pXEFdbF)f?Z;bx#WNCat6LP6La>WHKU=3hmg+qcYOi8|4rKt{Ge6p#g&KPTmnA)KwY6gK zOEm<0fmpQ?+1&`SUL>jAYrZ*;5`2d~n}fa9D*dXlNA~8&waX4(DIaZfVbACPR?{M@H-G#~LEE1nK1h=Oo^5H+qTHhpp9urqHoNpIu~ zf{l#ebtx3?DaMxB!qPL$V5VOjlRm^rgV>PB$>WV0lZatcETE_E(8ijKM;%0sy$naa zO*1~f6R9e5PQ_=2#T9!$M4%f=yie8Qz-auO(E8#)5gho@$Qc$P((P5^EQ6+0MNQ)4 z(=Er4WKgmI)$2p5qJ&V@3hUU6*vTv4!iO}8sW9ZiWR#dOwDH2Jhw?Z3&n6h1>gDun zIXGyMwhg1P!l?CBiCyee+=-;Ca&+;7Nwg>l({a=f0b?BbVveY>oEZn=5h&2; z3D}DXvO=i&ngG|?@?qPB+hir~jr@B3Dwn;2H*&3?KoPIP6=b4$qCP2EKbUNH&71^g1+5reI zD|e)aLF|SCUvASyR@?GFz)|9TKD6Fd<E6kyhK_tHd-5X%cYvs>6$mko4@O<8N1D(E6pS4tQhxgVCSvi8O$S-FYL<9U0Kh?7|dI@&BXm$ zikF;*(wa8toqICPpqH#>GtU4boI*BROedS)befXwT2n<`RRlNzwyn$0RHj5MTUp)T#-8%eG1uYI~flPiq~4YUkaedmliBHnTHwX?+51!jJ*PL zPY1_;hJjdtySIkp+Q9LG!QKmkg-C-7M~9QxfDI3W8}AQ6aSPc61~WDc(MgA!01m-R zhofSIFCT^p?+>}2hVf3rNnpbD0mGng!}$WjV2H+C=)#Fd!F|(bz0Y)iHQo<%nZ13< zkuRP`)XweKEg<&w3W4<&vGNePgc`*$2>Hz_3h2jQpGGfu zhi1xJGAtg9wD}H81ECP8diVF4vjQh361( z4nY)hW-K8E92e;^hIlWew2!c5S{Gy5HkG7gC^~URxGx+IdVi$HsX#UBrnc{PLz3>A zkmN2-$&g)EN33isbFv}>(@*wJ27XF5ihfBZKBbuDH&3%zKAtmPF;E^oDbAg9hub$a zn>Ow+A1<+WMqyRZ9ml1`sSKRgekC^e9DcG+j`j79SyGN^qKlhiB)iuz17W^j4C>hptvup_17Lxk|dg2qF`ZLq8=bVZk=goY!QL8NXZTGBL{lR6rJ zaOC~1`)9Q<>4$pYjWF(8ND)MD2aZ}2)d#b^d%9VlY=ankAnu-FXB;u_w z`v>!ErEzD$a#=O%X+iVY8meL=^I~2>_jIXyzcC7hWk&5sIxhOsOoS? zWvOW;(F488J5@t%rQ3j&!7~-2+mgF}GOhZlg=zxJ$g)t)avxovF_*4$tdqsPStGCc zm+TtG)3$|2XGd}K@>GNJbf>y%Q@cxZyJpkyeToJ_j0iz?h={%@UG@p@6|R2*uE)>} zb0}^{Pd4@@XVgKblMiRrA}8yR>y70NgY=C~=1%))ha_+Jhm>b*q;qPon`)RvEMOb? zOvksGXRM&UYMt#3qQQ5}K#tgU4)w*?=aqNMd3Wl@$BW+w{1p6<+6&4bxSTM2uHQS0 zCfsu1c)zGF&v@6`k6j7!z57b-e$o1veeH?O?%z@!%o)7PZQRU7HGFUW9dme}L#wYX zs`GVx--{fG?|!|D?Jy(_Hcagq1lY^xI9Th5aVS&vvWjABsN}q3;cYeu zxMXj@8_vuXvb_o7%pxO{A!&`}ZOj3C!{w(SN7$o?c#^AmrGD;rj~P%;e|9(6Hs?7m z8@ky_d5ZIXZ&Ubag0oL~-&@oATLt6BkI?%Q1}R3({Yc0EQe?Yg*u}SFlPuBFE0N9o z<$1kWstpg<=aX92F)45r$+FU@z5*>jLE3X&@wY#i_rM|j$)WY;Zwzm99d9!PYIYce zr=FOZotc`Dn~}}h@dX9PbA!?QA!@7BqH@o~kbS=P*usUFiAS_*ekBoUy&XZc_P;Yx0=gUE>8I`9 zi?rD?t1#fXSFjVJ*5#VK^$t?jQ66kp3s*IRsi%_*&&}b6VOyW4G62%nntLB|CdqD= z2HwGY1c_jm#_-Ew%d0?722VIMh!?cM9%X&CmNY~~TejRQes`WOBg&8`TYWw+F7z6?ey{>OqJ2_M)uta|`=2>>T{VMkCf&8ypRdK@iq0gsXObEqB0_{P$khnL9 zBFhPQwHl_jN)STA7&>v3;Ws8RLbyB59mlZ|&-?V%VtX69=Dw@}sZN5=2=t;Fcld&v ztcIA$QOXx;p-=Z9QU8<}2dB8!Ix&?x`O2Yd#@)?wE`p(I{}x1PL%^0$rc4JfA8~hp z?p;rHsqFr>;u^1A;%3qqPFti5KM$K8tuIexZl8WK>fHx5Jes;oMmG7YjtD>&Qlw@u zo2Dq#vARgYYFvFgjOBhlZQqDKiwLU(dP?Is&T6pPRDnrSQnU|7O-%wPT(V2=5B zr>()mjV4^2EKqR*h?{AQB!dhs;b6a2+CLQP{hw<#V|ag8_6zY*ZOT4IvY@%1?P7bu z&26Ir^4gZV_=KJ4l$yyYO;H z3^vJ1Pgb2_HTR`Ue<79W-_vyMlAhgj=j>a!j=!9?`9lXn&wS&OaA&ufxYOWl2wSNN z6n=~2Nq6F{IaaN|$Jc{Yd#)$#!vw~g<$~6KBBIeZ7qD3p@uy$UF2`8*hrzJUb-{J^`2l4Dn5^N3g-$4k(ppjhr?D?f9B8T7|l_Xv24%3 zp^UL-^_x-8c|APu5{$#|rYD{!<%}l}Ml}kf*;0zA(;U0E(X--@n#vWzbTLin z8|J*kT^ACyF>NG_AAAwVx`Y>Rua)gpaYkNJ-e}kc`?E4&hO8|w<-(GH`MML?**+C+ zEL0ObUR;iLEL2Qr7>lE#uXFKoFFCy;Aray`fHbi$F-JTnEM_dOdxLa3v%e&0kmvL; z@mPb0U6m+!%t3p(76Cqce3Va-wGRZ~WcyxDDw6ZwHh$fiwRYDcbGXJ+UFa3-_iN^` zEoVQ7@W;QGd<$l1DlL7>XFt8^_DTwVkbQ46+xg}RDZEFgd62e@$?j*NV!G12C71OB zBU0Bk+NhpXhUGj35BrR}y)XS4G@X^ThY?p7NCd(viO&T#)-mnvP|Z8OWC!T{<`5kk z{gYTXg^n=UZ55-zkBfhyKeV;&H9icqjgVfO?vV`2O8(&7;{kUv8D(T?qu~wa00#>H zMfI9)SiL$}j&69QU=~NLrxdl=-~Ac75U$S3Z};bvm8{Tr0{Y_H2%}<~llpEO5!%W#lIAp0{ddCc7<$RmgzPgE7kuUoB9LL!In03l%Haw6mwP8xq zQqTkJFfO(5IG={41%=RH%yt#(kvS=wW}?k^w)tHYZbGDY@mIBC!CcAQb!Fq7THhAo zik0FTm)MFMo)eoCH}GCBr5Za zM+W^v(f$gZ-mQj&%PQ7GtorpA@3A#YTB8d3rglx>jP;GCwB@u~HbA&I)ja0qp~c`b z8sD*o21nU=yo=xSYdNj2d@^ZjZ5`08db&xC)Dl^n^()%3+E^ENnF!rF-mUAYbZBCK zp*2(|VQO(hztFAiEJYE*yHKIUZVY&bJqvr#OfRC5AjtQ-y_vb8}ltUZkOqOwLtV8?0nk9rTpdv@%UYsSMT_? zOG|lww`O3wVjZUo?wQ=&OW4;d;f!qy9I@^VW#zJLXuZ2z?nTn(&J9Ce>`iI85w=j9 ztzQ@hPvt!KZ5e$ya9x(O7F%2G zzgz%wPhIBntdELfG4LOV5jy^O9aL7%0&0b&4W_R=fq58pN z39n`F@z-wHRFJ4nn3r#f;SBWdrjd@OR)KwR)F-e(8#*vM@~f@z zvHK^*CzYC%77}CtVyA5S#s5 z=IdP3fqcJ0q7{^%w=1k>5+)6QoQ(4zj|91=KUV923lvBrHrzgdYLKgbb6T z3@h3p)^0HtW)^-LCbj`s*mN#99;Cp7y3iU_fcjeWj&d|eW8{o-l&pUA3mL){0uI_T z8p9rbXExTMukEZ@qiPOtRJ9Y7%6=O5*!GQI?d@@YYP?xCP~ZAZHmhaZ!KIB z62uAl`(!SeOqT2o3ur89 z=MG`?zc$S2D$4+Tw#1G|LbG^Q6u4dKSZa6F$q(ieF%lDgIu$}0H>T#6XmQI=i6k|N zeZ(zFr0+?-A-V!TdX^yn#rt?KSAw+aXO-C33xBxdyFX|3SA2lE&f~c2Q(#_z_zfKe zWo0<^YG3C&1som)g-SW`Z~0qfbQ0f#2cTrKz&6iP$lJecQQ)Nal}djNtp*mEAcR2O z8i_cjhrVs6Tixa?1D_G!TKgTI#)&eyfPOvta0Cab zU5TyQOS`@zb4=}b5a6~Jp)seUjRZq$(hJr7qu&D4e#0ebNJ}cO=qCf6mxE%efnt=C zwmX8%loc6k$$GBFD=nr3Xo2$D&nEHvcoU;I6~Me(L#W(O8QIFFoCFD#fi^b#a}y^d zs46xtIqWPL^J+qm5gLQ*Ei%PZ-0Rg@7_Atq!9|k{6IKv}(sFgS5qu;{sT^6jG%34K zQ!pVjvnp@Q%5%uFFwE^RUbLIWpevB;B2dRbM~s4^IQR+vAX69s zkIf@!?-wf15VLXSM^NLo6)~hh`>lfYPNMcTgPRuy*D(yPMM6;Q2Exh3bS{2*XkFrX zzlVr&5et^AH?~R0fXrG*af~o00zUO|1o{JRRN*ZkV7{y&OG}36az+xC20IcF&lW>Z z8WXK>>%oZ!1EvZ1&kaB>KhiH3cw7|VhQRRT!`z${ptMM#Fhm4uE#YWwbzN(oCa0_ z8OEE_2=K(-me9u+^V!ABs-r<O)BtS&lLJqT1y+PWw6@KTtDS9rzTw?p4K_WApcm2^J< zi9@FPCd!5!E%=?oHLWz%HcRE$io;5vt%DEk0N!l zGuUm}KVgFwuS}!B$?+1Qpj1~KphH8d@;w<70sXG@-=}9=Pax0H!MccX8hL|>Qt>ZB z(o`CD#-;<%Eip~}=x>`48{8Oa+j6L$cxhtEZ`8xd{a=i|Q>>_Am#w*M?q%DyZSQ5< zwr$(CZQHhO+dliRI!SkRQb}KYm*3@hGc(6C$M~Ke(E&;)3#vtCAl>4V%jS~vxl|^B zuV_H1Sxc2+QLp{7QhsG=h>?t;=gaWyCOw5!%^@ygep<@^GB3ui;2*Cv08{vM%i6P( zZHuhwbeESH)gJGXAMKE@Fo)@hlhM~N&}lS(%l|i(y|0HOx*q}Ae<8K|5>R_ zjNXqwWG_Pa*#>J^B!#ooLJAQSaT0N@Wn+W*6|&cdXjp4_E@5<#@pW^K1Yc2VBmqO2 zm5#tp%x8AQWC1SF^k~0p#=H>{4!aSpQF>TjqKjhl$YSN~2H=@v zc1O%Eom>VWcBL)kw2ge@iy-Ac!gQMDY#YU5B%|znc38EBaKY48+oUCJAF*u7?i?^q zDxwEl}uZC5o}%=!dPZ{f-@rN{XY?LIVVvqWdIRvPz~9EwgW$J~W&4~8)4CCVZaUhYCv?-Y;-6^7?A$dYDaySa0Q4?yH3155{G0klHeXB+MYIA z;W38MTqr+vlrec9C*go<#Z4NrWW>1p~AzV!?=#d|8J%F`~Lwo+v1fMFMt35 zv|4rFC{a;X1*Z99rnf@EKMK+n!a~t_D#`U+Z?_t|3@>5ZUkGjdDx9?%z zsWCRPYIml!W7)K>R@BnAd1VK)dRsZ8vx2!b+1!e1wQbwht#bBGrFTDFrTZTVOP1jb z&vu4gwz#8P+1)GasT=)rLs^Y;oWVu;?+;6z3&yr3Yr7tyU+K)BanwA6ZX_Cy(F~_p z)(c$g1s`1gm znw)8RU)II1bC^f5#i<)v#_oUo>
  1. rS}LPnuf2M-nUE6a;CdV?WtN%j^(>W%<-c- zYA)LPR&eC|C$S7!CqBG8xR(FP3mn zyM}O^@J5D)_ct&`H>q?M^ZNP(nO|~p;9$7(O;@_3NrT2`ef>1Esb;eW$0a4Hiixf2 zjCBo{i_P&?{dDl;z~J`w^ubwupjeE%TbP@MhfbH%`GCkI^uP*htF?{#Zl~kqoaoTX z*h9y0opROBoS%C0-==+D;bd37XR63(1;o9 zF3?L{+luXP`qb!1)d_t4L7>sM?_|E*V_Z8Y3)9oY`ues-v$r$yUWL0@@L0IkNSRd` zoz+K2MXu7k1R>zS1jUo%;GAo#CB2+g6F4!(}Aj(evqVmAj9!O0RdFP2KleR6H$eR^OWGR3_I# zw6fbsrr95#i>miuZ@ac5=Hcqs54z_m24o>FAj@8QY|H<|6d97t#)tkUj;EyJ_QaDCl&#fu$nY{dIW&z84(9qn;0~ZX(_Cz zG{=Tv!GU$TVjH>LV$)v6Gq?QQMwoLv#2tV6S&f0i=h9@`G*M&_=2x~gv;ELM%qlnE z-4s$Z+{dK~r2|fDX`YpN<;U7wh6RV^H8b6^%~|TCGlMm1CF8}?9CXCShO$R-V>EKM zg}36ASWsWJk_OOZ0;mN=73!Kn*N>S> zxq;2)V^?R?w)T7is`A@WH#MtRt}kI}D2OzJUR2uFYrRRIy)R}kTg!xf1s%Y=NzLIZ z$n-Eo=e1veE*C#u{h*};(eI{w&`b{tIGf_={`|>B6Kv=3ee- z)fJ9RxC;<1Bi}-LudfVIOim|Gt6C0%eHTf-s8Hr-gj#A$*ogISVW^m5DS52$ui~0} zuc_~YGw#7sBZ5$8&z3p0e91ERUgILFQJY9xxUy`Hk#?~xPg%XnVOUQ~i%>aB)PW)a z0VXbasCtaKTqLRFSB>iNSF_l0MADd9Za4|HnY00+{-l7n)_@!$#+=`R#9$oyULY^8 zObwj>7%{J-dy1D8c=7)ldFd%e-8Q=8SX}iLU+6aCKDh zNA`l2`?4en(9s+snJI!e7}Lt5f&^+T=xh^tghR1$PPULg*I3{O5eS*iDfc6&P5?nrLyA@hA02&WO05U`U8&?lM!}H^bH}CX{h<{1>qM3$d0Hvh2Vty8#h}Z zUIV%hSR$;UCHfTqY|AI)@0HDN6KNJtGyu%2xczbJ?M`Sa8o?fiqruH6xW#mTQ`70) z2U6_C-)#OqVT{p~85g(g7N;hn6T;K=R^`!@4WjPhyz|i+B+MUncvBy)C&-!-GGepL z8JPb33vd&9yv&x7va9Kl^&j+rg%(UrF?91odKK$P_t-Eym)RTw?%F#TyE7c-G_YQ} zC_Lazt|`8*7W5)I;9IV+MCP>Q&v|#f-u}7c$UnTNtAk&gaaSk#ZvEm_L^^T5O8bjP zT=MBFjj{jFVa$JCoMSd_no=kL02)dF0NnrW#W8R;w=$x2v$i_c(6rrTNA7jN0EUa&qUDjo)ciZ8i z@ipV?;p7?%#kW^^Jbm&l1&nzsKHO7^XjLv85kz=O7P64S_|>VJ07Jk+V%29N-8a#G zPiD(ZrXlGMAE9{`Lu>>co6k`L&iW9wwDQB=BTabv86q&n4wLg z=)BNB-Ab4VmhCa!)J2GBYS0#SlHi_0^b_J|4ijE%;#6C~{X+{GP|nBL=#g7-1M%O(F&J;)Q9$qXCrRSj?ZR%bUH!ddU|_{5=3V>nW}92u;u(M8 zy+Khw&O~6ki4%SjN?^ycq$WvUzP8@2sAYg=BagolSs2KUt}wx57<1@qQdfqHg=kL~ z!`qv}88Z7;(;J&+RA#Vw$wOeNPz2z?qt{s&dZ|-NtP|f8)KnwOK|%Zn7t%OV%qxd7 zqDw2oSvvjPG&qb{UB{L>8nDw&rj(7S%bZga-Z|PCQxB!wSZ1oqkF}t4PE(kKd`|o3 z@SzRQ)ee`J)Dng*+S4`@uq)W>ukM%XYNo=$6aY*Od>BpP>gl4-n^X;#^7C)gi&&eZ z8p|m-0!EN5&0}zssy#4)6Yx)!NL~i8HQ)4n7z8K`-K&AL(=ASzHosP(NC!X|8ET@S z=N?Z@@qxIaRiz%GjB|6L1z#?vT&PWt(~b;yb0O}m9pA?>Ghg_|oNS<!z6ii zGwyJt`KXwkQosZafFle`c~Rbg%4B798JXxxR)RUOGTAxBBy(vty|Opw8tfBD)_qz1 zUZDsOT7b`w45urV_4N(1kqjTk1@DiI1mL^|TO9ZTW?wA4(lY@JjF>;{oScB>Uz0r$ z3TcJQW@S-i{PDNi)cfR;GpFEJA|S36N!GhRE4&b#>`{iWb_KY1KdEErbF(P#3eNCi zbd(3N`23P>hMK<;TLgg@{;O~w8Uhv=FJjgfum|d?@KeVSX${jKvo3~@{+R5A!7G+;C|0^o4 zHSgFxG>7-+4#;^TrQz_+r5?;% z=?So)NU^FNLIs`5hWX%ZSnK{2=f%v0L0m&1rbV{)&Hg;+g6;5Q9*%W~kuuj>zZQ5) zz)!hOtt+{+#;)Gd9bH|hhQ27fm^6cihD?T9KoJpT>n3b9!WZ%9+o$a;<_2F3RVG3N z0_dwuyn{=&rh<{?>1N{xbg?zEbxEcCL*2?om*jl-7qjMG9IZ&6qX znPVWqxE+Z0CUFYViL0i7Md}H}I4^JTAAx5ii0!1V7!Cj^c|krxMk1ipJT!pwMt*YC zj){FXq_Jxp+guS@XY0&~oN)J;=_rYgx%e{Q2#@;lEzsH2q<=DAO1xiUHX-~N79Wh| zwwo%u5x2Bcmi^=@9|zwJA&-eLVis?gw%HZnvgGiMe2DYCCFm`Iyy9N~iUsSWv}b;2 zW*XfwqoLR5QJ7SL0MrLRPAMs~0Jdoci_?&(Rr$IEY8UwXOw_$(FFIP~y3v>x zng3WB4B8pd5}UFR=9DT$m^q8-cwue0wqIKANfkG>DQTrkQg9po750~{5l8X@ab91L zy)lZi$bv06ezKJeThg74o|BBSx-A%6ypDU^YAxoXAq9n{Mw$I`A>z;N_v{cI<{F>y z56}C>p@j*@V9(r|7euJl@qTI&7ej;?jce%y+zYuA zsU)^@h2WH6s^bL2F3DNJR4EoHJe!fzq>q?SkMl+;L<)RpyG{2ueBZ>7Uh4O=A`8_-b;)N%FB=HGD#!%HhcJplL z;ZE#3lZyq=Oqi^^3khdbL)ra@imaW8o60p{HseuPRi?6SWchP6>}?%|qi( z!m)~Gmtu>4VPIX+zP%smT2|b;vPKd_y8-?kzFt3@>ZK+~)#G|8^#S(gk&`J_mY*?( zC}=Axn`$J|?jC+yMa_(nqYKa>i$|`eA_4|B9N~(Lo%|)vMXoTdSL`~8#(keq1S{vo zS{(v`9fxxPB1>$Z+LM#Po484j&uKXb_7y-Q_!R!mZk<5Z!5Tf8~Q$=uDs940Tn9Y z6Ew%DSEduRLkqx7Zo_)NDHA>I2Jjh@nTF|*2I#UE_?wz-&#|6@3yH~%YL-+)tHZ-E zMwMn_<;veetaMzVT&H&Qxj%RQB-O#tM! z)QR{z`_al5k6_g2w8+zZg~OzUpm3Zk+PT?bJ+rQgbmna~KI6<+?L;fj z`=VcLb9@2dP*Oed<{-YHk(--9DK$JuuXSKnKO=mkPL z>c3?G@I|*(jja7dV=|Adt-G0gSUBd;r*6I_qxX2DZ=oQzP9YY)nB&bDR)o3-yn!Bm zproo1vos zrrR7_TH4;7ZS5UBGkPl@cV;doZoviK^oEvXd7U^#qeyN{7{68@Qi4V2*T`MnyWp3i zW}~<-iGW-KM=w20=O;Tc)xN>ikAq45k+89QULP)>{9vn_)z=#kaRQgH+!%ao2-5I! zJ3O1X@cZ?!Z~6#Et&VgqUVbyt-O)0N7MRj+vSXlLO3eP7Jrd_yv9m9V{J{Al;fS@} zuCw(|DMM!g*U}*@T=6)BB7Xn6*M4Qd7o0m+;6`+PMq;_ies33Q>&v~|)6P8>EdD`dQI9dY3|0JyNz|k0ViKg4x z9<<5F05iSJ9G=xjv;{(g!I2IMdk_zK`_rH*G_WUFyM-Ha*#Jt*l|)+l$*|If#zNEM zs_19HIXop${WYjrRbuROq#356Qv+;-JtO(&R$z^zNx@3Vqe@feDf_o3r+BTH`J?HU z4b0FHqyBT7TvVk>kRc$wlvA<$5wsU19ZMMZlyz7^Yf;}y&6t%0g~nC24w+~g#3g^M zfEhwj65s#$E@05L?+_C>)>MwMh%rL3Bxjt;NRTW=R|OjoZ&0>PNKL3#wF$czfw)Xk z!dgg7J}9Qje}cy{iAu_vl5o-IPX$aE{agLc;MTh<rc!H(RWo)L$djJhMArIZt!iAER|6T`z0RaPrl|#%+jfXS&$N>OSe4!yjmEGW# zNx?}ct?kH;wvqAV=BN)RG)V)zDe1&DK&`FHMO_!%WA;h-Wo>k}2= z9wr!n?`!&cb$9lr>-xT13;QF-?$)5r@Z&3l@B_=x(%Q_@#AN^T8_4oZ-{f4!@ZGDg z{~O)h%*N3C-U|@_1M9@d(#XK#Qs2~S_?7GXdb`kX@xW(lxd#YH@QX!bb!IiuKcZ)4 zcy_gw0C-u@k;U-QNOgDMCUg;ZgmB}Y%Nc>2<6ho@l;@9%Lw$8G;3Vw7fRlwQ%K8t? z2+*xJFs3SmV4^#14AMhJv8 zBAV<>KKT0j{&IVMcC_~8+CF(@0EAs)B%O$2QV&82kUAJ7wQwW&z#nJ_LYSxEYtghMh0kIbBQiEHt2yAQ!tS^2DlA35D_5YDT;7;h zI;MQ}lHMbVMqj1o(}!5-<{(o-)aXYA}kVyju)Ue}yW!DgM6SRarN zQ|CrUVtSm%hTV>_vV4sU-8?K&Xv)&aO>#t1^1FqNs8rZDb8cP+_|-puupDRK%KDnj zf2EH(?w%hqNjGY--+uF_Q=ViWB_@?G%fbFE6SLvIkV*)Xan7SVgEImls<}Yp| zfYR-ciIF~@ro*W3;WW!kW*)@1-P zNR6K)0m?vJk_B2E^<`O*%nJOd9t5XV8AEX9`2_U3i)lZ9>0VR?D1jt3e40p~t6x7% zQ>vN`UW!I;h`ewAit%~6ULvRjljkFGQ@GUV{-cT)R6xtIiO>vgeCCV0n2 z7Tq5}rKApm*>Qvb0hWqFc@U^$VdBWzR|BAWEN@a9w?o$jhI?{>Z#xhS$<)4wcue?K z@5Wn#O?!Igj!=B((gQmHFobjMLNF%@M-w5Vd~D2@EASlpF&o61M9#5x@wy|?yl&E z`Ee#oDhMoEV_sUT8)m{S z{#p#D?C_dU;Ew(z*w3#~fUyuGxxkE#PsuAKr3RCjTd5_6Voez9N;1EKr&iDw{p5l9 zdl(5YhOP>O6iKz~Hvp%1)e$)U%Pv1exUmPBS@rM?8XaNdaj|GvginAP`=65D)11P`94B&y+E0|LB?o^l$sku-^GFXMFhiTGTQ`n?V&6HJ=HIPreD!c?u zJ!_Nj_3Uxp zo-%>OC54AWp;>(!u8rUG_c>s>?|Ia%sX%_-dRuO=@*;>Q4F<)Ip#3|QbdCu*xrX0^ zykR)U4Z5vQf`IZTb0+x{3VWIYXq;_Eyr=j?6>3$rKq174%i3BPoq2xMZ(c%EQeP8y zw16R(BE)PsDFDCXPxyif#?Yf?b0u#umcI%>0A`|mc>Tr0vcAGcI{8)T-Hi<4AD=eE zM%@-8PNBq+4&XAv<1xsfd76WIt&I}X;hM1@Iu>vut)r!+J9|uXzPAMi8gt#bL<#$j zAnH-d7DE2ZoSiq^#$)`t*S6v6(#QL;;9tU^7<&uT+OQ0Gz~ zDdcK2Vt_F*-A!gvq`Q&VZCPLOYD7PjDnKTRnT}6s-Vsk#wpb4mqhYr;NxZZvcPbyI z$q4gF*X*<)b}3u_Aj0-;eY|oFa=KG>@I^UwObU8o zWhF1Yc{Z+gI8?oR9Xl>OKJ8}(!Bq9Mibx?HLg7kVKg%KySa4#J69T3&kdQe0lJ?;yG9ejzz(EYUBB>FgC$s6VK++$z)f`OMQ#@#Qi6CDFe~08maMlgI%;5&SfQ-ko3eLef2)$&Buajqbms4w=UY zpzQtp=+PI@!uKT3oG(zP{=R5QmwiRko84+OZ9p=xUxn|71yOQXehJUuuVq8?Gb5hJ zN_=pX@hDu0H=FI%_hJ#o@bZbbBN6QXWErJIld7L z@iKWR;jsO*HP0pABrdGx$f_n(QEi?RdCu3dco?CV8{lEAv2&R%bU9m>2@Aaq8A+56 z^T#(fFUQZIwOuC1xB_kQS^eRK3Qsrk;pJpf<7U;;TI)9MI}jJ04=|q_!(iK)61#uf zuy3SJ>8L&nlu&i_f_SBtO-fl}fNf(%D`VgZ+eU_(Lq@Uzp=rJ(bh(Fx-ZLG6Mpt!$jk&ktDff zg@3#?H!`A{Op>KE(LT{PO(UqZYj3Wp^hCE5~t(TZcVX4y3P}|AJw{^48f9QaL6o zIFrPkdm3h({w3dBtr5&8-)x#caorsqv&tY7rtpOuyNC_9`n;_Ea}a(BVnl5|a?Z*U z1jdCTXG8`jdPGY4r~i*8RhvmVgzN2e1-Z0P$I26CX?~&o4BmH8M-n~M+%P9>z>-QJ zcr=9XrY!N!^GXkb1@n_cZMvqelVvv#S3Y0Pj0)NZx^~7m5oP`e6k0J-ujx^iWW`@i z)Bb-MaU=^D4#@eP-~lHCO-H)G`0YwZ$5)#$BzoEToz|6L*l$LQ~u<1dM&QH zMUKmk<5Ie+a2%Q8TOX7riU`dsT6eJ(Q-)wdd)9)kH_g5IVXz5>Q;oXXluIf~HcfKi z(eBCi{lE*|=*LU{df-a2g6ZpM66Df(^vuCCtXFut)f=wUzDs*>%ioa z_EF-bHCBtF!h$)!`j^lo#74)YPzoK7f%H4jV!Vdiz2CVLOuBzACWSpN9>Aw{=v$Mc zz`w5g=_X~gjN8jd+Rc~H_7qS=WI1#qs*jxwSJBRUzg-Na>hm6?~rD>w(fd9k@CHn6W*Czu_Ca#Q2E|fNa-wP#P zhR4PecJe{plZVYxrl4^_=E4xl+LTwlFFB}J8l%badH-4VjwnsVe7JSU6z!7|{mOP| zRYzY;sJaa|-8{?nZaP(ky&^%Q9C`}+b#Zm?V0~H6?|om6+GjKdz4yKd6mT%Vt8yW$ zVSqo6Y_Vj&>rR~84x*D$V96nWkk2Wrbjn!>UnI{T^_$6Fh|U{OiPih)16I=ssnT^8 ztra6+WzX_s3(=Ahuf~?2j-C2JhwpI2%sQCfikP{HY*j;2_~Bt|Ia|9|xBLQp&qaM9 zuzMRrL5C2>0fB&c-9=S7%JpJqWNtyNuV-#{BryC*O>b7-NCGn!J+Lj?sH)Z|U+y@5 zq7bX4^`h~zE(%aUp={?lRnw3xeHHyFx(xN+4)jW1Ev%#UThPYhCi$i&9q=IlV zUIPZXj6#_3%EW_>!5WUY5E{jlSs7_)G)dFBlzkhR%n|GKTFO;aCXL`Kmj%sPfGesQtRo7$AA^$$qO*jX7HF)1gw)OBvb0qw;uP@GMAUR zZFp|0OP{3{-_RhN>Ac}YjU1s2Szr^sA~y-><)&UKFIZp0+qBg}*h%}~#^a8VrIace zq$cl!!cGImA5oq5hO+_)mAl1zi!o5IkU6iHjuf+tOcyOny>AI1mH3Frw}E~vtadU3 z4c5TXIDQKG3DrGV%N71nTKiO|+SN*}|JRt<6Ql;LkW3k=IvfhT_8#@ow}IsNAgdDq z8bmk(6eQ@|*W^-HE7^?Svy8gXcQ*sZX#9^~@J|4go&qF8e9#|3;Mfp3AYdGDA@`9s zHXaGTKtq-x0P}xkH1ex8>t|Lg4aKz#f~KKxVz&hy_KH^>+jSKRl@crpUbBbNlE5Wx zo|AXyZP(8^$G-1B^Ii8(+;0KI0QP4??Kt;Vs#s3VhYoe1EAG`W)tBO7o!q=3ey3MX z!q?px`%Y#NJA~;cTQ*|0i5<&2ck3|gTRtLonQ2a@ukm<8rfZyNrY>Bh-wGynE@Hc% zW~(nA=DNKG=ojXxcwY@tvtf6Vd(9lWrqx?56^enbnhH!g6Is$ zqdy$Tg!Rh2f0f+MeNPB3QOF#jttUah zw5HYYpNJ{g+8CX5l5h#GE*IpTSznGb2SH$-NfJb%K~YXsyjB5)-($v zpQN29SQ$JVO{kpn1Uyq(1CY8vzSH>Xf!^#lkt=#huN zxwlx@@`Yb0a5_iRv-G5^mONTEA8J*Zf5@)itF2Zb_*GRfh550de^{Puh*9%N*{v{T z{IAxL=e3bWR&`qW$)0kd3zRomN?wZ2NYCmqktceZGY;E?PeC&pux)tgaEwSkxlrmg z6DZeuVwZ2u-U7&Q9V@yn`%4Xwi5k+N3hg38*c)01uR5|%AXF}>*x21HM1M18rMVz= zeCB|j*Iwu7N`^Kh$WL>>G)d~5vgXRNndm8k{5}Z>ueDIo>wPxE*X^+9>&=C?(^oF9 z!0H7ew%4Vn$y*0B{9A_I`xhyUc-Z;|P>2H3q{z4s^$(Wee^>k&(%RlY!?a>lInp;6l*BqfMa^3M)73JoyaE zCWp`;l{-Ug+z?*ex7=h;I9$~#&};~=tn7DBL2eYvt9WH3)%E#ZgW3D~o7VsYUE->g z7Sj*Y#d5ECI;-udel|x){>>fU9yms3w^9Xr@{02zg$m033(7H!8zrf2!6Ig@Cr*=n z{(7jP)+U#)#h|bxL}~D=&{xcLJ;#Y{@X}yKSCicElZg13l9_b_mzEJKB+L_Nu=r`J zgGDY)v+%r8$BK4YwAyemZ98}9if(q$CI*OMPU;v?PZr;7f6IjBu4sn^IDM!2{E1OR za&Rb=FEDPrtaMNSWieOJiIq`R7l0>dbLWMaBrdKc&99@=1%1_*v5kQ-66mmk4McK% z;UMTri;QGr+Or$?`1I=J7!Zi9-d4EfVw@*=GAHK`t~YzRV(0P4Lb5jY(^^hpx=D?u z!#l(cg9Yvy1UsBF`6gKt=efWRvU#ZHH`PWrpf)lXJ){{Eitjp3dJrtEZp*n#s{zIl z*nMDDE}o!k2qmT(K$0OR&30t(EU*61sO}h?{zF%>GA+)6^X`|j5VPz3Y zs}7iYQ&i4NOD6b5K0)l1!_QADCLFt} z4||yBgi+={Ml;ws7k{MK{N{ZP)NxGxQLo9oj6O%suWE?X19D5u`4)O+gkKKR9I=L& z4$Rq|Jy-v(oI|zA`VnBM^Jx(o$Ko{L?tk!Ufn&%4^K{*&+x^7Y zgAsbq`eI?I#Xujqx>2NIEpPE|LrLBZbCz@?OQab;3u3=-?gnWTJuB=X8uoxPJA4M{ zC`G9)TuyKKnN<*)jlUA-C0os)tQ=OMUlb?6cX>Ghsiu+HAE}y2 zJ-~xth(*N89Iu!vacJEvw1njqWa58P9^QaW+4`DN`Q za!?ngF`kV*1k~+%F-b>>EdQ&QKuk))qQ#t^J6Q{mFIbLif3C+>j66SDbsg_o&l5@1 zZ!GsO>Vp5g5)s}EZ^XL-Yys_*E0Vjd&K;EjCfo#&eO(JF4gdVo+chJ(sB3SROg!l1FK`_l2L& z(e!xP&~w{0EQ88B&ghduft}E5rq$*;e}=x0pSPSLDmuKNgjCOkt}>Y5toBpW#kZ!U z?Fm>E4NN%^iME_N9Aq{+)MLAlgPmtJZ|9TFwTv#}vWwBxhk6nr-!!bgo_c8kS5j3M zYWe)=AgZ{UHfpWrt&umn6v!?FfU4U<5=vhN&7SXXYRPxNp}u$d9jpnVgjvD?M`@!6zuc?JACV& zAGdK!gik}YkvF7d!)%q}d~)rR9x|2%QVyWUA1v3+8~Xx-a*K@Q6@Fut_hp34@%mQn z!gW@oR}K8zqpb7Y-LX*APG_j-1*v#02EP_5}hk0A(+02hPYS(5B+!FR~l z=hW>A6Wc9R`5LC$Q`p-D_zD464@3q#nV2{y8F%`Y8NSU;{v0WOiPaE`iDp>84}aO) z@tt71SrHb6$4n6Ky%6oT4@=uY2t!>ES}K7n`{aPTfttP}DPJoUB*h{uSq|%&Y1fRc z@Suu$EfKy|QND99D`u4NvYbi0e0T3(c{6cw+x5c5ETxSYc-XTxSx9u7)HAdol*+>* z1>q`9=Dmi(4z;R@_~i&?4sL&|@7Oq!PPV1sBn@_8D6<;o1=?A|JnBX_|`gqL3_XsZM zcM58SQlMJYm`8Ah<1f0XK7kVl-O=DR&*D%_gVRL%_(>n;I9DOKVX;rd7095ygq)BD z^n7#~DLPZ=nx~Av`K)IhWo>X7W9)8v1yE6F#$Ma&`h020ws&C~53a&@BCf%=$u?5@ zhs104e3lyTgkTewXy8jC?jQdo4rmeb;RtSIzi|b_=st`3f+a0;ipa{XVvNckJ+86- z6RKAxTgK#t@_M6oxef8I+P?g+KbFJSLm#kuZY#|&Km*DiK6L5@6e@;SDm&{Ry5~tQ z4h_=MO1W2?hnE}@-}mVD_9>7k_ZSJLW0}%rq&67~4+Vr|zL|=T z`&T#aQt*6* zFY7sy1>01F#K*;(_`@*oy(ow-4TMrPlkJHIC^l2U!=x%3Y(<~mMPgm!ZIP09MQ{<4 zC_AN@^QWDAN2HX^c&8>;l&b@Nz>6aa1-eS!UwG;uN;mPKOE>XuN}R{;%B=&u(%w+; zN!~;H^rEed>~K(rc01*&BDaEWUR-xIxl}s$Pm{XxvdZ3q7fP)KHRd_67lhY1K_Q!U z$T#6z6*`ErK{Ii>gX_n!VpLObXW&L#KR6ZHSr4__?p$ToPg; zEDRmw@C5Uq@K8hqmxV{5#Vac&Gz^WQ3dePEh5)t$Qw-5(bdQiCw;1%Lk*qPh27}Jm zy#^gKj953u(k(qeD6rh&4J1<5WD^#EUFY$gu3a?Vp1n(K7t^W>#cIW78F52+3 zR&=cHri+^gBoC=zI%QejgG!XaZPA^5qW&u7ZN%I0$ry$_Y0XQ%1N=#wSK&bzk>_fy z1ehqpuI+H1L?c1y?`IBQbk8*JE@lfXp{Of%pA4l0g?qUOvfn?_!lh0*9A?U*S*+s`1g3u7Pp zXkzor#Ap+CP9jl}ifHVgYQ8#(d(nI0Zwh&C^2mFazs{jEqdC3KwGHqJH$IeX$>(gE zUiPs}Rv%D3eFWvgoc@T~z?{x+QD2W0#MdQTm_OreT0X@3-YU`J%QKF8%)fZ7_0Ae{ zjOv)94)GN|0du~^cq%ZV<5#12KqLKTMEBjrINs5iE9aw6evFi6K*sb^KO*v@aa0Ic z?H*MdSTJjTLJp&)f5+YkLPpL3+kh}aNb5Cv_zDmR;=ueNbYjNLW#2+F!uYomXm1}n z2dtKcGw=LFNly+&-l7oLx%k^)c?`uuU64ef6^c#@w)8n|I@a1dbnxB)UF{hxn&4SW z6-CpwSn=~b-fIAIDWH1EM7p4>=wxU$Zo=86eFdTQM2i9<63tES1ZAc%H7|w*_W*hN z=c5ZfB!p`yGMW-x1Pe8+i--}HeX0=W4-C?NLE|{-1Z3e4srZ@gSG>|y*vjC`Hh+8bSS_#U z;k*vWc+^D~vT>^7N-H&9KPM?sJw>c@IwUNr#$3MMBH1z)!(ookU?tUumf6J3s&n_0 zJ!>$6xSJn*pY|bUo`Q|VD=9#jO&G7WEKc%3Gp#in;P*)0DhQW+5}zW{ic8C!Xk9kR ze#9nv%W=#wI;Cz%ENeuN{-Vfvqy8P4&nuj1{5QuCCRwR$syvWx?b%=Xxe_=$F?})k z{m77Vk#E57+=2qi5~FfR7!YC5hh1TjrKPF4Z0L!r5vYifhl$AzNS=mlQ;63Cd>PDQnWM=w@x0D6p|D|Eqify0D)I0 z^3-=+0Q$OXoE#9`BaQYTOT7|^*Aae&gJy5O7I)%dzn3PpQ1O91o?A*mNM_dhq&AW< zus{qh%rmCzC3UPWY0TNUp27@gkoeY$p^~$Iaq)rit0AEfG^rlPV7zJA7n(&9;^7Y_KcGXMhv6@0D6=bS4%(NQ?Wj0r^k@;6gd-_zyWq490TKARrFxlK&R#TL&7M(e&d~;axJD1l zMIn~rfokrACKgXFNN=~p|4jKn__LreDBVGun%mxW2coI<$G#mk-<|{x;sC7-#o5lb z9?D>b#DF2BS#41~D=O*i;Hei+UP{h~=(B!UF0&frD3e$0FJnYa=vtQCnnf+Hzd|lL zl`y_t9!89*6ryy_9P`Q@hfZ+&vQN%Ux_T6XI|(v`>h%E1KHrQ|yG;-#uWrnh4l4*_ zS7Qzd%^KYpS=X#_k~B(`JkRRS{i!as|Fbhgpd}jS4b$KI(ZD=|bOoey^fR&ol$8Rg$i4qPLu_o`2Fn(A2zg4#;s}+tA-7XA*beq6Bd$#R$ z(ifoBr_Swf|CmrdMb1{cg$fx>M8hrf9iOyBCf*hs0QOZ%g*99Ve|9}3X)isg?x1Oa z;2~#sQ;>avo~FmyljzfplNbn3yNgVQmuzrF_qM%x8YyL&0F}j~vuLnbtB^`|S}asZ z*Oj*mKDVSg7pmMxhgLA}O*8v;cH23tmm8U)OayNleOArg^s3H4gz1KF1o-wPaiZMq@_$TMTTZcT6@6PS$kmKT6*B# zTBHwlhu*H=Q|u_+1wr7OUP#;P#d1J3mnDISG&#SR-OSmiw&7WLfO3M>>a#J}uVxVl zrF%`^)TyW}>Y`40*|YN}?%6R~K_o=*uE^_2@nM(Hout*&MNSkiXWB`TaQ_%8PF1#6B*xZZU~ z==i(MiC~{Be)0e08U1CcbZ%PDj0ezW?kJD)uisbG^eE=C(JxjBOuhx>c19;*^p3H( z(0vVbnR%s&dq?U;|3V|q{p|&FpDqnBW@O)ns5rNZS0j4ujL$Az=2-_nzn8@&NuvAN z7N$yy{&_TS4!N^up%Y;VWg5>4SD0fmbBt2UBn*FX%wlUnSVy3;eimc>L;6<+)8rS< zEkt0^_8u?T1tjC*4btt5EOMF*Ol32J!juFG@$u~i(H-jQY{dCNQd|QJel-|!#1>CV zFi9EK8NCg3P`f-~@G+26VN^EJ3?b4@*k3$i-?(qsp@G>3>REzxGMkX^ePuX?9Ak^P zyg_B#@;}c@cm9?7&?cEzCJW)yqm7{krD7K;dFSTqyce1$c!S=gIC*xsbqiZ|dBfPaLqpmeE5f~(CVNAl~zX)f^VnlE}y}Yn27QDjr;+K@% z{b6$S%Gy6!bxu*ARj=TZdblH1gd--l@;f}y#u6?N27Iv_J_-M0d`F1wGV@Q}IDrBJ z!u;=b<6oli@6la|nzb^nDwZFHFvqqXY_MK_3l1{7j)-+jikb?Q$_A2_q@|_xrYU!y z@EU-#OQE?%twu{skSn(68(3gf@mIz?_8+c8L8p-l0rRvm`snc^{?qQ4O;7o6pNGjh zAiV+1E(fq}oZJy?bbg#2(YjEl?36uvbYGeoORhKVwEou9+bwlGQ2{og|vN+mX%of_(}*DT8vXWe2R%NM>~5;r76CPD^y04Rn>V z$H&n!CH|9QRamwvOwQ)+ocgh@B}+^S*mhb?bfkZ)}n!FoFZI`@kbmM5&*S7NY89*$Yu^J47iZ&wAwMegk|^ElkGq;TXE*hRJ387)U@(Qb;1Pp zv#8aWGrhO)`#FrUY&(pVNab;H=jXlSeq=+xzc4`v!PPF&mh&i@=d6>>!qPyzu^g;? ze}DWSh8~RzIa$2Kqi8>EqpH0DEz1@51Dtc`?il2LV>t~)ps*avH4R-QmXpsaT2Jbf z__0^Jty0F&n%N9BQ?cJZfQN2AIo7iIOma^X|H;A-jC^}BVKDlr-j!>NmP>&L9_Au4 zSb+D|TQ;A`Oc>xJQvwM(EmxC_Ok75L$lNd{4 zAn{^DrMv2o;pW@}@nyWVMMbwa1$GN6P`-TtZKWFPQu54$5YffQ|AR+hNP>po;uR~e z{8eR?)Q6(~^ePEud9i`$QK_dU2u-D?2{Bp@I>|Mvt_Z?+6XB!LNrs+NDm2!=U~e8OrdEsw%2azmS0AsTbqp1WrK;7GjG&bzST;|3PaJ5 zo$S3&KWcR8)_u><4OIu@T)|Vgm2^I~tX{4Y9@>WtxQGZdH|~*t;H-EH`-q`v= zr?WsSCo{NluCh&>Q9|4VNgkErkwVO^W=Hb*TIp@)z_XGr_ev@Z!}|c{lE%-E&Xlsi z*rK@k;5Py)Zu$(A{fuK7qay0)we2p7o?UO&BDv&X>rP@|SR(I~fK>HVnC5BlWM?Qf zS7PklwbC72awQFS{%d8*y?Wefxb5Bz*DOjAJNpk|YK=_!P551N0qf<>)xQ9qwM zje(sf=y+P9BqHSUHF;G$G?8JQk-G$>1Gly_-Glv|zwB=hgz6dgCJpEjNqPfTTsL>stDJ zLZ1{qNNiuVe^qjBEX0w^(qXckeeXl!ZX{4a-xkKyp%8xAJ!?STrNuC4r6*NDnCee> z<`e8PU*#b)Uu`BOSng5-tUUQY-2asYF?YG@Yl8k;{>{D2$9N|bpby=hPfQ@Y`L?}{ z)B18Mcbg|g*jUqY%9WyzQ4pRb*6D_cs;#X&U@|c)d`ZVKyYka3A=(1j_LS*RK)Ktn_z&D%j zY_`F?UGdjpZ}SEX(Dudx<=fU~g!WRxP3#L#Pj0;_^Aentw2+1>k|2J}al0wxnl&#e zf*Vt1r-yns7nC{$-g>G;`7oI4ssw>kORw3^BAJ$|uQWGOkEh)!H0C;{HF*l$Hi{*~DoAE9n0Q^%=#ECx=ApsoJIv4JKFrs|<#xkG-IVdY*XU-A=9sF&f;m?i z?G8g-V*QqZ1k2ql#&!0ZIrjEOyBR&qjOCnDh#6J!PD@ReyxpFb$z#?TTXk%7D?)R$EHZ%_buaIyX3k~fAqr! zrHEu+3JA-o8Cq15P~>s5J848rFTX}l(4-|@-O-_ewnhKlZ`h+aBta7SGhdWU@# zqn?yyD1(`+K|JREe!!aFZ*in)Hr>m+uPG#XDqIoW18$inC$ zTLLd+uyVzJlModwY|cFiDqw4_=s$E zQB`=UtgiL8Ad-Gxf8rl&0g0y2x_VO7Kf%`8hv3c4BdPN|YD04D@8XA?>u7g1Aidi- zw0oT)v}wF)bFJpkyPChWTdVvp)knvp%i*+!(=d?_Z_%?)X=R_HWKa2aV&27apQDg% zU=pQz_kq%YJe&DKC|)gua{XtD5E1vG2u7Is0=@64^yA{@4;=ZzDDOyei+5SMqRbs* z6CXWbFz?umv-=^FeI?gP{s`{BnDP+{35aUM?Rj(g^D0Q#a~=@=If!PD<)yAO9TM;% z6x@?$xcdXygWNG`nrP**{)rSOsSGEEH=>)o1@MGCQ0Hop%a`Ai0C|D>3kwINU!+I$ zVZSA}zn9;K9K`D3+Ndf@3ulxqDD;Wzj{3v$4r=liN*8kMco;)gSS2hjv%1?*r3^2@ zOByNu=BsR6>Ho}$-g6&9`};DQ&Rz-6HCrSn_4?fd{2p$1g2)fLJD4nwV%o?3YS>z5 zUeoN2$I%1yH60zv|S8G_y|4b*VlinddU|< zRZW5d0SQ66b@vq^bov8+<01s5W>72f&&QV{C+K zWn-OgwVv(jK!E3&wxYjx9jb6_s=)P9qLd9=S+M8GMnPprBm0JJl8lHTx#|xl2{W?YhpUF6>>;pt@ND=rj|yS)+Rx?TgIv z6$ZH=dSU7K-J((Wyh{%6jBta7N%{!2r_vP;1MFZGap)=THElXNFlkk)`Q>LS zF^<3v_#`Y9_+z@N19TetRKz9-26khCqD3*{tDQbw=wAa|%S=#BMmpHFP@-L$S2-{& z7R`bs2ospphH?Y+C=@p|pSUi#jM{gU9-7uk_s6&HU_!L5!8qS_8;+Nbo6xqbP=GoO+Pi2NXp&3wHs4)-cT`K26>^{dh8jtv4=*4$cA_{pLnI2^U=DunyQ50IiA;jf?wl(;L z2N}i%MwjmeZhlujK-_14nc(!b;_+k5J)3ikhossWle)nUr;$;YG|!dLkfQ#AD10+} z7fpPB5XCSkLI)90jFk(8D_bloz&R^vvLjDM^4uKM0$3TKaR#wG1&hGH@O{8?ay{o( z(mi9wfp@^Q{0z2i_S$2`e`N!%nfsmSug?#whkRFL-hYcaRtNU8`7=OyPb_fJw|nVg z^bBU)OmMhdVqAQl;So7LUU8a!6Piv8w1@c4`g0>6jftJ~SjJXicsh3I%&m_Y!`DnpJX8ELRhD-c_>(FsN%)DJ+ESJ%V2Icf z73To~uzO1ZNMef<{^owUS-yEnfqQ$ue%bb($W6Bl^}|`zgYsz0ZUArVR%7AP*;;_>gE^|2=s&`tVA)|0abb z2yck!M?v#l$LB{dz<4*wVs%f8{gpSTmo$HgM?EHjU0IA*Q<-dXj2gMC06d_HCPLL8 zPoqqnkvh4=wv_E8N*4-KpfX`m%U5}%Xj`6%v{7DNc?_+RW1}E%TrP9OEH8=!$1WJm z;93Di$FyV`s)aRR)*^|43Aubc&=EYUs6KWmouMbzR8`Vk)VZM$6RKraUSE(2w`^06 zAwtioQYBf3bX}J)+9K4lps-2}28mM9FKR_qrO@F?=0&x^lPg1}?OkYGg+B3)B2r;t z$A_xk-s|BS!wwKv7OC;=fJ0WpRA_~Z9y0D&wvIrSMvQhT9#!t}5HF|zB1@GU9%0S! zpjpM4E|EZOSmF4(Ah}fFBQ2btiYL|TDe38IGXD79Y0aJa(r$CLJKGRD2+^XNI+rYK z?69C&B?QDOKO(h_;jL%N6*~>1w5Gy{m$I@aOD4FFpEQuJ#x$E9Kql%)7c19nW)}>L zJR8{LlV*g&>f{XXA_FKO3mbwkGU(8Q?m_48O=`RTlnEjW$_5Tw%#I^VmWG8M89c(; zkgyRHx;E*dB9b-wc34ng5~?V8Q?*beNFS{@9ge^vHdgERuO<*gM=@4d-M*gv;%nmP ziI1M_Uu>{w6wH(w+oE`P8jPqe+8VK7@Z<6d-VEiRaaz4MegY~(~|I!?BCKj^5G+=x6=wcEgrJ)|+h zB-GnSDfIl%Z(r>aHn7X}=7Bou%K@L)%u=u85}+Ml!TrP9Hr$}XeCTqPMpq?Az2)TC%z4Yv#1wumHYwKgmkhQbAP)P29;WS!0=G8;6g$F3n*x=eMFhk=v`*G!b<{Xw z-eO^%E#cazp83?2%&)1jBv3Hn_{+Fd%B0N*J+OUfPrLBO2u3h(r_RJLHI#)5WSZP@cw=IY4vs)3L&MOgv(CU8#5Y$Bz?0Ig)FaKbXzI+q+N zqSo@fchO~E26%*3tW;)cdT#Al5XzH5k{?$!SFBWUD7RE0VApV}Qr@sGLm<;~<1jCa zw~P-F9tKhL`k!W?0hU7$Gb*!qDz%r)Q1b>KOD~Aw86AQd_sLWJ58^ph>-y{!wE^AX zr<%x-m6a8CDJiF5zq2adm)w-EktQ&8yWy0TpC%_xO&4>l(kk30jpH_CX=Pb2w2j65 za#XaL0kZGiQy$Tnq>XDT-orc}OSYw*y4Rfd;a*h{VRyu4+_&9UmE|~0s!%{&MH=~fALN;P1kL^4LmBbC>_0nT5Ln@G}N;VF;722dR;?9FL*Ac`Ye$K1s zkN*!_?-ZR2)NF~y-mz`lw)4fdZ9CbqZQHhO+qUiOp#L7X``&ZTxNqyVYLsTJs#&so z_mvmATT7PEprv+gT~U&1d1t#Lu~&u0E^tjQBQDkojcq%_1X3o^)S1L|?lLEl4o_)) zmw|v%hmC627|UvdB@@=EagO2$!&cLRtcrsc?9?xhkLf>A?oB=~IMTbFGIGW6#TfvW zhlS^Y_Yvnq7sI&Nm=Wcm#mw1VS+pJLt_e$-1%9U6iYy88Gx`wl7~Q4_aOaEeA(=4J zk}#Lf>07{YsJTT&#=}{{3Kh(VAI;ushwoVH?)SHL=c=}^wm&zM#{3&a#})+Eda zNQRsz3=VJeV|;f#wEJ6rm)X~Lmg2ZG0YXX8XOkr_PFrtkDOW3k`C?ef2I-RNBUbW6 z_zE8#N8N(5f?eaVVBC(4H~!J@m~cQbp1JA9LqN4J$yN7l?N&C+X~CW(f~_1)-YRN_ zGD^vj`O>M?8h!Jy4gAxXh7)r;t}`MiqHv3gk^*PHCUhvLdC{zm zR6d5gql$gQfx81ozC>fqqJe$WKyFZ;Lx1nRWpeObq05`)JNf<@V}Ait!2WZ6^blg$ z2@2y7TRKZbB$bMsfnB=`i0VNpv`SbOg@Q=y=FW=i%1fE!1lXypbvy2fk6lPySottF zcIh8=Lq0VF6gy##O-CAbBOLZ31n&lsJfnt^Jm*N?S1J=yGn#{U{B7&JGYGyixX z{%({$bCOo8uAop~7*lyp1z_1|jPfJ*WlxMuSb`VCSE)7CeHhIwe{@((uQXML>|tUMaEb;G`X=32@CoI^fvA;nuTu?`f}Hy(p;b`4eK zmz1RTOU|#$2!|!oR2Y+r(CYbxU;feSf6$Klz%D7Ex*?FP&yI8@lUzQC{)diQOEs6E zxn(C90c1MJaAcn97?n)InbfS zXAN4QY7d;0+b>w{xalV(8RQM=DqLr;&1R#is;?|_*&QKZ-+j^{zm+o&=MO?C_h{bpH!pQNv!8(6I=ruq;gMh za4+0pEMG3(*k_LD`lI^$RmI^O6d7k#KYg5hP#anfWb~X5GL>QGh${h}dLxY`M zeZmz&NmQBFBx-|71f@)W8u6_CkWS1OMx(ZNi@eAv&gvW?>zH~P!!bONud?Rym5C0M z2yLzIWsO8cA*-$Kr)-Q#5$&J=M#^{4!4qW7cAcz;_V@A!=t|Zky%2jH;div`cXszV z;&&aX6dt)S=*c(di+4z=Xr5)7#J-zPJI_GpCLVJTU?0P;6c3yd?)A++iAZGXx9LX##?f`fy@g}x)^ZEOXv4_*$;^k3 zYwD}~w%+E@U#C~J#r&Z1Sm}(66?i85&+wqiSAVjp+GK{8Hf_!6+ESLC&zKkGTLE1) z>I=IwN$Z7phwb*(3IDWc-@sLdE$;jNYB}l#^-1ZtSr2mtcnVvjlD63*`)2m?8ENSJ z2?cP72OdD2KFcBz*fYIZ-EYyuvx1!)5v@gb&53<&j`=+}2FYjZObqSGQcV5xHSd&RBxsoU_VY4?Pi+jcdH-7A(tSY$9p`boe*PgmKQ zuL7Pk1a@O{nB+fiQ;cIv>c3>(u=Gb+ALz*4;EO%Fn1LZ(*Dn2|Y4H76pBEurjbnN+ zs=xajx@qxG(=jolzJtMSv!NQ2*;|msf%ROF#^jnFIv3M=5Y$7^|Wl_#sL3wwvIB5OmUMkK|G=H0K{k+rIU?J4OWlW!iX zOuzcIZRd?`^=zN!-Tq~!iaD^I;o5Jyue@we(EbluRAflks@)67fy_t6@n@=n?s9uq ze=IO0TfP8r3Vr{FeKva+S#2|aSNS=XD4JZybbU;|Z@I!2EUos1#;Y~_b~9LaF}X5<)A1q&yP?W)!m@L}BX+8Ex&V@*3L;H~o6mgnVd1>UFi|+T}=@t3}^E zbs0sk-CzHoX;Ktw;@`gCTi?4~@7s>Io`VZ_0zbHZp9U|mM~A$?e?8*^{0sv4_eOlL zRO@$QSfAaTe<4d|MIIOG3AFa+A>W=Eh(-6XFQ=#XjB(0@EIu zU@6eLc|q#-RT1=(tOS(qjW15FV0OnPz;~15+8!TJzLtaXsNOp;o*L$Y^vB;APL9;| zzOefq_S>93?uzaE(&7X>?j^7W^LXEzbbc^87W7Lh`aJL5EB+xM{9KIm-P<$lj`|nB zmGQpU`Q)kH$GW`ff}5iPi$RGUQxdBq(TlaCC6qylA>oTfD+H?x?o^MwbXusVp<715 zMq6zc+OAQfh*RopU`FDsfRJd3;33UO4!ZG(xnJ_ zk4bx9U6Or~#AC!kv|WbwTUZ;4a!_l}P*Gh+guwBUGY%k!7N)dByt?s~w$*TuDN&SK z+itjO0ia!Zg($KIS^cw`?O>{y%@;u}*o+D!C0^=*=e;;L7l>-)0b63Tj=H8rR7rnB zBBJpZj9v1LpbSoLX;?+>7QD1y*4_A4EvmA}VR&4M*h$F5el|ps4lAQtFPM83MD68V zY*&q~n`aDXY4HkuXSph&#e_$Z1F(MJ%&WqRZC<&()-eBA9k^=IlHP2|wMWk?=$ume z`s%VGF9eBGVizEl3}vEXW>|{$Yxj|qyUX;n6F-7m{pR5WEnt1o?cibohGQ($VN?wz z_ms!0FOsJVKO8fFU)(5ys4du<_tl(6zsy9Gf-A#8#G+B#PjG#C+p@E9!c5wYw?vru zz&)yvZf;k%z?{C7s4-{OSrKsiO&>@nXc#|i+`ycK&tRjao@xTfDzO$cT8Lpbh&FIF zVT+xsbe~3~tjtt;s5eq4r=F`N&B$3e)<-l(5rQx;FcvzWr=K@l==R}5TXJ}EAd)A6 zOu;95E@^WXoyU>2`-sJ%S@U*&on^WwwrwL2nU+axUm7%$Ei;Y?YWd6-xi(2Wt!@<& zmp1xHURS3hbEiGXk(8>SoM?qAkO8$5Z4!+~={)B`;%FCyIqZ+!%EK0k|ABL@Sz&E# znc?6sCq$auN1@GsVJuZfAZ$n(lo08nMw;~^k|HVXlH>4na(bf53nfu$jH0PWBOR$X z7YYukb!5MhaYh5*&R~x8Dqo9w+Hrb}8FlY)z(NIQ6y$H#t#!%6) z+08e+dwk7g0WfN0aOTc3<&hOencKq@1WT$qL2pTaE6UZa9^_up(1e;R+W)jwohSE4 zV{hjke_~Z~B5)+`P?7q$`BokJFeQCUqBRYJ(+aD;kJzY!iBf&z^rosRl20aCj@v** zmMe&rq>EfI>o>Bz2QqP2w<)r!PahGnhA&p8GtkA(Oj@hbBEqURNS59TtJ0!lOnAu# z57C|voM>f1h0F}gRz#5x+V)B=-%ELgG&@+7-8QwoRuoFBR27Qbc{Wg|hL`Yqtqyy2 z1kHti(%$}a(d1KA=@p$;8Dah?PvK16NPl0vPu6(TvJUdH=(@dHJaViNF}H9nOkZ!) zc(RouIPOi0^irk8piya10%5nZHL)h$M3HiKuC*{|u`$k57%@F`5@W3_lwz%F7vfg3 zT5V&%h9fs3O|2_SYB{=E5N@3Gxk0s@=%H5e0oDhQuLP`D<3paQ9y%%cDIhmHtY3ii z4nRi^$7m3b!{3&}G1v1Jhi%~=Bm)kl`L=s93ArAHTFG{BH;Mt8NxNtrH`gpnX4hs_!7Kq%GdFGijq}Bs#$IGD9nUxD~aKv z)i}ZpOZsXnW!Tl3dY%(jQg{K^_ZbqRPEXM+Y(vmh+|1Og7u%0kB{(yuz?|fs`)2q7 zD4|_lT4SzGItK)u9JtQ5RHBF8ANv|y0wMW9iPXlsv=oFZ)t68^67pecu9SNtT zZ@K64R<`eDjyE*2wM>2k4$n+njJtXG|oI|LQ;zzCO{-kC^9jbOMGA8V&J-JI91w2>nI0kXaRbW;NI*=#?;&Cd{fCgsaV8-Y6j&b- zVGxX#g`#gIkiqc6qK2d3vk zavvxH4j~Prki)n5o>TB?_Z=JZZu?V zimz1C_7rCBh;j19ShuVV(DOkZ!>=U6bZMC=;XAU=&6sGScPQl2E$mLrs}~MA;thdW zbMd_$X2Tpb%_OMXPB7qQ?JD!fc09w=A9iYZuu3SBYbGC=E)M-rTS$aTE`8a%Prh zdLiurjYkEXOB*Y~ra)gdHO(W@uZxN|LE9L8eT0^AVc2j1D3!ItYSCD8l5;X!gEVBk zL&oXcK#awE`Tdc(Bi5e+ca#~!39Dp|Si#mQs`HtSo{k5;rMlb?XzDF{gKokfiUtH! zqXWZL_I0ojqB2cxGsvie;RW<@relivIjTy>a*wRG$TylM?8cM>IQgz9+L7aP`b|-o zY7hPVhVtME+KMm2hj)b8AKUl=ZuCeK{o7e1@@!%oV|*6HrjQ}gkioHUZ3Af6K&o#- zSH2NfpBzVz>`cEXX3a6YQG$|Z@!_fduh~qanyQg^s3r1&)G*CawUZ~f}?@E z&WNNpv*sCFC82f7Q`h1kUd!qXPkj zvj72M{ttljUq=C2w4i-dR#pVgOcOaWx=Bqyg+YSFpiqQ?kdkFZBncCP)4>?4;U`Gp z0A_=R#B9*vE48hwx0^d9s$B-!T@|X$kKc2?_Ii5cD9Jy) zqj_&T?st4=yT4p#xhH@2yw46KBF=4U#%GfFbVik#J*nlBc3m!V$Y$UWOdkOVJN+lz zbC30DG|WuQ+Go-TSUU6!&czjrp0#14_q?7N3^N;xwDG67mgv$=l4s6*th>cL@|h!x zvgjRGcVlwo zhb`N*Gh&+CgEjbC{kD&ta(Zi5-9qB@=q5LIyObIn6VIo#yMx*#8>LTVn$5|+-1hFy zVOcz;3HqU(yv^BNJKaM=z`InuZXH~dEPQtG6OHxV&z+uayd-w9lPssF?YwVLxjpzW zPyCjjyvqZ3D57c7&KazC7|m%icQBT<$*u(PyQ+&m3Yk?U9pahRd|yPpa%TurPb7hd zMZ$hP;|E@v4neFO(#}b<9Nrcf{(wwvlj9?9f8qAKv{T>3?aC)xyB`zsy;*mUcV(ac zpj{!3Jt~X4$NL?hIK*9=7$e6#+VAGsyoY%-o!e89M_$?)^OBG3*}Sz6Kh5XAVYMd= z9_QFUg{=CI_h}zOhK{0Vb?0A*%D0o7otLhEY_}@aYi%lQZA~Zb*Dtn?s$f2acT{T{ zBng*$f~mU5aU(~43f8^!8E?&*kxgc-wKvn-`C}~pSp&ddJ-oLrTV$=YO7+aK)6y$7 zx0Y43ms)FTdK=tsxHHI&r~!bsWwkoMG}B2Wf9?jAQoN3Qp{BI&Y6}1Kxo`lh*{hekIDVR~y6W-qn@yq~-aIc72mo z$_M;ocB}*C@m+WexYySKr$rO7pMKeQDiWN#`#%x@%Jlb9gJ8Y%C}DLa1REr|CVCQ( z=oss0Lo*R#jG>U3bw1B=)CdXUd2{haiO9It5MI|;1KE_&!R@*p3f#L0r$~b5DYiai z`_5X!6t0>?oP_Z~Zl;5L9K&)%Oj^ll^|Ln0vsjS~4L}VKT`EN^3eR+InlSu}$RDix zGz)M7tm*PNq_=jD=sp9bHD8b6c~>F*q(< zOOMVT42{0aR@(szTsI)xcxzLV6jzG*jz}5RCeuHU^G#(b9S(p&z*4;bnJ#b`6+$BD ziRy`>M2U#7hdhcG@xhZ2Fn~9iTmf`NY8)n>=|~rf>zH5+g6GX^Z?S!t&1z;jzhuh; zHo#-uESVGLIyCdAq0ncQazVC6E^@}<{biof3{VtEzyaDDOVmV>G&5A>-^yn~KQZ7P z#WHUGCO(L#kqKoGaza7}_Bk2x?mg)JC8AD3ha8r^t*C8up_qUoEczjSl9VVcxCK8! zM{0fnBZ2tZxcoR=n@R>YsQO|ZMhlEZvikN8LKK0UC2nTZi1c~`SDuw%&_KMqauWB> z176nYMXz719nA_3i78{=-^boejJ$h-zy|4cAiZ>?sNbe}dB(Hc4|tTYt9@#N`(YQbn=iXy znID{YtRg|u&&!_47jaq7CwWFIeOxLQq4S$yjL(zN3&!J`$qy^v%MW^X47yD|)w8@! zItAvAX3R{~mThmdx3v%L69`+ZUfc3re7w=cYK;Ow8))z=$xUiwi63aW87gNFa0X{*| zifII=)+*A2nEN}=S$}HwR+ilQ{@xP-+z-k3AyoW#gK0+eG4I6uu< zeTy8@i+lgnpk(qHDGf@QcmJSY;;k>$W0k~i#zXgNZ?Onw%Lw1~zLH$khN1mF5t?UH zulardaIjzKmxsQ0)u7yxGq_I{#zolY!hyO8?v{7U&y0aOk+^39ud)0-qtqWFlrl!4 zby@R!kD$F^ZJEbIObF;aR2c&zPcbiSJH#ueNv4TfEw}@^>UdL9nn)!z_Qq@5px+@& z=o5JNLx6G$kFHK@MI&-;D>H88c5Sw;DMR;7D)Umxp+y<%A{#sFpTf1pchz$Sw2*=h zgB7jL0#l1loicmuX9&@<4z}gBqeQ;1NDsnws@*I&I^`$HBvXsf_1X1hU{H>kQKZY5 zvHr2UE2c9zI_;@}QmtB6V>r4vg0&VO5t@0JKVBNctYSL>DL?f3yqGecXa*Z1Wi}CE z!e(NJ^M5l=1|#6yZ-#0Sno)PHl}_WPlS3O=$N~_#w6GAuNDW6W=cwe7Eal&z4Rj*8 ze)0VgHHBS8K#wybHP9tlBaMum9W6h~uSn<1YHF%P1S{<;wX+!S3d$`@g=Bm2`-&Nx zl)yisPl}S=2e-2d$1!^(`_*Vbh>GzN)sFwb4wTmwuVSfk{PbF^-JnENVd)YD1?8!DCv9w$mgrW$!cqc|F1A0i3ycg3xRq zM6rLoWqfq{979*69gtgEv$c_Qm^gt8>mHB}@FeaSiWRG-NhEF_UBx<0>e9QSLJCgwxbMT!mMuF+BL z?MrV7LqfBzr$JEf_?$1O5+;n5Nj$iU&j(&Bf$N`iS``}1-@!w8Hy-H4j(BoPllYh| zr4EeE-@{1woFs}0&Qqn4o|O2vOw!b$Sdnn3*ylTw-P?>H*A033B0)UcS%SsgC_I8J zLFxhrYE{<_P9&1%uP@kJFU-}O{~llP=Td~J&uQpZPoMO%PF-I}IRmtEfc1%|1|bl` zQ$6+;SX_LV?9ky#p5rDq1SLTv31&*M_|J$~7=u(Mq9Qt+tmaC3r(~i&D`gc8cBdHI zfq}z0j}z}Q$}OBr#hjbjb(R+YVJQW|JnCdIPCUf6q?ihTV`1jpKRTQ>GvZDA{OI)` z#eMJIgCEVVy>que@?9H6&)&g4N6ur3nxKGwCBjShlGmVCPJHElft;bMMP}{_=-`4c_vI=VRON0#)F72dMM|c{`JJdLX@&A{SQV zQ`!k_-!09}LW{B*BBF0cAK;1lNMo5Y7fhd-o77Rga(h4|hVQab2iT0sd78{UXeCq| ze-d$ELv%Ab=FQS98>9ZX#0+&lp1s%wkuRhzeP*JkC=EKJBv_sYDSAakh5P514*j2JL4&rR>H^)gzv`)o~*7`XvHU$xD6 zIg;MrYhc}g#y&^}l|8lf_ccn0g0vKGNB4DK%_kyX4A!4TzsR%m*Dmv8UL7Y z+U&v7M=<#KAjWi z%Hw?4T<(LTo;a7sVJxor(C_rioSX5-*&U3y{7u`8>>Md54{9xh-;Ct8pt;QV#e&yL z6Y?s|by-Q0JXTp2Dk*msaH{plwGF@cm)R zjOsimn(PUBydST?%=a9#;CEv+pEDzzwQbfn6ESRDBYiAT{fBc^Y$u)7EM8{bNO*025SD_b%rZtZjM(F z@)1QC zQOcnu#-0uC^DPyuEa}P?yKE^>ZU91e@=I=Cwk&PrRW7@DJ1O67B8>(1MF#yc#p&k@ z7NY!97paaD?+`1z+&KyHp8?XAC4~DSjQh~;SNo{03-Db(8vJ_r8R!EF%mW;bV*u!b zG|q!`*bL<`_z>bR8QNhV`SUav`k`ET8@c}p(8?~ne_8$VNgZ+yK`9foP1SDFpKAUb z(aUlq_U!m@4ES(PNr*QEsxV9RyHf60C6c=Wvsgvgu?yr~*s#{QR6vw1tZS>utea%% zQYxTq2;EFUyyAh~w6}|`+o-HU{zw*Ax1}-k6S#AJbXUR9Fi$g~pZ-vTM<2}Ty%6|%6zH_rCb8gRzi~E5^#Ce0 znnB0-3c5i6m+3f=G^K2S3*j_7(vK}_@Si(i!=ntMZgeOCuwVj+!K;9K62W&x!fs$F z`>#EsuYZ(KS=k(3sKM5Aa0g?@d3nGG6o2p83AYh~+dvK|0`6ab@do{)1M5Hd2)oax z5%GZrg9JS=5O-mN-9Zod1U(24{XYFe;!nb;U=33$x55`44-=0W-8FLhr@0qv_|o-C z)|3$C<;Z&l$R4hq?@))u5J-{ysmLC1*dN$Dr0;X`>A*dwfLeBAb(CKKR6wi0HdFD* z)r}`WH42FOaLi?V0+WTl7%jh`vJcfO;``)|-kFNo_v(i}Jb1+xGqvn*s|=s>KR*c{ zX9}C!ggQYElSz6qkmZG#b19S$xiXagluRUkDM}x*UV^D>& zee_I4Iv|F25@;tPcQ>g7nx7^qamV20yXr}8BOR}Dc$p)>$$XZUCq(5~8YF!yMX)Ng zrX%pttIwG=Pa$QVYydDS$|?>wu%)wmaq^VrzA1FjtSCantmTm z#hr_jjelqwa_z}hV-qv<3uu3`NL1Ob`QT_O%sv|C4&T#992+!$o4p||iZEu#l@Gy7 zAMEY!)!-IWhsu7O(wP?s@@q7jWYP5=^OcSO`GD>4kb9cpy2RPK1KK(fv)|Dek0A1= zEY=)(;*uYP#j&>Xz`anrJs}v+$k`zxjXN2h0Vi;1 zQ%WA$W{mSksLdD|II#n6$&HcV(6^1@;G7%lEWdP4n0yZ7+Iz}^5UK8A2t+v!uY0te zOyk3dq&{iLLsc17)JZbz%YL^Ox~Vn9iS>p`^~t<2T^hQzalG%DVos`jj23W;vt#Zyd)-&1_ce^AGfiNMXO2$Y7$8; zs;tUg@!gwNmGhUD0j;CGGHCT(6zetTbB>aF6t-ov85wT6ry*7M4ymRo?i%_`@Wpgf zacrIhVg+8ohvEC)Iue!)U&(I^&s`P6S?mJ-dq6J*Hg15fRL`EcPVO^;k~ca&xr`gQ zM%>1gzMac6YP!~4Lv)V4kw5gC zf-O4vy+RSflT_b$qUx(1^)waZ%SWQ zDm|+-F8c5-rM^$p!I6CyBCO5p8=L%obsqwEMzgqd9Wj~7fy%1dr>Q1R& z$p7B3u6~fxzXk#ViU$S)BK-etSpRpbBGdS(|4mgSR=2hIEOcQUa*h!EN3%bc%xXgv zgi7g+9IqvUnkTcJ?wbwW2Z$h~UYQ2?ASBN0HYe=+<@*78AHNAG2}n7>u`;D_*+h#j zoefqe%#j{}c~hz-0{xB(P0puV7aD-$Y($Eq1v@>|&dGPY(8TJJTvoRE&?k%$QiYj1 z3~3}_=N7ByYMFSmTm7O4-8@veE(3^R=03Z8aJ|k#K<+t9PThzFv-?t#I*Djnj3@4O zo@+Y1Yx(5{BcH+O?ml@1)&&kg-@b6Y4m<}+#W+`k8_PeOe_C`tw&2m-=n8p2 z`2RHZzmLr|Lk4kX{6|A;K>y`@oRp-gjg9^PqZ*zU`!#+D2nb^cJ2wb9Hwb!hh}G!j z>}(%#2&6xqTkuW-7eqziG|bMF4p1@?Dfq^BtabzrOgzg<;|ol5_=R)?Zenr zNIC-_C1Mj5b{Sk1(QrP%VjpNJL=_`sUW6slaNFQ1X{fB znHib>0-?Z?fUd$2nz0-<4*)fJXgS~>=1a((tES^jy!N0Rn579YN)uwFKK6>R& zON|UFK3%CnAJXX9$hAgI!<8S$=q8trrPxyODj(SrvQGLCJQ{-y5y}bOu?iyk$cs08 zM{JxV+bgx27!s9|R%ecWF0h0V?=j`sSb5!cx^OTVB%{Hv>(M!o^#{`;6X`y@9naKg zUX~oHTHLX%i%8`t(qfGtC=Wm?cAds{k#&Yh#4K&0hJL}@tG|{FRrJrrn`?}ED>=St zTzQ1s9<&9eBAdgt4~74ufivUWoJp|~TJGsG!w1SNdWn}cgP>#FjIZd*@7 zE1egsbTWMbApL}0F#k7?z*vmLFaE9}@Pmivy%#j+`p`lDgF6f95yxuScp*21qmQJ` z#O4od1<1*FM2`Mwhx+!<-!`B0Ek^S%H1&Isk&P!8nxkO6z>sy1Dh(>c@X@l+924n4veZ$DHqW} z$n>`LCGMVeMI*QKDs z6}z{9(rLC#`zSG>KU;O!pjIEd?3gU42bM->&{(Lo*ifc-2~M`-C9T19n2b`RxLr4! zCx#fK$GNGYRM1ooKWU9LR++%nRNrCJt}4vGN^$sciwX5_E6sCp zq~vT$%F|8Tn5dPeawt8?L`~7aWOkq}Qfp|^>6?b!w`(%iNP%-eGBnl{&^DAsX@{=I zX)>Z${>u{g6x*7XVHeY#GH+?5T8(WQ*kqDwC)4$E^5)=5%bCj@Ju}&;w6!(WAt5m( zfB=!N$%cwjoBmeX)rkLNfv$R~i;kchr6INS)~rL9?D&a)MiC))pPus4eUPc(pwJ)U z6q;fL$paDg)wwk>W|(g`kh_KU%)P#m<<8OiPTZ--P#~quaL8Y3Nw^fsHSfDq*j}K<9 zo8xfudZO<;=N|#G4@VnP78w@UWE06;KyR*%kf)m$dxq;}Y0@4JVf0^222`B`>c43p zMZ%X?RbMY2tEjonbrb*4o}Xsdnx9AJ0p37so;^Zyb5M^6nDV?~iJ1JNgiKE128Z&j z)qE1?hzG_j!c7TgOGJecdq}ZkvH1}9_ZihbNo#(7zkd-vbi3oor}M#^hUJm$;iY)< zF_$n+MDt=Ih)97fU!-FPk*h(Q!;Zp*@6%gH zSFh-obGfY$^peVnS`S?#J|TKy6n+L)pnjVa9B|xm;j0o+=4seHgJpc&5kH|Ew>Wnk zvxeEba*BFH7?l$hZkXYr!R-%fK@*O3d0`t2?%TWy>RnECCknhC`MPH)?VX0RbRMNz zRX^Z;jeFrhc8OCNf*b+wjNx8MeVj=Hq`r5bb|lwyk0-x&M|=RTt@V>^v8S#;6s2_7 zzy&&(Ccf~9TYWx~&&aAet{40H0DCgw!P(lYV%R&G9576#7z7Xd;!25mK8KCXVi-O{?*w5(lufMT0O`C^W4fkmMcijmZ{iE z^*>WZG}t^x`ZW7BFC^CAF5)$MWvkWGQs4ihM8giY17Lvy0kJ>;0ipljSSV-e;%@I` zt?FWF^M4pVleLwX1pkw*Q-n9;g#(;P{7Zt zc_amQ?qSAkdIIIBOLxBvTXa=G0}8jEq0$JzG9BH*sXGVn0(N`)CPTuk6mwlpe8a|_ zA_>-4IXbgdmnR*`Lh_G2jA-L-E2_oJM(s3)fSr{8(6*C><(y~gg-vE}&8n#}>3R!7 z%H6(+4MS4eK*QOczX6_|jSuxEvx8h?to!(UtxCk;D|3OEEn^8a3^$Jq{}}1Up}7n` zSnHWnT)Ha~^}Hg5KA|2Iz}kO?7b?QTkk_swktPTr^eM5(;IY~ZH)yr9_70ZVV!0CB zN~0EV-h7g5FU#N|wwEZ=DhycO?k|^SnGYxqU4t--KSsLDUKGUHWShqJ2;vIpojmxY zTXOBm_m5&R-NIjCv?q)L{}TnmN0#ZOC|xKhb`gJBBt)3i$=)SS6w+VTv)b5j z1Q*Df|bhQ}^_joE%0mkBk z7MW?IT}$HmbGCMIrS zW`PnsAF{`1tX)x`>jMmF6x8=ZAiTu?FtlHt)X6+NyP%=R@)&p1GQB#fFhHQ?Mb-{J zn&}06`Bw(_$YKEOCdsga-fm;Cu+4#I`eq1k`=l#R9`(;>cPKE)Fea@rZW@zUL_x_M zQcXa(Dp8?DFBsF%u;ymd^K`>8?_9e=0HYP57qnv?3{s8oHiL){l?WVSVG1I5o*BQm zZ#tN_12D@t-GDt!py{ldhFi*yt9W&TyeGP^et`6<;2%V*a+nlKA+d-2#hn61_;7bZ zzt~g)i|^2JOUf{-v=lJ>&>Z>y=}cN>NB>y=Q@*>C0|BA@zh|wT>|N}Q?f;J_#b`il zXs@O4Lk8y}#*M81WvZp)iH!jcYFmfrl+y@~At8bT!Qp}S4Tpq+jgSP%;*?z^wOu5a zZDO^&cG2QlT9(Zku`Nv>mEz5qHo0P6YR<7N%b{n8dv{ho80<2#&*n(Gr4)qZEOyEGq}p`UeE2JITD>6g{kM6(Y$6iFByv2${a$YOsY z=16-q;;=m=gGuYCKumI^!DxeV)V@5F=D^!GJya{Xcf#=Rmo@Dc-mgIo!Uyh??yf-( zPm5?{19@W!;cC12c<96I4w0ohqb9fTpCAG`a?H-xY{(yNf&5CNH z=@vPlL9G_VJ^q`%eRAk))UI84_HCzJ-scEnvt`JM|)iP0<6AK?rUyhy4cW-Oc zzKm#KRPJS%@7Th=7pfpby!XuFsbT8JXa@Jv_R+JpfU>gPU4{2=n!~mZ(vq+58VXG{ zpuT)<_3rA)skxYNY_O1!pT`bo?C|Qg^hAASb$91@(18dVTq>;n+0PX`6I6HK7sR&1eD2EBuCv|EF=Go5)ApU=<6D)4g81l3-!=Oa z6Ga9B>Q~n{vPB1B1c>bwl*sbPGn#yHn^ea$@W6w5hUoGNG8&#qKD)VvAJ>$~NY33| zRkN3mLo73l`C^~PzWYh@-?XlcjdMOAIT+8>cK7vEPv617K5y>_+L87enjMly>ww7Kdkh#O zsMFb;6I<>!kZa}YpMin34OG!cuY<~6HZH;Pk&rE8A#7qpJo?4bp&?OLQ0%bT1KM+1_5QQlhrLniiOA)#&7OV^o3(8|$s{j;D(^aZxz#_CU{ z?-xj5+{lxBY(I{HUZfK*L_p)gFVEJDZ_Td|%jIi0bEj!Z0jbFc(Jw|)wP#W-nk zG1_Mn?m5;%iD2PKGnpIFpYp?KA%Pa#djKEXPq&?iF(W%2(2A`V30;H5cZQ#~IdB0T zuM9yklt46)|A@SUeuPq5U1c|8^`&#aeSm`$NbPVL{Q?LT-x~5VvA}O2Tfr)WQRbl_ zad_Fhu1ysQ7+~T{X5F!Y0%)@Um=mqrQkUZ_mNJM$iR&7D8JXMo&?OXlBt$+%nQ^0} z-5_A3S9DYw*b=qQ>-DqZD`Jv@D?|~qtY122*n5nC9vzkT;=t>v#DXII-jKkV;K*GxGqxka5vVh(=NR`n0?z3n+FM zkzr#)6Zgq>r2E;;a-QIj$`7K_3@6cx(FY{xh>kjakaOl}$j!RaO%VU5fImwz+s-xbK0a*+7A6H(9PFmX8et3yP;Dfw9a`4scW z;pk7CqX=x+VGTCOn%q{etrByF>xBgr&Lu365GgU&Y7nj*lf|YdcYdBB$d7I(cM`Vi7|J5wYi})yT2I2#Q)y!bOy5FmagL z%qIYKX0v-1>0x}MTSjPqt``-LaY{?n)I zX#CjGSWz547=MkDD4(uL_jx$2&?38#JCzR-qaU8<(EA}&?7GAcI}xmU+XY3s+7OdN zv=WJ!?D`qNiK?SlIUn*;{BhJGiJF&F(j>u2*J=5b*5B(<@+D%!bdscLlBR0CldcbA zYX59|i9zrMVNYSPrSpRdi*b8xRM&BDrA zbx?z`Oi_S}q)-+df*wJlJw69Ix9<_t= z=4h){HOf^YHd4QAZzi}_R{EUSzJ6}Sz8psbXN+>F{i}bA)Op1W61zXkit8V4R*H#a zPG!)+0Xk<~i_QY6^mVfz!?lq6;brgZ!+~J(bG?spOJ;uB&SK!?elCmb@0w==)?wx4 z{GBW~lCW$q;^GK$8U`k0XbJ#g=6;&|2)+v^#_iCvS*H)MK?!y)$YFhQO8=`T0Hbku zIb$hG8KyVg?DM-_b^4XGn=rXW(iF+p&6>123a$KiZ?ne@q{Z#nyNqG(MRy*gue1D& zsx70!OJcd7Xs_GYPZ|^sWK|PVZ!^JDkHIt(tjWnP1;QNx$5%yxA5N(G&9rXPyy_69 zhR$@tIwtgEB;beCzZUF}Sze}!gZD8(fn0eR6i-s#M1jAQKxc#NQV z9Q#Np3pbG0G|6>rklhpPp%(J1mMKCN0n#(y&S$9Hv zuTebVxRv8npC-Roiw*Pi%NJtS=bhxen#+@go?uPa_JA>ncf76D-H&*^Yi|Sp( z`PpqEWf#U8myIQvMrt0=!=OKna~1WE}o7&}vsWA>mKo^TgL zkiZgzwV(OkcAm@Wua4}D^0!a^sf}P1Eg@^itx4?1x+mc&OJ-2*tFIu;$;VL^me8DO zA7c3X6K_0$DVS$yE}M{x#pzzh1upt?UJ0>*?f4TM`fZ&a`7N2eRvk>d}bF zxFa>M1mLUW*u3Jv2fncNX)-Z%Eg8Ih!@(Cg4N(JRm9Q46mPj|u5&jT`WuQHFAA8W1 zp_Z~1(qY(<&uoS-#2RLmM6=x~H8rQd$tpE9>J-w zqepjZMcQ*4tA96I&yuY$9ct2FtjVh}*`?1u{ z-hiqo@bvCyvzxG~QYljZ{q+JFO+HWdsfj#gDeN$7P>NO|U zmA3fA#CXv4(Dz&wPF7EUQui34q%%V=nN!po86nv2?SY56Ijq4$-W2@ZGCEXLuwURH z4Qs&&vp2g_|MIba+uygJZ<8C=xI=UJkRIT%)-Q!LFdtu=sVu7wUW;}T?2dEl2*Yq>P;(%t+52h^MI*a9)i1`2@dU(nk4`r_m#me<+RiYWXB;|WS5474bBD4g zb8*Zno5T^><`$HqHm~$fx)b{eL&q&tEp&ac>aCQo{jHV1%%FDXetYspBsp1~QgjCF za9N%2#&=;kSzl{eWs%tzH7$5ToLmV~%PrEY%I=R~PzQJ(9-UCq#wlSb{E6Whh;ghu zm1@6-YI#(;*tg6)*tE#BFEH_;!5eLxy~le3*(KZd!rO9%>w3cV#jM>3rheml@8uPr z(HTDM*5cjb#QnxL;iA-nY-4KfW z$CRmKmA8exkrRVEb`i!q{AGsjwUJ=3!l9;*ep=cM>QhV8JMrNSt?(JF@L3F}DTZ?d z-V8_1tnl;;(_?wWCMIw|pXwXggxP!L3OCd0Q;otxYJ2i%&=H2+uMd=`5e5{n*687a zUk1YEm-%opv(yth@*<1(&$TJj6h2$%lU0&mB;=De<-ns((B6kjKmzC0dg)J7b4Z`O z#vK>VJ$VyKM4Q2)J?TBoyuZ&NX|3`Oc%|amE(|&b@OO<`r390~)jgm@@1T8rnn#SR zb;dTY&dpELKmy$}20A-`t>?y>I+=>y?=xCCyEq4eQG?Rs{l`7F z6L}c_hYRL;HK$}}W&@sODJ*+Kpsw%mHa=;0py5l8^VqvNyN>~S>6c+CYBpuhPXNc~ zUD@zc-jjh~`EQ+{tZ&Fk^@Sg80g!I9AAMq;+{FH*Vov!a5)M7i7_Tp99SJq#OseR~ zqU=Iz(d8dK_OS+=_&zPDhnIE2%|@YbJEAf7CLshsz|^=7)I%V6$Lf_t+EO=l(r!6& zhS8&Ky^Kx0`~=-ZO1po(!x#B-2{8p8MZ+{1fDh-G8RvPSBYBeSNetF24_Xg=LI$jR ztrg|Vg8fO2{fW9sJAG;r?&KEW?8J#<_5~t8;(;M)z&HI%bX_C@wFq~|-^%x1XM+;{-rXJGY7_o&pweVnnhYgoYeFN#SfhG!oAD|# zqt#+SF8&pBy-29T46`)?M-EEeO32USrRU?{^Zf*(dJ?&FQ>^as5R@PW^BmYvN&U=& zaz9t{D`J7Fg>i=+YSdKJ?B|<;*^YR&(D~25z9-DSK6zYQT{KbFVTo;6BZ+S^euIl@ zcIpe@qp4;99dn1S#>#L!e=KD^ukUO$2AfNDK%1a-^5XIzB)R7+IE?B}J&X@!gX`Tr zno5dbqycI1-l zWy2Qf^j3hopMb@!94mnBd1yuCaIjM?HnbTBLn^F5l$>a6syD-X-7d&u+LxV&%JG?M zy~+Lm!`o<}e+oSFA5f7%|Mhmqe}O7(4h6iZ{`>{H0osi z&fkrq`EUH|`81t0-W8u{Jx^j@C|+lvPX&c6hr~t`1O7AFUwOOPERSE0Z{YdigOtQ; z35_%Wg^Au!P4MXQ>A7V~zo%D0Qf1 zdm3E-l~NLs3xE$gwB0WKVCGT)rmLU8J6Y;NJ*qx?ulXg3sLg~E4(Hua^8({0tVBCT z;4y18^;f%#R|Zkaa#u3-Q~NsolHMlcCcrax1v7SV6XtXFPkffrGsWWB+Dep-P{RP- z4VS3&w2PwmdPm88=kYq7qxG2O!Vd&zf-BQdJt;sKF_qIYSv_CyM1_f;X^}U)p@CjI(-k(?40ozSn>=ee0u~p766ESI?n|+L~ zA*)FnPEKWV$rvCXl{mbFH<5X5@ZT{dPZf$O_y^PfDlZWH8`J+?Uicrd{=Y6_p}X_T zbPddt3%!>U&CHUNik;`6FcE~0ep?WFi+byNEI$eydTwNBPoSNbh+^g=Vgtz`?lalG zvn;1mK40DbAe;m8U`EwaT4{|$0F}|ySliHa8r&ofngK_-&CWhTKpZb}CecbqFJ_e#=`5 zwmQ2`xT+F2hTXC4=Z95}fy<4V!{H{P4SY65L&;(p`0pWSIlE;i_goY0mOx%iCK7C{zNGC- z5>6PN(n3`5P3*N$#_h0i3oI!o85=9zs13>%K{^bIW=bvD?Sg8pFYy1~>2X%|BAg2j z1XK(2FRukfPZtY&yMG&1rKoGD;Ao(IV}Q}DSOODfJPIZlI07k|brTowkRc zxlC{Ioo*)o5!4DuU))B~#pS5nq?)9KXia}trf-`fq6oSl%eIrz%o}8O-6UA5f$dCj zMl!UGGs1SO3$}`TNd_Ag!-vO2&#n zmijXDu0XrV;^<6hc1BJ@Bc;>ihWzqUR$A&J z6Bi4EI7+J>s>fxf>(SXk>B~YthW$L@z6pewH=L85x8hJo72eX_}~-ZO7=@Z3;8R&AFe}MYJEI|`q`KOx;K)U zeUQiez$7nHAw6US=Ahyg?Zeck^njVajGxY$1f|yG5>!?gglVbkdW;CCa|zw0Djcc_&hQt>tE#NA z436J_W$of<=Xg_CSOxgynWjozhV%?bYE9|~0?ozonX4(3V(ICHb{Bura8 z>6o+~|FEU*gs?n|5~t&SS#le4`f(D>D)ZUhe$h%@_5Cm|!MDP(DM%CL3FqXT8>NWW zclX^IGrmD`M&E^GT~)-@qh)nlU=Wx_(ecHS)DT+_QVoI1=Y}7phUha8imEFJ(QsTMmRA6lc#oa49&=kv6O$W5|K@ z04QNj2t(T{n8-m`L0E@1X-brOv9p=(P5i;T>%h^NOBQpc3`%`wT`6nm-=KZnz+eTO1Q5|i^pU?`v+j>T|4FOuiS8wmY0FjCYW9^}&Zci$V1FiPYVI_#b$ zc25JBtokt64UU5`T;M)^uQd(Lx*vwxXmF(er3i&Ks?uK;E?@dekDwK13C zk~C(Y_!xc&T18)M-5dNkx=Ty(CjSg~ernLvZ}-zhIGek)8$s>oPC#Dh;xMpt{^vhL zShZ0j02T@eNF4KDl&%bLadom&Ft!4i{BKH+(a>?g(ZKp%O(oOOAFaH#O=g`QP9vc6GfyglkG6-hvlBe_VvFA9dP6kx5qO*|ZZkQJ_-v+avM$ZkEa~M^whe*S_C|kxjSf^PQE_ROd^^0R z$diha5X;3})>7C)=8|rubY`2VzRBTqJ%ssM)5}2oicW1MgUZ%GE1~(s;32b#b9c`* z@WGr(c%3F@pl3S~P9^mpT=sD4%lugEEd53p7V-;!Z+O!+Z`Pv}oslZI#SG=7Qkp@9 zne~#1b;NgV5zw~e4Vk3`cmk{n&lxqr5HD(~u^BpCxT6hGB13zz|UC&A8O}W1-^0R8cT!s7svCAo0z*+X( zm+w-WFeX!eXRq~UTeWphQnfYJos6<4!nzPEcgTMFF4DR%s!JI&3_Y4LC3M%KkWO_J zVz7*xUyJFGb$E;7G&%d@I6Ku@d`624am-V3hKp;k%eYr!=)8=)yZ%bzQ(=FTt#VYL zu!kVY?VF^MCqu4I$=B@?TdXmG&1%d~R;hXvox>=lRgFPV(>2r_NCJ zNJF1r(FVbx3=3)=-lSe#`hQ~(&rv!@7U6+jJRtV*MDyE-87*`z}84h-HGY|I^J-YTU+2?>y5$;sksoi`&wJiPup1U6m+nTjBxCPaZ z^4`KWgTRHff^Y6d&k(OOHPbt8yEKkDOZ$W9-OBPkYP+KC0gcutH5~1Iv>b81k%k@F zzdxm*OkR?gm%wisa8dPu=e;?O=}d$w1a_0DS`!_)`UTs}z_7T(4dShDhDN8Mv2@lC2cj2H8I?eMe#{ zd53&;f)HKmc*zR(u54WbL8s)HBSr@5e}HLfZmsu(JRfY;u~v0*BwWy1s->mnqEde>O|G$5pGvbv{s*9#yZ*nuiYln^-Z*v z*ZX)U)F5bACen2u-_Wtr(-W$NYi}gd{Zd}#-4Q_V>eTugYIZfpfYB zUHwL?ndX!y@@kAx&>vAw0AY|B8C$%Qj-9JQO1QXw99#fX}9`ZFRn(1Pl4jv|5?qr9{ z`IgIh_NvVwuMgzcKy|E{?!1HAV0^5R?qdA1gWh0xC=Ru_tEj^;&#%YE?6xgcrdng+ ze9KPFvE3|v`l+K^*ziEXYQ=^&UCS<+g-R?T>SVCwY|E^hc0tDFHc}*NRdx7)6Rt&e z3@Zk8ICG_1O;Hr*l*Z=NViT_qKE|WkEFBO!=jH4DnEt}c^ydxG)#i~gYnv263R_P_ z2mG5dg7$*`CJojl4;-`w`lv38t)-{2M9%6JL>Z-u4!3-q;2sqPtXAl>?v#@~v(jZq zmCXYS%H30h;1Ik1GBo%GVub99to5=(IXX>eWnYo38Q)Wvt{n{&5UjsTaUp~I|hPtaDzN%tBTJ?_ALh}!? zI}?V*bR{52M8zGGcG0RZ7y}^NLjbY|SP8Bx8|n8F+55k?KTl`oJwkjYePZb!+777h z@q;;UNOTP`o=3#$?8ShxRHf(P@W|ie$(cz$BJK}qX!3SigddtTQDG^cs^0Btj$qd&^cGGR|M=sWP`~9Uo5c|O> zP#|aVo&!+a{yk9cZ*>Sn2y{`#x>=P$HmvvME;}v^AqT&}nuzeWUP;d&LxbUq^4E)84zAfWTGGmdpPo5dp z1DEa728p<`VM!D{JQbEDf$T;qMhlNFXcgtV5?^K`)&dsw(1kV}OCt#zZ?(?5VqYyh zvR;61A7HInCO=@NNXd7LeF}_m(UDdPkb&o z9t`Ud%?7f}h1#X1)g%|jGC59C?@b5EVuq<@z_WjD_2TaZl^d^d_z(bvq(n$3r#3 zGA618`YHTWX+KWcgGaL+ro@bLkMcSQtngcyM5P293hD`Rk4Brs6j8tpmzb7C#`>@q zhkc*?4$ApoSSb2-ath8Wm3I&md7-ovm@JfeTaNlHN>v()q(1}1=j#P%-2hti=#j=6L6SnPVc6F2GZFAMR*8)P|{Cs5om5;Ph1-DOMaES=u+~ zWU>r3Ti&&O+K*gfXLi-V z4XR!tFu~ZLE|s>^=fVQLMzFGRiF}3H-o}X=4Ima($Axzfcy7ufZ|_BRuvxIn$4}X( zsp4Wyc3GCYX<$h~<3>xo-D6vb4S=KRt}J;;!BGt3GW!>iR}-b)gcoczx}#3LO6t=b zl+C((!zC~6|8i5aHO53EKnGzpxbF_XMfv>XHci_mvoDb*;Hw2H2r0aL0~?X^K5;1 z*on(JEe9xQCQE8P&rCN-7du|Tq$A%ny2<3)y^hUDO6pU250=O1;pzI|2ET~BF3Bd+ zDiq2mVLeUOd9LdrN#&w4P(t`)Wv5*R$WZSU9xOOO7Q@bTM;Dk0MA8gF@c=yTh2?3q z!tGR&~esz_hzvcdDSE zvWKm>GAZtOmM{!Vau_~gC+hMKsHyqZ{P$1tVlZ=tB?V=&gL^2&2&n7B^6`nA{cHyH zAq{c9V$v1HtoKFeFuSl`m{SRL=Vs3Mn~~h8Zfl~S85hNEj?07Z4U;LZH&g%Gmeu9- zg5PZk)&%0s)EyE$bo2vLa9;a`waLf27t%~S1-p-D=q`OmX&HaT-U}jB^s@5T`N$PR zci-5M&!rH>f6|>Kj1-c8%bDD>USMbGLgeJba@3v9 zwFhbg^_|NwMLR_sQo&*19Sg@H`40r3yUCbaUQR5u(#h#?<#kCl%VcH#l>)TzV9fSW zx~Ffnbo?%Z#Xj@s3Qpsp{W6sSHzx2Pbqyi{)xwoJM50)2}Y-VzqSkYMn&(n=qJiV(PqB>r<=4gtIf^BoEZs1)U2W4!#&N^i1J6%K>sFR-_~MBg$e){MwJW^ZX^{&uHQDafr5pfF0v6O4q>-p zPXZV<;kcQjE@Nw4P%yI7Ze%$ak@WA@Rb6*F+~gfJmi*XV zFdT(iHq0Kk?CZUhr=N7_A~P>l)a>iMrQ>M@Q0|Bp)=b2{ImjyE`U@rl&wJ z(jUxwvWR^xbOZH082}12UzH(Xb$?B?n#`V5g^pOJ0AWC$zZ8{D@1#u~eEE9b;-j}9 zcIW2iITufa?|K^RzourlFYGdW{dhwuvtgzy&3F z8`N0!yr@)*BebOrY`q*jWR09;lM*5-1Wj1}((lR>Xm0ru<#@EBR%RP1m-5_4M!{-L6B_APea zhEF?LE*4HyiG&l~^}3Bs2G_99Acl+T4=ie&$rO&1_ukg_bXA<*0aim^jCu z6w)8=bMqBd<@y7wNvwgh`R2esl1{P;r-`!3ft7{L?qtpoExa8^dQUx0>NPC3l$4vZ zdY_iQ3(rlQUp<3if7`2<8#@zbOKUm;$<+7eiHTBDQ-s;r_(4}UxMpm!`oyx9H1m+7 z2+m_;vL@FOse1?fRPNQ#l}0eRFjhz59d(CS(aPVM_;NWc1Qr+x`l|}q6!_ESuoHuU zrP5Qna@T&kzIxqt7C*2HUN*s}CJ<1?6*O;LpcVX$pp5 zWeAj%*#Za*P)CR+oSGn*OZELr6*x>!Rhju!bf{|_#dUK}(|Tq3aaP^Q!QY36**cUC z<{Vfy)>Sna^w14f@8ES69hF3Y(^#ALuB*lAV@yz!3XQNaS#?KXsr8rQ2V`j5JFw>N zSeLkX;3q5Ud} zB6L_yO*yFt)8V=tq5_Yd-M4cUN~hekdN_PZb24OCvl5nHxI8PGt_ZzT%!uy+Z z$jgDHL&5+2L5KetLVsAqtq*e+XUE{kp3pkcuZF=$Wk6wn?R&K!c6b5{09rrf(r-Z5NUcc5`xu5tJ$-uJrF3%rg>Z7)z6O1J$DUZU^ z>8}H(Lr|*!G-8pU&fZb3U9FsxS!3&+PudKM<_?)Np3SwCx2Lmv^2ZZSqAs}fhstYw zd>1=2S`F_o8t7s!OQ26zH5=ELy*{xnZz%f!1$tj))gijIj-N z84c>@Q#PuCOCLr&{94UAJbFv5q2CU|jMv?6PKkW*vPVZoU76;6{uXBJF0iU@&fTl$ zeTk2tFe%0MCHzBJuG#u8{@0ZJaVR_48BS^Fp!9S+7I_%@k869*tSPg2ozJNVBVxK{ zW!=N2?rppsr)xG+A7@aVfV7fodC8LJSmw|Lr&cO0D}IXQoxRqYsA=HK>?SOL6m(p= zaM$njkbBuoI!X;t-C3GD@tvv9^3=x=do~Y9cA@K49uyPYkCmc_~ByOdgGJT7Gb9e)aDMOAq=1oXTgt;P>v1 z${-UqUR-yFt-#@MLoPQb2#XajX}8Qy-U7~rRwq=~ujCjz%0C31zrh!wx+5cRkWRG~ z9s#P1A9)5zip$^oS3Sh10V$8(7S@L*1*>A3+ZYhH+Q(!MUv+1_%7ci$HA*4NgP&H8F~ ziudF>I zf*+iUIND?^}bTkftAw|SKV;}OSuTGt`%*A2?e#QM=tU0wt=`*hfcpr=EkfxnCyH%bqe zCZ@BaQJ)Areqnfh9NdJf8-(_Tqj6b;m><4NMy>KUWK--)SMF3b)Aee=p8$pAq?6|q zNWLCn{U(>T+T@GYN@0IFA(RcT5p$GX95W+NGKF63JlMq?LpX)_JOz{>6<#9w#ZiiJ z3G=frOSeIl=wr*U^TI2r2xUM<2<}XlgF}Xc1DDOA@^tP*W_DX52JK8&f*6hLX^Ab( zD1X>F?H5X^bq=62d)S35=^R@oeTFe9==_$_W$}FSC$bjwMM|O}am;#ADO=k`Ku5eBW@^(`%Zd5l z?8w~b?C;;-!~&k&?rFbRWds=A-EO9wPx)D#-(IE-et=$4@X%q~EB36>clS~wLF{G? zVJ;ngXjb9T-5oT@YAAlGQBQu|`{UZ>8>Y;I@?<2@s3t&{cmKw=SMLr0o~_ zS@5zv%Y9%u}q~!cJ>-ns!g^{AaOKi*^#m|_ z6&kJKQ@pxXnMc5fp3AoOBT;eDHO^e9$7%G~C3VF`mrj$Vn_#;QI?%oGF^dlcuo<=& zzr?K|JeDqAWB05O8Zg^*q4~^3!1&L|Da3noIdd2NG3I)ctJ^-Iw_om($IxDW6PvLO*f(l(CrG0qYjzf6F+Goy24c*)?T;V+`U9)5=*P}NUBk4Ou*@2iXzQjn|J9)13lfG(wi&xJ-vI%7ybhq^2 zob(bRhOVZlrbx0_d)Ps~+e$jMrAeet78n}UJ`|s;Ne_<5<`dT$s0SZ+r#a|J%gm@U zs8)=|%6e5$yAR1be@Eh{e5X1%`x>%8@R73pM7zl#!)zEJD^o-QD}()fUj-pz%w|Mw zVY!#DvccRpd3&N^gAi<;en4jr?f~1D3RW@ z(Fip$lhteq49&Z`E|k^z`A6J^shbDupfTY1j}p#2@Nkm&B!Ildg}oTOW{2gJL;pT2 z)yVvyKBqPNkLe%|2jP}$un9LiTWU6{!c$LDrtWO8Xy^kD%eybVh&nloM`s)*BYpNC z(O*9!JRRPE#pTi zB`Wm(awTlA`3S#B+hVOk^N7Dp%-IDi-V)1bC)e8(g(+Cx!l*k0`N4QHn54KJj>Tz0 zb@P+d&5jdidb&9QUC9V2gLQu4+tWK{se5O zse&!JQ(}6@XK}z|!y}VKU_nND7$Wdw*qijoYsf20)5+uq&--& z-_hjRs(41}0)3snEtgx$9fj{ot#2;mzZ$)_KE*A}@oTzD2xZ4|l%TUxjlxswS8of; zn_n|>m!>7s%<5vPS(L*fsmnLrh&?n;!cC1tUS^(2B3R1gs=1q-3?YVYoK-Pvku zmAFQiy1(F$aNpms=52TK0HCB!&C4C0)7;ga(=Czj&(k&mpzhxU#GJuW;2I?Dp%qZ2 z36_H@W1)_EHyp%cM&tvF)w*`#jA1%3`mut;roY;bzaw?^Q%dW^?zng2C-E#l&A_lv z(2q9jf%u}uDpbM^p+f@!;O^kUVoi#3bOAvslN*((PBum9iOfZ+l5!pHm|&fY>r+!U zH)dY!SzSpp@(OXLBR8hvrK8}5xq3=P-_f@7(eKxmO{xtoiF{?~csWX;F^NV7Xclr$ zQ_x@wCmMIWzW04vP3NanQ zt*LK{3XSpXE82e>f!kMkK;AMRxzQHID!Jg)Oim^jde9vy>JEE*7z{%$tcH2y?~$Mr z90o)xILHo{y~_5VQuqF0oc0Y@zblEXyKDKAk)695qvlIgyUK8AEkjCl(?f!#*4sNW zx4WmaA~Nj$s!#Y`%t7VgQ8h8f%huf8<4E9oTUYmDuq|1i%Wg1Z+A7p(&LacT)s~t) z-C5L)%bMkOUZ}ortpd||-E-Cxf!_zcvTcLi2y3Y$koSB(ko%KW{P8s!DyS)s!G-j8 zgJo#y%uI?pZ6-H6lQDnOV42aH!4Xv)-P;(e=f(~#ka6b;(n}iL?4*@O?mKOCbESnH zYTKlrKS#_oK5+H;$aFe5RS%b$D(q<2B;%(oorG!No5MWBMtDYjdpQkBYix$ zFGL~W4(_Lb;6Nc1L-bg4pDMHrIczT`IdpDqGxlbC$6h+ zEYdjT2Ga;HdH5sq%pnZD5sIa*>*IuHgSF?lO5J1-Z;mj5b8^|Dyfg1)A1HHX^J)kW<56+$wickWx}K})^CZ_K{HEl9D@*XL)HkKU;` z$wY{OFL20<@Gj`qRZv`5Rpcme{6oXiTMBVYqiTe!GKz=~sKcPXDed@@PjdU(CIpW3 z7uv^6XQmexGl~`iQhcqNe)y*#dj9=f(q4@39OwHMyZnb62oLA+>#S6NEYkcvGH?{1qPBmXQ(Vr-M7;SjgaM?zoan45JIl%e? z^;0B5AnRj8WzXGq71+QD)zXjg*6bonCM74_Pss-l&|JRIY|uQ#8%%gIhls91mRGh9 zD`JY^tByhYBVC@UiW+yI*eGu3yu&Quj-=MN{evlkChn2lxw2%kJZs|1vMNf2l&E>3X1Y!MFOZE+Wq+B ze46Z0y`P~kkzYLjdK^mKCx4%{Tj->)Y;>q-f{R?Ma7f$QdCN7#%f7`3eij-PIwf6u z^xZ6$SGXe-`kyidi#bO|pzNt;5mb!AHR*fvKFk#U?JlNUWM^BvtB>}!9JIHXS_I>NVIWH)dxJf{}Yfwbkyfag^6~$ zE$RS8hAAB!+Mb<+vVhdo0kjSC9noQ4=SapZAV`W#$Rz0UJ0yB-lo^yZcR*T%yhFC2 zmPp#@5bT0HK#PnJD>1G?T#8&tWFkc8kuYX2a=TI{Zmnj%hsr&TxNq^P>cGScn{f8U zG3pb1`~_k~3X`}CuJIKNg`W&Z=*s*_tn|&H!S`O^zf*#g?ltKo1Q3ul!v8I;FK!92 zG5v3JkfQPSkMR1{11d!h2ih2(u1=$!MjGkT5Y0ogj)Cq6H#tysFduRqpk?ZBDTAHI8lYMxCmbCYXXXb44h>FUlmbM^)H)( zTMNzz6=OH;LEM|ahx|mYeW8$d`D>n%P}#_nVPd3l;m1YBZ|MYaumKc92U6$?RCi=G z6;v!mdzEL+jtgfFSE7v~S7gd6cuJnFB0?&+xw&UI4LAMY-!lQ*CZIWAd|NJTi4bHa zZlN&A)t9ZxTAPyVgW?NJmMhCwmt2e(U0v1*A)iLo0|f*MG!|p&Nm49!6xFHa9F55%3; z#z=>_`_3EUC>mkZJ7N!3rlJ!v$K0OCr;bG1x(Z>Jd2L7VvL5*QCmkK!QPd^UShy_B zrXq+I(+LyPuH#Hm*k8nwTRb&DN8oDSnUzKQCrWITPLh{uNt4=S_?b(uO_ksNpj&8} zcej^Et&4w(_9diNRC$D+@|7K$Y3FCC-B(ARqP>>M>X_9T$~aV52&Y6AxG9&z$0{V{ zJ$O03fW>AEI5xq)#z(h#=-cC9y}Pe^{}PUA&VU@Bxb&jfegz?;e;_+RX$!J1iab^8 z&OZ=yqP-9iGMW3kZoYOxWg@?jFW&*%NiRT(v`&n)lVs<)I$=%(PG`oI(Y~IAoy%UwJ8xB9&VR%Td+TYp3gz3$8dz3zN1Tn;8#3U3qH7=&AYRLf2_ffmnxPY#do zY}paf3%!}XG=EZx6oAauRqj+pcgwL*dPvt@bBLF{d`~}9;h`}4!C7_)s#|CBi-ekg z=^mMXr2!p}njg8ApN!VG)D~6-#X0j$mNIYkzN7vLym#pyy?5mvzIWLX$o=vaVHZ|J zK#`+W@FD;k-}~Kre&@Vn)EHI!&)RdYHRrBcdu{p! zI<&LwzJx9GWSN#pn}D%pvfp8PrhA1vmZlB)#;&$%i%i5l7d)3>3pYy#n1m4FP{NA@>JBNtIyRRZi`;~BsZ_ItbbT&XwnMW zPx1=wPU~8WH$8!Pi+ezofbCNJFNRN*8Ls@)b}6N?kd5BnAhp63yE%rt8UL zze${mB%ib3g8dPOgci(I6Bv6wMRtx;-I7$f@jA0jNk^IA=G+-j@$HJnry9;65XDYh z4Poj2MmLya^RNTz5wgseWQ5#<)6J<9I{H+YroAqjr@dlkE8|tw0oJYvSo;B4j*VHX z5$@4^xEt#Mwq2Z#*Vv+nx$OkqggoPEVO%W1y9ayo8Q30_s<_ig8w&gxDuNKr#o83< z>FkX#A;!AIoE|-i)S6hqm$?;UCR&%iRyo3dG3fX7MU*{g8tmSxbV!2MaEXG=6ln-q zfZnj-z*D7G5PL^trZu4J?sPmvp9WaU(X7f(^yfE2T(2Ys?clo8c4kp`Q!*H2>qt9$ ziZl|0Bay8RfvSwNY=+>S1ONtg(w%;;HDfY&ONYOJZtp1a#8!}W-)o*&fp>CMMwIqz z3Qifh4ietX9N>)q>1F_RW)T?bR$Ml}Ovif_TN}k8qSF zib0cwUZe~nB%Aa7u6`6Q4fWj5fI=O}K|^HHA(T)X`m0rZjM9AR!5Q&S*?C?I zDrTfS$nQkPYN4f_O?9;kmAah}m7%x@EQ($>6fMN5>tg}?L!acUBHTV@UypmTN@&!D z=12>@(kU-)$l|lB?+Xg!rs}OYL%ruip`e#);S6!S24D&1y;LVO`|*|F-*xEjHEGJ6 zpdX~DeG&wll6qYup&lR?c>_N0EgYLb%X4qmJu5L6=^QYO9DWYnEQmT|z>nsGBdotz zQ%0%C?y^pJ#T@xC1oa?}ykuBFAl_*#rxDxB=Km^JIi%D(JQd9@fN=6v#PGn~;}dP} zd~i%XoRav+O5i{VbWPysIkXMhy~MH__jE3!sbA3?-ZmPWKK&0NJekwp%lRvWv;S*7 zmL$N&LD9$@@Xue|ssGYKw2>Y#DSc)*dRnVWb)Bm3F!N#^g;ua=E%j7r)Ov;}5GYv8 zj?BnIMQ$})&%Qes-C68T>&{+(oWBA;S@gQ?$-|T2*1=Bbp1Ztvx^moi_%~T4C_R855|NjfIN_aqqq^U1`fEcUf-NP@D+}!q&fT=9Bl~|QN}r(F{2t3*W!lPa zG&6sk6s`8{F*1XEpemR^owYW?<4EufrQWI=FXfe5x4NEGkjH6ea$KFI2rCliH!i^+ zFDhUgTKMyzr47GU4GeUhSliF$(%7(9n4>>i2PU?#h>f6^w)y1qmf^cR9Lm;YyB3%$ zJ9NEy%#U(gZin|4fW}hI7Q;z^o-#k;tu+p(+?TFhdj(0U%3pAv zrXK*)UF;YEYcZHD%;i5asrSjvIs}+(N%6rSvpKHc_&QJ7*71eBeJnKf z?IJ(0rJaigUWUIDm?xl$G7ZifEvXy-={eKpXR>K4082Q^O>RjX-fo=C1d?&p>;1N} z5X%URPMstVpkU<O|3~YEv^pzI@ z**a;Kwg=uo2`yfb2rXZ*a-e+5_gFG9UvvjBUz7(lUeZh6Hp3ckqFKMHdfjolM-b5s zxly=AmJ{WCLQ|7f$aTrO^6y)B{eJV4C(0;kS6I$}puuT6TUuu_Ca4d#;%&Lq=b8D0i&`27 zYWP1Tde*M9*rZWCDKgyKZ8z#CsEEkvHac3Z>A{Lr4RPLK5Dn~w_o?NzjnN|trY6*A zldD~|Q!G!@0JV2l?6k`Cd@sJ@!?ms>#GC2HL$~9bSHHfrl8YsL6V-Gk6VK!iB4LXH z$$Q}Lt4b@zHz!6jjQM$dKP>cpCKbn4li(TA$^c(BrWoRpMTv}Pj9vfK zgL;fzUsWUdiU0t~FSbbi2d+U_$TM9G7Sht*Aa*WfY zkK}F6paYDp=}y=Hr2$`^*$^J=HF=vKeO*Y4WhC+B!DUpnIkF^+sF_W_`0iTu zrThBZBBDw1aU6SQGQY`w#?a*ppv&nISJ-t6AR^CF19VjmXQ=n;>2%svs#+6FH=r1xodtx{WW!a@u3tC(oRC3N?ZrF5gqCs2A~No59~4asy8<0qMw zBbf+}(x>!1xjm!SSS4Cw6Bl`}4@q5T6yMPN4>QT{8|WWBkC)s&;iaJ#Z(d-3d%lyo z#`*`d^M2l$&x?Js0RD9)Zl2!~Qr!!v9{1Q&ebs#p8SFJpC^OzM{8Qh|Yi|3UAwfV! z{x0;0|4ZNgr}6oJR&W{MEunuxxValL^MaBJibj=@!v@$?qUqF@{j??pU3!ZEBu~euVAqT@iLgZUDka$_O(t6fGzm&n^ z1&(0;hLHfdea6ZIEf)IL1E0TS51N;1Q33k4QBi=zWyXs4)srY2bstvPM``FDr@dA? zlp*{&^{)l~d8-D`uxl;sqbKO2wMbN0t;f`)h2A`jt(^Wm#E=-TDbMoZE67d{V^%jrj;@>UlPs7#tLxEI zTJW%C#I-h?Sl#ZAVMUn?yJ8_6C}w%Y#HYa^$Q?4kRSx4hQ<)U&PD~FK2j#VdZjiWZ zb4~E&<+87{RQzByEAec!46GP4cAImN$Bo2Rd8tXqx0lkjV7NKL#ZcS$FmeJellK1p|F|3t-1KTbvQRuA0O(wjG<1)l(_ zrW!D^0OKM<&1?A3M$X4J*(&~9w|`VdUDYSHCGcwU-mLr-MTNJWrF08)EicEJR^z6B z^Wqj-eq55|+^GnM>JuBNN=g4m>xZnw@cE2d=QxPy75YWAS8%3j%yKmBd|La394PTb zq-h^h!&ABzxx{1fLvAKuM)B4+kmeUKQJW|byW_NQ(wkO^zG?&Z;8$bu|w;w=ff!167s?&4gmiAziQfgjvlUQABJQqI{6m!Htj{frIb}MP2Se%}Yj%(Pr7f zHRy=I4Tf{$k?CSHOL07EKv%^w>(e6^*=tFB(o1{-SvD$ z7ddv2`zaiUbKNf%SSpg4p@9zEZG?x>>K3<+xizP@EZUt^a-h0wF_)pwdYi*j|=>ZYoz$I#y&W{q+KDxQ==lh_Ii>8V3 z8y%k|b_n6Q2I3C<2euOJ--vp1s6h!xmb?JI;I?^_uETcLu>$OtfWh~L{FLfyXfL4 z6dA@A+?ASVK^+qA5H5zgEJirROXs@ov4(zaP83U1+4Kx{8TrK$(&gEqK0@DSb%5G- zE3EZqn_g$kzR6f%!92PHr7dXl%2*o1k5|BQAj5Ef3F4zz7?#+Lpry4VcKwR)hU9(g zwr6S&h0;O_L)k;BYD^4u%oL`ZVcR=sM9-&Fq-K5^1G!Z53u!V6VBQmvmmc*6JtKeW z28-6z(U!LG1@|%!+ufkg*GG7=HSPh?rwJe={%#lpaZPdKneYs~84L+^E<|n`fJU$W zoe}Rl;{_y=nQ)0I2s|zDjkEmdm0p%52`LCNsLT_wrHqsAYdV-FUzt&6g_#-72-v6m+VeB%X_)9jpCfC{z zjHB5l)j`GlSAK91+2$t0aH!-EBt1AU9Ew0~e@W-i99QG5f1I7EBhAhi)2i-_bI!mo zJ!a*@X|^lP)XmJBpOBlTLvMl@I_mYRBogzU~Qe1*RgX}`^ z*f=Ce#gMbT6%z&x%)d}F^W`iMCprOVwIb*sz8wPYnAq+SWQq7V|UvtNDp;2-SvOa1wH-4F}1 z@0j+zGF!6^llg|IJ=2$-;bQi+>fi9Re7XatOn-7GH|ubbb4 zSbfSAAfx+WF@QIfgcU1$mc7K?rQs(<#Yq6pl_g9O3`z~_$OY973Y0aWjmwha9`q!yR|J)v=3GIV+g#P8fW|9E-K}6ImET|eoN?J%Q4vzuHw7YIv z5b!sad~Cx(a>6OUKe0u`&I-C11I^bp>DNW!@A2C4iw~_A4U| zpL{I%(+}O_zKi49cgOq7-KT-S?duZ~1g`)?ycVW|brY$tCpz&i)Gdt_kXD z$QPN_gfvhoDtt?y=oxm6;_zI!o9OTxJfgS*r(5zqktaXxVaEOz2{LcQNU)C#f~W2% zYW-IW;L((-o8-htV^n2-oL-BM&f8A`ty=(IpcgTOw_s1}_LvJ-T(6>0+im6q8LDpS$X)zalH@M*8YQKBGU7<;!05wz8-5w4|MQ1F@HNw8$m$6B{BSWK`}FXQ%2Ufje#?b#^+viAmPUKuOCZBH7BZ{w;ZYXUjgYIhChZ&JZtEG%3nI z#`^@H_DYxL7c#8Pvoa>Tu-@PMGmY)7L#`mFXioh^2V9HAV6-@^D$Sx@@lwfVCz~fk57-&cS+juE3^IGeypdK_vnO$( z7ZP?ZBtH=ADM*|skM+v$-24dBzlHduL$=Ca@oN- zuH1kxUSiLwDA5a*j2!$y#7L6#yy|xNp-}XZ_+b_&IwD71G4on}YCL?JFuPH;%s2*m_OcGqGPjrDOMwb| zDNw_uGh&7HU3O&V78ajkKTDLg=-@+hpnv&RrnC4!bNWK(SHfu0S-@RAP{du0 zcBe#?dY}y7vvLdBISl$}Ih((@p3j!S=C=Cwup7&*Ka%6}j_p5ptMM);?;|p>TGU2zJifJWlgHOHYGeq44IH3#=YIY z!Tlst3Ay@B`m8*dEf@53Nlh`t1zkP8T3RiwnmEwXaHKh`x|m8&VX%xb!PS1e4ATbNX?nib7x~+)~3XIUh`7#AO0q~NI8&j57 zID1-shI%O?0o~CS^MO|Sl)Ilp3$i?9kurFjYgqGcLRqqx3yh}6g35L{^VCJm&cshNwER~xE zpx4{c@k1(yY+`UH%jjEZ6Uw8&*U-E%r>o&cF3EkmgfUvmo2q2uC(3XFaC``N0mX^} zY9B9Tl)W-hcAV}Q8~%4>qq{C%b6z*{D%TMUiT zAQv!L{PbJ-Q^~g&{(#`4>KcFc)fJKFs#lV7iB^kzc2n&!FbST-Vr(1fiq+v*=Q^OQ zkU=Hd4{4{`lla$%+EesQ57bcjM})@43oH%vMEZn=UXv0&-{>7G+~1TWUO(O7j@m!G ztVqrz>#h`LP=>1%zhvqNz^EC7Bp`nuM(RY&`!jyyJA2h!IsKzbG>%n#Ju2;U-;d_^K8%0qkC=Dxa{dl zn?r^X{cb+6(7AJNuMJUuh1?cv3*><|(fU|>I45#c|To^{@m28cX^RS8LKt{6|D^nEF{sS30$B2yUJQwyO9MXzDdA1U3w6`~Iwjth}c;rU3W+O{Pw< zrpE%d*HWiQ-5vOqtlbN=d?l*jrgy72fVV{CpUGClS9gTOdCHeJV)zYan5P3^1|Cg4 zAypu3LuKK>!*nnStm2iMN!txfLX=?BLzitq>etw89t5f?xCh{B!E+cf*_k2lL=-sk z#0=nnKar1F*uvXzSAM3)3}aV47^qB$<2(~~YSz)fm)D~^rcdRpsskmg`z`&mRM=2J z-gxG|#tWA0x7_jY2Gu+I(IzOg6Z2L@N~}@H(p_cCZ;yjkgkRrY{dv4kXSQ^AOkTC@ zGupP*v<@P3uohdRn)Lh4N6T+OR#M*xR$WfPB-;A0R=mGr!D?g7 zc`2Mef$?f(+?y~mzX);#otWw7CEaIM$P1KLr3=|oG8Z`rALIS5zQ;L7OeMErCK%8S z{Yq7vXR>zaJp4vHKG34V8Ke^P@B`;ej+uLU z|A3a0NSlIGkCtv5@L9PoRIFjO$|y$X#a=F#rV!75cf#%a1D#Q-$X2VoT5X=f4)Hf zX~T_X+>7qPKtRkPK|qN9%ZAGVZ0(&q|D$B;O=UwI>2n%!3AXHQEzp}Q8W)mAbw zJs4zIO6nsQOEwRYcp?-zo=$_GZBOCMe{PbMJaPKr`Pg5*DHyR6woS4Y>@wOh{y zU)SsN;|_>j-yCC92X*&cTwf>9yLBfN3B%aq$A}objAlwhB@q3Pm3OcJt9z+7Vg#qG zM0p0-)IY~w6zG=xKs z9N6OYP9|Yo{7+WVLKA!(o-&%Efe~^L$pRNka!1B^YbHiDaZ8Xnk^9N$j5F>jJV+7i=cI=bF9J|{HJ|tQ#3nzsrCtRYgL5`pS8_l{<6s_ z$R%1~2+6Qp3nZTncv>)w43ZQE#n6ID9uFBQJw3MNSk{ zbvd(G#g3BC!ZpI5u&DPQ1}=Vq7ns;T|ogmc}2ys4pgXlocaDn4sTGY2O8XA zvEq_0ai_b#qcROL&J0gOW-6j?AdE@x8>w><$w+I(X=@JH#|K9t2n-iuJ%~7;N%zHG zRbSp||I~%PYvC9mDgSzTOK>9z_JA~)qm`Zn{V=p=DmISs^9zJ&+j>%YUoW|s_Wx(b z@ZpFU?}q{bK|}!oA^We4p$ssy0hs((p=Fk4q>hd`z8@kf7b}mVG%T5pM(#?S@Y)3+ zec3GzC@muli`Lv09_j#^CxyVn6`-6oz129a)1r2{smsIDVWpwNG2ePt*&=`P!F}!i zf$QT_eE%^mH4kF6whZfVc%-b$h+vCYeG4T3eyzo4j-~1A_T20C=3u(D3)cjL@2#&RF`j^N2x}2$ule^ zS>=h9)s>Re+{V4QK!Etu;nnZly1a@rq-Hg5j-wiuJ}S;AwZ#fu(N0m!5!-^T%~~xs z`77JzoKE(Vr}Ju^KxTLnA7#923JeABn{RMoWQQk?JOz-gzyLqSS~!J3=>V)88TEX# zcmOZ5h?I3Lkl2ejk6c1o%oy}3MNx*QP;8z&JaGHfYJ-QVtBM3a+6u@%Ur|Q#-EI3b z7n!AvLA8sVf}xUpbP1q|hVmP8e~3o~!?lcus~csr?7>0ax_xqXvN681{)%O3v@ed^ zXpP|jdmE7;!UDB_IVsASxtr8zuG9?38jux?aV`9tHPuPKjh~l)qlRzbAb9?+f-kvt z5!kUZU*$kg218_pe7w1hF+a(q=OGv&sj2KjgruQDb;U#C_;_QKEkWiX&3GDR=@^+! zxq^{}35V$=aV?9oBRc)NeXWm~FCOt1!1>7j`EqsGis+y-3YNK>_CSlgB-rY&4tX+` zDUjVOE{iVQzC3(3;>3!XXir~O95ch6o0VkC$DmoC;N{4=%=yNQ{bDD}8y~ix5w!Se z9pN8QHlq>`6+CK9c&#pHE1lIWEJx6U3BLYL^s!S?5*@aTje5K^75N@5%m?mwV>O^w z9}Gq5fxyDd3`0*`VMwwb){*z)X9iS!U`^P9IT~*)CR&tHP9VXBN%${ZY5X(I&2eCn zTarTs6UFLI>2AeKdLSlK!R~o0=>aZGceo5pH+bN!E%;6!0Nj3P!?d$-x8sH2BYzj* zV=#<}_KqZ=d}H!18t@JHu7LipE!4NL;5~yiXq|xu3!yN6K|9i`%l5LcbH&@G;>6n2 zOQr1OkES#ex!94c>wEK9Y!BZNoidR5EUiK(qhb6)cW7R~J}J#uSc^nYsa{Y&X@#iV zN@u?qgadk{EjqPbptk*07z23F-|@fV96oF)nh;B5MWTO&peXWTT0sXs%)sYr)2@A-D$hYw3=rCe&yeYIpR#i;+U&3ts`mk- zBlGCuxPh3mp+DXcFgsP*SIc|YDpI$ZlB;9eKCb9F%MSoKK*qn{fvWN;aLX5eP@Cp$ z{l+2Oufxf3&=Lum5xK0>x*d@LV~vTIPoa%RIycK@cS1eM5JR-k2SrY8#5+{7)2Y!?Udtp14o6j30 zrh}ibqjg>?pCedQb40x9cVgK(bb-^e@QjzqAlUXuOl@7xta;(+G@8pDL{??2zmVRi zoG0H_G|f$0zvzCCpO5lZ2iK(;?pHbV1HBDAs}45%NzOUzyifQLDT!4`DDfG5U%=O!-KkBbI;ua zk6{l8`uga%Mq(fy)Y>plTvI7;I=AbNG?P9|n{E@a1RkNrE(UNVb&s~u&{_GAw?1~v zq#$PF3~JC6IR_tBf{4e;!YcYO(Tu{f$17|&=u9;1B?~W=zaS6uix|xa`!SV~s4KTe z?+3mU+Tj>%FgfCYD4=jXM8zwJji@}1oO#f4+wdNU&d)t5*q*leoQNuPTob!p`J;y% zuNP!xOQabtS<4-VA@n2HHv}F#Y}mW3&dk$rNrPOtIsg(7=mrSM9}A^b3Ba!eV#3XZJLY z$tpBFs9!8D>L^Hby}yK3_*pux5ys!V?g?f26~8UavDpJptJ0kNkKPh;EDdbUXo^=g zc`z)OhA0wX@1BQm`u=dem%0{xYPlh3b~Suo0)3=_^@s@Pi}C&mxiYs5o)dvkuO6C0 z>Jh8rZ?3;2rONqXL2(*LK{v8F9rtvLNuo->ozTWIy+&{C7MiTR2fF+B(gW{xXEOc1 zJHa)K&5kf`M)P|T83Qf!m7JwZW0lD~t1F#eYb zQn7G#F|~KMlQ41txEpz@0G!+a|J>pitzxTyDum>R0ke`4Bpq2wNgbh$P)Pj(`l3D& zj$;8|MdoMKyi$Z`c|GbnF)_o8EctHuei6$&y`8lRgX27Qb}*Xw z%NWwRdLdym9pClA79l-ZhLWpXj~Q*rP}@uPXd*&Ov%VEi^KRx2 zH59d5qXVYp%1+p_x2aeIB6MuEoS!qR+nxL9l5@j-jhe`YGR_vrAi^%q^qV+1LMW`x z16N8fRbGJRGW}{xU`D@r?aNbm1p@c>cUniyXJYv$C=LN(8E+}8@$AsWM*0IHR+`~M zjG0$BQ3gR6(=kS+ZwX-$b{h?Y);gB#w+yakYE0a~cvC|;)Gf(2EKs6pFW zm?D+*rg`mjv?zB^*ocPZU@-3~jNcw7K=>Zw@l5slkx&ddG4s1j6Kxucp*+Q*3VeGx zAy^XlLQg*qd17X{MW=s^5o`Wf7t1V3Bxp53!?Y+d7lQWkDM!g7Zj^|g3B${~=1|CV z+XZLseBg@eWjIt^{$-F~Pk#_`jX%U$t^8?i`i$z7rEOwWZL|m8uHr{!@(1f<*hQH~ z_yN~AH0*t?m-&yO@(7LJ-Q150Qcsw~RzW+_ zKTwYdTSQmh%7ggOh{sqyPR*8Ll@XxeWm}Bvr*Q82B|Z#iO+q|ME9$9c7f%*K^6QUK zWR|KQ&;S?8=;jmVvqK7Q)K$3odvjC+-}nVqlEWtcak%)g&ymPQ*bQg)VUCW3h@%*qbgIjrApeBE z+b^sI<}d7JVE}qBPaFPJn0i29n?4ADS)6f5HY+7Tx5+Y26UKR!(k|L~8 zW{*X=hzN69Mz7YL4f>7i6iu=kVwuvYVy4PB}36EF(m5n z`vtDT#PT-$dB&@s!vK*Rp9nFSh7_^0R%RJPh$Jq?I+h7|6`Qa7Y0EJH>VgZdIU=y$9GD#+y8+%4f=(Hl>jv z!u_H{I|=Zi`{B*O>mUi6EE-X~c5o1+E ze_bpS?QELz5XzkkjtMA+lA`c)9U7lsBc(2cBtb4hMN%n#R$|6>Lh@F1{1GVP5Z~g6KQae9 zU3|X^D;Ez@*u^2lA;=++3{8zp4b2R{)nRD{kfMU&{zo+U7N??r`x^~ef1?5Wzg|Ji z&efLr|Hi)tz`-b7MFyFwhmc`YGL9HoLZGDpAGSc*y}FY;3( zhHTn`^{rFD>UKK@E6h+}^Ju%{r0t~I-SPE+59`pOXtx-BuWr2A-3$m)t!mmEAOf1G~+{{ z{8Xk$YRjAL6^G|3bP7{4-c2@*)7Na=a66{Q0A0GVcq&+f)af2YteGT_KE>_;B`NtB z5_Hq{)5z{4bY+}!4wZf6A4_=wn2oD$ieSrIZ!tqx+K8Qo!UE99JVvmZg0K>9J7ZTH zjJTzEkrX1Ht!$#Tr?+pet!BSa=W(j4HwY3UrpAoiefhB#%`QFCo0|2CVCT*%3op^< z@P6c=`qhyhs)&o@c%lhrQMD`R`>aD<;G&$wbt{Xeim{`gkd`y-54ZZEhQF|>{M7x7 zIpi|^CMo+ZW$d>GKvRfndVzZsCW2Z`@-X>s_&2sP#a94LD_P@#o31k?^;JTb@?8Wu zvcVC>;Xo&GmJpv9Hv%gc0`r&wniL6ll!#|=lrxOmojZ3l))2e&A`{b} zzLe3l-;GxAce5(c2ed=QKf2W(_f_Y=&@DdIATs&lbVMP1Ii z+FP279|R#Wc&zxIHf5`3r5_SWre-P^aKpw}^OW>D&0BfMCFi&P6V4&EA~MPr22@FP zKjE+zg8}d)c0e0(3dF2h!AdE6s`jSgPgu(uXu&R0!C$w(Qj;+iPgX_0wK8xV9 z1O3FykH;+n%y;$Z++xX!?est85>feyQ&7m7uZo?d{MPo^K4+}y&jZWvI@l#yYJWN` z``y$2X4>dii|&pm_9Kt{O@uoyw~}$uu{W@!ed$1A^PzQzO~+{W&sI58q}w*m>ZbL5 zFM5KB-r9+;#Ny{X=`-7k`+y#>NJg6g%houQP`6#L_{hzdK>Qn(R6Av%bu$OKy@EqX z;`)-lds*8$j%dK;MMpot4z!W%kaaThD{J8fN(V+3*i@m)M!R1J1XMw?V%)xXz~dp+3Gr_>T21pQ_NXmS{U|?+S@H@YS1odsUxe*PjnlzsDL}=> z)|DWDgH1xW#FXI!e~TIfFH}g1SBWTVO(A_JP8oi|C*5DT06o0JBxU#kD{LI$keJUN zxQ7Abn57?@A6F`vkEl4FQE?)f^&VPnsOB+piRBDwxmm>NbQfhuygbe^YJV5u67zt~ zeCIx1sFI$|Qky8Cp%hM!qf6pxN!tSSzSS>O3CJBA@U?jCxBn-h7J5@T;r|AE&9{Fm zS^M7z^*?>!73E|G{<`FiV!i0OZ7|87@J3*VoF>d;x@i|R=OO97LV1~u2o{Lv-Z=7e)8Tuo^sE^Sm z2~p#O-zsJ4{s4*0zv=R(EL9S|Gn%`)|0{g>pv4~mRxb!hDG0w zI7CA(g7zbeek42rEMcgq=1ZjhIz#pbBj&kd^T(mEK!ILDH7|1wC|2R;y}$Nz+xv>I z;r`*|6S)`JO>fdxIex@^IAI=NNotHqKe z>yG+^pVxCMFDgkD*1>D#OlSKJcRZPl$7}-`bn7^S+2i4l6rQ&uiD(`qMB1@As+py1 zn)P(V_le{VJXVJ>uc#iiW{s!p1_+!c!;d2+|^JxhB#3R*1tRvaW zhc@PA`3<8^)Q@0-l8?i`BsGr6piAWV+gc>#AwBu1f+7@x{hsWZv?>(dymQ&Ynglon zv@4nI2+X*Y8bL+yT6DS8_43;3W@$qq;;}13)PRJ_i*a#8r5$K!!Fz{Ics4$_< zp3hxBQK;|`4+0iY$5=z{Ss=~S-*P>9X8X!7?t^z?4a!jKFOLS%8bI$a{m_Tqf7I>_ z;0W@FjHY1A?BBS!kBM+s%Unk4^TtWP?FJsb{0L*QIgCVHdy=5$PeHYtR+T+hkc=3KMGn=7eLiMQ{W z%6`!H;#V-0X6?&eqAFmFH`+s06RzYTWfhwszBDEXsLWm{*^lbH$b27(7Vb%7kZg^C zvWYbAYg}I==OG+JTL34Kuw{h57a zM?EcFip@N-;Fb4_B&q4BPwkr%%Dfbj7%HR=_14-nRFNz<PQ&kF4~tH$K~3B5t3ye1Jy9>*ivs$+9YUM`vRRMdGg z37wMzs8+9TO+&dnI7#`jZe{-z9-To}=`!$S?vcfdD?D>mm3u#33Pu}4F1?>I4~u%q zcl<(cGMG6}M;A_}Ma^Z4W?Sf7%K|Dl<`cV0nQ*$)wZWgQRrPTM&sYUX$#7AA- zybQ%^C;z2v8OK*nSch~f#!M7Wyx2gk9XkA{_pttzQ8NynNsojtdbX8H(S}={gtpe| zLqRzDgMnz$C+@9?8EA3ZAnT+=0u!NBtuo zx}76wT(p9>Lu_MC$h#H1Q+Z=H^bQk#q&m>FbFH~u zg=VZDBjrB^I&8uHPMyj%f`G00L3_~anB(%9Bd|`Nkl0T}V)*6v^dn4vypO)!hymS| z1CdCa9Y@`SV#=peu*h)#u#Xw}@Tr3`b0f23ou|X{#I2j;3tVo9;7p-XIMARmw(?4_ zCov?8mD$GNr9Cp6_u=+O`lX4`WdZNtOB*8Cstf@NgkuRv6fp7H3L_qkdfAf02*udicr-%BM=@cbswnoA27TRl6 zD=2iBAn1O`rgL;cY4RPLcpFxUkKFrp0x1=4kpsG4@^EiW7^&}`n1~da&xhhXVYks} z1@(mMqqyUvNb5@9`jsJDS&xNOjz44BcgRFP03)`=EJT-~7N%w*NJs3Qb&xAph^kJF zVcqu{@4jSFwN14luthN~6xVN!TW8KqRq)y#W9YPx$pQF|0ZTBo+TT+0(>)K>mKz9a zh-(N+%BiA0OGIPU`WQ2xlJk|l8A0`<4%8Tjr3l!8XZ?oC@oEgi;nY}(i$~bQBBbEB zCN#;JMwAn5AAyiZ0L%ztMYI6skVcu{Nonh>K4!TuXbvx$hjv&QR81Ts0H-dM z+AUkd9VZ=iY}>YN+qUhblTPl~wr$(CZ6_Vu$=7?IbN0KR=j`*1ajpNg)~H#t)~vc} zDHh>P0)NF$<(*tVu-7g(GWK1UX;ftOV%ohI>)b3uMR7)%4g0P1uJ4Yp9E5F#yAB1R zBAOHAHj0VQ9?r7dov2RfSE5~6C!Q4DkPmxBtn+=+{gq17Mi?BC{^UnPK>yZb@pmc} zv2}Lzkhin=pND9J{+VwgXCxQNL{wHj?C%KB=&s?yl&}p-l@`l`74A47CASgCNY<&m zRo^Si|1;s-WDa@u_ey!%>PTHR+1|?51Hv}!7r-b-O@o8cS*ET)q9h|ky-kp}bOV6a zCvVsbZXz+@Ne}bNRj1%l6rGX)g#)NjJZZ(1w!$K2Zb|F&XAa@5xkKHL9?@j@l1|SF z5o)J+dUD+8BOeI<1H|+nQ|A?`3RqcT!+g_dI+sQaD-Q0j%@+~ zQI15Y(Hl;D2mZp)MXyvoFfm<1xs{}9DvFI8nZL4bAL9BXWiR^_)O8?hDmY8J0EvCx zF~LBO&=F@GlF{4mUV873#obxun0S9Z)&5f6+c! z5dqRG|7b#Ni?-%E9UPWWh8NxP`%c)h3vtq!@E8Wx@6j32b1zoTOmd>LoCJNuc`i{9 z%nYVcbshc%)%HOao?yw$^bMCC}AT3o>L>hEJ|-bZpk zo4L|sx@4Z)VaYM^3Y2fjUA?bLit`?Zd1g-HWaINUIWXd7FE;$o@eco4nE&gTCB#jv zt?mA<1Jb&J!?}XXih@V&4M;{WWMp`Yf+N7nWLFH-3I9{h``C%go9J-15(RgGm6JB8 zpll7elM0Vh+GKQ6#K36b=^gs*zW_8zTf#g#Cs!#xK{Fyjv-A|GN%50@kdK0jr8O`z z)<4!i6bLj$9UUVDLmeHxhnY$~FxDTIc&Z2PZ*pJ>?$EvQAJd)wEZqOS1zTqmM^ghM z6M?^P!_^gB)fJreZ@Q)LbVuAmUcun~~4VCWObP4w%V zc#yBGp>A83pQ5UzBh(D@KV(ZtePey_IB6K_zu70LtGDC#$FAJ}fBOXgPq{VeegRQT z6ddEv5&zYc|7WVz-*?fH5dYt@D=1v0Yy=E2KFTXcsK@E+r-Vl7D2a6hOazQH41Ui@ z-$>6y53nIa4X^|$IP2d`2Nfrg=KOQ*69fHEApDo%JfU^b2$3;=uYycU)iWav@whb+t z^krE_Yp~ryDO|9ghxdWQ$N;JgrCoaC0MzPMGTuAgMX}RXO&hn&Hg}V;-THkaTPZPE zXmRqE{9RgC~BYKe1 zTS%1>oEXX&iz7$h&Z%sUYQ(LWzd87;+O)_K$Rz4wujW1nt`L;1?zv4cbbJ72RPlCs z34MgcOrf%o^S^X52FmQx-~)w}TE}+@S1AW{`$!arY9q$`MS~%cfQ_eeu(JtVx%s`n zH7cGP`F2bLQy?Cs-M<|GYh({o~h3c}I(XZ05gp zDCIL%L+9cmc}5rRhQ#&FHFpd-x&YmII=_C0jCiqv1H`S+k7m3(ar zjh;WZ@{`RgY2UD!QDX4CuIlZ!yI}b~v3(%u0n$GZ3S2gz`+}Vz^U*xz&he zw2Bgr2URZDZhi6=28u2ioXy6(_O&RilI@bilaj|)veX@H#MCd54=L3JEloQr@J;-4 zrzfm$aI(rE+eKbS@Yg3s!m60P8wPtF&eN`2Im*EE5q?E7Uzk#8eMn;Tp#_ipsQFsoGtIlfI7#2oYH|%~J9}al5aTg4&6^bf6F$9XrSr^y zO-&w8s-KEs#;Q2E&Hqh*c5@*5k+oCRWAjQ#6w^&g-wKsyb1#WtHt_KaLVsKQIyck2 zUh$DobfJVZ2gG3yMr~F?m+?8ICHwh=!P^SLToaQsaHxjc8UcTJ$re497S|?mi5IquEzd8Xj-pvX_fYmUh zg4__0CX>(`WcDO!1tof}r{cYAx#a#LdznFqd*E+bMKs3-B1+l^_`d?dlEQF5=nnw0 z{w(bO1%&^r=J>B%<1)efD_|jSAa3HFTNGRsR!&YvP86IKAJ)_!>^K>fq`1xGsDP32 zT2T22?bL)MiG-0c1VuT`__SD+_z3l&0QIcj7y|>8|Mox9;J`m=u&?J^k%m7Z{~7-t z%=zDNoQvh3vIp|*8yw_+D#8Dc`F~-;Ocfh7ED>a1Qah|T8oD?_#dXETv~Ly_4JuGz zKtTCGmPi&#lx;nj$w;gm>9DN0z6riS6u8mZJfAVXw`A_#rmV!1QXIdNTe+U5e2!YD zH&Q?Lua3KcQifzf8772dhbjPX2+s`p2DAbArU%Bn0pz@)hCtzt>T}nn_tavw0V*8c zMFjb~caY#Fec!rbgjD;}e)`$$*8UuU)Yhb2S6Qyvl*w0S`uO=3#BaV-N8=WCwihbn z-=pX;jf_5`+0?4bwZf3uQLB`9+-U8wOm2f2MRE*l*7LDd_Fa2g^|#xj(h<{sjv`}u zX=Kz2!T~hH+gIk9i9R>0dCK7fYqn|g45WzJF!|7NBdD2$yt=k z{U{Ao$v$j^h5B->Kg2?oO;!BgDcqu39u%71Rthfd zrs>n#a|I(e(gAF$TnTJ0`&=H_slw5}kWcLRY_^Py?4paYrd{JC$+GA*nRPI1gscX@ z`c(HFgYIJtvI&>-dfHAOLEU~hXm9Lrq3rjQ(F4ZObA2+I z@I+sBpjN+OSI@-fp)~Y{B86lx#<6h#67F=g4@j5 z@bw@;?}GHn^&l-?0#K)_Wn?q^j{D_gHXv!}B`_s|crFjwY7UpR+|y6;A@^}M4lD&w zuV%+SK8eG>T|8pE2O)tjP5mfG>YdqojZVoDKD4mmWv&uZK8{?7f(^rcHDB#sEXhQIz6A~c+J=I8%VeH7S#>Pt{^HZZdK zZ<3GSltmIi9{!rQZJ(h9`>s4FM=sdzt^6X7TtOIOp?D1}l;htpBUHpg9dDxP1LH%QITV}^SJ)j@3-f{8~#A=)Ue z7Ak-T3bdhagb~!!fIXim&^RS-iTg-SYSXaJdQF(=rV%^Ij98V5{F-vjOi*R$UTvmkKh2GdQjB37GnpU@F@vy8 zqtUsLE-F1N)P1@7;x(iPZ1hc_u5?Vb0jyjp2JG3&%r#6pW|e+|aLbM$*R9?;&UKu8 zrxpT0__+0iGSV@1U=%q_Ab?wM;c#&c0gbLO<^ql&XcCoc(IMZaIp-4V!N&J|H2FOZ zH=716^J}i70lq#c(D{}eSSW$`EJ7d-^jai^1LG_ z2$+L>;6u6ik;odFOt{JR8n+U%*^L6!2<4}Exq-#gKcI+4QiZXV;^HlbUa<{y23ZIh zw1;7{TcjECGzAf-f57f?Ej-bfDqsG#@gdRl*&Lt=Lrlmh07ejiBAsp|5qm!!&fl zGR{EhXuFOwD!-^%A|p!tvtKP!C$A}w&h0U){b_^A?D7duuN!DZWRV|J1g9BhKwg?4 zw2^p8r7pLBPJt1U5_3jDH8-x$W<$bo{%0f&&N@tJ*1=>m%w6N9ezi^bQ^&lE$aED| z%NYIL$_@5QAc1XS6naDPXQ;mTGPYdDxf>>-`yduAL2BCe!6gDbcA^}=Oco~(AkkuD zni~TK&SyBDnPe5Q4i`$JM)E5r&!uIb;h=K4k(F=1X}l8^1!UMtMaYKRe74~{f$`VY zo?^#72tZr-i*W3WkR1u=Lx17JcLsSHhlJZrV6VuKIaVO)xwwRhA(#s0h}YxLzPW}@ zWOxA1bu^{Bn1T9h`3H;n^ni}B%LbDJ9{kA?5{IGWQo-KkBW`?e@4tgnN6!UDdfZOw z#8WXxTV!E_QAxHcqUZ2RDo1fd&IReDU&{?l6^+1!OYjM%`NGs7Z5WaotsG&HWeAQr z$P#uY-0ELm6GJu-0a?Qzg4qyah2)qnt0G0-1m7KEDp+$X=43dR(dR-fleRsRgPtj6 zbTVv2ETL`*ny-qnH%FF|x=KK+hin#KL({g-dUGE5{t^N*-XdUzS+6cYkkE+vxb+V8 zS9dGU$JVy~<7%Pc|4Cs{Q5OAcsb6{07Rv;ghbilZeO`j8bf5VkIPg<`-D2-E@SDsp zWi{kR$#h9qA{C9u%~Gb`#{3&%LM;7Ws~Uq}G3v-r-vtY?T)71_ZoY+}>CZY|vh^?2 zt_K}&UHLv`d0kE)^Xb07!t|I_tdHax?gqf(po!`4#>w1;d2iL%Snm437NG&Qasf9N zkvdmB=_Z5~YJBypPbRhvCLA8gCPeH1+loSo%e)o>2 zF5x9UxWndcXgt!g@q=9zjpQ(#m5p;v_@M8h=Mo?Q@0HG`Z3_`DWjN3<+qQwoCBo{L zQyV+EhtWQbv=oKJ)Po7s=5os5)Pw>lfzsthcSG08c0AS(EPrROa}~YhpzvQt_A5L2 zI8N6wPE?#j$QRfO5>DEoNfZ6O;i{qh&i$Olp;xBcW8uMrLjlw;C+!($@W; zWx~_1$vKyfev=6Y35~RxTvH`76JIfp@A`@$OJjUORPC$VIos35z5+$A+w=+aI$EO@ zZ$WWMRl-vriU)G^zWOau``*+72f7%gO=W6}GJ);LqfpX+?HMBtx2c0ALikgX$mVV<0pwzof+FRGi}mY!BArGzz|S+ z@%V)mdNq;fA=MSG-Qkiwj|-D^J@z>|fwS#QSvBHi9x{$Hysw_Rw%@**?`DUuw1H}H z{FXdcIGYS4f8`SaYvtwWE9@n}p^@O=jv*GKa1%MGA{3KMNn|9a#J1!>!k@aM&vSMK z(w#kj$BlJY-eY*URl$j-`XocXEf`Red|4*$sv?#|l%sl%7+@pLN_;L-yvZ17cJ{{F zj&;}FYybTkFc6No9sk@l08i{oeiJt^j;T9)P2~T&q3&BfY~7DB4dYRD z7;%NDKC59bnYHU44AX8c*&*q_^;vcKX%LEc$MJ&~AqI3$2L$5Dq+QO~>#aQ)6cfYCB%r(GEs9NfQj3ml>p>q_48%IPrPA5ONBKMbuQ9L3IH8 z7PG7Q9X?dSu8$JRdJMJLP1>}G+|ukZ-Ic*CBK;{X>J(GA*W?FVM7nMet-YEPi=VCM zu)pj&d7>)~?3cJyWdd_Tg~Ram7_G~qT~#Vy z%C&Is!)qi3ICWG^Kg;lBJ@6<#m&tS;6E8(Pv~Y@Q5X?*+*VR1?}4x8kT?^!7EC9uz|3EE!8UchXgejl9a_ z?H={RikV9d1(j=wY3!H1S?og$y%DrXq&$j!r2+9_`)1D^>7><)nT9P6@4$BE?n=FS zj-HB1@sx~2%)&?E&nU-vLd*Ltfz2z}rFYU@yz!E!1xS$0;HBh+qLU0!6XuqE&XZB> zzLDGJ?&+lw`^_nf_8d_h@3Q*vg=8cLv}BbPOiHe{i}u+dYQ?{G!Tl$TPEH<#m?k1CcbK9UD> zOel348n{+jL~G4C^Ow(mrwJ7~;0NsL%xgOtF1>8E9;Hm?R0hOjWYMNUoCfq%vRE(T za%HMI3Y>l4xb`1W6w=_*A4Niq)!cLTP3KPu&e!hclh(BiaSzXhtuHghLj{g{>P9eolwVN z{rTOd`~Wkj#}Ej(b8b3)DDDOpq}b5&X#vPyW2IBcnIBp#1Rc>Df{Z9qt;PZ$e+gKP z3o0XLFd(LkPX5?lNIW8VAhKQ^Ax;%wuL7wSe66(TY$@S95LZjZbwjwg&0Uf)zE4mS!=V>l+rD3dVKaee^&=IKQ z3*QxuW#I(P(}4qfk1qoc&wo4b%f~axm^3`PEGBn$gKUJgBa?q9AK+VX`tgzQoJ_|c z9_lwOrJb34T@xD%D-^bm-TsEIBYnV;!kdlWZv&M5+e4H`dV1jk$ggMNf*$t(bZ}sN zVo>48izJ_YNL$z;qk@ZoFV{=4A1YE}R^J?bHU;tfwg*aW^lXuVa>z?N#ehmwNm^^I zy*{0yRUkc9l$IW6tpp0kq;BhIH?bnh`_Z>;?4OH39w20d_})Vh26muM+hs=>CP$~t zYNHqx{TdsA7R>wDAMDeRrYYsn=SjiuZ3V!3@0LB?r7-sKb=ArnF87()j=5LU4Dc?2 zot1t9LWbkmyj~O$gbvU?xx>w*X-G8n<}%hY)(ZHU=1Ms{u?jm)1ILgf64~`yiO=3j zWF?r^!Q{9GkkPa6@m;@Um{$f3wFE*>N+?m^9fk4#q#Yk8a-V;DO^e!UB}fnQ+qWph ze=Fkt7q7~DFiZY>8=snuo3a{;udSoml(D0K2kZf$S-wOXAfcpd1tKa)&YUu{B2V7C zB+Hm=KR+tX$rK73)+o7XSxK{~SYm@xnUklu9w<-UB_`6Kr!B?zEW!5zudUPYeC}li zFg}u9Z;ZQZd*XdN|2RM1XnoyESNmo?ur9#7Q~y01IuCd_;F%jY{s>xtJBm(qUy7Y} zo-E+Gt|I6fCDoTDgv?|uq}Dhf zv!YxQ?qM9YyuysUtw9plaL@CCMmKNg8Of_~SkjF2Vt`@jM{NVm-9}z?6*f;w9(1PCzr) zfddGS0ZD7Ii&aSt$8|f@!VI$*yA6)4y3lHqAb2vaXSYsvydG>(usm?2FaX7^Y{ltI@+^5Jo9T-U~gGuph?m=UiPWIV4s?d1c526Y?3p1kVhPzV9n%LxZ^-R6N~T+MPI;OR9`KI=|&2i-Rk4ntsgm31-+Lk$`B; zNX&JGZcjOLvQpr){^*Z}5(eSx^#I`;oJRJw-2v7mVupPOuEuf)QTjd)PHQ;SL()-d zzuH50ljKkJj1qodr1u09RTBsMj!6XW`|Iv|H_J8GyY4PL_@iNx#H}Rh{vHM&h~GfA zq)SFb=>D&{_f>x6N^o1DbpQ3*tsZb|<{GH<7w-h9RR8k5U6uF29+vk+0z21K;`xaL z7BpS1R8`@0*mqbWUD@xGw-gkVYD0d!N)zF+XCZ*GnglVNo7(w3qWGpgum!e>U>)4+ zfWj^14bt8srYN9W*LVn&V`FB}bLs78{XSET)7?u{MoSZFTr~7YWHOub^QniTpI3)> zofHJ`LylhFq1g9FW?21)lL<9iz1|nt*|#no6YGWE6m%5ziC>l@GD}`B`)lM5WOkb3 z#d^9q1Z?vUcj9E(rI@=l>WqmEFaA5Ib>yzo9tD`0AdbarrN-&Pwh> z9@OtC1B&c0SuT8Smzt=QCi9k*)pSK0OHJk7P}`Pp)hX3d1bnEzWB@InjF@nWxuQ%T zra6tbG=a1R4q(~QBt=siMz*G+92O%oX|#r=1)^?>OaY`#ZumX?%l3oDn5;}J>#1qF zQE#b%$VgtuI`1!wp~W>=$M$TxVr(`SL)3*;Hx`~)Pdj=i{eC4kdguMH*fW$4-9`5D zEy%R~0+mC1 zl)n@Jo=XfzD687wSbGEYBzNnpU6yK%2lFb1euT`yJ~wVi#zlJ9By)L%Y{0yb4d{rD zVQE~4y5ki+!Hn9CU%0;U6?34I>YHBY8Q{7s=)}4gA0yi7dA6-;VLRIi^Gf=*#$%wj z8~FB}?3^xRrWZ-O9m4$3*Yd%j7ZyN>f1`vx@`W>}PSyMg-}O|(zpIqiM9SyOKX3e@ zL!BSi_iK4p{O00&c2^WB?{{3lviX`Vd-sT zRZ;0LC!+Iy3+gC`bK zC-h<{Q{v)9a~KlXE@s5S`%a>QEWT#M#`z^Y5kwr*>nq_bdIzu-ZSwY$CTXJ_();7B zIlZJSQi;iERCssFzUWZ!hew8o!#s&%vG?-23Kj2x#g1XIv?{t974HSbw?xv>`4vIv zx=?2WCL4lN+M8G;4iyAgTbz;6*fIPQdAyp%C@sYuZ}#v z;N!x!BI4USCku_@ZrWu1AM2|*5$#NoBa_WJIMA!SGVa_$lJ8;L*tyr-06IX$ zzw17oaA7-kF%C1k^)hjHna$a!AwBu#*qd))l;QT%?`PB;Ww{Jk;|1mB3x5m9p zWgEG9d1PPO^%GUfnj6j=laMi0aFo>VFbWuSz*r1G4A+&`CJ70yr<_Dzl1Lb$h<-o$ z#@ub!VbDn!d{HxYB#zD+f}V0Np>P)C`R=gl}`y#eVLSk^HJt?}RX0$G{tNY@UGEH_n5U*x}+0qOt z&H5qa5204U_)B>)qtB%Hbv`x6)Un#8)3Qvqzv$Zk-Zw|9!q<3pDk(kSLJ70^3Y*QN z-)Gb-kKDh)fCb8(%mB&=(HI$&T1tI4Sp}JRqu+;i ziI{kXEEmrQMyUCEC+f;6|ne;9)bNYoW{aTE)_& zU-H12xHWHTcyf3rFEhSiQQa}ao|@EHm+RQpwf@X!1G^m}pEfSrs5&og$c*vB-6<)# zAH&t^aeR(OX*d4$9x+?>3CUIF`#s2M0|>7%EPxyZj&ZSZtufJnpW?`<M?TMk_3^)f_%{0iAk^O4oCN%mLpUww%0WYUpxrIAaOWqDJ~ z5NURuf%RaF7_D5nas*1WIIZU5*_Jx^HC8BMbBT^@FyQL`NKb3cfGf|o?=&&TG@GMCH zMgpZY+GdO;{1o#u+CG%9+z9;;4afuOUV3Zb*Y`~clh+C8TNF+jGtfraZAhnVGCOm` zC*fON++`VenlNg`tR&=digYR;{izo~Uoy<3l7vwtv?3<3x{!hJ=`?*w!s!l53Sjl7Nk6pijM7LTn@=)XD>%n~CX#2-f*gZw|7 z$=TfQf9BVy=qO>CAp7dpz36KI2jT|`!fKL8%om|nXi_%8&Lts+$v_|GpO9+ARgT-j zgfnft5p~@5VcEHe3Jy=*Lp&3`6X+^Wd(;bxqZY@Ha+pnfW!-gNeZCw;FMl)D2QXCc zL2sBkiVf-bQo3E4x?s;s+5i61^MLGO)+ z%(}hPL_Y(bZQGi8f>}1_=BU7W1Mb%vr@Oy;$S7Q2)Sk6TadoNMv<^Z~ErsppF<*f} z40#do9FMyAU1Y-y`YH)gq(XsiJ6Q@jQLuK7HB@Diu&I4!Pur=@m*Eq;?nDVl)Z)p| zxxELL%yI?>89#75)UOuCI3tk~bsKAIEMk-PeN4#Zp<8)=sU7*9WRZT^(c@NniP*-% zTY00<8A;kH3f9UJ&&lEGJIMN=i9c}?ri40E21w_-u;fmBw0i%cQRy0CMV$DUd4{27 z2l6WM)MC?y-Ds|Alc@~ezIG2wCH=~m(|Wa>0ca;wLQ7BR%HTOpKMB~h!MP!(?yqWN z3Db3cxNQdOq=k?umu^>ehQl;5nmy=+%V;{GY|jt$enT5#g2fuK95G;upn+X?uiC|Q z?R(bjXY75U*Z_P5<7#2Xe04 zJl~lre)x5@LI?l#5ZNLtHH55_p6`r66>=jPgS=lElYuI9D>2$%&>7-3$}1rUCM*uT zByj@Ky$ND3k$I}{3FJq2raKNlQI=MY~^>ID(SuZJI zss2|fso|MF)>sH*sj{yq<2WkO*aI{j12v7IgDD^RUb5e3+B=kk1%HzbDp8z+FIgMu z)~+@P<1*F>*=q!<{jaX`bmlXfv>8^J2M*qTIGbKp)YB=gJSw~Jsxm|v_L8wRQ;IK& zwML9M_o2&C)t2WJ?)h>c=DrLZ- zio)U+g^Db-g=oS;MHi6J<4NbX64yE%OgBSVNQuHob}}7L2;M+=J@%%Wk`iNsQ!lA^ znO7P0X1-rvpGZA0N3T0#ie7b~_X1*+{6{vY?fLsvQ5Jv|N!soXU1~eixz_!T+P1AIwyZ(PbBh>+Q8lo(s(}~iuvVQlg0<7$enVSduWzs0@BseEp+iX$e^Lln9P$G3A@WR-Gy?CmERWDyPyRoF&f{qohS9lBeR2`epyCcI?@s#ZEl*QE9YA9Dw^wWUP%o99I7g* zb~H<`b48g~vt+(V8^NlJn;CsBBB2i<-@9$W69)Ci$JGarI&F<$0H$;28OG^n^64;5 z^J2E;wf0MN@=)TaJV;TO&7)n`r07H)fPmx|jXmrU96%9%uYBAqQ97@DzCB#`0D}0yJY-{TB$!vX#p0(yv$J4xkQxQ8Dv4## z7RB8E>|yw;F9#2XD2V^@<9Wz``$PTTeEGk~P}s!D$kD?7U$w=5(xM8wuWT19)VLT$ z?vP?ZeGFF?P)l_>-cBGIA%Zb^wCN z`%*#;5u_Nkl6L3maYr3Q`Yn;8Yf1Z&V6&m_HJhwBn-n~aRk7xZ3wPlR9~5f7C=C7x5RUAMuW};`l?GJswJlHjdfMhV9xDg_aYqU9aqB>uRF> z@bG{1Uae{aTvTBcrkxhg5%|vpnrL0}_;ZCWF>0P@!i!hcE_r*zH@Z&MPYtf!1)Nr3 z2E|b%CWebzOxja(*inOT;DitArdNA2wWYgERK{r}h#I3wyA^kE*rzsVJ@+7Fa9utj-A{n#~`U2oGR0xJfBOvIZ)EoZ>@PVjc3OF zezIPa?zf4Lh9nUU9^y3b0ryDEJvjqesx>=h{URNXu(N2qi<94`bOr23D3RZ1+d4Ib zszb4yk;h>)Iv&lUcN;@IKN}5;W4B)yh3@HmzZVzP{A1YxOwO+4TLXXhz(@6V}tT^4y`UmeRI8(N>WF)gx%zI<>tb+t1aT8M1nENX`BbAoxZF->4~qB!UQkW(Vz z=$S`i+!IXBfGLL>2AL5HQnUyNy~v<Am5%zclsjObJK}~{Va|I;E;0#li3S&eeo9V*@Oy`z9l*S(u6$DM@ zxU#ukea|0#aXLvT$N2oa_%JAh&+|3~Mz2*9xl5PM7!6FkaSSow{*O zkn6^qc?r8=tYZ^`gg*Re$PV_4nix&*BFKkV0C;ZBiUfSk5)7?-z?Db9$K(!;+;y;Z zu&R9xM#mY{(Q*M0uIkIFudMj(_eh6RmZ#YmdhW0-~JklCM6W1Z=k<@ zJHh<7*0=u^izIAa?X3RiviFbYvXlpLK258WH@W*T@Ckaq39zPr!wq2xG*{=(m*^qR z!$x6CH@n?4WxKTH=hj>=v#G39wL-BGsby82M^yk033%wSY+Te_U(?iTz;$nGv>g29 z%Vbx-YK#ZEp_jsDcf|L#pZ@jWetY-k2F$K{aJVO0F+C{Lq5;P@cWp$c-7**q+oIBM z!)v}m9=z`J40UPW#n!9#}^DWKum_s~g3I44Ub+c{;9 zyQo$5?`w;iNt-zrCtq8V<7?Nwh^1tsMhS5Xj<0R$L0~QD1p=P)N}p&lCtt;311-YM zXrq*;SqvFiTZ?eljJzzsNTpep&MH&#D!}9bB`h722cgNd%C|B+E@kq_6)zx2V%5;E z;1d$gOckG^(h_(Ms8yxps+)?ygx$=f5$)huK-?WmX;WuM)<6wQzvXE>frY;wS2Z9X zf~!whPA6zgIwOECRS-%S+P$j!`6 zM{h=s-hUTs9%+5bN>tekqsGMxy_kscgc}iv2$i15$mu3qPT|8PogXLxaru2v$XRmG zMfwpT057cB#SL?o7|JnD)J#p@ql?NwR~+|6L#!a!;yE8s)3X|ES53pWE@BZ`6(iA_ z&7bd=qF&o;te%X^@nfIm#@#&gr>SN-=I4Mnnf(&IA-r{8P2bstdNJbQE>=F{_FTV# zhq2|F!s@fuv4m*clK4lHat79@~Xwioqz+}dGbp?gLlz(0F zH7X}{4zz@0T3LbL_Nk{kF(N39Nn`00><0?X)tHi^5w)HK)Xb9Td5Dh`#5s^UO=DP{ z#*p;M=C<^{qBU-YRn6OrnXGlF#4kDtkI|Qa%UPeBP3XU8sD&za5 z?Uj@;U|h66?e34EJ*;P+8P&Zf;=lK6D>kY-SE65v^E2v9-3?Gyk&m@}_6+f8?yfOR z^;`^4bLjvqyL9+>a&8T1ac(8`bGfSxdpSPRalEl@5$=Fks2mcfYS}1b8b=_v0w|=0 z{jR&B>T5NJElTDUqQP;#1|j0~Go#>dVgX;L&**Q?3wz2q;Wg@Fu&5{OP%4}O>LvR~ z-evnt-c|cpEz%HN#QwMaEtWEnI_!b*aM`+4p+WtdH!ZW8hN4pLQqEOz^jJX}ZT3)X zTU&%#F$b^>RWn-4+bQ)?h+f2SVx#+ zFd|YB4Ww42rG?QY%9r1n>$c(ag`s#Lbt$X9DlN9^$j%WL@&$HF@(ljW`k=Pe(F0P((5*iR& z^%?wnD%EsM_6wTWYON4%ob@eEGq7sMX@_E6!?1#BD|0oKF%q)r9|Z<;B^4*s2ydaE zxmt-aCW@<_+rW*kOqAu2Tgw5aTFA@+LIU6H|AtZB=ht&r7i6ydAon_w6H`l>UGDLs z=-Uw0bb}S`;Oj;n*Msb9ZS~Dj{-;NYA#3(D8${WQui=mv?N{yh5)P>V&pP}PvNO^Q%c5W0^MBlZ!(K zM^=^VInyp+=C`g9_B#`}+9Xd@3#T0q~uNh%)XxY{Ix=n?Cg*5 zKW+CeNp6ioJqaA>NcMmXd9u~Br32;2`X<$bP=PUtGSK$1wfmdbXTsVIc7PgoF;d*& zaLx-kR|!Q{F$S+`r_6|uhu;o97`RY9<|8`-UeLwhRPLcy8H3pGMyw<@8!{U4D{P;U zBq5c=Af~0%Tosq$DQd%1-B+I^{c|49H_k06-FW*6;3Ja?fMJS>6>~kynj?(kUP@*hEef^ z3Hg(4lXxgwOX?|;+_LM6%h2Ybq9Juv>oa$asdpkXUAth1k2?TY;d5^rI{T#qxsb?o ze!pnZ1oHjP0?)X^i4lN!1c&k_U#|*Q|Eob^sPor5ELdqoJStdAcwJI?_-LTUEj!7i zP@lCzC4OR{2PXGhX^xnXbngZUdddaNoc4zjvL*S}uvE=YUOgZjIUpUc6Mi{a5T5}w za8F0pnPr7|Jo)f5s9sp+Gtsr$EdFij96N#;EXZ*iP-i0luFRn=f%Z%8Z!@ZOABOzV zYO8ka?nwQ;=?4bBQU#1Ji>1=MBs~SF=;1eiA`O5-i=IWk3v3=3MN1x&d@JDeDQ2Wp zolCtmE>&x2QB#HeV1kmyQ8mA=D?_7}E2g!W9O5?}KN>m7 zvO|KTuA@9jGeV0rUuAazR>jsf z4j%+Wx;v#L73oI0ySqE38)*>f?(XjH?w0QE?(mQAInU#9ANBb@;}W^{{bOdWd##z- zduH9A`*CXFCK$5194FV1K3D05uy)OrZ)+E;wNs-gi3*X$&y}u_WdjzGwI409L-0&{D@yEtWa- zf*!06wR~ZAkzbQ@>M$`CQ8y?!eaY(A1U=5!w+&+DlJNEp_S7*(b-AO8(zNDG!C&cDH?8CL$fWZj`@{|Lyd);@<9feerg;8%Z ziS1V#RIT$<;XceuAW*8QVK(Qi8kvMl(n1PEH%> zCrO4aT6mp)g1adR0z2Jv>emlpal`rZ6$8|P(Ci=>5D@+IRRifSt{5a8pRX9qC9NzR z90j$^bxr?s*$^bJVvi(;{FE(5JE4^tDvgiN^nx49(Cd>w*jRY9w3x^)PFF`27O7P0ubtC#Sh41ZotL>~@Z_53V_Q-wZhDJ3Y<5gUq;HfUVh z)%FpGS@e~A@Mj^In9l41Enz=D1ej*uM&AJzNYcGm)J3bGg9~c2wfVbj!Q{l{xhUC< z)XKt(H#fr{x!O*x1dt8m$|Cn?)KgPmC#5hm#fd5vv+C=;^_i!WvXsM1F+grH*S zpqG=H9@#wb^c+e%-fjIDp&c?DVU(94w^+NbRBXK{ek&sr-6P46JvY8Pw$v;`XuvO% zQ{OP}G$-IZSI>%ED1NK{zO+%Z*DA*hY@T)gF1W_lZj>_VNDxMr4VitXrlairC-TY$E4=SJ^xNI|OMa#ezLbwlWqsz|M*n$+U!M{|@_YV{( zTA(l@?}(7_)djOGp?pJJyokwhF*XhVh)p(6<`aETjiGSWJGn<=jkiU|EEve7+&G)g<7*qT?40yo8Op>RdcMTuR699Xsu zHXBMkK-R@D{Lms-@Kg+;%+0+%v`m62T^%cCI*ZSNVbqgHch)^dsw?5;Qo9(61(pN& zF{UAJgva5Gmgbve`ckL{#}UX?#Nb5p74`*!>iVJG`&uJ>4;;wq#84}ZA zBj{mVKFyPJ&{=}xVK}IRs%UsHLc)5#PN31RC|Df4nK1u)@z6>+r~b8akPk6l#;(k8 z^FZF|8+y%cYsZUN&=CIw-3Ai_u~iD2-GROM!PglT)%Bkxpr5b0WnIiypcXRQx3*$l zHn&ehe&y(c0%MIvYYKcAQY5%SYz-l1pq4r*@s%)}5cXcxam^I2lP@E6io&^2lwqrG zF3Xc{nXi;ik(L~&2t1F549eVHc2b?}-!>bYi&x-K3hPo{@l(myhOo_ z^OQ@k7YVYR{#LI>cUd6c7Eyz_Sri(J=)-YaiW*qc!0l%QiKS8ZI;mKty3gX}V^sTo z736{&T6{Av`Ww)q@uW!)caV6B|BT$8i>fk2Iesn~0mY&YGTZUk@4aa$zJy zUWg7{mSW{RlXd~~pw1BXo_^N;Hj}wd>IZPOYHb{GO%Rqcus3nMXu)gr z2RRG~>b>qr;K0oDZ$Xf0!<3^E12%XK4`3G%(w_Qk={IjXM{&EuwZu6gxIR) zK1>Y}G7Ki}9TUX8iuA(1P7~{xI7qRo(piIHF}u6CwVcJ(X~0Y6WUfctq5Xu9Nk zfq{UEKz@Bh_))iw&)mSsT#xAeUq^%@X*CJV@6S;7_Lp}b%NJ=>RRZJHn<=MIv-vZ; z<`M%Hx|@U^c;&TU7^fz9p-|E;n5xn(H zV};ggl+Pw~bg4F>ha8qsp^{?FN&M?3s;Do*(0wjxZK34gRBhBsQNq|4!|~kHjuSb( zqVy-V%PY$9-x@T@_j*Htp~d;g(grx-c(}q;FCW&dr>E9B(bS1RHgO-VuVoW@t?>dQ z^^8S$_OJcTnKjV+3UF~^>ZTFAhbMJaPIGf0?hZLe55po{5o|rb7QbSr8p-&8hQ}o< z%gKaNub6IZEt@uQzi?S++~7C#_BGi;3tFLlP9Y0u=VPBbG-PR@nl zYkrib`Q)<){K$Ov)Y8&bO>7KbpPmb52d_lnCR^$t4Bw!flweS{88F|=obX)?cA09D zcYOvZnCUKAqpjeW!cbcx;{<(lvJ8%ImEx(tn5la_;=l2rJlthc^yF!v1=4&Wqbsw8 z>P^$9!mB@AJ}v0-7_fmNz3fM7cO<2IOscOlxG?!vYA9sq#-~gx+|L%;P<=(u6s;6p zAjU8yQxAy+N5kO(_2+D`{74-hiwFed@b1^2ZUAP(U%61F1mS|cY3puLqEy5mT@uxJ zR&~?L(uk_5Qmr>zuGgoC|E^kZ;3bb{6Q+uZNWuEv}fAyQG zMPuEyU0!#FD0SA)yv}EWd+;twA>USPWeDPWiw&$4rfuFkGGx_ji8~4q81aJNU197d zmAN+gI*?!4bd5vY&O7b&T@iiS@Jvs#o2fAqS}`D%U6K1b6D!^mOe?3b&AbpN2C`n| zJa3Y?qHBsg=sadeX?_uC+-QBwnTU-6CW6MaoU}t5j?vr3J*^HpgLNE~#MDMdC81gd zJnl620Y7{zTNH(`iG3;M+(CbJE?b$atdJHf5N{51UqM^YGzg1+k0%`hV%KcOp|!Uf zJ??;HCPECZ69GS4Y|2q~Ur@K)m;47?NLubK>C?Y&}m@o*>Mkyx};L|*xIY;*PBH>0{MuWd zWiYfJ8d9ZV6E}gnN~MPOj$ZjWr^>RD*DOz}ll3Fl=R&<&*U8b@Nl%gQ>Zp(WtaIfmgN`^g0 zz6dSmY>7`M3FH*_lEfY8`dQD#EMv~Q)WmzI+k6u z6N5j4h@x^%1jdKZyCzKyIm{9V~-we+aT=-6-cVtrq|-xy)PyU^7Tw-r2_Z?n|FxaWd&tQSv@$;!K< zpJcHLx8qWvIhuQJhTSn%`^k@mz5gCNl`>JWbE#I`c~?IuZ4sW;Acz5|@BZ;bNA$%W zogfbj`;ar=!zb}KJ9bIVx+jz83ose^@+K~lCv4tcN@BnZFFuwiPUpNRn0T`*l=S2; zzR6u+C6qMJ8tD)&!*UHp?S~=iMc1jvojYr<2I}N{%Jg9Q%D|{UVkY;3_GFu*_w(Eqj`7g8T_E(uQ0-Gzv!Z(F`QX02N3?Iq!&vl}& zA^7(L^wPV_3h%t_Be4CZ@Ccc8ymF3_><_%=f{p#$BHWE*2VFArDnz?0bCZk;CP?Iw zlD|BCIMRLVXK=G?^AHVOM>{sXY!Yj8iQxyX-4qJPa-r&B#$ScxtPsNfCdg@l|G<;b zGyXmuN>meLB1&AJ35-(lFcIwN$-~scwF{|J@pBF;^0icpIa5@!UQCCJkFQMes-t6D z9onXy?uVq1@`=*Hr?-Tg5KxO{M-W7|t3tvtAL*x(KCp%^7GTF3H$~YDnqtjNC-@T% zz@wJ(OXxCmuk({cNT&F?CYh<3vCcsjqNDV#jw7Pu#*bYkA(X-~m|#ar*q~n$)2QWv zT3Q)+`-+fsr@XEARILp1XOkreLbRXI!|tRf3PPR!G|13t19hZ!0Svi@D?{R@<4_Jf zq9{KFi!eQmKaF2E61wep<|f{EEQ6_fqZYvaRTA62zRt705C0aA>3;W9)**ZEjip(e z=)_JQD^Ihb?k>hSCVP&dQbx_y`G#w>@UYBE_iOT`VmN6foF|y^$WZ&x{*j4EES{y; z@mJct)|i-0bV2&zcr0C?DOO*Pq0Wp4GfsEp!!|Cz3@DQ2lA5sxp`WnRTXj;D;-a7A zf#_|=9gweM74(aIq?#0Q9o>5IQb;%qD||l}FV=CVbTjMrZIC?%TO3cZ$joqolzawo zP_UtmY=%xJCQgYDgk#pj2mEV zC_&Ak7NO)=m8n}!;VwCEU1@XK#fe{QTp-Kws*|wi@|`>p-ENF!rS%%2eB{dV=s%hF zdAyQDM=0U1+($T$^1n9bb;i+d0v=leZK&4bdlS0D-3gk7Q(6|kw*y;`bo!Qe5D|9-y{>k6NywZr0F>c$i3EA)|Z=E>CVWIo5!- zEVO1bF6L2VDJZEEfVc4FAR^JQ8joITjG|p6XM(+!7_}G6KplT+bT!Tue~unH@kzCh zorA-6w;_k9fEkmyXgm`*>vc?S)k(-o1r*1yS4`C_chF=tpBAhS-t;F%RJmor9Qh~p z7y;~Xn;Z6@%3e|pzz;d66+lBgFu|XU%MyKCtUcFC5+Sa_pEf0&E<}u-T0G9rtxJ8p zmX+RqWpotU4V&$vJWC~AccCu0Br+o`v+F9W$K;lxxs&wmt)oilra4sKat|kVk0{aZ z0JVTabK}ukJ@dQKh1`ZVq>DjjA*g-NWE1tYalz4!q#Q%Y;xpaQE!(Aa1o(6Gnh#*> zz%ys~N4VTZ&awJqgr$#Cmbd$SsaCCFJM?1@M5 z4M?&6&{LnIZuzV>k?mN9EHHu=pep@u~&AYX_Si4oWp%w(ukJD5*>7|qQ z=DhSR+M%e8yiml`p`wB6>0Rxa3Onmnwj%Ff$D+c&7#KmYoJYuFoYjv=SBV&>82uejI}R?U)T-O21!`0w8OHEWF3qmDCfg89#07ts~hIGMFt!~XW;o#+*$=MO5452%@akO2BGg|=4v{cj<)b_%Fw?y5dG?D^ISM@Q z5~p&PD@hxG``mqtgN2>1zKkWa*IgZiU8!Q_@%6T@Z|Yx}^xmjmjmTIwdgo}gF2h&j5d%fJfBz=pn=j!XfE zwiCeRofJil*~(vZR0ckl7alSpEp&uBH(tsQv`!{sM;sp>H%2>@=R0O~yj9M6nKYex zv7dWVl40!XCBN#1)1-x@i!dY^BntwzCpAT_xGGd6H$`q3AUDNdpl47-sm{0kA!=1w zfA6u^Us+#x(IABUNMxop9gKn{F-92GZCd;sO7ZfATIJ^%m0C!`z1CM7U*kz>3QnPm zT(R+?r6<-m^h$L=R`OZg%7cf)ApEBkv5W`Q3xh z9({%!i5~*{@82oimV!Yp)V?%s3v@hje>Cb3YT?Mb*zWHwCDX$~PK|j_U0zQMgIH9e zsN7oO7u1=ccNc5S+2=J>AtAst^JNy(8yidHOOK zBWYP|A(%|2tjf{bJI3>4Q+9^(6uz~Sv1jDTn{{%=b+*vZ7`U5p&BE?lD8VQG?~NuD zb)@$=8qWQ|5nxyHa z>+S4BVhlShA;Qan7Awqv=3+_bMeaknSi+tvYZ^MrqCib;$@3SCcfJBIKamhg(t>1r zE9~gRVi2HX&gsUZ<6^qhrD@6YW;>N&V!DjhDMdGlzGi0cAElNu_0 zs>ls08&(YgiI3o}zBS|U7BwI@k@PgLGKOb8m<_e3W?n~nrY)l+ZG?i)nz~2d; zklD(gXdN;7fm2yHFd4wXZBL%8RBzx_Z>D%OdfSIFv)!Dxt1xlO7FDg?!=Nm;REN7V z7KS$U4>>7eNv*d=d#3lP^?#fww3=VEy33A=+pKQb%wKGnB5~I-JEJuv7%H23h_<}k zbmPl~aXh1pXsZRa3?|KRp{=`>Cto%9zGjGTf4`xbD1YkK@J6ifK6uple$+T-On%-^ zegK@&OcogDDOi(3Xg%N&hYRm1Sdyu#k!!hp-0*F2IKPyMF)*yOSLuzJfT9hh`AetZMj{(1~2xYBW3fl!728#dDPi(k7mhq z3B1?$%7Yi;fQ4KL-m(IxxfjphXgKI50NPZ|FBekbTjoNopHzFh67|)O@cq2R16023 zqIa{1yTz>SoN)+6l$1(RypGM})(vx28Qjf=QwBzzL}&;e{{`-+Tex&DHeQOGutuFV zOAS>c*7EVAK#HPfQv2nK`*i+dJ3d~{N3X}YQmFu~`MkxEF_>*5Ea|7OcNNG&nMFc{a`2@(8>_qD;eCG-FpMpgE%JCFN|j)L1E z2a-CqL5m!k+o<#u8oy7`!zdUN)8}9T5FT7;ffgtc(6^wz7uv6rRCMUdnx!=-C4%dQ zD0$*s%!ATCeNrwf(y3U=okI;vd7~hT3YtyhSCr*txVTUhhvLVU-t{#tE)+OeDMTuq zhTdYZRCF*9`lB*CkijH~C5JkGt!h>Nuo|$tLp2e+;0!B?ltF%UoJlij!ezQtF#=6| z83lil?qq>!gJ$b&34!hvsrED9NN@DZ!2b*&e0$mb)B|au} z8DQ#vl+(;i=*(Ztw5E%ck}Az;>nv{dIdYq`8f331)`Gi-%s;S9U1c_gR4?Va{bJvQ zbkDEG7!sc?_gWmw4>EWra1F~6nT{TN-8<|UtoZSmeU#eI-`5cpiV{_cS83rxW{z)7 zlZ?Gd^;q3exd>Uj`7}jW0$sh%(uhn{Ogq_e<_3otkJ-VzEnMJINBTKQ&)$+^z!}j1 zRnLKM_b#ek(pu#k%lFvV{4|0dZIHI+UAwBKppKVUlev-tpl|(tRy+W$>EVyt}UB3oC zj$q#`%js)AD&xez=Lva4n8kBGz$oWDozY_PJEGV{a>3YOIWC4es^2M=vTu4l&9Roc zFMdX%B+^N22s3;;jISx|D)C%d?bINR_+yhlZ1WvnPhatVu*SxoY;bN`e=Fu`h z1L}4mzndkY{fRBksB!&}ri_7NAMxgez3MRIMn@P`Sduf$w~zz-&JsG#w2Pt9o-z9! zl?;oB=0(9P@JvE?0?W_M44%4Oz0%@i%iTaqP&guHI-!MR<#gXlUfqs%!vph#$w1e< zAMf@#co=zb89;2#B>2!G&9RX4(jYI7nfwVj`V{^;RmaOMsho5b;gOQP4K?543uf{f z=!kCJL=yM#fEj|$m;Q^e9vUMK7h;mpw%{yNxwm z5JwK#6Wh4s64uVF8X~xGo$ft5UoqY`w-bMTpgT}N*P+Ls`Y3;&d52rTm8Y&R7q-_a z!fems7(Tp}kTE{1V*{6M1dBkvi5f+)(`@M*;0{nNUR`v@@)7|EXqEKW55|0-lH_cR zO#fb>OcBB*ax?B5r?o3>#5;KI7et`I9_S<<{;G)FMBcofbwt2mQW)VvH0b(kgP=Z! z1!~jm`B}F14N%Iqiy&3C1=(N3Y1Ns(>hrHtFFwp7_D&_f2mi1c}kXt5@ zo9W6ME>IL32sg~mk(27VCkv2oNS(TkdBsnSKrOBBrcNk8TI4@{Q~9*KLIrlCQspAR z)l?AOBGVCz=~m)pF zKfR38>0JQ&nt$vATqv{b2wbSJd<$Hed+Z6skZYj_gryg(eL=vJ|H3ea8FHJOhnver zsER;Yr#BtH{xn+ymBPsQ(W*=%`eqZch4ONMriJo)0Fp-@(<)DM+>)T4HU9Haz{LSh zFcDb9yq(Xrw~2uCSlZl?pm7!cC=QOlj|^dezoz<;QBxH*WY0_E2AKduQ%zeC3uI=& z1mE_)4Ts9ZRjn6AgNuD$>Jbwc<>p6%HVlyQ(f z(kksdgb0j}B@!zIA>HYDFFhKJS5ZB3$d=Hg26=VDg2NVB2?LT(hF>2c*K|2|@Tjy0 z{Ng0eGEvFA-9GCSD z<(a1TLy+`h+4>gJWCklKgx-BqynYPts*@UF0A6PZ_rKEbQ3`OKWT(Lv(j_C!`&#Y! z<;fwPu+h1(V&)6wAd=zH6P6A+ttb%5)>eWuVMjk#UR%C(n*rGLI9qD;fW^XiGcjfz z**n5ycDA%#tJh!@9Bd4MFyB59WyRRk^b&iHQ&J1i5R=fy87{J8ON?574)5edE{hg= z$$F0bShm>@yQNWdxO23c=AX5`iAPe$I^6ogYNeaE(K+qP}{JGO1xwr%^oZ|>yfWhU=t?oD-%f!o;+uK9BK5dpGyml%Stm+jmET>D_7Dd9vQLdnglbs>IQ zm0AQ!&%669ugV08yWhqR53upVj*jE8Qyi$=-;JD!eM=~Nuc{JDFu|WG2H;4R+1HCA zUqO&5qIauzi1bG{N`~g~?F>s&HLl@XgsIGcTybZrrN#OGaY(epG=KhUvD4#g93bXb z0pFh96PiPtNN;isu?tyo6f+Qz?JD5!p5?N{0~5fzOj8jH!o zymub=^n3CCP@DefEhKMGj~%J z9N>Ur|A4Y_JZic)PR4p$dtIK42bL!sonpBBlvRj3_!Q2(M&a;Pl&%j6tRNIPXL{n6 zilvh#FU1KQ^5LeZw97f@bJYf!2}TMfF;Czhp~Oc6&%{!6N*GVr3NoucXh^MFOnM7~ zgX%G2O?0Qew~( zYaJ9CSC!Da$cA?rV?6=TK5^_z_Hq;Yy-E|IIKXne2qPbYs+qGG;ZwCKs<7sJG~5cR zK0>93DIyX~|EA43kvDMxsO5(YhzL~Qwi^FZ`i2)8IW5?d{fq0bnMy%j>rf>vbX>Wn z?KfH&i(#3cRb|$L+B7Bwq@<)|2Uf*UNz2sL5wn9K$fRXLuVHrlwEXnu>r>svL`JfZ zTQHXW`G>V~ObT03swPxbZCF;i7|xCfj3;5ch9*=W ze6p^=n;bb9gkfS!?zT;-H2v%#7}#oBQQEEEZ`CyQI7{tY_8+5c27h45{_^P1-NzL+ zv6Kp364Jy&J}A_TD|M;sR%X0QxEVGZKGkLy6;+n8#)6?=6OjJRJIJPFw5~OisW>u* zn!%|0aLxQ~JbgU92n_*7sjNL~;a5@JV6yh902S-GcB{YiL(X0i zASfLmQ+wiMxP7D)o--tQDOtELMmTz-b06vu+h;18ytt^?OFN3p5#H}8su+_wk0T?C zK-M5Vb}T9{>kNe1#aSdfJwyZN!;sqOKxh!{d-1bbDorj~FcXB`?>w%JJ>8eim}pEc z?#&!i<6GtEW#DC5k;9H>^*EpC8^1xdCl;L%Xtmv;0mEukVH>FeG5SU<#qbW%Q;d5~ z$yp|Q?X-}r*!Se&eNz3hC@&$+Ya5DWtqdt3F=5Atjlz9`CBztd&WQj)47L$6A$YpU z4@)b+A0U05iQZbnUxJTz;?yq{4uiwDC)^+?xs>+5zZy-@#G8kWP(q z(VK*_jXh>N4>0RAy#FYqK%8U}pJaEe`MF71fGzh3iE9K$lx$mc=INS++$lXTxq6ZM zLa!yP7bSOHd?C`!X#DKii@i>8?2EPM0Q3~tcY0~eDh*G{__{yWE^Hqe8|azc>&Q`A$(tr*{Vjb z%S?}=`Z?k(@}{FX{mmLVVeGA>5vP=R80jciY&)`)N)rfHI^(34LPXZEX#Vc1S64yP zH_jlzY zBV9x&KfE;RbcQK$2FZpdkX|@h>)-OP9jg5!wScac_nbK_Tm}Zxp4a9H!FOlYxvyVd5p6v+f;zXa{f>YFQlmh0{_Ky zz|JB#->x-_s^xeFWQpIKMOLq(-Y}34nA6D&8XtNf-Rh)RkpRPy;Nh47cK4NN=~b{t#3jwkM1l*#Rur`2D@FT9w!UqZt#gP{jSkc$AOkm z(lcBQfQV(eOW2-f0cT9GT;~uy^}GR|D}9eztj-}geqY7s=8MA{unXVb-hutPvHRur zx;UFuSFdEAdrJHL_mO_B^lgzTpUDskVr3wxoPy*zrnRR#7y9)HsHz9mTY z#KbhrW|P_lXhM@zLFT2IydH^wpYhO%OgQTzk=p@pI{>U;oT7ZKBA+qyQkzRM9(D%F zSwbJ#Dq7iw5XI||Rwc4T?t^P|;G#Za+*kV!Mus>n;I;eY75k+Gn<*Y=cBAEElUz2k zh!wV!6LNULm4#c-!=8yR$eI(Pz+RrQ@GFBAk}Bng2+splTk<1G^Tl^!w+ZEkdS^pO z5wAk0EBXZSHE0X|FiakS#e~NjM=CR8orWFTGItp=5)9RZSqS`Z&>irKzCs%tGK?RZ zM+0yNcymm$ASObA9LX}e_bS=Lmp{$2G=9!i{E@!cGJ+9m-Vu!&%wphh^=uXcomFAX zgzoWe746VQ9YFY7b02b%bJ>)Ll=SlD3}&o?>D%QM%ytaVp)GV5fE;uNDJjp{=> z%%dk(Ngsp$mM4_t6Gvi=#>Zyx%%BvLYG4!Z6jGf6{=3z|W)xWTx^lY%q@11tE42tB zSt=FaPub{?-7zTjFvA|Rm%o2W5ysV-w;8t^Pw~TW*vC=V){C0nuck7?i0Si z-WB=z>twstn7dItSRrM^VHR$#z|nJ#nO9a>8qXMpNo)-}Taw+~EgTv^xeQtokjJ+U zXf=UjB>i_S1sqqZlm|=LkM}iVT7;8*|5HHC{&1lkVYTI_j`o7KSlGkIa5@G z&V^rC!^MDK1{2GK>2lx8mVTSx)l)bA8M&je;2eAY1g{;ADQ*JnSdh~pR+k0Iu(mpU zqF6mo$$h}eYAn5r-=Y}j08(7-;zsbL;816=Z*#DxY&cJ}+Z1_R!1WlCNw7G$FTtN5 zT_HS~FF^SdWL5+wL5AT&$jO0%$N?j*YJ63dp<(k9P4VnMZFWVQg1& zyAmp255g!|ZZ3ml65h<{$a)sjEG=)qHtD;ITT%rf{v`Iy{+V+2u6eI>@1{4UrHgVB zD=uVKeUA!u(wbluME5*QnQX#lz-=L)$lXgwNTG$YZv&ge2D5J5MZ$~6;a@58aBD%MT*P@nF)1wxnF1G|ixmeL5)MQQ?kNOyK0Emi@~l}*c5EtfRv$)+s*6{YrA(z>X&4fSczu)o3< zccvf;q~;ise#15M+;TmC z;anB?`n^#L7R-41W{Rs5IOSbYl{dl&;XiCYz8;qnewW0Sn4C=88cy`1Ftgr)6IXE9 zn{}?-hYscahJ-o*8F`P+2pkWkH3E$wAw39bhsxT|70bkT8|qk>P^0yPv~1XI zLrYfm%3El3|ANu@IcA^Zp%Mz1?MV6Z=cF)Oj((Qaf7B)7DJiofbmf;iDbHBnp1G4w zTRNEx|LfT5jq2--8t`|DQYPfq*-xJUpCE-<|9QBUu~1W>rfO*3mp5P>ciBqjk%>&i z+;`YQ>8A4<-l{WeOWg2}kx00Dx@=!MXR^=#)cFEZi|*c3ERHSp)UVJy3uaPQ6WX!Q z9?hMfF;sFrRr;>YDqF^{L%~mWi>jCPP~O}TH{y+4*WDBM3Rm+}Y$T#4i{xUNosm++G$u^uEjc3=H@U@qM{XmPIlq zaD0l6$@44n1D~fqW=XD|rH{W>|2a(HEe}Y=-|xN;MgUt+x<*Ef&#sb)4FO?5 z&Iac$&)k;QiWjV-cV__@hX3(uqt3^-U?8ejb>jn(4(iv9BQCYPyd^ajGOj!!w|&~r zb`&f)*t#ldH19PF;k))jo~ z*NavRlb4#CxAGR0p*PL@jkO|WXdmug4}1O!O%8*=^32m8I-Lsml;1|)fD&O$&}rG+=}`Z#gaq4f%EPDiJD-$tpcmsXub3P#VCxdjJq(PD`L;fUQ7cI zMh{}%PrXqi&#Wjzo>Itbr=Efzl-c7ux30F_nC_62@i;j`0NCT~XRnS^vO01=AivttDQ&sUqGDm;CCG7CmO#zl53kF{rg!c9ti!8 zXRu!(#Fd7{4}2sE7B%^e#;0zBJKsbkpraKtxV5%FS%>FNb#x4S2^@tddTaQv z^YGr0Pn^Jxguk9eC*VuUpLSzk?aS6K)-+W=<|}q=aIw6%7G!UF%?p0%#?$^Me#lR8 zh4B}J^L9C@JaLN{-s8ir~akq+8rIDyX^lGJh<=9o3j66jhQjP zgD`}k&;u&iYWeu^Qu{mV`tSYyJ!QBx+w&L!IV+C7^b3e(^IYLiivT;)ZZwkj&ugWJ z1i6E#@dF;a&)CfTJ`ld<7V_0(r~KS<-KADf{2e0M{f>O^>>x;(d!$TS=8vof=&%~N zu49-bBca%<8hp4TSDU*3{PnewZ5mNkT~tW?XCVK!(E9?%|X5*+NO`8|v*`qsxT`_Xd*cs=9+;tjuRx2+*Te)aNQ%89_b8q*Ek>KTyls*=;pH z*-o*gOZsnP{7hx_56~N`n zZ(m&kU_HN_oImN2pZH^M%Fmj$KWM3+zucMs9{+?KFq8kXryoOYo!|IZEg`6S+%vd& z95e)wsu?JcG|)=)m%%N=EwLliz7&2(?}t~6RN z71Z5efAMeH<17-OKf+$%9X|o`h@Jp{Y8ZGhfLiDeAZi`ZZ=6dZAZ=fY$M5_G&Torv zw=8`=WZS?AJaX`<+lkzz4N#yz^xYT0Iq47Vk1Z#j0IEG=7=DHSR`2pI?Kwah5FOZ^ zP>Ryq?uDHq!XDWb{)yM2Usi7$DK-K~SI7hSn#GURUEXS|K7DVm>ce~M#be0xap@@= z$cLZ26fg)100004K!Zr85Q148KS4=Q%f2JjgE(6uoGGovo{nzc*~`^|7{pM)lNotY6aMomJWuO-pg3cO z(=_uqNO2G~5XrKqub_njitW4uGi&iYtjx2tyRxxHtZgAO63Z9yXaF^hsCDEX2bsQ; z00ZYWVlo@CNUCzgDTM-K45YNAyS`MaF{8I;;RGy}<_&u5`zH4f%n6w!`rIhShK{)} z?|rNsmkvfWu{pa|4y(8mEdvKCkYN7jmJ-m3Gb#jgUh>ajYr&A7h%wKV=x zb(mxmODHa=KPA%8;&rL-$)J3!xpo*~(&3O%_(J_MX)j>blWssJiiXzUH}-2$BDi&N zuhxW5=iYSgQ*b~%Eo*@8wj(wA#~3LYH_*9O-0xw9yDXK|g-BP^mCoGie)_ll#2d%iD2?zpUj!JNdd49=sk5 zb~`(DG7h-LRj6r;K_l0-&cFg{rG%{DCEd!Wmlt{@=3jvS6{nklB^|>5aGC@T0KoEp zfm2(%{|~7#DLQr=3@D+upVYEG%{n9}#3sm42^;<$D&GR6*G-k<{|e+(UlNS6O!wx& zT0m1Xw+p79^4#7>I1C)#%VdA$%I=9dx=HcsX-BE&7|eWx!tFztLHyyei5M2;s~ z>57w4V@WSXCS?0(JU?Hy64(@XROGM3C?3!Hv3 zaA3ZGGh~p?T{?ee5llDtohd_vgHlV;2z6+1LFFkh$6sNJkuW=eEC^aAdkR~(uU>m} z2iW`rDhSkYcyb4t`pG|_)})olA6AxFXdsQD@>wJQc)H}U5;1pL>ol#Oc8ytfUeXn{ zR3m0Ei@0u1b%Kq(O=9<(gx1zuBsehRQ)@-IxG9WfKAGV1EHJAc5Zqk{?>??V?IF?p zDTGB*LNDviH*?mzU~!v)mW3AlC&IR1$GSYmRj5Vv2A`h&SN^$A z@oO%O=VR7AW;emr!Ovy3-mztefo_`hqC#HIEfc!bGRo<^q+(7)vrWe?UP&1oJ!uss z4IKXy&O_Y8;O6jU1NS4oBSYTnvlq0+Uu%h))b1qi#F%?{Ghd2hF>-RafAlB7X8i2q zq~yf06@R)@|9dAo_6O*{qH;f|NbBX_&Pgl*0KoV^q0-3C*2&qx)|uATz}m&c>3?mW z*V=wg2W-ec{CR>drW;$_px}v<&8q^VjIsmW`phE|Ss63#Flha5vD0g0I6kGj-#QvU zkYo`AHS}rrPQr<#*B9$%YF2=J-vxieH>L#y(=>a-z|_pTYr@;GW`7+>9=yda=b%TQ z2f-G22Jfr;9B~IvJS0vK*kJ?L#d7s%(4L1Cwx4Z1WR6Swb9|;YYc3BvFw?cYZlA(5 zVaMGW8EhFVmh%Ii{CZycMTkm0t+%mDyG6s@)Sm{KFO~U~ z;a`m2?h=-nFLGY&{mVL+Kd8rt*Zb@499UEkni&>DOhr)#gxxB#dJjAZIoBRygRF!7 zg7aC*)7|Fg83qi@If85JKn8I#x8?$L1wtDut!%|&HH}r6 z_l`>`EQvZW_>BhlX%=@XMB-5alZ?YonYrTgiA=#&bfLw^M7xWzuFu0TyO85$e$)<) z@Aa&d>m2IAG#JsYbq@{8`e!-bF_y)|-+2E+`rnFMN@oDX+6=t~7QT^Yy8+BtvyXM) zXX3%#Fd87_3TGE5w@mi=jbg1d=^!%=a7{nk>3}T|8Cu|*k_E4ooFh*?1+S}dQSd5t z;$A@nd!peN4;~x;N|r9yZ9Kj=^EHj`^$%M5df(33r@b5!YkTo;3C+A?Aj6?R0D&{5oEu#CLc zfGxvt=?=CmA?I%$=RTGmx%=@(!Rmo9r_$fs|5VO~kxj^QT&(-D6fE||7mHQsgx3kQ zj076&97e#QSfAn559>9Ek;@Qp{bwv?DT}18AHZzFW!2Z7YUHJ4yH;@C1W)1b)2y)} zou{w{pj?v=6Q%QK7K;@cLX-XhO(k1TN1094dDIW;xD|J=Cj*D}-_&W+oI?|<_*!~q zNL(STnX)#53~WOb?xQzw{{a6n+8mX`IctNH9jOj!4!5fBtPPEPE<=JH180=DK?$o1 zBoJ)@xC+_S1LnnuN~#m%a>#mVvqpR!el(~tD%m&|5%nAf5=1VY(OAQY_W_I1hG(#L2=hm5xy~@iM3bP7 zHQx>cz*yP$L{Vqgj~;mU6(zarhd`-Clo2QYH8bWQE3}m^ii^RFIJ#Df-dDlxd!#I15X`;PA-~} ztOd>_UZRaHCj^U28S{=wNVt%cN3!^vl|~gSl1#5zmn7PXg|IZ#50;Lfh{@29XUyG| zUALF|vXE#pvCIWkW3|J_0$gZ=uHQin23o~)j|jL$EUcSI`hAW0jo~6*Fqyk+NZt*w zeSbh5t{&XY0$#L3d$v^|?M)J|e--O=#p4NRK)e3DFdQ)y`dD(aKN#(w^A2n$lL3p4 z(Z}PBF>tE?E1*C<5@Q)(%}B81?dR;Q%r_6lzCmg&wF{uGAp(VkS-?$~r(Z$h+ye_f zZKb&M;a$@XeNG1=^C8qWAC~AzTkhHVnAuta#ITa7-ctItH^6)GDT&Rm^IA*W<<4Lb z{n1q%86h@O=7UrN+D$%&&c(^T+WC#!>YsKTr*U&-=U-TtF-)xV+~a44<8HeL1OV@8 zv~@m>i|i(CkF1Ou$o|gY0#1%22YV~=HJf3@Pt8rsEZ;rP;W1N1IDaFp;F(CE1|p^F z*)jq!UE9lJMwUtB^jBiA!hU>Ac9O=KlO~MLP!2;CR8rMd)0A+cwiuz9IAXhJt`Z`p z>&ZF>Fe6D!+gz$;=v5*f`fOsI+%f(>@o^;UI1FBeF2Occgz}Ii0I!O0~C?SS)N7lDReZv zo(*h)(L$za4CpAxrz9#W%1{|HB4K~)?pigF0wVA_%AP(4WcRN{klEw@)JzC2)%tNB z%cE+Gb;K`4wjBM(G%0_2=tFvSWE4SV%3-p)-n3I7bHw;!HWE3yZ{#Z>GNR9l#)8E; zUrBjIddi>@l${lg1&#)ZTc5QQi|Apf28dH|($sCw_>%?WdWv+Yaw-7E|Kne_O0t9C zH0D1Qf2Au33?{;5Bc)4A+(~sWPIby(HIYkEVH{Obhb+!hQ#i-fx+`Q) znF^DLknJWV-oOLj^!5uY4s-rvmXMjIl*SRg2!E;kRG$qL0-&s9q&NGC#cd>rRJhWV zCD1Ah=PSOvb0Q>XKXgijCkvESc5)HYQMTz;QzFYBtPxiiGTFg-r6b@)dF4a+u0Rw@AUMhaOi+>TTnm&OQ6$- z_#HzS4qJY5-e*8fMyME2^+a5?>%E%#4rlX)?@Q0K6zBaCK5o>ei`gQyxrN9X<)`I- z2!xisqRkrT6Yt_p-o2=Y+FjJcP=4!8Bw-T8=q(LmvVmm;o-z-`nNdJlD;OUh>Pti~ z?0oY7S0ieX@s6_=7y#f1=KsKmGB&YyG%+%8HZi8Pws3O(Pa{fg+ipV~!SA#-y#t;U z{sm0XsT9TOpmH@^Q6OUTBDW9}e$TYNdK)xS_IuZE#mWd-Ao4>|rGix7&Gm#m@6&0) zmlfkR$d6p;4Z*uBXuNc1+Xf#et)5=ZX9TRj53i|f5)349lxjf%$*F(M{>Z;^iL2Yd zx^vrelbSvO8f~kwST_O;D1ns`h04XEEmLf={7&V8R713QNFfkEvH z+bCJv2x1Y}`xqOKjJeECp2`sx_7?v3q5k;=$Q+qwm7x!lrFQ>eHfv=wtI`UC4`i;( zT3bVmYXOKGSekT(X{uBFoUQRd(<>xXXiDvjn-G_qLCP?u{sk9!pDok&aU+<#@C}j# zYFELNT%H!O8Q3!|v0%h5b2`~kpVC2h`HsorWYP^UdQHEcXVn7f=1gqbIQfDqb$nei zC$?N1_AKH=9v$*nK;$l(m_AbIi!FC`S9pSyIe7_PZ<4jA9bgx{)dBF=q~t`Ry{L(Y z>e$l+%|m_`>ac4Pre_snvSQR4qbV}!n<<(t$B$wN9NWE!AQ~kPY9K%LDzQ1ZT)G9S zc-2%i!=CD-WZ3t9&a~R0aLBR%EyE>EF(~&?n-wgS5h8WAvbTtO&cjXH1a9fL3p@i$A&aK0cfe<*hCl|J(wq)mTv;Fds=Fvc z-ptV(vX3HwINGT|5ak07oe24_Cr|cM#i`wvr@ORZh`*7(Y+gzDV_VN=?DiI{dGR1dM(O!C_e<=>*) zr#}UEA)yXPhdVdn6Av2HHE&Ly9Hxm|d+}|dfBP*qxc+E|-YJ;Ywu9j?+qpSjUUAN3 ztJW597ORO4Vz5IW_V`F4YN1jqTBNqC2-0G>qS^>>zz(U>u({u<M>S+)XLNV1m5>5Rl){QE>+yn@0qKQ&dXj-=( zn2wS^H(~iiC_bj*j$$f?36;nIK^HsSjTyC%%JQlg#)z}kL7|KsLjDQo zv{WzzdZhJ6v`Q$$h+Qp%rG+is&l@z$oAv#t>B%TY)i!{c)f;H+?c(~}Uye$%|7Hj} zot4#V3z5_?b!Y7E4QNeAJx%c9=h8EvBtWOm27#fLnW&-J zsltY(TSE~O;CBQZ&4>iW<~s@z%piYzwsrXGW-&4pl7Y)e?ije0(`O2}KZpFqvZ@`cmXm9@#6xcHdE@?9tdk`b49Q}MhZsEpxus%( zVb#(|KI{|`IVl^Co7+Y4w{Ot;!vM_9PT$oHzI8EASB`(FamIuM;6NHfNL69DT0M!; zQt#N17yW}dX4L>RmJ&(&=7wOpSFC`5imtZ12LO~`IoPUOHSFLq1Ti11M+-3OrS zOR=Eq$+|g3V$TR>&#NdabtD>=k!*v*XqHzPwdpftYqqNqXbDEV_EV)<&p|uhnpg*2 zXrBlT*AKw)1Ug7s1!a@3)ex!C(~?ZBJoxoB3+Gyg?F%<4$~okB6fHku)#u*gqS}R;ZWsJViKTCP?Abbpw{XPr@CY22!k3%l?P%?@|r0 zG+R@rEPG0EfFN8zPPl%TvFP0d!U7a@DGCuhuf6S)bmduSmM>lzP-iox(GNeTdOp!<}gccal@h7mihOF=|3uE z=QYVGB-uHp(t(yrSnW_tG}i=&S>oMtMfE@7bKWMKZoXj>wP~Qjp?YX}>Gp1I5>*#h z%k_#ZSzDFbE-lY=jM&`wua~F1h#IC|!7Irs5-SaphJ5WDdC@2}PcG>#MRYgdVWf6& zpN71I=1Aj*G)IlQKedhki)oi7wJ?XRUro7p*~}6(=jz44eOVy zsK|R`5X>s@NO2LYCBY)2;`P`ZYU9MObJxwyTF=Wm7Wi>l;i712X{n#yRWrMgr@5lx zz#aG$ZWEO7n&<%aB(22_bq?Ut<&~6?gB@I~(u`tkv7;}H{SJxJc&;({n)YG2 zj+n+{Y?<4Auv@R

    =dtYXLd>*OvZ;JAfDZ7E}7`T`Ia_6HvfX*!4ItRaVnUENCw4 zi)Vp2K-;KKz$l-fonZHFt@*J5DAhi)WhKmcJsf)X7Z?3ny7I!=n6m*wP(7Sa{bIqWgSj%NSR*U z+JE2S(XcJW#Q znJ%~>+HXshRSdqF%s8p-JeR=@#QyiHWinmb$BUdLvdYlGLS~aiuR8Bv%o9ir?xx*c z$FxrAwbeuuH6Uz?Dah+KYiGeG3P-7GW*!B?`pin_Y4ka|Hy_k{C{%9-rBsiCN?I?=SXiXBdACUL zG&iPNGW0IgfDWjyk4uV~R+>z+)-Uyxr5(Xr{$h+6$J@<-!%D&!$q3f)Ul{!Ss=#t} zC0(M7;uMU5rbkl?O6avtmZCDrQ%KlHQOG{gx~A7;OdHPf_$1AmvfNsmkO!Gc0$C>2 ztBO=~^^6HejUniQ+Rgw}T)R zX0BcZ&(c54z6%?n(4oDTFT`;5Tg3ve;(N7#ShjAm_P7G2EuS?yC&_cvVk9z3(&p4S z%2=Z^8|&hxEcy2!tLf!|-2M|w{}j~N1En9<`W1fr?qbAL8Zxte zxJee*%WQVEg170+WogBbe88BfOT5;w;j)sW-r!s{=Q7*tcjO5wa>F`T5~Eg9%5Lz3 zKp$f^@S1dOXo+%Qxaz z?nl96e|N!S*;f4McR9L7dMjsf%s zsj{h>8A#$2k*zPix8mb1)|0e#_dzlVw>RBhMJ_MgN$~88{CKweY3FuscXv;V+n?C_ zv8Ug@zK0%3=S3TB_TXlBvJaqCOSx`dv;M=Hr)jxrsl{5kof+9)i*dSD=;tBm%i`=y zgxGm+xT67HH27~g{SM~+eFVpbe1;S-M51^reR&G`M_5AX)qBW#4-C)*$P6*i>G}YJ z>iKipRMI9 z5&jD6kA1LR`I~?4RT!2uNe^|ot=JJeNWk&=g?nBlKFheC!t{7sq|IwQut5F`LSc3N z{3gc2qN3g@LkMVgOe&7I!2ekfrf(1CQ9_Zz*3zlrv{jALL)Uk(sy``E5-FxQ;Q|BK zwKK^eR6ti07*YyR7hU`#0u^RM+kGFCa>lYfodR+Vvvx73iWq)Dn-oLjM*c04M)8n1}x zvJG1PhND*5l=aAVI9#f~YzMaNE|kI5D!4ATt9AS=xK@<~R|Nl3BZuGq5FH!euM)Kz z_TBcGEy9GatXZ_eoBnkl-zaF@f>o?Zp6>QYz}_@B%QC4Rf31iWfE}#FUhWwD6JKmO z6dHtHy91}oP#0q|AcOcK{+U)5Sn13FA%=0Ea zv7>@%)pE%a!UfU>L5PTDG6d{-weRG=v@cACV{!ySAfIt!?wPxBI-`c3*OfV_Fl^X= zY#HWfwHf+d5ZGHmas}m&Kz%s|Htc_*kx5o^T`P}io1h}QaXxa)pb_g=Ec&J1d@B4W zor3T0j?E85RfFJwv}v*6`A06`Vn55k;r^Or@Fhxd2{ZPBAz$tYnSu`!8)bztT{Jr8 z6hbOw>XSSL5(L-C#TJV2*m-6l^eyqAjW0`+%zab2^|5J+RFzP>%O)5%;B1U7g@fo1 ziSX5c4iO)=WS$TW<45Y{;in6kdZF@B;q5&mYK|JI-PGpSaf4g3;3)a;e+}QZ~y@4|EGPx&eg=x)xyM$*4feIKlKUM8ahrJ94J1swesEYGI-V@!KQ8v0kS|6 zS+Y>l2d-v$Oal<|#58dv`Oc-kKD9KPYyCy~YwPFI84`6S+g`KZT#3WOm9DMFzuTRn zx8HGYWv}NKH!UGJ=!d15wXF_Phxid8uj>|em}?T@V%QKceq}{ zT99_yuy!a!@thT?g&B#0whn`*BU463NrbfF0A@JP7YNP_?bydawY?tVe{F1-cydDT!vftmzIz z5!WS?j0hZp&pI5g9U?rU*9^vk24}8zdw<`mL>u8R(!;`>qof{wfhcf$Gx3ugjU^vq z!d@zPLALCa2+)Owem1{Rz%4k&==>lWn05$JCr+KlBd~yXU0;SEaEW-H&;S7`G-FB` z{jj#exsEIPY89E8oQqLGPwe&g86G+UvPemVmrJP!LieaUopBAL)hU z5aSi{2gH#@A|X`>WajabL-yw(B6fCGDTwHxLQL%8KO$tKP)%CDX=Ppt0HY|^AX9rH2YFKaYnevR z=)m4La3qj93I~8$J8X-Wfi;f|;zzlGXwKKckr@Vs3US1Rm3z{xxbNIJCSpex%;ydL z%aOR$Ag2nmA#OLsna@A(Xtq`x!1MsHTEoejrWUo?zmL@>7An1VXAf6^_9zNGq9{sL z@nSgg5zz4!p=aZ2W}#L!ss@)Gr^*lB7jgUo^9D;wuYNiIWI{LV4M-NJn0U zG{K}SXVTTT%!I`}1&|Y;NY{agl#xwp122KSDxFA#Wcm#-g&fOdI);&@-~tCXFeS(s z6JorpPfGkHvZ8@zy%=pbmp1;e#?+ya?(Q16QbksIdGkgxVxcjas@G?%e_%j0$E*cp zH;(J-g~+e4I0rp6ghn-NhEbtlfK=}eg8gUf+OHtsxUqE>eK2+jRmzbkR>onNN9cdiaP1^U)qnq`Np3BC_}3rUEr?md7%aCCpQ!H z<+&WvB0bw}!AN=3;FX&DWCNW~kCBmiECyM@mxHHqc?NoIKJiHXmJ%28yyy?-Nv7Uj zlbrjOD_)A`h~ZErO5>KUx8=DL46T4`6Rx@)ncv{F8XOg9PsLYYEw6LvKNz}3uk#qi zJH{J&SJq@y`({AEv&;kQt=l;IKnJ;tPY0RJVd;8)u;$4MCp|{F7Q6qbReZx`Yr$Ww zyj<+^me(8F3;alqR^s9=cNdV8Hfu2yP*l`*%q0?#UgGpqP1XA&c9!LuwAz$bjrAj4 zS}8C+@9wS5)Ynn3xx{U^bt{(}Wdjcw&*kUuo6&>*fr)ZKJ#$*JVzYvm`I6Cs@hd3u zLe+=Wg0oto`QLkaXN@cV$#(O=w^A-F9VtrE(!b~X=9n?0JfTUu-3PmnS>XTuJXo@m zY2NxLdE-L*Ki~_TOdKr?tSvnM@dc)KjyC^!aj0%%cff)0v!k~^Ezx=%HWVx`*%_>& zb!Mxk6S>h;`l*a*7}@HPWRt&T@%tHbO@g3H;wne3Q!HXBG4ePSZ8j1o*0k$lM<2uw z&ilmq<{1PAkIufDHrBxSGuS!QhV48pI9T3Ju}6X^-M~VKIMUq{QWcTBjydx9zLnu6 zJCWyLA4bj;i^U^^} z0s<`h8Foej18%_}<}r{q@hYc-rCiJ*F$jIXab$;c@P&-BRSs+HY!}(ZHh7t?6Yrro z6XyI3?{T#76@8sY%0pXB%v(OLl7zgV;e{4qThknc42evU9VK#$R_mn@A1;7Z^Q>5<4&OpMDQFmL`obu;)ZhCND&$}EP{aoNo0aTrI%WE#Ro z!hJac`b-#11Tx9-6G|*)9lzKk4EBf9&l%=bGY;yKCh|#<$4cSr?Y0y&4A1~ipBy+r{&qZ zj>3r2gbzDU)8acruBDLORP#gqzH{sI8u-xyLr3(g4s6^dF$FwGFBq`_;YgDGSrqer zmQB>K8wBo`wzvS_NH*&$DAfA7R&f*yMpU`w8olQ(<%T5Zs;g5EzBnRh3sp9pOzYcW zL4E#K$G>lLVx&2JE1rldap+N(clKg!Vv)LQZ}xEKvZp=p4S|0vN^qwO;O->R5E^Gjn?e@nb-?k<*JHX&tbeQeQP4wpPmxlT;i9QieRgZ(N}^1q~y=C|A{Tqg?Ck zCEP97H@dpXZeENOFDRR(+QuQIqH9gm&V}jX`c&_WbcIuy6(l5j$%PVp~0v0*09IQbs^WT zRJE77)T%yelC#(0T-YuwWhJfa4l6PD&7#!>Zr6sRj=c!}==)i>jQAxyrE0lZ#Vj5k zew>$kEm-ETbp89rm-3k0`(?eey8l`1i~AhfaOS7bZvR3!Agv>Nj$2GP7q_UNQ2O+69F>O{C1qTSl(Uzclbe^5PH0-j zQwY;pF->t;N8lu_KRa`A>=1lf*?t>o2iox#bZr!kZk->G+OwGj|M)tM>Zdtb6vvK1 zv60;d{xq0J$dAwm=oFe!Waa(BOw}f1ZtxC6J@c2hJ8)j-q2pZC^{@AnOV}9>YCai+ z8M-RNe=!y*MR~``bK1%3ObW&!6lG%bK0kHaJeDZ`4`=5XTnWEy{fTYcww+9D+s?$c zZQHhOPi)(sU}7hech5QJ-l}uIyj4$CYS*rR?awQ{R{y#i0w|gqm&c$PJJPVrXGpr5 zX-}sS<`c{7tE*r9ji8AUQH`Yr3OBMx*O!`3vIZ5S35~CE{3kI}8ttyX2Gf*z++$qD1gCpPvQjNbMSa~m{JC5 zQm9VgYH&fU35{1&r&b1LDmiFL7k<$wGXzf%`_euwibR<~vv$HW(d-?mRsqb>%H$u@ zj&cplE}uM`&>nL*1m%GwJ z>cQY^`(idTAr8Byn@;6Mss)zvPVouqy)^vThJ!M_3|CXPRO#1c*wLprvyfy7xAz*DRHFfaou}3qGLd^-gnnhjo zm~d!JlY9L$^5+iQokM=FS^v4kds z(mqHF`HyPcyb|}`E;e~?cA$X(^$k&5rSyQ{fc@Pxx+`O;fq@(7DnSDl5M z4(nqB#--ZUvqz6`L?vA9R6GJ3j6Vcp1>AN?o*X8~QtA)m_T*ARlNG5t;6#Y3fh>VL ze2Ox69+_$JFZkGyUG9S2q6<`ntm0Prhk_8xEO}@zcSwmlpaXU>YJ8qdJ)*{V-{{bg zzoSR&4nwW8PF2d0&5IsPkjxTUpti1fo17K?KJR%aD!_Yi6; z;q=irUO{%OQNgX1UTrVS#=WST{sUt2!?(xR8@`Q&G@#{cubSx_|GJon?D)owRS)xi zlA6(ep=l4B76)ka7?%~8u7DU?eUVu!b+BYBTQ@@HSFdNq!;|eOM_Zaeo=}d& zIGIOKpx!#~G`q?p=S7yhv`Lo@2x-NC{nXvYUvk}Y6$#S=_3(K*QWVeA+Jr@SDGxdy zzs50d(B`++Zt-F7nRV@F*W66lfwyXtQgj*I_H1c7OtDf|!-MQvy1ZPgvH~+(Tn56- z8~k1?6}5^Cp$K3X>xqh_OU;&Yxh&Zirtfc`sD~(ta zntSLH1KAM;B^htzCZ!Jh9wutk_G~Mvv=s;(?dEtqIbu7(pVzh~hPb2|RN86{*~&xD z*>>~VN)z4-acioU!*_bZ;vBwDnKIn)RPuLWx1!d4)<1nuWTeTcr{r2lFSZhAwh`)C z4u05yfc{zQRb%=!O#`O~ce(;Ah?c(dhzNDbVeuSmRw>!Z-rPmPp4w$RZIi`g-srE4 z3;uGM9ycEg&UsZI*1})^Dlr`&w+IVPIbGgUalsUOa#D{`+97orVrfV2HL%q+sB%4B z)q%(IAWz@Ap2k~8hMX{bf!*Ll&933{n8F*#Q=)qa41$3`xQSa;M2Otx)$P<|3s~(c zM|@L~0q&sFgArxc3n5N}j8~mzb)Wm)-k(c|Zp1>`8I^2wj@NOtYcmT2&wm@R5INCy z7yIEIO&<+Y<^;2P>)_0)jytVPhp($<@I^#;; z)VJZkQ2rZ5QG{}b{sb6HUi5F@*#3Xj(!UxGbY>QSl8NEp--)M-lD*Qs zsKxTW(935Su)%S;p50uPJ!;4F|hH*X)(n*;`xghb6ii9v|1+8wvEzxQ(VZ9tkarwARd;GBy4!`ao_(bYMU{tc)S zUoEzo5KB_*0`8d`{#P9KN?plDv{axGb5So!ASFahqQ-*8TP(2)jcS``<&u%&M>wZx zFUe7)B2$S9w?wTHVe&)R5D5`3>#_4CPj1*MA;ad_(5sw29gXW+I+x5Kns)?JoKnZQ zTt=08PN$<7)Bpu`5woItcrc7fvap2r`02w>pgW%ecD#+Y>H9Qj)MG`LWxVu7$&Ly(2_QYW}@ z9*wBUUToF<{Xc?qM3-j;$_ZFKfjYZH26Qo8un|sU^m?m|!C~jZk%;=KBsYj)Zbb2G z(rH735e)cdX^JBl6$$Yoo)4I$w&Bc?AT=PwqRbPyP%3Vq_i7{ZqyVF2^rUMoG?gk@ zO*-O)=Hf!=Lfl|}If7O_gu)BUw{F>q^CZcEG$&11`!w>ZLCV+X%Q9H7CRjGg;~ITC zFrB^)cPZIQKUw`pZDO-jh-A6&Dzu!i(hbbDhR@NDLwinxIjD?N@I%(p7dA%yq%_Zk z4zNb__gsR=a#g7!~U+gyG z9VXpBOR(Zd02?SZ=9%M3y@d)q&bqsg;w1(T9#0{8pf(Cxs3HsJ+#rqDnbCr-2HlN* zVxNwTYeX>4STKfWdj1eEjh&40M7`nA*{C#pD>r91*aj;)zx8&(OQpjmC?`janUubf zYkct0nB-Yww_>=c$9K+4PC$cn44uOwXd-`t`9q2F5EATwyclR(_L@;Wl{L5++PLWj zJK$Q?b=LgE(aBdcBa=_=od8LmZ#;#(!V=;$)lAX3_Nx!tzFdOHssT3OHj;&r5kvcF! z?7xYjnE9RG6}aC_TE%~CPdnA@EVXijdsF6WcZ?=?Qn1ZjyP|+AL@%LkJ*!30+nXE zT+Qb)_w<(dGt{*oeaQMFb~&ri<_UZ1%ZMs^_Vh{zL@zEyH+Re=nbF3qXw}Lar?P@dL6Gy4eTH-2qc_nS(zKgKEYn#9J zDtkFJB6-~UdRoP<=fTl(uFpYWIoVEqsg35`s;jV;<_K+DDQ2PWywK7{)a!TUtE!9s zJIg0RmOfn*f%`>?-~Vx+%gjx)D*!#E0q80FKhRUg{}R+)`F+!c8lQxAg^erj!3y5y z3y}aCi!!ee86(0f$-=e%x#nwMzftztokuzDU!K%TQHLLYX64zJIi_t%UW!(Q@&dcq z&-s=%+u;5RDwmbP7HsCZ=r9jm0U-rcoM<+Au;JD@Ku{qQbd9<-XuF3q(oT$Vw+v3$D($_#c}~l9FqSQ z60eDsbm1{K`FMCC^Y__{?vyTJ5m)dK#%pvhk6>@y)&kUmgobIE0*#Vb?8xCQ6>suaHc>?+C|OSP>;?%*5^u%z<^os zj_67|{&E5xKp^QA6KIRzQj33~{(MpsiN*Mxh$K5md7)bPWDf?VCU`20l%T(NoTfOO zF@X>*qV0ofW()3&8nGTB`sB9Vo;-2?xL+6P8lb2L*)a81`b&x~Bhj(n=VCiFpng@q zRKaRpfMF%l(davf4N_2orOlQqCo=G9!pYT?xxxVUS)(<>Q5k=}^#6Jrm{31Twk!pr zZR4_45@nsVptPDcH3%%VsC8|lK4SJ*;WpZp|27Cphz~QC(SkcfWqg{IKWE! zQth<{&6y1qokQUA5EoN~{9QK^#FbOTj;4T!o`u${vH*%lnd<{wvZ_A;{toR5R=10K zstEg77m`g)BoXf|Q_xgi+%rQo`7*oABeO0!kKZ7xiYneKj7ifZrJ^v!oTJG-L{q4CJW;cv) zako{|OZ4k8pTjhoRxq9=xc611xt-uf9II@WRAwQ#_UuWkXhw69*>!W;_s0V0W=#FE z1-@+%1a?8LJlqR0)0FrR1`C;~;%@y!qFMgmgcI0`ZNxLIg@&RkGEw!!Gi-&XqB`b7 zEoirTBmg-@z;>__P0zhqk7P7ev)UvYsd94-cd#0TXY##TCmLz!13Wc71DsM!87FkJfCYb(xu-@SZM%$HBgYjO;vZ&i(4XULT;)FDLRVOLOmgOSnbgoE z0Pf{My})P+xDL75_Gv&k+ulDzt12sH#&%TL>u-Tq$Te>4Q7YJg9ze=*eR`>`tgra3 zVrWdU_lIc5vgoO6;rqzx+Zgssiopk}d8oP`~TSVvUR(TDt z(JDh~5pF*BWtF;Iug#@1Kl^=^$^mR1@u)l&8!f3} zL#~^sx26pnMwsA1t>)TrbWo%=tC(-QZj_04s#e^vKe>m#8zsEWUmQaB0D2_U3yo)Ms4SO zOZs&Q;rb&Zy9p+*q#|FBl%!0t;%l#SuB3`TL&;t@TLII6Gy@!77S%{h(!?L2mRt#} zf%Dk>31krqOxd(l80OfOJ5JZu?zfky_J}j+bA`s`x`XRcX&AG5x%xt0h(cG@p0#k} z8q6DK?HrOK0;@iJ)k~);$P#1O;BvvGxR_-T#SyT$WanyDwpU6W!v{zBoTdRJN12LD zMJn8ct60!dZ<=q$M7Ze=+)%i)_Z)-_ehFDH*#Vhq=iVqMlrbwWWFx#YN3GmwwYe_X z)L3Z!^1J~ksj(4KJ!RG&4UwfBTprjk7{ipKiJ*xLXN$^^!1h&TH{$e_{8;;#8DW|_ zQ_DdP6GuZHDIk3+-U9OJb<>B+iRf{P-PYa0tKb{^8sAZ2n2j_@st37s?J;ki$ta)C zdx48`=^Rv-`+W@%^|nLiyxf6D-i0coMa5^-y+-qgAgdS4Nswm_Au1m@zPN6+h6! zw5dPAl)oN5*Qq--qVEa`r0;TLx{o!(#(=uMHo@1q5l{pdVozM;_?IDd{SAMHU^5k3%me z)dqe$r5(7cd~yVSiqB>iTQMKQB68R{#eD|*0+gTufs&bXLcjfSrVCrL8)iAemP|lh zGwQc(@J6{IPIgd+Og8!NQ zv1KQIbPvK+ri#~uZXi4Rys|qn(9Uz>7a=l&d%cta{@kc6YxAWuMWCg(iG>ZxlhYm# zn@Ve^6*8k#i|ZFUk74JT!7vNFc+-oCHySySeW?|A$EK2v`F=)%m!gaCyH8r|`O|v> zNJ)8ZZ-*;S?JYh(v^V#m?3@;Reuo&=HF=HkaPG$_Q7yjCU)1_O&f(Mve#J=FrFwTi z^Y3_l{dnGy1@RGA*FSu4b9tvTtt7!P@cQB#i4Dc~xtF&1Q{-55NAcg9TT8FnOA63j zVvK)QoihKYI@Pwr=0f#7RnL|yq4P-3_)dRSilw%nNR9@?E~9mY!W<2?2z!`dlV5)B zw|m$yO~z=`x*(OM8jsAId%e|r=w$8mr*hpU*zZxx9&M+&z)jswpQGQw=j&ea=W~{! zav$dtJPo2_Vx-0hEm_2M8#g>B3RM%!E?{$3wWMe7d z@gEPWkl)4!4n=a2%B+ZKnsc~m(CV^6Q->A%>s`nNp&;q=2B zb?s;ZV1hUS zBqVbK=j&*5v7bwv;OSM9E}sQ#amLo)MkG{mfFxh|hF1#JugFGC$$c+(e8=H~@b-z- ziDH4nNG17YhYh-H@kJX(I;e1HP(pK~NCQHBn~asqt|aOHYOc+)Jc!04add5p0$Vjh zU6EtJC`6_B@ZlZsFT*Ipey+{ER#h(#kf(YsQz*?sq95Tg_e9qYySH)Vf&7Y$rc}D) zR}W%M<~FnLTXwG6Nl0tIgKOlR{0O!dp{Bu?`x%fA!GB%SVrQ|KUbi_@Ha^E8nyD_I zhd^qh#|RBITp@*oqXe0w5zE0HG$P)v5b~oE?_dqN5FNM$y=a7&<6=v6VH#=nP=#t} z@eCtQTGe41MSM148fEm*g=$#71DlBh(1dBb@*>r|%91LngS!vMw(07Gj{|SRc(5p1f1K-bu7Wr~Lx&C%Gbh%vV{R=idM$sd4{B^e zj>!Y>o#&PX=e4_3sh3XOA1XELc6CB6OXbrJJ64<;lHYugVwohzVdV)`ZUd!O+%x17 z485_Is7`!3d?cr=;-DM{)$~t7Y~1pCqj6^rCml&J!=I%%s9HSwW_-})a%eJF9T14E z9?CQ#H5Ir(aMWaZUZ7vC?Ym0&L|Dff$hlX~R8Q0qh_;mUQj)^FsUQ5>Wvc~GTdi?r ziL>~8sXpwqYJ)hhjbK{e_anv^u%lzA^lHM0hWL3u9nT;ncqJ%b?Mv-DH6jo+KYNPoMfc3 zi_-qmTTgFLbWefXn44M*3eM+#ed`z?Bd$VvmE<<@8+*6im%L5)m)O@kO+Q^1|159? zK0mEfIrHOZ=duozYp$GLm73yhgq@GXb~He-lNcU22|!C4Xo}N;z{cIb#F)yGqc}C% zh*=w~pQet64aA~}C<@7@b%_-v{G zMtt`84%n+DdA4BZmD*W-2}ebNm`GHELm$fAA0Z?stj7^-VdGTQ+CmeG_OegLh`kgO zEDw_Dp6m$n!YU>#*XJ-Pi44Lk_U`hNwWcGH%5KcwyqM1bP!tgll!w~rG!o}`ere8=6a>|E z9I#)QfTh*)FSHU?C;HR*ukH4sLyI|eYBJf<)*)KV*zizUnT0sfV&am=2DGe6t+GhQ zzOPIL_fblH*@~4p8!v1Z4)YTwOhvJ0En5Cm=2<^8SdBa4uYn9@u|cy=QLT!!tH%1o zgW5w;4DxU!LCED)lvmV$L=RPUWO+Yw+zIxn;SkK z!7cUI!o@;Kxh(*K0;gSZ;5oroRm8nuI7+3c%$93Rxu&D@eKhS>;xVi=I5h7z+jzow zqfF-}pQhwtojPmU|U+O(Jt zTm}?<`?v+&3nhDE8r$YQma7U>ok)I;vmU6ANH%2yEvEF5i8 zA+gE+sT7l@by%)qtZvm!v+Q2N7AwuBXBCsA6eG|w5~dV04Oio+RhwAosaBgfxIwKh z>2Q@?Rowqaxi*p5ORqLj$V;&{(a0*Ts(8<;_gG z45AHQXq_}L#co213|tHwh>_q|BP)nz@RxPjKo3HQM{Qa z9H4u)=Y;lYsRJ7&F%18J-6Fx|WiCY&!}UxKWt6fnT93m-#QhnsMi^0X`0SHhNdHX9 z8BNWhsIPk=XlR|$Z>`Q9^r3c4OS{xPf@~b{?O;oWPJQb$Lc7ZUhgbW%+D}Id0`))u z`hOt^SQ_2zPcMr2DzZ05ffVgCB2HS~AJ3V~TBqb>(Ca^gN+AYG>MUsexC+DAC42vi zhKc2Lw@Dd!mGyz$kyA}Wo>bd2C z{i+SKnO3rFCIUo6HGh%!kdTYt6Q!@C`{rvs ztO0xv%naib4CRP&Rhg6k$kNtOho<=9w-#W^`3Ya1#T*3jU0m!ciZ2|L#X^tN9J2$0 zeg&u9=$F74a(uLI;AEyt+hPW6?s>?zCYhZf3UU*=YWi@X-Aq?UjT%#tME}OXD$Vej z1xmxhl&B`a9YUvtmJWIjb$U0tKR$>W#Tc8$^CH%_QU)GCPBFVc1&+MjKGFF^7u8WzL4dn zskd?J_|;Tu57vpxf2Uay5Ll`kKXBGlMXn5r&xGslBt)k5d^od@{GjA2BlwHM!vP-- zMt-qey9&R2exr?3xrjC|c9|Ei>UB7Fc?#zYDW`fq$X6xIN-Cpr&s)~ByEGh+jZf>1 zu#~rHqx|FlOy6sM4R+&(h)OEdD2VQo2*MVZ0<0g(XcLyWaAc*>|{-ty+cs(`iv8Ki~qfy~V+;{ff$tM0hRsw!+T1UVttf~J=z2!8=t@BR;=YnO_59uQDF)k9O zr13`PPWwZwB*s{|fBcKo)LEDC(fb%3L7|_L(x+tjw@Kcc1M5nHYNZv5nNif2&u@oI zhHK$rWt1}n!t8Z^;LC4fhUAK4q*#vONFs(%F39s?tK-E7^p|OZ%SjR*TYP^ji zy1=JFfG+yYXYiGO5aa1VZNc5P1`nh|RrJ^DDnhhi%{qey!Y5HH#|7Ab3WoUWl{$P(3aR(VpXkrNvbo1c&FqqF~jXSBpq_AN}vde zbFb>alWSzE^>zw&nArt|lOK0`ixaW4w>;hEDde`>-_Rb)-*k>3&$j=uo0-&_JGM>- z2XsB#Oo}7+QsY=Kpt;5IDhWxTqgz2KQuATfo1_g;-z)4H8BNpf-WKigSt!4$Ni0JO zMuv++mkeibzyOWP{L$$7Oy0)4Ld2+u)Hve_zIV=;7%t~=`pP*fcO!LP7UwaNq7*_}4k|t5D9Fb?z@APIXO~X{^r}%B zyxTM*%6L0}_>5IRkj1V5G z#+QhC@U0Z~ImLSKr%O|bCm3sS_HqRNs3vAi&K?_$e`zXc=uG@8Zm1f60OyJJr37f-dEN65| zWSJL)S<~ZI#(Lq`&oygGeTvi7@mGwjY;~xIz|bF64^f$UFf_}#NZeWZ&#L0u_zj;m zY@eEdSuBSCb~~d@Zv4prfyKxEXKsh>-vX;?1DIj^>vrZkSk&(Z)Ces$te7~U;@NXR zIqerazzqkCd( zZ5RFytJ=&0chrd%$Yo=+1()4+_?@Gs&@V|#j77#|h}pIQ?^vo-TF0bEpSE*1CH(>l z!N;4z;u@=&2zegD?0jCbAzvG{@gK4T`@hMOHrCQ?3GBjsXdhx2WO5P}Jq7xH)=GDV zp2dOJC=|fNf^%%9b8sb~5xRVFk5M;EuTVvhe{OLsgOiwG(!AqjcM$JeYxt>EIbBXl zNzQW4LzM0~M;SpDQ*uqI*rw!Cg<+{d8%cmzHsuzq!$e)0ECn&H3LTi}o9a#)98{Y9 zFOda5GR4c_+Wl2!RsxU15hmFl2*{ZfsoR#R++3=Z)lHHSy$hMBn@mJdA9EXzMJKXm z+CO-a>BAM_iEhdA7EdY0o)fgW1pQ13&y$ehb3OVGyi4(?@&Q3p7VJw&u!9)p%E>;J z+)A8bb9rA_!Q-PpNx+~)JW&v)AgbDUb`g<$YM^6XC>rL2bwpM9+f&fnfGNZG94l1k z&T=0QKbQ5U9&WSzZ!wriz!H)R?3&*&*tM86VW9}Vcjc%{z!(;O=SH3#5>D>ao^v70 z$3vwnloTUZ+DFURg}K)H7g_SsVh12gSa`$`+hc-ta4hN`FR+@iFs9<2UH(9onLK=} z*o%C%7JHdcUf)~l>SV)#1#ggPEwHxM6M*dWpjx8gDiO2z7UvPOsq`gD~m!+8FXcO|~{P zmu|i`H4zDPHajqeSU%i(6GJ}4lkDN1K`H&>W7P_8Z2mV)5{70ZQm8xyN;=OKJ*RZD zgNvU$wCi$nx%Tr4ihw;xhO}0tM)9**cns{E9C9NQ!3C#@OiOcoUhFPwvDAk7yXGCL zk4xegTA({En5uE=Rax~qj0Y&HMTHI?P!FWiyMTeNy}Qbu>Tg~O*-Bz+24h%4``%-g zTNp1zGIJzghoq%e9X%W77qm*VTDbn zN#TChEGqq~BEi^3!LNXWi;VUxX7v!S)T{f{LUpEOoZ8iGt^ixlrEV_XWBj1maq%Ed zQXFhn?pblETD9cR0%?hZ7;(Jzpm;}aB_nC(Tv*L6AjY6OW8!6TexwT9?u}We)l_IH zu22@_!ZC+8>ylH^aa#LX&r9N_BA0YCmTwnxfdjNUHkyC*`MK>QlD28`P~<*%Hcczr zPFuCSq6|S*$H3)WZ826Lv-|wg%jZvA2X~Tphv6!?``{Y%D*w`1)ggYIJ({ymK?$O- z>K9*PV{w?&h~Uds{gChjb~%N>(>AHVP5KeJQ)H8UJ~2s!5I7}hnkqX?HH|+Hg>|xg zJNvzt=J;9ak(a(|ud=Q$q2bFcgI{F|f3!-xT9{K?Bab~%9%r1unxw-ocVml2*CjCD zDNR7>DyQr(>TH$QTHAC=A!A&+K<-Fe| zsmF#}1y9yv=)^7Jk|Nvlm;8V0d|L@SWUoEL)s?E)v#=qO) ze6cVpkgdzs`DMc~Z?`=Ov((Gg!)ZrYhYi|Cz#h0)?PW%f`|9;xKIuSQUQ9dSfXJ$i ze+RBqY8Pa{obi$*oOk=4?}Z0@#K;eAGy-}{g5~j;b4XCBx*#tLuV+WpegQ9f4hw320H^u+ zezR!R4q-RntSk!t=?Ni*NYYg1Jm=g4IYE9jfNPrzA2~uXFs%4Q)L=rGNSajAWLmj! z`7`Kz0EewQog5)Nsr>|sD<%RHk(iIPj0#ffP7=HT5g*%rq0Xb1U@F7gM=-0(15Z@# z9QeX4PE`4YLP$xSe&|&P>eIUnEW;T zqdm%!O=Wvm^lP#3oMIT4_a10?UIDLN5KE?8pY1S@M=&15h06Ed2T#Cm4Sg6{uu$WIrP7t;}KIq_8}^K#r4Rlinvu8h&5Xe*;w{aqP> zMfu$@I^>&z#>Sv=|DY(%z#;yL+jlpQkZ1*ew4{h3n%nP-mZ2`_koiG{Og*ax#Atdz z8!!w?jpRhh8ineyH$ zvAaDt6&?+M5f>ZbOpSd#Nya{k73z+HHA-|jr>v%=L`jcHRa(?y2Eb^X*o#g`S^nf2 z?GQS2`^gfrw>GQ&B8ca#5jxNu?i;WWGi8ii)=y%{$d6K#hwwoggJE|jQhCa`qvn+Og$C5=`5siWCwG1#gG1M;ah8FF66 z#(E*Tr^`Zqe&g(kuL>>JUFC^-rB410A#2IX_ zSRWatvD9D?nK?=OJcivucUI$)W3px(`iRG{$)&Xuq4q4;HPEtOK*lieAvJMiSZ4iQ zd;#TG9Skl+>S1&I;hZBYG*_%n9OaRH%Np%b{kpE7`q=)&y${joZ0ccm{2|8VZ9%-T z-)y4}Y7?glLzE_Nb&U~vm{nK$Fwpxo;M=7OQ&do^^3*~76?zk=y4DvNv%_xr4k3nxz0ZL?xp0Xu_j$!hs39eE(MVme=IH0Z3#Qgp(mjY2|pi zVK*{q1UBr+CTV3k`f=5A*=|BM;MS!cpVvCF>^60fi;8HP2)sscc}r|Ekssb;jZj6I7m`}TbWmWoa+2Lnp2B&QJ;16F|$ zQq`b2_bINE2U$W7XI9ZYZ#jD$#?O!!h4dSDg0SLmMQr^URp~&@;LKH3bv*u?xdm9; zh%1nMfkI;b2)Abtq-{ZXk3~BUa2oJWCtVe-Yu-&JCe;H9IZ-ZWgy04FRjRj zlMXMN~HiQ3L$rkj(2p=Ole4WJsF4whW_{A=ez-?dN-g zsOXnXd7REknPBn);d~zFPag}j322W9(-an+(ag1ZQT-^D-U%1|a!g+6UDwJeYKx}l z%qmKzR#)%+J433JcCfkCpyr++>8wso)J_4{YylsQRR%In`%Ec^XewGi+o7|pxf*@ebzV@*oxE2@poMTD>XrERy zzk1MPeEl@x_2PUI`f*I@GXL@Zh|>hg0pzi%{|shbzMnm6KnWKh=4SS+nW$lKuQD>^Px8}CKjgHFIZp~jX<9G z5DwMtq~9KgwLXB!m_Gz{xjb~MJXNB7wX@pS+tTf?<59O3iftZp?j*lXTk3|4=&f{> zTL*eVlXRUR!aJMwRH?koLBT7sxyLwh7A592D|ZCWeBo-s!x76w7_7u1_$0?fu6&4a z+)BWj^`>W4xy)wofWpSnoMVN?InRGypBDC*i8Sun@kQBT$C#CYh)oSdjqQB*gZ=zG z{JJLiO1nJ#I_6`%@9{5F-~PR9>WeeAfS-<^lnAG83(14Q!dAy*V>n_PoG=P^efmMf;OB(1=0GlIL@n^)PU^3N* z+YrE4T@)xh*+Qlk2 zEQw9GfKRdy5im*?t=Lj4E3FdG2_-H9br&_Jyf+a+`Kw1 zeS}F8JxHDLgK;Xc#2`bR0y^5Wbx2jk$7_JFy&Wds4NMS#v_&zbVE~Y}SOC(7R3=0S z7^o3LX!|bsBw>0a)arF2aPQ%6lEPB2Sw8h*l4*?hCw)_ zL_t3NYkTvYE=ZuAiHTOlI;}prgV=oyLyN06I1(4;NmBncP%*8RTlp7hE9zhY9d=wV zZEW!9#<0FuC|5U&#wr2Y`E9wC3ay#hkWk5%c8D4+W5w4=(!sVXoymrEn!*#>z*OQQ z@s)hLGz_r40{e%lQp~64+NYChe2AG?C2CI;aRZLxh1+^a(z}afwqLQMw3WN*2;3g}lQd+;ja8U{R7?I~UI~H1eAM+)nUfs8jy~dPH(+#nG#0p0wr)Oa6V@ z*NPdqE9WoO_B#lIyFfEP^F^3MhcKt7xPeLe$u}a5`11u+k?n7U7$L|CgT%H;0*zD# zK2I;QX4yk6lKx?*gF$Q!w2MmY%N9Nx5_P$@os?+BTE1F0eIv063$K9E`2w`P6PbFFyW~4=1#qrWvxseuD_!dJb$S+sl!L^ z04pb6ho{v6E1j-XXEp<@-`GwJI};x0#vB)i@ za!H89#u&*^*I4YUIKXtFchKRIJu$4)M7t&-U1)XM>P4W>%m)d8h1w>-6Ha>@c3S~9 z?7CW`en;cT%=p@=t>Sv*;)l0iB1#|0MS0{g5_-;cQBxPh_k-vepFJxlQy%Mpknrla ztCd|Hn@q8c83aP^+p^PaF8C)yY@XKLBj}qa@aMQ(eipE&V3@+;Llk!yx@#qfxS|&_ z-x8jz7wL(>(M*EtLON%B=^#z7#SX$d}BHwyod{Q0+gx+sn02Niv_ZUAuRGu$}js+;&3- z%GZ0W0!&fvt_npZHmxoo*iO5F^@N0x#t5OQD2f-G`n4pm^idkut+S^?P{^DP_~Ju! z?y&b2Nz)IdcP*38?W}P;ty%J&z*}QLph2=jomPzsglI7z;cif1G=<3S4I4wQon=4G zp^w?5K|4bu?{~NeR_SNI*>w*?Z&*Vw({O&->vnbbwg=A41)X~w3-Kfd991(lUJK=Y zmZSJ+S2AQ2BQB6R(GlosGR6pv3;oiHv{bKjeUu6r4!S`?9?_K^qx)erkly^=)|gc4 zNjR|raP<(G-HC3Lh$lXYUt+ssMFEb+X-LKi`BnP^d66{yxR;Afmk;0snWg;m48TqC zWhs9=y&CZSvy zLw{%zlgeVw|GuS;zLOTv;OI3l7sj(E>_-62vHwd#;M5?elg^9P4^^OmiSc+jtEzIm z_s?~C+~oVg==pu?$*P;q3cIYR#lrTdDg(g3h3(?2=~2~MM7r|k%p_bPiwOo^yE{a=&X;-I8XIkSe!>JkPi!T~LE-2Ybzpib^f^Kf3ZlpjNkCca{g`q!d z@<*YttM-^VWdkOi?&m>HcDw;B0MiT&0<4v>)b!^^!aqX#l}Ote5~uM7Q{7thSddyPZhtl=kMJ+d;rjHfMU zSgNn%k#gFVCUWRgOiJy7GKM4F;JOQ^a@zD&jy`?(i;RT|BaIy+-~5P+11(Cs@S+@b zKYXO})Gu^AO%&9bQI$jA1A@aqz9dXBqDc?&4RdsxQY`t$6OByReFZ3Q*h1Cta#L)@@OIX4IiiCJLR%OlO#wfE zSi)Oup%FKte<=adK^%!T%tX`2Ud+Uj7IoP1J8sHQlfN8VqUlos;LTM7cHHp-O}OzW zHj+rwwGX|nm9}dap0Lw!r0BLn)WY!kqwj zf)w(U1nY)1xUDKaf+|K+(;}%THYYYTTQlY(18aV}a*MS;v|uR48`@n*M}!}AgpHD6 zrf%y50(e@mQOMB7F)DKG_3sNam!DAo)1x%ms_zmo*Rz$+|@pTt&3Rp zPDxVJW?ql;)wo=*8;-Lt`=`~Z)x)og+b=)Yk6pGo7*7yJ!M^e|f1wiYTm1MQ@N6Y< zlDMG{gXBN0E0mug|DBuJig|Vx1C;Lv;Hu$2p~1NRPmNgqdy3to9t~2mi^_}wXrWZt zA5M<@eF&1aKc=29(Vv;;^V-WqQV00l%vGPA#}K^!_kYF!WCzppa=a8ce_>b z`cI|L;`iLR+to$zk?HaZ0+pa7=15LSQQ{<8GT1}4QQ;kxi4WkzV@E7n`JiZdd*`q+ z$C1i((~NvUTH`vs%AQ-FrGiQP427fj`h3Rl1ZQMVR@HDSvIZdELMp&ECrmi?GaSu$ zPMm@wuw!7EBF7tkPArw#SfnkHMJ~$)q{{4~Bm|7*rl>`yrE~FeM^~UEN7Ob*1kAy4PCQ^#c?o6f@reN)4)?lAQO^Wo9J?s9^Yr zG@>G?ie^Qel7eyzw}2|A?L4ZI-Dwz*j<4W@L4CSM_X~B(p>nDHwEDd%nAfB!YbUBL zUJDf!z}ZMg0u#6KN?7SU~W!uThSi)&};Be%) z^8OUB;TpzGva8&$p9_mmnC8itKLe@jn<|ILeYQSbW-p=o2sDHWPqOgW0?h0sf792*-WVOl;Bq@KWs<2jWTCeBei%fku?;O+tPy zq^4c_OsF}Ldj@&Yl{&#SxLU1FR}mX+9=os!>nXU}@b?Y4+O$q*Ao=hL{E_Wm$nxH1 zI}v+3!o`MJ{i`cvK91QHBPXkBNz(lvjrs;ffvQq*N=%7GYhsFHFZmuDd5X;s+_P(0 zvw|oz_zvHrO;#ZkpA&gi>XQ)(UuY4pVDw;$lM-0zDC0sGFaeD={0D6iQ8aNlLO5nV zgEzMb+bhnPPd(Hi@yCaUy6^e5?w{eM4oZ0%;$EhO7(_T1D_FGZj1Laet?Ch$j(%>g3|}+qo)lNjtBNz9?Q%?IrTMme>~(BG^TW4YaeH{f z1-YM}R={&sU;1+%KYe&Q*pXZ5w}O17C<`+kS5A2HYd-wJlH~L;`>0Wo)XaU2*?k{` z<4#Q4Pb_h8?H#tvV7|@j*-3vme9XYWHB7%MM|@Uo(IQPkH->Oj^IH{lW8c#ko< z3o;oIDOEoLXc^QO?rkz+7I>`q1eL=@_tpNOL^sL&gS|~%xf0l)TA!4=Uh(=Z%ExVF zp`5)A0XAq+?#nw2)C=fQ<7TrC$4aFZi^&%w;b6g%U!JkFGZ-sYMJ8P2KEb~)mZeS;a?@Zk&Rtj=1?#oVavq^T_8??ub$On9$X`0KH8Q{)t9$4IUdBJZ z)Ew{7j`I$Z2}efM;;&rJQz3Mdax0?Y zza}lopk2nt17zz>5U34Cwwzkk8ymi6a0f5CqLinf->a#P&$JYtsPIx%PxII;Sp3mc zUw_c@ZhP|5Og(_sDj+nE61$aOYG!znKoKI4Osxah9l@XE!!1|puh+1Jn*K&U)9AIk zeDi?gRR%vlnC~S(-aRMlj ztPv6)JaXJN6Zpf)EzGKz- zF}%no8x|)x3uWdq^h<#vku!>^01lTFG(Chi9~)I^V?byj!+{-mgtDDwF97@!!52-D zL!u-|;ixK&xqfCu0(T&+OZ0umGr$?Q-ck9F@~Qqbe+}~-G^Ej=sE8S5o8ar!%kHn3 zRUjh@IXW1& znhlJS<{@PoA8(*Kj3gHFXo>i}%o0OJ3zw!|4fn)CJwEDXM~x6LPH8MGTYKF_7F1`l z0_U1B?LQMH6V-^ORp3VjH=!rK!Xo~4bmB;gH{$Z);T&6a?bwK_mnGXMz-Z7z-Lfr7HUGRB47~uHdt%lNx4IQ(cw1K*} z^J+JZ*DA4hz4ondA6PEaDSwJ1As#0#Tw%EAhhiz{qSl0Tx=THW(NG#lomgmIrjs&L8xtz03th8DybO#Sr-dRsX}bQV9)E<|3d{Ti6JZj8RGhXLG1YE_w&#$&gONpOJU zU^@8MHqw3NoMJB}QA<8Fa2rWh^@z`OEE2%+F7+%z3bbkCD(}d22M5;EZx0&;{}aKN5X_+2w_f5 zdS}8YXoOldhQGo7ZOf{^zX`kd4%+^BNufMKP`up9^K{JUUf==+w-^sYLeh=5S5dQs#}{n1#RG6j-5FqIAs>uOPKt-30QIO z0xNFFihn9@P|Sb&vX-OAnhWV;J>VpB^k(j)&2P5IoB2^{c2s*qfJGu8YOIXG4n&Q? z@tak5{-VZyE7CyJxUP`k+YJkj7Ew!*z%8Ovo2VXj9YaGckA6W44h0H z3KY}6fwMrS`Psic+4+vJV!`g2S+&gLfg* zm2nBrkY_Gf+kpg$Rmj}F;E+q@;Ui3f`2EdC2TOD=f|DsM8BJ--v*)koCI$_-mz1JD z0Hjdri21RAa)$V&cMY0+Z|z{G|ZA1jq+_9@5yeD!hcyhe`O~ z-*JGybhtc^<84ED*hrRmbcD=@F?-_Vo#!-P$ZxIiOZz=pUXI8-1ne_NO6lH#9CcsM zuUxv)QC=Y3@lYnG#@v@6(eA%TKv^Uh0ei8iB6?ZUwxF#?A){Kz`6J7M(pitn?_w!S zb-AK>dFc;-LXhjGN*H6|;QK#!_Rv_j(X=j~M@Y(+g|^%~y$N}h4a~Z4FMnBOczYii zc!buAb|Ed{ICe7ctX^(q)Jp?-m-mbf^sc3D3E7Z0XLKea;)8XP>%zCDUm5m*aK#Eh-_x||Q}BLDx0l`o%S0XzZA@CEY! zsto^M7ENId$f5y*=v<4e<9^Bgn8hdK5=v-NMRXca+@NTvuF`eJ<>R<%yVKbZVxuClevTZo*w%mDdev>rvJB^S zQ63IgT=)X!FN?Nz9`kibGVE!($-F*|ZLOY|y&+4+n{=qR3r`fOB$^dMK|v{&kwh$fQ_uxY-Um)q-pUvUO2v&=Mi-S8j0`JP{Ag$ry-%m zunq-^(~9$OR91^_6C0phRXqsy0W-OF4onD?1WMGfh_{&F;P13HDbAqcQSc`rrYkUU zN$Zm13(?uQ$tO1FW&$?{PoeGOv10mJE*jC;a8lynSGx$g*rN3D^E$K1`!e?Q5DOUm z97<8luu^)zpnYq>@3~VO>7F={l5$cR@5Y7EDZYE=Q5118ZIbpdpsmS6)hBs&d@DGc zy)B7G;B3rJ|25Tn%XJL-cTmNf&UjRBF+^1#V*@SD$K{B%;Pn!a?d)^hGpOT0-!{$F z#n06xds&{E2jT$Q$CZFQh{r5OKWp*RWR>C!VtB3tgvb?bXp|fZJOHSmtB*k0}vjr2wc$`Pq=kJ(s@5my< zo8KO?%j*dXhOBK?tZ7lv#^E+9tC~KYJ1*Bnsx^iHt0H)Pbhd*cUTl_=QEe7b7oo9+ zx|u+e$t{$=bXF*<6x8npGS*oiNN~m|C<-)wu?Ei6=Te0J6+XYgvlL#WrkCP7Oh~A} zoHG7a>iaIOeITvP(Q1laYb;$DW|EIyWT^0;G(S$96jr_nEyo!fC{ev8 z8^v7nP&DiRLg|gLSrP4hJb{1?G)cY5FrJ`&24*!FvBr_Jat#Z8k#jX4EdI74ehzbQ zO=n)0vPg6)a-_$TdvjyrccLa^)yf79(0*q^9>BxCygB2SZ_p+ZYj5t$&J$vKa&LU1O({d?Y7{S5@5oO3nlXPgUeZQD9qZOm)S9vi5^1Ih;b0&!+3hyYml zfY&w?bp1W}(w##X#J3yG?J%ha5X16GihT~`VY1DsNiA8_I6`3htmth*+ zg=$`G$q5u1Fq9Q$zUwEllOEocI;zMbXdyukSssM1ulV+75nfk3af5jqQTb$BElFF% z8mHL|)p&Gob1{pt6&fs{3(+1%@`!5Xi~F$$R}UFtV}MMW4bup5HRy`EyeG+GeT$T4 zHpXzdA7xYWvjPO#PboqZ#5lC|g*FhW_<_<~X_Cby(tfA)Tcx?y7dhY!HQTHbF$Z4$ zda3r%Xzy8@xQQtC30!L>%aYF$uzjSgLG z%pP#k8nZ>Vv*4V~tBIi1wNI@$UD)a$*#i?wReS^6_We@zw|_Pp_2+*arLrCVBVa(TdxYx0I!gadl4*b11D8EMv%33aig1$X zrMiFFHGie(axK_d6Kb>M&We+g{f&)0$27;RiP!4ZkJ9VZ(Xd_eBzXAh>J%OX`Iu#4(D^0W>!!@WZUFp{3}UFKVDTP4gZ|$Jkc0_7qR40o`_*w=zB$ixKBVT z;u+;1ogI`2F2`w$pZwe4ZXh`O7{Sbj(J|2}Lo>gwb`XX7)5n#vmXG#e@jAWqJg#5v zVA&whxw*LclcHM2_>`+p<+g}&B8zPfB2@aAZmt0UR!B|D5qKYYAN|<@)%Pb+0WK6{ z&uaf@%4=!tU7FZm&_uc-K5RkZnLY=_6R9EOfGC-Cqg195C`e&9GF!qna4wZHu5X!W zK1>;HEZL+K^GTiBT9_oEe|BS(HeqaNW$mH#LueX~V{WJPiu+lGjijVACx73Hs8SQX zKOw%O$fM<873ZdXRnJry3__M(5%op^>E4rNVNIo4=}#o6B*1f5>iRMRp=_8!G;Hdz z*NfdtT^YzFMuJU?l_x%fYM7gtO0V9C>O0+2en&UYC!r2(=qEicNDE*mSwn!uof{4H zQ1(^^VHieQH2QR48gU1IjMT(snizjgpmFS}-bl;-QH=re5-{D@`7Jtp;D$Mup2FT4 zGe&Qe^Qdu*S|igtK8i7KpSjCyR%;m>^tS(9=tdBCjt0b2tnG{ooZ!3nPnn!$=;_II zTA}z;0PC5j5rkvcon%E=_OM7K_-h#|M8v_X`fF3%Mo9YpZ327Wp0%KHx~B(t6p*f%IA!ei4G`F2avBTw z`;&Ozk=0R(%RY*+)@sHVWGM)?anEsW(S=aguNYFKkxn8vr6anZM8M2xhsT@r9hQxW2d zrh1}yEuNJ)SEtbs(U5l%9AvGg9Jp{I=1LB>Tj9>QPBXfji^P%GkA_DXaZfH; zs0~JCN^x*GgyC_L3TQjt92$gbd)8-_V*uI{ds?L`z!GV*NKI4dqsJZWOAjDlqh-7>eP%04`jzt~{#ZPDhd)wW%1*GC1u?MvjspNcOjRY-J!GK4*Z%VnC zJL{IPoUZQ`%arBsJ<-aj2BuWjGq5<1=2_0;Pj1>$4}1=51GPaF45)vk_{W*dCST_8Qm9t&xxCM$KMCch-yFvpCuaH1xU| zfTBS8hRg{&3f&paLqKm1bFue^Uthb}X>HpbKbe5`r6E$YOEizaaMEkEv@eG@nKwc1 z?3(V0Nz@^J&A3@MPG|N?KlOq|J1byT7+tXgeWqn!2aL(Zc{@QQf}xM;yH@owAqjxW z8Nw7+mDYbs?-e@&O&(LCQ*C>v+EhBE7T)dkHq_GWxBY@oV-KXha|Wd9yrTEUHVp=~ zt=z>9CQcWbUpj_8`{fyN*}Pv=y(S8!aZ>4A+S`o2YG;g7{`+-Gb?d3K@bEG@^m#w1 zZAkWZD&hkU9-~e}kZ)M9idmF@y|-Uy``KfZx6GdzqzRHt7m1v(EJsP1s1p3S)n@J- zvb4>(#%__amCnj0?3v3!$6X;nlG&5W(_BSPi3TuEbz$lPXZi_Uve8;= z;K!|Dx)+)#{Kf-2aJg$+I4twog<|hmeni2Af)qYm!i7#0`n}XH#Kf!&Fh(@95!WzY zs$5WJhZvGVH`e-}PpGF^EyR03fH02WzdBa`haUtw4mmMDvbuNl$Tzf)69CC>?Qope z3sovGxP=+TbI@r6f{TbGYe0$W=ezze{MK>#y&HwR81=xztsmdR>#Aeq)JawPW7)lO91(pG92~Ibwu7|g+ECSA$ z1a11mc)uN09Y}`2TtToj_jRw+Y()|$*GwdPn`%Xj2#K$MnapSsi37kOo|1X3VnrD|QUuBd7h zWfJJkv2~HP))$nMlQDN^vfEDXAMOUq{#?YcyoEsp6#kM*9FL21X)jY`E-nND56dL(C(BdYyA2COHj|5&V zLX;w&F7xgMUiH?8#;k?P-;T1gWkkJf%#DxI<)F&0x!-glm+7cP8(&3*Q*lt0TOnTj zPrHuKXMMhf9&H-Bq-&Iorw=ho)_0|ys1@Bl=?XhK-zRiuJ#~Jy#9y=^9Jn1s=@&Y| zmUO|YyyewdDc6?zkSMh!ag!WPigYGCN?5|uQjCm3Okx(@&G|b zbKkDc3&r$)zuVgMHQ|bAuq{93jBJ>V{KD0)+c2>=#wNJ{5rQmIRB2XD%$~)($yjsy zEENs_Eh0}Aky|X`W^+R2t@tebEE zBE6h~-YmBuuk3?IE1MWbI~}($obQ=3-)#RRK_WyziCLdYzwdUEul38abGe$W-Sjt< zje4(meOEWcMOmPkbNMy`w4-xF(~>NI7SL0b(!xM)--K8Kp)?_yi6u&h5?`#vhhns7 z*BT!#sdCAT@)sX8`-=~%k?(K>dE|tT;OO*0W?HvyBHCcs844q_H#r?~xA2sgK$-HR209Qw?WDw(jdqe&|z zIjQ~yQVYYdh_yR{jWc=LRQ^D~A?t&dU7&td<~M9 zG)D7F9@97JZs+daeIh}b+|s_%EU6$_KFLYdUbBQ#E49gE=Z7EHOe09(`-F&-wahoBIkjO49#Wb^S0tr6&P0)%aL@d< z(F0;%!8p?{pxA#!I_=kO9A58Jl_+DOpRjwKx@RxJdxM9TK6mf(2^W}Ik{>|IP z-p~ZCu`tIHdZCshF zm+_^%9pim=jao<-Sbv~ozRh)HzW#@0YtY-m6cc!Cbfg_j(HxIhXZ3%?Iq-*bcAFW<~#boed7!lBP}2$hy9n+g2)r;p-$3f7%z^M<~9-vz1pXEK3Ouca`w>%n)JK&A&8}?=uM=bshx| z6Gyx3bobEV;TOC@lr<|AlPL8HgcrzMj8TY{SM)~AJu%vsg!xOHJ8tU_68V;Xf7dKm zC<7AbDnQ~q?0<=K<-f!^g)|-d;Nw3t9(t_5X2YO8W^iFC0#e{sRW5YN_pNK*4hYW` zVe(0L2rKQ{{-JRrWcqEHC=bA>oZF}_-zh7AMH-5K1~W;EnRxU*I{`?X1GOG+^vLva z|02$3+#&xV&IJXSmG7mXA^)ZIfDotJMLGNq@ut5RF!V}wdkVT3t>lOxm^pw5X* z83l&L{8#IdHqMECV?qS9$!Z4tHE&0h+Y4AAW^RC$x(5X66Sz)wW#?=xtKqR2{t%CX z`YJCqOP?0RVAEjCOn~eMbIV>6N8TqI;6l0?@k;5+ky|0y!^@a1;hx44|E=cBMY@MZ z-#O=waG>66zVtm-JvWFD1+o)aVf#)Ei$xWNiDX-)?=ThCh6oifqAd%#>rR!8GrB8# z0?;d%Jj0$8b-#-tw1n`}hpc3t5zeq>!ZDG$kg9}4&6*w*1y;_QtzE3u=#pz&Zd7wW zMA$~wav!MmNN$Vs2)U;4ST_temwjJ)u0wKR!Z8z?a=wX)szhA>ekvq?SSmt>DjG0> z{ki$4U6JS2qiV4rzqcbw2PjdS@W2&%KHUya4j|ojM}3KqUx9(=BL7fs_35(JjwU>S2Wlw0b0ofC8-^l^QjXM2l4$#$or? z-I&IBuRtQz>!BFaSdRyYnFFmJh^Olez?Y4h!4PtGFDgKm85wV~!+f4GKQ{ z8>XjX(T2%SiqJy*E+~ZI$yuO7qgy&m?;l1Z>#!i6$?fSoNF!|-*6Q5Yp6h5gPQLRb zej>;?HVt;;TN=bROr1%#f0{LSZE#?%Q2X*riIQ$vW4x+|8=kx5B>I>$+Hu zF11$!FASHZM*3`Xmkm>ZpE`&BPd)0x*|AN}eOK{0?Am8j(pRK1rI-JRjgg*W6Sjl- z@}Y5w_nJrLClnVMyYarWsSGo z8254ZNZ|X+1N;f|86>@XSuEd}m7c1i>B#d_r>R`Y+>1+%{7TP>zLp68%>!NR?2mc8 z3srayc)yB;Y4EhB>mRw=a8QAep34NR)B_l}B7t~=PvK9&M)J7x8F}^|fp}I0C&9nI zw2AZFyWgK@ymMqg$+z_*W+CF7cZsd}cWn9J`d!jlxwcujdz&k`4d`9O_bW82Y!L~- z(|Wc!G7{sG5X4Fr>Kz)HLe)*1h6c%Ac5Ndxs zr(%#YbD2*V=I6$*N=d`yGyE7IYy$3!y(<>jDiX*uh!IE7^;$M|}#Gy4+roi+uL1*f|OGxJE81<5}T~j|@#8^WwY9a}ShLB$nB9kMZz~#DXajo^sZJBD8n%{#&PsrxIpWduIf4ECm8V6R4oZS#+{|?wE~&q%P&}wMS6j82>LoSD z4|?QnV42=LxKzB|A=QpaYmDf-xXrdT8v?fLYvb;OOGcjofWT)cT5hN@7Mlh_jT$a9GBG^>2PlAX~5Fd&u zhEP}Qr7sLtbXKPdD-K)Wk13&-CIn&M*$ELf3V-{V7=G{V>qXP($feX+VpZJZj{f%0 z5hfd$(J*DgIR0$Ke;!E>aEReWU^A7fRw%10>ceQL(dI&_Z3x)m1Uu5kM+zOTS1GZ= zyensLtQO7L9*&k7gQU)Ow{3?K+4ht6_k&DdeUL(MKiaCNm2ircYVf>xb0;%++pz; zzQGw*r?b;1!#$4zPHl1c^Kyb{+|dsN0W7YD*KBOAj{;7994@yew6U}_F`4`I@%%wA zBeNMN+YGG+!%u6onS@T}dW#viaa!#rJ(fuH3k;UXC*lU z&f`d4Y^)v!4NhTHM-ToD-#}EhK^8m)(}(C)!RlBr7*1)%A-_yb!n9XV3oUi9oRB`s z*MjsY*ZiG?V{KHb$CzSC`Bih%E3S1x2}##b@#Ro65Z2!DBBLK5@Zi1PVAT_5wkIz> zFbC2S4AF{29p`O*`CeX&=i1ur^fofH1w*xzu?-+A7aB=+ z(OCSMrpXA&Upl}`_d39<^DyE%5wIf`CEQ^lZprZ}Xfq{xzg!a`St~5Cognd(4NN_A z_H}`Z+HyQPKSVX??~g1<7DaTRRrQ|oS6cEgx+OS7iAJYtU9YckB&%8mJg6TqJOUDl zgV*m*jP1{Ur1|?4XK87OgEH)?8Gl zbC1Q3^wACl3|}2yOv3@bgfoDc#5rzL=6%Ou_WqJh>!#xJn{R^)Z85PB+*6mSk~(9H zVH5+V;dfXn{zO4`oi<287CrN@(e2MR)oaGl?D{E_O>$CH4!FZw20UGXZ zTMNzGTkwDg&GJ)6cTtNGqAE`!A+;Q<(Srg@Qy9WDRg$r6O@WFGmvwr`dkXlXlB+{W z>KW0AdFrql&-ikIGB?<0dV(K0dD(Wg>tIXufC;_xo57dl6bHA2MhB{9%^d-jCGC6d zD@SzcV=ALI4#clUzBY+DX}ZV7vx${+CMq;r3biEhBsd|!X_*rY7P>Oy=gqvufNF(A zoI>lG`=D64Uk9`Zc@N$?TWd)p{kbb*Ni;R4_6@Bq>LFNh(4_6@K_I^DL!6l}p}j6$ zTS+Xdv#QN>VIHF@9Ia3bWYh5Kd~3u5m?^2u4`YvHTB(64c=Kw3X|DD-uHVWnuPoZq z$gO#41Mwz4JwJI2`G&qKG3nwsrMn=soYsG53O#*+w_pyz&Pne1wHD=MrD6{sq8w`| zU_SaSgIcY@`-o|w7P{0-n_3M#Zf8LL15uM*B$EkxaS0qm>K()=kj>J zh%HF1UP@hQg(XRKQDXL3#Zx+26U3iNeib45f=b!LXw-VLN~}Ou`2ys+>{dfPiCN8M ze@GRUS&lZ18cwqR$D9(mf_XTZNIJ|k#ekc{GE1<9-ivIIOI{>R8sg`G}VZPq!)Y9P+eVRJk0c$N770o3Ia0W22hg^4+<|Voap?$qUEd41Z4DDiYv`sfe z**^Q&MXTFGJ$>-)u{n|A_ED@(icmJ3Q0nt1)6_&RkBJL)^{?~+MsvynURB&!F~<{1 z-^if$C-*ub6sW;l!dg8W2n1!GXDNxrqAAuk4DC-U41^v`JVLTA;zeYMp33{2?H@TY z0}Jo)z2o@Eoz#@nlmYIjHQsk>0L=Bb4wH&3v8NvLHCFv1a|^P5t)pg;*}9BNuFbm3 z&cmG#tShX1b=5oM>%(N{nBg&O`Q-bQ`V4306C^jR8^_Y-oAL)l7Vs_A7go2sp45eN zI4(%$TY)l-D)K{vKS)Ya!uZLAH5gv^(myIE4iY`Im7DMxq-G0|ZTRMgXqi!2eZq?dh5zvu*d;M3OU7wNi7m z@<_ESd?*zb{F_Y#m6|>|8%~dePWy4NuIg0<5!1;7T^>+9ZjW?0E zEE6InnffklWbG1#2UPkzLwlnO4&CMy_)ILwoX^BK6NPAmD6#rF-lb`yK2qr#HKlKA zh9i!D+jEVkj-mC+zu?cK7M($$Y{@nNTE|;++G|k{W^-DTPE*<=Y1%Ene&KsARTv{^ z;ScYmkfHBo7sFqxaxPjck+cqPAQXhcUD^>tk_idp`t$+|SAz~z#;<9QR6o@zr63qp z4wSsca7dEE{<&TK-6eQ^|CQEyg<;E_VbkqPw8UPPRxcNRg<%Rs%6-1w@O$V$?g!Pe z*hV0Q$v#X8E3>OpEay8Qw*Me1;d`--0;sIAv|&UmA?VUbJK5TI~lIsEAj10--oIgRY>s^0c1U8@ymB{noA^M)z!F-)};2 zZK?($jN)a12*gL1M5tooXxMc33zp;zw`5`r0Wx2FU@8qb05q`d670CzQFt}`E@|)1IeRKM=t3QLGZn@atJanir05?P`*%5hw_`ymiiJ~hu-DGhHG|-0t zn-qeK6Hk>dL3$wW&+?ZkDQQe}Qt?i%m#7lX$rlT6Y0S{P8T~8bZYP#;#BzY>0caJ~KU6(2TM%VO1N_F|J7LJvgf6kn5C98P!V z$i{WARDFd-&Gif9f>f?^L{SKUIjurf@J`heYLBKVl*b=0E z@?pj{Vn;Ax_h@XgQ%i)-%Q*3-9@gyg`(yIFGf#s_ByI^7dy1{$J!mp6X>gF9^8>ng zY9-&b>M){{A~h?4bmbKBDq5CZkOSMKu(2&nn%N{ zv&S01{5;r;;9+NJm|0>|65TBiO%!EA@oUUx;lfGg6qu;qM-0i_B^4^su45{8Td4xZ z+;M2ro6uF|i{XKsoO9ly~|5WJqvu8jG<^P7v#kZnFScT#WRoTA`IG?)O5gp?}A zU?5=eUNm>=Ft?12H8@0kg>c*v#l-}Y)s7aYK`6ioo)dXsH>Jas*oHZW2Z>z73XYubURra}Obv~jm}L>b#9xY~7byD+lHARObB_oEp0 zv5OM6F!|JqA+?!6?sWs&7biTm3%rXlQAIR~d1{}ixk2o&r%v2g>`Oh)n$2b&@U006 zU7Jk3ve~l$;g24tj3FN#5Kz!JHCv{j@LVavio+NUoql>loNblq+)i(|)&ak2E_m}X4*W*3steq){t>}YK?M{$OdF}$m&*o|6|OBJE?L5+0t zfqbk5OhU#`JT39r$s=(MDLcU*Qi}e4V(ft%znc77Gv=i2PlMUVL>H%!CwvRA7N$NvOX!GGE=>v>mFWSC8W=%wa6ksi5IX)BHDnMXKg21qxRSnB0u^j zJ||?vNmZD%#%XoWA9U)+u5X7z)*K!kl1?h2nH~HiK+3=bRtUD|SVC6w8;>s&sZ8Gb zGY#kkHnD)<0m6(9db;b&27>onDig0ko&pZtZzkfot5{}-X42DAhq+rl4*S7O9;~{& z71IgoD}Gt+nh-#aiI%WK8`8z7APuYBPc$bN8gKu6essu7rch8CFkp~}QA!xQ*~#PA zBz?MxA65Er@iUs^m*)FlxDNsyAXTLAMiO_9qwbEOC88 z9fGRr_Mp=Y1(?e%Lp|7lyoyLvTCae@-SeurmQ|)w(!FSSw&hrqmelqOBk~Rf9E80*FJYP=NEzF{Njl&M%sd<_;TyBdzd&N(en{^@Y>A*6bVD^WEF9=x#hIRF=|sORX}F?;V(RAHjg z#%|9}Z||PdkYDQB>euSHYxbQ1g%}4-|9UZhZS#ixh)4Hsk&ocgOJMW|LREoA1Yi4 zn9T(AVQ9!U12aOJPX%Wimg+L0q*g`2T7Jp7DvbPsXwq`iXbQ+L|9l^^_73QI$d`Vc zR#6A(c;H~k#+owE7>bP4=Xq-Phb9PGgTE8Va<5+d(-Q2%Z;9fBqu z1VGM8zE^kpgC&vKTVlkj@kX6Of3$?Og@lDGRzWJW_#bzWSf`i%We9nZ8?t- zan9s~hgK;Jt03>IpCRtP~ z%vKkV@B76bEIs7c`B-a;)wn2=@pO(0SLC`HxO@YeXs$$5WNz~=lzb}4AxmN{Vqb~y zw#GIc0hPIkg1xVtk-icmuHyqyOvgWT%_%8YG^B=rgygQx4o|iOJ0SzX$i(xcAaLwG z(849%&-M|<6wBONCEr@%omKn|xRf1Et{6SXq?Y#xpkq+pydz)Gh<8wma+D2ccn<93 z{woi11KV;^x;Ft7nB$8(GHJlEP@$k4`oh2I6OkeSpG$rCu4tAYxg$cW;B zF~;LaW=2``oKYTa$Wh*K`8}mTEZlUV838tMQSVD94V0JfQPahgj=P)cAuL)CXR)xM zA}|gw1rsKtDwtjZfKa>jHx4hDI&N}6<{HuT0=+@<@6g+nbs`9w`k$fqFfa};3?|&~ zJfoZyy*BF~cMuH8hA=RjDZ_L7+wa8^14;4Wzu8QhZo{-jm+(A);X_3I8 z_eWTb4)t$hb>Pt3hD?uE+;bKI{5d|l1982XJus6lub&iMG<*u%fQKDKk2c%)D_@O& zWAw-ViIbkc)kGrHsR765G`pZOzo{gx*Mz>KuE|blJrC`&oRLD4O+Mn!E|Fc%OEcpZ z9oAT%75P53MIWzG+zAHcD6MKI`DRG3|Dg)I)z(wqmWI}1A z8el3DDv(ssC<^0%xXwd0I(=_LHL@!8REePE%vFJ}RqCV*UH%1J#%8a6cM5|$GuTJO zS!oBpJI}>6{*D6>v2hRp=Ht0sj%j2>UmW6v(Wh=tTusMtS75YvtxDeRO`Mb?It|57 zgp61y+aR|vQK*>vqja=(cg8MLH-Fv~yKZ5-X>TBixSn9@Z#{Usf3@PDuaZ$)u?U0# z4cB?XOa@4t?lVY){2cZ*m{w3;t)uTx+=rj*`3m3u3s!BHidsd8Zi7v-5#JtbgJLR9_p1^QzF701@ zXAm{}sKB0ay5Y$H<<;CCn$ej!L7BvyMbNQ(oL8cM3;RN0=cRq~T=(d+@+zSBz`k$` zv++1y=wC)-KqZIQat{pWB(h=!k{13!&s(JvqyJ%OTdnXu`)l4ibAe9L-QXp+d+VB^hpmC<=IgvUlToQ# zu_VxGyLd`hKt_OT`3gA>CClJKJp3rztZ(l=R>>Qlzv9^0rAgZ`s55~uil$uiJNc6*LI%f}kS_PE3C9 zrrw!l?=n{YOY}=q}mV7C;9w|SjTJPqd$>XD-+Zywj1isEk7dkV$ zza4yAI-b7|i$?N6SFU5EX_ojSR(>p9T>-{&C&1#Ofj4(ew7S}h>+YQF1E#7cO-##a zUdOZ9N4$=b$tM%3qNU3;VUNv9#+f(6|1_G~KJ!) zv*uUCLQiBS9( z+3blTG_|rld02xOS_pK|5|d8I(Gnt_*xpXYtI?ZsyNiR)7qJ0OQ#+CC@l*!rc)pQH{hUYbg>_$Q?jQtxOwQ4B`PeMR(mk}_12ym zj4Bdr#?*b$^wARwj+Z zOh%+mC1>@~%}K_9i6s#@Y(Wj2l67{Ebq)LeO%>RsaxvXKi((tE>v?TcN7Q-HV?KR0 zq4GFu4Y1E7s|4eU@}ko=LC}&D&I3-rV3oN*G6!y90)3&KiHF1^J+`eD@8tl`q3GHB zI_72Onr33Q*;j2qWtEG%5tt$DQG$*=LSNJ%%)*vJ?PX(gMJBpYjJHOJ-jFq}&$#C? z)UA1Y#%XLBWbs(YrQ|qr6UDMSofn9=NU@Nf_XJ{|(Cr3o>EvUvjVh~Mml(uZ6eTpo zx^7m^flgG8`$7i4E6g05<1`5-C7l0+E@;3H^luy1gQv=qS1j4{N*o)@N!C-bh$*!Z z{{7*b><)d_cS=R)q>x)d*Jp1f93A$JjYkS0`2Yd6n~jLiok%~joY)qz_O?3Sz4O7Q z`7+hLccgi>b4@VBb(^L4X12E`ya&Xq{WQ!J70a-{Ag!NF-Rl>U&fi@rk(udSFJKW(gR$1a=Bb!z_IY2Q`Qe zOF}>!0gr4;?#22&DYqTrV&fa7N?AQ|dmHKu&-6p-!mj(bH;xR>%5*x>&64?^+AYlQ zUP{nWZ%s^~3jGEgz@i&BBaMt7%?3W|!C5Ah3B;=Y_M;j() zQ4(lOQH!pdC*oXN{15(8&h&}UYQ2$w3c8y?i|`ypyd5`|NN;UjBVmcn(Gb!6Q&m6x zf^I`RhUnCj73;Ag-tN2Eks5NKaw$=+{p^D^E!4V})&Z$E7U)mY&bHCBeH}2 zoIFQ0&&rDQ29t2g4&=H(Dd`FlD$@(0MCOB5fdesBrh&@8oRv?XIB;~}cVL*5JMnuopv;VxA`x06e{~D4a!wsd=j1XcTtVbB6dylCoSriPbsazSh z>*0hmr5H>O@?XJvQ0x+yZfMkpN}2fhpJIw3bV}U-6oq|(M3JedK)xv*#cysBLU@>^ zj~Dce@FcPW{FaC^L3?;w@{ozCZ;{cY)d`xo(6P@?P z7CFl(O8%iJNnpu}BoBB1MZbWM6B;)y2n)hM1oRMJcA8+WiL}*32nc9IMVgu3qGG_7 zbL_z6x0JFpyM$@Q73JN3yo1h8*@RuP%}ju_Mo^a>0FA#{D(DY^g)TWQzOmJg@}BdD0X*W3j_i3~U39I!c8r0b;|c@-IU_;53Z+Yu zTn+*d5Zb4F7k{ER9r=M@iJbD815uPYJ@HK&j}a@Jx6k^jUCiDh9X}Vo@YL2-7HBSv z(Fsb3sFxOh&;?(60w6gpt~5&)QZ3pO1?u1t{>>0@OekQ3Bp>lZEbOm)j(d|$tbd)} zA`F3@3h5UW8GZN-JkV(oS$7i6*Pi8wQ+~OERU5ATS;BRQvc6Ou#~kiGD0x3B|G=xT zgeG9gVgC)w3JQw0T*8V#e|o4^0HYgWcO87JRwgIuM@5vTXuw-$p4c{`oLq!Iq_{LI zAMp@d>7D)(8BHyO2%bD=R5W(JCM`?E2t81~z5K(J@ac$hNLUCyWFqn*_Es?N9i9>k za)07CcqXSIcnp9vee_elJQEH8!Mn(@*W3n|jOm{PA<&E;D-8BSi6P1r+%A*fvWc3k zT2j~bP`}-UR5&Sc?KU{eEMQzV);3iGW<|6mK_F+`EeFlI`O~yTLU954A_qj24@ua3 zBT3f~h+5{Hz<@l(eT+De?^`5t8U1SX^>7F+lpeT!ne4DASa$o%urhSuwK1_U^*DE6 zT&wsHRwR=h-0*w3vVU2z45%{?(nGN`ZHyWM@W6*PQ$A!m(lSF)baBwVia_b|%a+_gDc%~Ie9H3MIPf0^naNNu>R$vwN zn|=8$1mz&j2zVkKVq1=*dI!iKbzKD$3~!f29l_BA^{#@ zlIZP2L~aDZb9f@q3iXT_Q4{SVq=6c+>I(cL2NI2tke)%-n5iFrFdk?lrJ}vlcHY~! z-bz7R%-+-2Slsi|>(LFvshEH~dTj+Tv?Zi3E9l7eC zua`bpk2gBG=c=nXI7|FJyJwglo^r+>t?TXP=i6|~U{F1nZDznj zY_`y?U-DNiA@$)lzGFQadETO(J%9=7G6gy*7x0#t=lz=Q&{t2{fU}RlTd*~P zkfWv8)LXyv3w_O8Gp#)~h!}I91Ty`aYT?R)Xt92l_$%2Q7(gO`bRQTE+m-WWN_6W) zFAHU&CI?8skkHR2wCrr{7Q2)fWl{C5Uay8?B9IpGjI(%U78c0W46cS+W?`F!yo#MG zq?556+YIEC*pQ6Wp!FQ717xSOVl6*IrvRjI8Uq6lTfs-c!3z++)TPsrNgkDP?N_TL zXx&9$%%5yCz|@zfmr?$>lDM}(-xJ@frcr6@GGufqQ;5qalpkG7;Jp4NL#Qqpj>abj zX@%7-BQ$1x0tPrVcki8|zUe@BS%kQ4`cm-;MWnvRnMDCEJq~C|Gb^S%h=9tsM-Vn) zpn5g?ILpv!@G(v&)Fqk5-}E=)A+p3X%e!7bZe(w+m?9z#@KFGQ-X|SL>EMYCv=rti z7$=sc!XV4hFkiVU!j%@!N|1zCWY{h{@9Y_47JY%bX5#SViZAKx=}S~cd{@>{eOb7| zAgNHAHcCNrrENgcxe#oiGRc~J%mF%xp473NzQZ;3taN`7Y)pK=G;MA3VyYY|F#&jO zWRkcjm%0jAv40eGArZ}{v^3S@Es~;Atu?GsPSo})QkV2e(hp&lYE+{wF0`Ujm7s5o zWSE3h5J3!)n7nmQOo1yV9~6ZjBGpMsO&*8MsIC~Q$imx97uN6i$i0tN!uRiWV*n~w zh|&0JFEy*M-~+WN%j}*v`!)&noTVB{)mKh}-Y`>s*e9xv_@CQfhO#?Jt+ z?G>+x)%Fc%nz+X#U-O7_fO=Tu3s~35+n~f>=r#ovN$(0nl9cG+d*X=)(#FpkN&n|m z;tQyUW~jmM>p;&%2hCB6q8r6zI~eKU*Tr<<7SGSmdtiDGx=P>A4Rl~^L;vWRl!n`w z;_cam5b{9=eNRKB1@mf=K5$TuAjVCLr2gX1^{EIA02Pi;IMUCer)Dsn)Oi2vQYk1s zG}u)-C$Ya)s|WISXz4(fn;`y+onaO5fSOw{N;gvKpztQc`)nXMtIt}Sxs92}7cv%Kc45~IRS$2@vy~I|rli)KREytH ztUR4f*VjLpbj2C!q;#aIN$_WUFzR}D5+C(W z!jg~Bv<^mfJYfldBtL5E2F2Im>F!%h6e+*G-UW@+vg4o-N)|KYuUIj<+l z?8C6Bw=fRpI|7|32k{B=#94<#E}uYM@9H)j3)}?=ml%GJS)@b-R(d9<8<`GCgCsREVDan(oSC@7m_pe z{^9IF`BenFKQn{vd09FDma%JHUvC#$3GzZb=&Wuqp*?Bz%%k#|8U@eRqQOJGlN6f*IZr0&54D}t1JeBCs#L%Co)tK{`$zpJk^YhS z=-n=iC}R7Gp76GJaJ(w|d%NBdN2x$8NKz^*jt4{@kJ<0ZqQAPoi(_5MP0zrFETE+( znKK_x*>-*X8>AdfFPsaT7S*23BQ4j5)tY%Lr2mhvYd^33o3z8v-Kr+8@Y6&&R|9Dr>iSk88-2WGqVj^LI*N%g@3wN; zU$=Z$O^~f75KhuOk{#jXLij+eZqNg)dU{~80MhPlk`hrjEP{ygRDYf&K&;An$!OmW zY)Q4$-$Cv=Gokm`lSoZebXDjoYg6Cn6PTT>SRP80ER=NHxATQSTCOmvlQOJH% zQD6-!3y4P(S;&VeF*MZpC`*6+Y^k6B0{s78jdJO8P4hnh0LNngi^0%;Dwaa}R#pc3 zhL%EB=EgQo{}mUkW$k#_g78zL>;GVa={Z6;YkIn2PwniKP-8!u!h=ee?e)MB**|f!Gon_ldtO&m?BdAA_kw_O80BV+V8E*(v-C^_OmzShSt? zV^Dv+60A#s)aE#sZ7rKXZ{Q--n5 zpG%LVOYd8+?2m_k|3ggsb+>yoecs^j-eIZ}!wYRxZwKM%&W12a?yeH?LNtEi(63OyW+)-sv}-qEr5-ZYAHv>h zvW8QsH3wTi^N_?Cyru|+DXf)cc}DqZ&as>6s)L!YGW~_{MJ3e?5A9WcLJ<@vM#2!3 zrqw*mg;T0~J(ZNYP?nHrleGg2ih?N54fn&pH{I(G<3YT}?Aq1tMWrBnl;8WDlJE3V zO(*j6cVCZ7>NwBVdR&`4gvb-4lyog;sA8tT`78q+nPtqd%}614w;L zj|69oTvkPryZRDH5itk2_{eQs4wg=7)iPkbSKF>-m~a;NM^E2bD0cribUE3xN%YWA zSc{iOBVazz_+Jqdg+;1*CSzq9Ky$vBD*jj+KkGl+!zbHYtb2g9#2Gm6efYj9UV{19On)!H_^w z;_LCA%2V9aser&k^p~g5*0Kt(UxUAAbFk{_$%Of=wuWT)=Y+xp3-scTG?Nw!*9uZO zDf#L3zS&$>^J15FR*vcWkFzZ|X<*JaQMR)=ZY2>a2FqRx0(dm7phCssmv%%9R$#T^T}$KI45`A!)+Ml!~&fNh>Pmq^tkYC1$Ug zgO<#=6jVuPs(!18NAlU=0fzx$5YJ4$AE_dILmqFoy)PIXQKZO$qw+nHsZ2iY-H?dO zGNIf85}Cn+emK+t)5*Xx0+ZR zC^o8o-CzfSbPEJE>E?IMYPZwjNkw5Qz??9e`Z(Y}+o@KE4cf#KIuuKbT09_f67O-G zl|HtQGH_+N$|6L!rB!kq6u`;h@|_3&NUn{I$17jzjriGmusc|0@{_VJaau0L9vSC9 z6)9^Un1YwQI11RU!u!aoycFFCQ5e7*=a7aUE$NlOSvv-9-1Ne>u9oOG65CL`f=xE`3|4Ac0qfQGF=02;g3V#d{GJ_cJZGwaKCF% z#45wq5}DTpNUaqV2DLtSUbCn}2&+$pIXNw4Gx26U78_+-bJ*&a;tgA@9PEdWk7>QP z`q%RfpEDS+*TH*q+>r@jNt+3IintEd3V9Y9?spqo4Z6N$92;de)xyx9YcuAnP1V*y z@XrgAwA#A;la$KlqnmO7#qv179>dQc#oeCnTEGeDxuW_K>!Bnc-p9SM1^mhgI?&-( z;dG!yZ~bW+?wKU~!m>ALe_(e$GKEP)F#w>^XTlQrBz7Shjz0x&Zz5a{zMQ$_eGlqZ zI{mddmx9fNF6!eFE&$~|IlvwG>y%->ymvrw+^Q4o$Fj;wwN_Z2zio;VGs?XiT zCpqb>E+sH5n`F7S$R@pXr%KHthGme@H>ag0W?|mce_*VR!CDl6`Fw*>QZcrZjC+-*tac2O<;r&y65_VTj+y$ zpg<`!{nvt1$~>MIS38Wwl$bbYF>UIch{y*%5z>%X;;smIR<(?K#|#015xkY~ijo^+ zo{VCQG_9idnUKFlOJrrrOTg_*m(LOEq|@6DB5z6F2e;fPpcT4xGRKESRK9amp3^3# zx1QUzNLZHC9^df929*68sbBj=xXWvMn*2oK3hXbQtG>r~)HQe(WUvM(t47DGOjuZ; zoVaDtC)U_8*l2e0wS1jMB>%uq6>BCZTya1iC!>9#Otk1!L=Kt~?SVjUAk(WpCTJwM z#JwDk=&stD9-GP|{+N)2vOp4(hluH7!;c3BthK^LG%yKAVBCfVhlF28Hued`FU%)P zehzy@hch?|N~q8U`?1~lZ*$_lLWa-|H`(J^Vfnqqd@})?GaICH>)Kumx zPU&oHV9U_32gA|*O-$_Q8{L(c-K=1XuJo)=pv#)XoD`)?%`MXb;!$EM0RwuhG^xND0(qdd~A4{sVLw#*wdHEN!c8WvZ| zcY3nk-O+J!S86OkLWX?iaC>r|SQXerl++gkf2%1`kfaHEg~ZNTs*Ycm;npLMTdygu zA#;E9Xg`Qrt8!+LU6FT3!Qu;YsEoYPCme0uCH_eQNF?a;V(p2_`)zD}Xb5t_ET?uh z{uI*Q%14#p{V`CrSoXPhV^0b>Od)gy7cZf5EM4bP_WWf zvyF!wXbIsSv9Ws$RJgguriro@Wnb%6ChD>Ydd$~Z+R)l#8r9zqxJM&H6kq_UW+ zBGen}%$|xS604g9Z`8AdeYAlZ7Vtppn%h}7MS?|lax9*@@ZLbSjvHDB+Xr2t5mrS3 z)j{M>V1}sh=8&`zFA>WDECA{R^(zBd$Oq}x3nqKmse_h4QaOkq$Nr-3>+CbA?v^!D zKnWEqIGWASG!#*&WnkUig*(nl>*}(2Nh|+;Q2xGM3|3q3?j-fDR(N}}KpwEGG5eTS z+gN{=H|Kmu)mEb%x5Cx*dfRia&NEv&nOqO6!}vkNB{sPMgRbg|CQU~|DIiJZETDUoy=`*{_Fhm z&uf10uO|3Zb*ZfwhY*yj^#;e-55@nolu&raR)|M7CRC_51Jz(c9j(X8!tn_*ofE9cJ`>)qK_7!$!83y)iZ> zK65WjQ9wONXte!Ga||jdY!=6qHFLdDF_R=V#`6@!05OMyz%XmA=dRkN`dzWm|HH^7 z5)aaXS&x54fFK4}6E*xO%1>`s>WeH~JOK25-Yll)fb5D1ZaG(cK!BOE<<|y}C(E0g z|E?dVq)d$4=>Zbw+rj7M{_BaT1H0qw=P-lq@n;rKQ z6$+8jM!n_wDADR!Fx*mIh=Y2a!(^D~WOg6#hJ+ba#Kqy*W69!g_Y(Zq979Fu)Sv@o zOOo;87}+lP;m2$en_m>MUUrIj$W+yVKN_sjGi$oxf5qR9N1EWN0lzh zv@Xl2yoD1pqFaR8%h_V-i>;qSS-oP6*Tg+J$fUpZiB(3P{D4UovCCq7qQ9)a)<#G2 zPhS3JyW{*GlLL5b+3x#yfn#dVata987&OCa9v zQS^hqZ=ro5Jxe2G6mTKgN7++EQ89QE#i?g;Zf&`L9P#Y$EPg(!YoiI$Pz2Zryp z6bG$cikr{Na(F)wJU$F>wv-)Xt8tXtwI)wRA3Hy(IW!}!-2#ceo0k|ro;F9;^aM%S zc{BytOha*Wnx_tGJW^?$uwYpF)xcrIdL4T15ahiP0Gr^PU;epqVI@T}9oq;p&?q+^ z)&*X5?&kSaZzh&50`^4K)(QN|VH6Do^tL zUvf)>{=~SxaU%b#^1^#Zp#4GBDhDB4?~s(WDxjFM-3GsEOwzCXEDY3@q*+U0D{2vR z@Qa}Q6}4cgYf-FY%RSpRt;vIVLG^>+W$h4QUY)7huMempa7O4wdvCA_(?%8Okp(Lq z6YEL~L!pmoA#QMi6^M!MU5_0=v{|sW{~3quk~AHw4)|sw9-@1F3)WYSozvR z7dcC1(9~%-iK=3+tBEF)X5$bE22e8d21TDp%8cYHy1Avw6M{w!Ho5H43cspT-DOl?#84=6b&RPC!(R0qz# z_|=1#3@>3)cakAS2F;CPnvYbbtIu=ACfAh9$law`*9s=Q^X7d`^zF#WmJe@b*szG} zmlJ$*U`>bT5Etn=VU&?7D8kBJ77FI^a7MqnM99i?sY(jTGdO&v;o#&Ak1H0*h4&pP7 z6>yLErEK24Fxo^A;Ove|fjhJV$QRgAgSjH$WfqxICtEAvlII0nXK>q{pEX=T;}|h zN>37swCo_VsyI>g{$_^OwjZJI;;?d}6nD_{8U^VKlngD@_Dx;vl2mDe z?8+-z=cbE4jpsC>I#P$X4OEpe96B`DJ)4q;slhTtJM^Q>RccXma*xLaIQneTWJiAkHCHD368Wq|~L;L}kA=M_`;xZ0g~h z*dHbvB|m~5$MU7Ypxxt*KxT52M}!W4Z){wU%Rc^rBX&0{$F!R{gR%BeLhw}wRR_Na zL~9816JY}@g&=Mj{Juiqi}RJWAG?e+*#;^&sgsQ|go=Hdtwv5=xI$VbLbz}$k3;3=_fwD z#epjtA8idgmlIqAR9jK1VxQ3RttggVdY_bOo*32W{@d#4{Jl}CaX$as7^?AM38k2) zD5IHERP`Cc1ze%ovMknXnhefJuf_dm1xJRZ3-7B=6c4V}Vff)(%7?dckEhn2vvI|F zDqgU7mAlb-XYB8ar?~m6_BrOiDn=cpajd|UTzG0v9h>=K@Apddd8=zG81mNI@*By4 zznwoUanPREs{onStGPm%c% zGPDvHM}t}N=g zv&*>=EoR%+ei62>skXpJserU#|8^tAE}mKk484J1oy2i6@3+OGrnkupEZQgY@#l8W zYp3_E>!(j$r?-2=n}5wK#@sF2*N*L5_Y2Iw8q3Cls#4|d3aL!a>$KneydPb$&X@n; zb9KtA3}S%*0Ei<0-;$TGv5CI3m6L#-ot3$v{{JH{4bOk%h4`B*D}Nbp6VANmKD|W& zodsd!Vv@{{%X%aj|B!wz8tKqI2=9 zhV86_Z&#;Z?DYiGhEG?=_ab1~F3UI98K?C0jB88}B~c;ZS5SJcW%Ul(E)*Q|Vm!s+srW)!s?)go*}(701*g?y^;9PC{0c z^b&d_f-x5adDia=pfeKq8VmdgAjMW54F7ihbaVQ-fw+t0$`*?Cu=~J``L(zIxjDGP z(Sp%t=i%j8j1b00w|+efno2(#Vu0o*ES+PTTp8dS(K zG~tfuD9Ch-vDm*P(QMLwOwu&S~n8O z0k}ok#Xg1YbE)U3%MS$`&=ZY<4$YhVO&r3A7i8QA@dDcFMKj((M&;)S+f*IF&&^{N zSRPA4=847I`rNH6D_Bmchr|A?0-~iS_>k&?E5<+vlIUdn+8Z43>;akIy<3SUB-5;h zE%z)JLPZ5yCAdnDl1OwU)hBXAyqrz!M^UckjbS;;WLwK(vE;hTi_ByVAVL=ZO9LN; zvdD^(Jq+@20Axeg=(~Grn75EYnT(e3gC6D1MZb}K&}2DGYH0J9w=Uy5SY0O$XgaRQDxA;SVn@F!1DuA?8D2CJla7g*y zqs>ECy!dt{*0h~a6~f%FM9wh5+}``bOBzmjy&)IN5b_YC@H6WirD+cuK>UjASgw@r;wvSXR&g*XHp1P&P=-UBD2h}e7_t=BaKnIv(^-y((zDQu z=vX5q0oBMH-PCos%&SYw`{5ELEY?T{mh_nflTV{XgFWTsrNB&c85b+jergN~McuFp z1CFxv*Pk!R{zH6(9?E>ART=t$M7gdKh`S7|xb`Orukx4q!Epua*;~xaTDm6mTd1$)q=nB*7t#aj zBdHOn%U)4qf^~AwWtjQTFp&51ZF1^q6Mt>u>G6N^SN^I!*IY%#X#+|@cyn^LcKS+H z%M?9D$+>yPmC}}=#ASjrWJYHMELYv)LWAB1pA)bK%Tk-eFZg~EsXu7KD+iSSXzK9Csw|ZG-#nf z8B89fHiKE?jA-+B74otqRgp4H(sb8_$!2wd9V9^(bSMep&ThpJYOUxUpYY8Z*A31J z6iAVHt7gWx!P2jwvH85wWfAR|q=#vgIZn&y8Rwkd^B;FbZx{6GgwLqV4d&1i1FA4% zkLqUD&|0QS#cZu^1`o~QJcZtDpZ@#Q2kxK#2YAh5h3n((t1D1d18;u6KTQ>>ia?go z>r=8PdWy8WN+b)f8PfoDFcYIj6@nxxO>&Na1md_#dGW=4t}=b&SYMalzU=_?VK-Mp zC#AIFYZ?}Xvh01)#LbQ3<>}S~gs$846Io=N+t$kBU+%9A0-$IGw_VFjhJ*(wLCh%9 zax+Xn;$Z0|gWrhMHyEh|GTVihr`Eb`amW^PG$#=!3!sLTj zTiLT|_OfjrJ{LY%;iKim>+lVXOFEYW2IcNMCq1(XcFUb=*f}2iU_Z44Ji6bvubU=? zl+^5k$4(SvO#3Z~<6%Tm=2*@`#xIqa8g8 zLWgDs42LSem-)sRudF98mxFo3bj5bz&Vv^Lg7Yw0i~jH+%`ld$b=S0XipotRBKUEv zdab`zbU#|vWlD|5%SiL*SuBw_BN}vL z_`CmJFQe-hm7zNsJ_msM(t{Ef+~pSc83V<4=cc~R`%1e#TXvPM+fxc|@yF@04DLtD zUxp<4R?S5l+1SnLQq9}ML8s%sfB#t#d3!7=UmT9KLQ7=}^!PImhN^dn)~qyK`clVY z`vUAen7Zoo3MMna=DXS4__sC}InA~ihAmW%OD5bVyz79!zROO5I5ym=lY%6j1rDZ+ zn`|Q+&;^Gi6&uDDkK~kWJQa8qXA2oFw>2hVG#-Aw>d8f|*W(JbKHp~mr~bsNo7Cq& z*Uw*Fq*KU4IwwUd{$3r_|5#7P_QrrT6c6UnbW|H97F;K~yGwk3S}cG6N8>Gc1{QV= z6aWC11OR~0|4Ie=Zx#{LcQSU>cULraa4~j}GIw+`wlQ}2?|Js6BPIvpe}so+D{UW8 zG9mFLbUt9VgxRz9G)7Oxb#BPgG7#~P(`R5oBHh^hzFP8}Kr;EkK<+eDBEn%hS1x~F zh7un={_7fu{JC{aJyZNW!QA%#vGaEJuL?Q*x;z@N=lqWGvA5{u5&VV#&w`o&y5gES zt`4rMhu=>>oHAkZbTMc&J9@g!u1|X_jhRI!Ya!#VROb3EpXYBiatRd!VZofk|HQ`w zFDwxOS~rxcGv)jd1wzyp;AT!Mq~J2#J6W z>L!?x0?tEo)Bw;CRzmX55)*?pdRtt@3)52&_9h(}*)IE*2Cq-R0Ov0kXFFie(K^RY zI9Ou5DYG%ILbqfmT}hTn!ZkYO-PgtObzmbb6mLxCoTpixvLpx74~J9GF{?#=tv^|B zh3R&r>8kdd{s0yi4{^1GJfHOC<(DD7#Lmh8t6cOzHiR%W1-2|h_RtTZTugq6Ch{#d zZJ%U;LxaQ(O6~r1W_WpvZ!0?+;BX`#1J0|4bj4!22mwRLdPv+bi?fdLW;9m(op}#; z@x1x!z4RUSvCZYQ`Ewfj;{BzfiH459I+_(&qUIKfCm>J{3H-(0HxJx23L%^^*58oG zI*av_91<b2#8-;r=ZjLUI>9|)D9!bRg$9xEM%F@1U<9#XC7zC(%$|!C#s5eBJ z60MIkE7!+f@X80Y)co$;D4?*N=6l3q`&@-23fL%tm1zVVTN{3tuY)Wv7J5DFNm5L1 z8%t3pi!w74knizJk0Miz9|6ID^L4+zh)rseNMBg%dl6*-C=BPCjzR z3`IkWYCqU7m{iMj?F{npDf-U8xnrYnk5sifg!le`W}MV={RP&l)`MEuw@%eJDRI`} zlZEhR_3p-@gaqhKbR<0rc~$u>^&N1?Dav~3 zTc?L#kqb-3@f1@aY^RV1O1EtbP1Fri zml2Thp!?hDgtC_t6W}dC_XT8FL{G;T!3RJO`2^rVB;5JU<9}VgUsQO$H_@Dh} z_)cL`AeR47h7$fMLzAOB7W2g|*A^tk^~tfp44ScL`1%()PTVNr3b*PTR}?0K!`qb- zWkf_usjBx@?-*&~$pt#*Em}DeO+OA6i|>s_B@-Cjppbk->Y;)ZM00#Vtk80{ob>Y^ z*7cN%?lq?Y5QpM5L`S}IGR*_$02lDpB27(TWi3S&P!@>B{@{Ons4Wz$R?MtkqU5-k z>XeLNyE(O6W(HMYR&wk57*HjH2gVjXwo}|rX&=SY<1Vt3mPHrH)X33h_cZDr49N8i zqypo$B(T!0%2dh_7l#F4%r6P!WrXR~%HJJ_^pe6nKz0y{93fL@9g*P=)>%M0BpLwo zWCeXG#x@l5qGu8)6+Hk27vS{rf&K_sJ1+T)2>Vf}+OeAHDCcHYb(w>V7Z*g{Sl9ag zz8CCvi*?Vkv0Z=eLYJE%ok(D6Yq(R>!i*_39T&E{FEdWXD<0*pmO8WONf5F}VKKqf`)eZlMdfuSdP z1t!pmqgK$W)N5*TarqmO#0LZZvBPjvp`um#17$!;FN;w-#6Df(gLWY~q>V}`O{yG= zsx`4B!8F!RG@_85@fX>#pZCZxw_J<53&{3R?i4e>__EEx)LU`6(5H^FZlcX#qy`Ti zE@qji%ZN3mLAoEo!C4h)TkD0-nG1T(8`mS!`Bq`?!o%S-Z5mE7s51QTC4q9|2C6vU z1Whitlx><=Xp&jfi+;ChUwUEZ2(-~ZBm!wk8Rh?sm#33}fl_GikqkNY1O$l$NMMBU z#1NsbC`~>-W=kzo_oGAhINtv~svyMJZx0q3AlFukhbon#L8xEH)hHmtFGsAcp>!6q zkMEjJA9SK&5YIP~aIaF#E$LRQjzJAl7n}SHf5oke9&~Q5>bPl5d4-(RYE~9Wl8Q}5 zmQJA|n!*%3Wnk`iWcK?C_jcCi~&4 zKT08v!h;t|I38tMf&c2iB0#*=N3S0bEN~;Y5m2FJFxp|_jyuX1TUL?5xSoDVQin9R zyAmv8tC?CtAV?`M{p0J(yM03(_Sfx395 zP=irofRt|Zn>`7v@6Z&>h$D%RgRiDAkd!s9Y+H93X!z3w^YR7jdD^hs)3Ngc=IJ%i z?cl4OJCX^4)J!c`{T<9RRt%MD-0MRL)zXRued4x&%`+MquOqJ2jzrTd=$S%I z$Ou07e&O_de&pEV!4%1urohP8D7C4SK!$sm!Jo!KHTy9^h^uN-XoHe8?%`ssHuB)T z8$EOz+fr&{+AB9#y{5dwcs9j52$r|_@cl0E+O++R2GX&dd8OH>VUz0vI?(`h?b)LJ z#Te(Wd-8_bPF-{#qs%z9j#OVal0Rl4Uc{?F{fySL_>|Qx{5_qA8L%6%%UzVvkF7rh z^zK{`rYC!U6h>xr@unFiVOQ2;XXGKWh#z&x3MxL{0MWb{C=f4tx?ym0B6^ZKnc(~Q@kaf8%vC0?b<^Ge-hrE^J zX)=ql`Z!ZXyjz>Jd1Uo8tj^|$*}*bd(+NN$e%DQ-;SK)oz^>B_|qd_`QrQk8#?bg>d0zg0RXc70RU+KS2+CN9QU2^V(W_=ZmRl4}+LGZwZAs0x!z*^dPo=CyIyYq4P^&!92Xp&)hLI8-ko`RPa z?v|&k$gMarH)OT6kk4jju5|twwB!A2|KJPxUzEfB?V;izdQ8Z}K;}&)5%Ys|EawzK z&N=4Dx-wR?X%x*4#)w1v6>V(iP%dVnrs2rR4b~t@&_T`gl}oSlgJNl5&E3ODmci0F zKhSKT)?((%#@Zo=$y$^0$ZlPzccMO4wCb7cV}Db4Ff4AJY`(eE(zn>_@^_}LF5w$v z2B=vW&C>^s$MdYKt&g|s7i#u)^xUPY*k}X62|e1YZ{S7B$;D8dt|pn|l+;nWUxpiTBELpcp ztL0F7$3cKu&1%o#co(OTv!m1%%2#2^Ewx$$=Z<#z#*HJr282%*B&S`tvBOzsS9Q(d z`rGF7(H>f;OCI`*zPQH}+#MaAd~GPF7~Q7l3)-^{KONrru1hxXF*bA;`AE4uH% zx1+&YiJZO>qHApDmiHz@`>oxSV=$Ebs8@@S`3WMa9s%nh#3q^WMoN6)&hvE$)W_)E zNjVVd&Qwk3C5)oMfEMx#z)dOpguUD-mS~|(HQRF8d5!04y%ehx`(TWNfY4x#<%v|~ zzZC)v+WCc0E7^Q8p%*`(O6KzWy=*=zX3x*0Img;C?AcEw z7HZi;6MB;6t;zknW$+`l_&h=)sGb&EKfxUkOVXYX&Z_gacJQ)ppTAi#MK4K1N72;6 zWZx&ebV{WBk>yK8$3A#d0Q^zu;2TgIJ|(*M(5rBb*dY{HKbKulPl=I6kdx*=iR|LK z2EMZY&OtOjGAMxhOfV|cg>43}ou;!ye-_0S4ETHxtUxqe`#R}KuAw@egE%{w!u(>L z*#*Q(ZmDpf1PaZ-UU2VA0t!9M@J#_$k|g>z4_=v1!lS8r7m=s6&WI3i__|SY8DS)K zyGR`u&pWevKNiH6|75b)ftVSRZE{1Mx*$f+Wj`)Z?oQbsO9XVAzYN@m359 zZ@la~QKUB*EWaSX{oG$xcs}r)i#RWgv5kM1K#tMZStzeqNDW1bo_z(?$Pff!9wJx? z^$oX0lY3s6jjGNlt=^k4IQ$G=-^5uH;>ho0kMEZ)!=J1A#3h1zKv+xJ0@J|h^ zo63|n!D7HjnHHLVvvJryK?~uuGOg}z=z?W;hD!{uQDSQ1j?_2U8Z0dhGUIb1X zOgL#c1UNrN&lO{%3SOPn7RBNxPahXHPM_~iPTr9>yt+EDHos$WQ>@8iAnNR;M?C+B zvU3O$B~X)f+s0|zwr%^gZQHhO+qP}nwrzL6bK|aNR&SQIt%|6K%>T>G_x1V$i+JR6 z3AnM7v|7A(N;0b;Kd!gEz2b=lQNsh^{pXEZzt%oB-L#hLJC^JEFE};L11>F9Ab{QD zO~Ha>p`mZ{u~?*lMIj>()-S>@SW-ti4OO)gdJB$?aX8@|T}9YDeO_Bj#L*{&wlXGz~p{o(9g&CQrUFKMT6Ix$}V&06Y+kdUH3@*r`vGg0qdY`{RxdGTX4gK1)fko zhv!w%H_XIm3t6XV+rF@W!Hu{~InHH~6{{8x9PEpI0_`mJSD=)k25 zTcjDV-WHXR?A+Uii;yC&kL{6NE%0$+e~a7KU=8vZm8`OVt{PsW=idfT*wK~zL>fc9 z?YMcv6^nKESH;X!S(sKYq0dE$wgU!=%GNorX;E%o=~8w3zSW__CzQbdT-XF!2xBwJyUDoknuA%R*kmb3DByJ$dx0n5vA(bx+X zW~FrFM?DM+I^DY$-wBI-7s{RVnO=_Qn3y+#jeTCMZk`DYyGP$53W{kvWAC$mxSh!0 zz+d7fS$#)rKTP}3t-h;lO-r0Pq^G7$sfBisUeaxV%o>?L{ZtcIaejZ8z+@is#MQEk z(S|~v`rBR(oQ-EipQMfJ>a=bCn2B8xa?`d9{(#mRp29|Pw!jig_YFCH&s?;lTT;3z zE4?lbXpV+W+j+&~)h8X)?AmQxhG^PSdfIY7kd7sDjCQWgxjy3Oz-rg#PwuZe3=+Rg zjRv)?Jw0U*;Jo?VYr8m~m$U!btO>jj@C>6dwdpvv(I%Z#66nR~n87GPe~RJg*Vyb6Q2%JkgF)u~8k=2M+sT~l)2@T{^KCsWyf z@IVRk;9WZCW&`8ip7`qkPtxuqcd>%IrEFiId`DaPj5o8$cScV*Ol@S&0^Zj*8h#$H z%7Jo2P-vMI^5km!hdz^fm|*hmk+qb{mcc3Qp&M{Dnow+Q%E&C-8(SY2MI(A82FgO&c~uDb)*n(!pvvyVc~&WXI9tcNSR#f{ibd zEX_Hsu+rUrHjV)^vRD;LglGqdKq9%r6V#5L?+Y0ZvTy$IWx(O%iHB}l0}znN4YmX#g<4IvQ=-Y10QO4@YryEcAq&w|yuR9XUGal; z!qSr&Ry_G(hZ>_=oI8%`BnrI{qDDLu63(&A3df6Wm|f(Pe}$75V#M4+#)Y?`A=tR( zcVa+lVY$Bnetw-(rxYFBLm`fxj%JP&Fi0c*nCycJF{6i4e6a!#UMS!(R%^bDB?m2> z3Y|Wc`MBih+5%wWga`0i*L`{oW#K|J2)HieuQ7VjUpTd(VR2^1m@O{u8_8#}4ph!J zPxsXU)?3wO;7%>v+i;p5VB(}=V`Ak=g{GVX(qd%4B;lkHyJYIu%wEb4RbPi&--a*t z&ZM_7yeEq0A9V2%+%Cq2aSo|22^h@S@=Wk3fMekE)nnTW>z~)9-~GkTe4U5ZO66nE z7*5|05HkEFzU%QYI#<5uW(A!*Us|W%k>%=Lsp^8t4k@>zWwxWCLLJ^c$Jq3O&?R89 zhGLJRpOxyUx4Z;Cb8=?FAXb`=^l}1`1&R{h#Wca2S5JPJFUKV-)sGu6%fDr0tuTc> z3=$^SeK_UVFg0wN-QZat1%FZYjkpvg10G?0p=&Fo2jtTWe(MMS+2hho(`qXvpj3VI z{R@11F+)2+nro2i)@zDXMn-I}O8a)RA3l4e4DMd8ClZ~7QmGN5dO-c}olL)Yx0Q24 zD1tu+*(VuaW=@%_QAjy9RWvBIawbc4l(Xw*&uP^k_wl5O!AXv0aC0xBWIs=`Q84e9P94MCsv{(#k? z(Bec8sjuA1SVIjj;=6EW$lcQTdtm>dkWa64Hn@(PL2_CJkHXf>^ki-S!DcISnaD%M zVVXNzK@OIhzIjWT5B8Wc&b7%M24I;A%l?w=)}K>O6NDndU?Ep$M0Xf$AM1j6mmg=r z5g($wh~t8C-6D2gx&bTk@}e(t^Iz#et_klGmN^|Ld>fJJhS%J2679g}HrUqugxVQM z^|jRfTIhjaEu4Qg{^0R#Z+(e|G&ItA%g}bGh}AI~EoZx~*ot*quc3bb>e)Vhz2@z; zZSB9^j6TkRU?6>o_&se|wEf2WpVHTX z2$5d{763rr7yy9%{~~>b{ud66W9hib`q25$sF-Sci4qyH&*=P@X0%#2->#>LEpA$r z!wm}}W@aw;%R_yx`ThPw*F$nSoW+=IiYMAH ziG`o{B`$kQ*8`(!6v(+xbjwetlo<7(6Zo-Vk3lIYME6$}Nb7n!bNp1*YbaGKMSL&| z0_Q!0WoCjl3<0Cg5)E26jY)%XW84#5qi4R6f`Ev- zJR}FY4@%_vf>C7ffWCVKXcR+dYY9P#3mhgcPE?&I<1{xZA*J-U=YNtGQ>+pv=LEWf zY7^hL7grzB*EUv+Zd_yI2I_;2b`w0J?XR_>IEJ<|X7v0lW$M2nTf?0u_wg98GKs`7 zXue%8Ga3P;2h7a3_g4>n`?&`ObaxuEW27qn$6FqTv6F~cQsNbI2cxXm4Py{^IN<5b zq{CYNlaYw3S34!}Ky&n|<7Zyt=Ht@@!PbSkc$R0D_cpwHX>W<7YfU-d&|TBgE(W?^ zl`!;%A`0AbDu0WcIVcwnl$@eWNXQESk(BzZ3LxQ-F!jj$A)3aBDg!Yx3#}oOGBVLIZ+W(;atJA1d<7VN5B0Vcu9$un?&=$nBfzI;e8PQqdYM zZ}s3tLkF%G_WMzU@nkh=(3y6)VmDIehrxWg?(bnB)PcjGa*H;9uzbSt;zG2`8 z#cSw40TLdz|26%RaPHy-_A!&?Or>3{^Q zzzSovaT`#&h`>_=LihOKn38OdKS5noHQq^z;FsuVy4qOfWHnvQ!91RmF~&IlITbAW zHRyzs^M8Sl0$-Q6jh)X}`?t!?3yY9flAo>KAdoAgjn%HKi}1{PB$tp!W_5T*AYk5<>V5niEr722x+g zBh}FynyH8rS+cugncKJvvO;H^6bToEim^%^))L3so<>*YkdGM%a=TciqGSc=7~h{( z&~AmWK?vox|Mr2!?x5!|W7Y>SAcuFqfj8v55aYfrDPM{t-**VkFlX#tTFT5hjlXfFzL@dOQw8I2&0Gp8<5TrR8z^aiXh~*CN2R4YkB0|T9WhRiO zKNyilF~WSBv%yybuY>j-&gHF}96Lo1AV0|8*Xlxnet&M&WQOtw_k%l}SA(-jknG4$ z{X1dtlTI`m6>_T)A)`F3A-)vw+{19hQh# z?+SB((DW@m{#UMk?%yYL(_FC(%aGTXtvgk)*KjP@3G~u_$K_ou(o~6oN_dNnD+AIe zO|TLbv#KSLZ?h5*A3k)tqH9RermbX5dK^N-3{q?Jq={%In+P;+m?yTvDTrRvVF=8y6nP zNCOBK#20-P<8l8T){E8~-guy+aH5#3ab!B7Iz`g~J$zmXSoK#aqRQnZJAw-=yTIm3 z@j<=`@{g=)LwIxa{vSEkW>Eh6dWt9Z!0g^~8glA@^uqR!T8USi0#{e?O0NC>&jsm1RK*%oj?Twc~y}38|f(>fd(EK)|jxd$oVgHYl4|80CwHN>sFC z<46GG>_X;ZuAwnG(eCZU(eH7OI0iljiu0Fe4l5Lzr13oJVDfPqxBTo%!Q08#`%VGi z?rxAIq6eag0V{EgA+Bq(e`d&cQ?e_Y>nfn*+ z1YXV_$nBY19+*OtL#;77dDJeY_;|>8((ueb#@TMiXJgnkhf5y9L$N@UYfHNdrE`d0 zT<*Vh+EFLAky1aIz}!djh%ZMF;d7@L`q&8eF(cVY5{u?E_GENmz7ck9Yd?COTD^re zzA{=FFDh%rE#cqX8Hd$s*|iy`sQz}k%2uSR%?h0VizH?|tP>a$<6Pd`F%s+m1|?6x zKja%X;%8`H{*b$g#RiB4=^D-W1{vS(_1mxQjdE9M#%GpCV+0rFnfK=>!Z{j#-4VP;`Fx@q zV=z(-M@o>GeNu@jwqSmmWeCQB7VI>jHUj@ZpP0!N#3?3h{WM%FA-o2hoCTh(|JLS| zRH{%V(;1%fwHA^kZYXCU3sY1XM@t?1vW$dl616n2#}5GbQs4L{Gad$O7I-TA7p8h; zH`-q;405A>nWLe`2?AeXkL$}XqYP0J=31OAruvsgk3>uNIR4P*=g*X4A_F{P>wa$P z>L$!Jyb=Q!X|Cxj7?3B66~Y_~tBcr!k=NEru4!{<9x?FG}{6s(g~&mRwC0C z7(2IXfl`Gk@X49+_H}LnD3hIWEBoM9(VyyD8ec_G>Vc9o@VWl-%f!AWloJaGWnw_3>_8zL9=J9|<^-e#Q@nn*ih7^=diuA2eR!^3gog z4FudS)VUH$0u(vb_~f+T=0rRmuGUy;9LeH#E6hY`$u(7x$Zc0*{|0ZaE0dwDE! zg9Nwegl8iVSmIw#`n|lp6AtqW9hCesNJ~QJ?9(W3Vy7g7wPKB<}ERq9r2 zP?O+&OR;xz;EQQ=kzu7 z_5fJMqJ~CIMKWmE4D_=FoKoZrpo6u`=K2?@OocRtE2aM=wa=nwxd5g=;+K~5*BbIyj<@rMq! zzT#tz!(flD%V6vzO29%`fG&;z$~B0&s9m_t@qSv-QfV=PMSQ zMwPfv=;~KAja#~=ooDE3rQAE%ZS(5%9bL=r-W3nin&uoVEh;1cIeH*Q=vO*lM_rf}^k0t&12s z%o2{cA(5}yGlPRdpZnIE@Fv;!&Zf9K?YEpYw@e!?LY?cDSzpD=glJFu6B!B%76O1P z1U1Lg9WodC{{qNX>i1$7^LoQVn0K_a~ zkZvmQ7%^XKM~@zeFm!d|BY$S`K2|TvqvF=)tc5GZf2lVtg$hscaq7E$DHyoT9!ZF^gw|xI!0Gf z-Wd80%2c+1Yj-plHyDDbJ>LeT&+NH^70~4=sbcpHWv%lDOCtSb?S@*k@O4@^q9q?S z9bj!_aL{QV9_y{VLv|mqX39Iw{AnjQa`!4#+8dqjTjhCW0@of=Eag*%9EOP0+3t8^ zWryYlag)6`5Dwfxr~)7Af{#VI{X5*4;*9xk!%>iUZ;um;T`7u!+wyssJDvNaL6s~8 z*_^xqO$cf&`-=#p_7e6XzKEjp1gD@(P+&^Yqw#>zd8tEZrl;w=5(45vY3;9U79F&8 zq}QX(QIL{`o;7m#hG~m{p0!o)C<^8BMT4uD`%>j+>?h6=#IZe z1^L1<;obD}U-p2eysJ-VzdkX$r4)RiWP=V%AmkQI9xXN#N;FbSrJJV+q$gBm+{VWl zd1@!XPMCSS*r#)bmYHi*W68ez-F=w}OTX=NogPiRpOX8SBrqNrGzLuIVG{<-e?-^C zYVL-ftO?l!nWxHtV%9UK+=V)V`@Hsw$?C#c2JTOo?F1(?=ZePTe;70(ST#Ma!+pu7 zz^=V^0XO?NR|fV;aGXN8Qd@ibsfnbm7s{okIDGh{)cs!l`7^rD3^n_huI&VtmaUv5 zOIH?Pp`|6m+Yx+Q|7|4){sB)xNU0g-$YsIU_`Ak@9x|~id&Ca0t=#rj_1}^7bpi<_O1YR z20}uiWpIU`rSz~ko2%cw<0nR2Rb+CD1!8y0vIkG-!FX@Ikr|%TcgZhp)C(R;hV!#z z#gL=43^RmORpYg{DiP{?({J$y8#5C*wGU6BvpB(|>|rm+F(wb}(uBZo+~)Rdn+A48 ze{06VL{Ey$aaLeMRg5CZFR|yeMx1{g=)@NMNVp^qAdCo|*tWzid@>RYLnE?rQbA^|%-Yy1{e?JKf z1JGU&MZkB?=31!kQq4M)|Vy6 zPOKc}ZfsnNee?V~>Rofc1nGP^(D3Qc>%!asc^|LYOScf7Q8H=!GqBxK$Mou+xx%Wp zTFdp`&yMCj&KuYM?tc4K$0oA}0rn4KKMx2t3bu^P&@()USZ{yCaO8_3>7U%m;J=_9 z^MtBs%m6qZE{&zw#uTQ@*kuFjP*%c6m+g4=mEd(*Z*41qKzBLbn}A+GF;nO(eq`Bp zIaaPVB$<#Ob<^foK4T>~^Y(cFYP9TWHM7~1bF%qF%edfp(&d_RiiPw5J2#vnkmLpd z35Rq|iEfTA;!ScxZku)M{qr8qoL_6d4!k|*xj8!UVnk`((?IjpK;$2)3&(2299VDg z!yH(eZ=7bBS~^-V;D?#_>!%75wKLk@BB$2CA@()|)seb{B zQ$C|^S+$ETt%OMKS_&E5JS73aaHYiJ3@nPB-EhaCPF|Q4VuuG=C24~WNWd0>Ym&7Muj@zpxqy6<=?>-J~zJTff3pQF@l=G4+eEj z_u;>K5&9B>f3m%Xpwe5((j*RaeF-oBdRSe(A3ZVdAIY)Pf!z<|+i(?4%{vi?VXM`6 zTFUgmX@&NvO{W4r{RJkE?UekM;nDWQHdTp!Hg=#&_6m)zwn_EEjp5T)@diqX`k2^S zL_$<*wlz)PCU$KADFJ1x2PymJ0!t)Taffrc<#K`TY{92TMRSOY;ZTh2v#~^Vta6dr zXBG6T87S)DdCy?S1ZRvA^sUcp>0f1k0&vTy(1>aT6~bifJQR}hJv=n#twV79{VqG4ccKHUQpjeo^ zM2dwmud_IDIOD)=ohk@Y^ZTgFkIP$bW?FEO_u=~^6*3CS)WR$J%7j25*Ks#2@UuZt z4`IUPfUMuy0}gL8!}IJ`b<;B69Bco8ILGyy^M^~8@$%+7d?7Js9#pEw9mv;$NtQ)_ zrl}jR#if(XGiwhHc+$%{4M)H@S3j%sGGUyt&q(*gk=uJ1ElJ(*+p-lbWu&v^`Yrrr zfttevv7Sfc}hkVrB456F5mtHMgXo4_G5U~e6nt=-a{F$ko$+_i zWzJ_K#@QVS-p_agW7ZoGv=*6``6H)bphWgyKzJ8Yb6GA~#M}8_dt%!11iZy_CWVBg z5bgpGVMIf&An$stbltEGw#B~Q&V3pi))OiXEYgt101!!pGJc_8G~lCADH9^1<%ZT) zwwDUft12$+>>?pRMn%}(Y} z>LXIq!3rn{7en?JePXA4#ptCQOW|laE%>;`cZ${*LzXu6?*K_{Fgg`7*>3JWXVR+1 zbx1T9qmW(C4`u>|oD9(3^%QDg49g|5UAWX_)Pcw=wK#b6iH+Vv5$5 zu^D4JlP)ubB}nDjaHHRikCKmNW1z*M+|xfCWL|m@i;Hd3kMux$lgI+=yXuA$9mAMLRL{5HTqN;< z*W(~4ex@?TDCC$P07yls5=52B|1GOoG|KyL4p}`4ccEafWygCU)lj~dpWaiBt8||$ z*{hmP&PdB1msG~hmp{22r)69G2*b$Lo0*nmQmj!PwWSYNV|Pxs$PHO0GHEh#62WT{ z&_?1cIA_g>pT1<3Y{ER@sV`s@P|%!gv4uuwxk0wp79_b@^6>GjYnJ@BGN5UKj^Mj@ z`S=dJ(J5ht#+F^r-HW40R*diqCbcv;o0Y)!>5@FrFF1I&I!(X zQ$jL$adl>u10%d|Z*|&!5iL|Bm)eR1N*Uv8U3cM4StD|T#_cT?I!gmg`()oorp6y57eeKmmhY3*<%(y%sjA*sGy=tK;Q{L{gmzqvy3C zD~Em3`%o;>d6+apa)lI71&cv~>~b0#H<1azNdN~!aA%1w4v2K)Hp!40gO4_IR+3LS zGqeIVRI*AZ)=^Q)I(IAPfene%1I@e zolGFJ9DiiY{dPZXOf43?amq@??yLyh;Cto%W`w?7rv7RxZygTOJsZ;c7uttP+~Qe% z*%)+4$LD;^;=b%2g35Y*s*GOm#|kKXy?i4g(pn^+-`Al`B1~88bX( z^~c3p1IeQ|nKrYUM}(VVI5~Gr36|}hR6|62UL}ZplovA=gOe=YPy;~C>&f#buty;eKcPz1Cyv9}T8su?ll?rPQFnZPiL&9fOM8o(} z{gYq)^e?qB&?4ibEjZj)PeIjR*%5`uwyrB`@+mGc*u4O5yNpZ zVxyNbyoMHVK+t6Zg4X+*7m%On?ku=-7EQKIjs41c?bTze^q>+r@s4v$MLuaCe4*6F z=^nM^Who%?#YD?XKEi?1YFbgc{)+f_LXqiF3}Yt443&6T8cvah#3LqNG3ZPjwNPkM zP>}+bBw7e4y`W93T$XZ#r`e4vpi|fc=f>pE4^-R=^1CT5%PVu8s_76|NO9tm>R{^; z?ybj*a$9(oz#{3NBE=8Yp@5-sb5iJnz$h4D*1hbBcWu~>@TPsNdf7aMA(q~fw%k((qzMQMoH6)hvTTPrck*oWo0M6Hlku5dJ~S8}EQGY=>rkMWj<@=0@PjW#Y%3A%*!vQJ0$>@NV*Fd(HHf@23Adj^bc(zu)26@9zi|3nCXiQ#!>*b zE*3`>;OQty{o)lJn8#sE&w+rp&iIi~py9quMmFE3niuNyYu+#7Pu&kF+?}RYK)r!RVHea5R3q0D< z*=ZEIzks89-w1xJT9e9rXbNZs+(p}=!7T3<(`J!0`ut#wl29ptU4Pp^wP5bm0(g1V z`u#d??peBHrDLj4U;8RsQ|P4^Qd^ikb<-euY)a{&fF+HO8N8;-VV%}cP@I9h*JxCu zd0?uPir;De<3egEjTZmw?MbPgz0>p=h)?i7PM`&J6>qHt_y~~Jfw}kp>Fw)iKbSAW(8sCjoEzZ6l9HDgdsE@ z=Z`Lzu@V)8=viB%`M04rk6J_ehwh!8b&b%yqbW!xo}6c=SZ7gwwhw~^ZJiamIrWxE zL}s{C%(972q}@%1Lc}eB^?78`rKo)6m_muh5hC#+x&>na+0-Lr+72{{ERLmlvc22! za+B&SGO%X#?SG|?Y!~%&)gS->c>k?la{u40{QsH0*SNGjWJUVD={2YTss^qq?&@$A zH$;=Yx-LmYGr0cqjWUOXQyDMfpLES~^U=F|m!C?)u~TDz3c!FMR&w$WC7g;)n(4*b z((drxg5NLnbcB7)rLC!@?Z06EhhL?}Fxi2BjFfx3fzXo0LnQc#5Ibes)?y%7WXRKjd&LoH^O?FYt>Ol-7t4NiY-- zu?6)mggGS0ekZW!ApDW$$abRHba5cm+?K!SSP_Sl zvX4oBN_(gJBICv#l2FLX>cVaISYV;~PbrYg7&jrXzLd5L!u!+YcG0&XKIfPy2UuC= z9mn@QK!^4jHcnqJYUS5Z(BYKT)43NZU6evIZiA1Cm2We)7{}uL$e!|0gA||IlaG*) z*v1#XA07lF9D@&fK5C{Tx?>AU(&lLlu_y%;f6vaSs+@ZELCOYOE26KR>D9hlaeFW#M(!SR_Montw2BLLqnb;#tVp#>rr>f zG^!EHovVct=-}(bwgRJ|`>D(@vAEf|M-6 zw7abGM&93;PnHs?+K~&82ARf0GdYw+T}c6H@f^L4+G>w#O-3kS58+N)@_)kkpoeC- zlXf{Fq0@;^43j0C<1HbS#)CC9F%3xo0w9P9NaMmJlFa{w!gJ_zvS|B2BzJCEJ zuOFvR{b+l2=4t)}k&0~KY)Y#S^Hcm5bACDi2?)L+4=I1d3i+#} zQ&scj5;zRr00N%)f+WaF(YitgW6}l|zkKF0**pq@hhvbnvgGr)(5cU2|1X2Vd;Y zNlSAQ)@jf$FSalunRhwdiqkF!)S~%FLZ0iYayYkadOKLZw@j&bbMq$Rdb7tVFKF?e0Q!uGKsX%>sXs(pCT1q*+>8IX86$_-EAy0q_pLl zdCiycvv28Vg7Y^)9;ZEKv?gK4>f^c$9JRqn)O>ihd4@o?aL!uGn45&PK z8|ZIhHB44$lxB7LHwUetPYMQ^Avk_vXc?sdzkn$fI{sa_IDV30l&;PIO39CxbwaX= zJ(^9(&1ffJ6z}%d6W$N`7Q^I&WsoU}akr=n!PV2yUov`UY_zTLEjbCJ+z1o&EzkiZ zA%p%jdukZ%O9WM>gZZ>KAN*H0vJWS8NuSp?YzO$t}^1KnmCnKE@sK3QqfcSaeh@ppm3Z)RnqMuDS%)N50z zkfX^gw5&c`Lb9vxH4?!xate&F4g%e-Em93)1RH5@LGfU}0 zKqP`wM4N_8(Me^GI|A&)*ht_OL-Ckd(t%)Emzf*Qgd=YGfAmErshd60@y*lVxOwPh zr5^d)ZFoT1sy9m!q7a~oj4Y(!Htn=bT&3*ki(uo`7BeQ-O1O@e+PQKmN30HacqZ=< z11j7!Unv*9U2b)ZYUK?xBgFg{QX3;U5bB5i1PNE~Wa^F)4-Zmc7N5&-^Wme^4H;Na$|n2ii`X zM#A6qy5V1YD@qs^V5u_Itu;NYhp8vKyi{1W_kQKcu!GZAU5T+=$v!{?y)JKvcXMGv zY<@2=UD~-MJpNtc)-+2nyZ30U7kk_;jkDT-$&&L-qCF!-uiihb z*qJ|1dJ1MIHT-^)yOgoT0jco&2%#W9g5D%c0OIS*iK2}p3tb<$iVq&i6HM9Imnd_0 z2GAjMK&9C;DPp)8{M{Tj5E%c|X78|^PIIDftJfsmTNzFyJ>PPRV%6VJMXO=F-O%cZ zTy0_37QCcRDgut)OwGy2Sj$|pf|;y?mp4TW+cv!94G}Nr^i4bdx{66ijVKGS;5|8%E>p zDsVN9nzyZ;?CPdfvesEvQC*B#w9#7Ck@0n&%Ivs6J=MAF{h#mwO{&;lA~XQNsp9`{ zPFhk$M#bFN^}pBwwx#0+=ZmLLa1GaD<`qb%`__#t)K-36%QSz2n#cCCy_rbYOh>i%q&eI*YYg0~pi!!N{*0-EI4OmQJJmGrRUHULG4`~P&r_tCUBEXX&fkM=e5VW;>Yo|7R zz`*?Z@ORh8>ASC1A`rU3act+XXxZHYw%<1Pe`NF(O~)1d+<9gq6AOYX2g&rTu5C|; zk$UUWrGU32yC`l>LdbrfJXnKwEOIOx44?)VM|E}&s*%E~CqH05Y~VdAiIpG(8-ngM zKwRSORs#mp95Mc3h<=93)_4-$ksWLyT?Qzpl$_%c7uDPFRbTc!F(-revIR3J6CA=< zo1v4qHx~3trUCWhGxkF_8}HY$F$(|@U|3OqAjkVN1Pf-Fi$l;4jll3M3^+X#avOVY z1h`DnN%+02%hzRJ1Mwg=3pcmB&(m33W^yqT2f-Ks1o=xVYX5 z%{;~(abNIXtbO=w8ZnKp-QfDRlbbT<3crH$=(F!AL~7 zkU887R7L(jf3lN^b?U)gRivDvK>)A)%=UMrOb7{dz}8UhT!Qx3V9`U}k3%`jX3fQk z0JJ)z!~&g&z9a(;I{1Z5Dj#iEU^e0PpjcRQW$o)2|M6nen#+{o^#Ia+q$6&zfvovY zM==0=8wDH}sImB+t0z%v8eN#LXelFDLP1S;ehAy{0I`-HgD}{P&|x%JjiUM+m?g-~ zHK9ownpmBBgdgw1Lb~Ct&wj{)rYv*1+n>l?q@0?Lls~E#sdG52coN$b3Co!MUO|x5 z_Z4zLc5DTOb|Mx_@*T8q;d)Wl0b1~o`0{wHgF4KVTSbxOR$NGI!iHrY2*kf-Y+r>@1@Pg=3TTN%N0=jr7 z1!{{CI%J3VQzTCZ!7h&E=JVdDK7Q4s3sy=+Dh_7`FeEQn9ftz;jyTW+2m*`1p_;f# zgu8k5b~wPT%6k!GS99@d0cCz07Xn)6r^*(x8ly1u56+P5^z)BEbo~Z8E&2BI31X}x z%{}rU^&ew&P_w1E1D_gTQf*NqY;X1`v>j3*czCs{NU^{3@l(tfi6MwXpFW%a`?jFt z%5)wOYBCgCeZ-p5=oydRGa=<4JmeKeg*3ohi1~sT6oEoJL z9>9|W?Cne4nJE3|QT$%4{lLU~g`1xy-^>hIx?hQU`%KTj^$oJp7K)Nqoxl5xv1GV1%Ek{{=+G+f;TO z(SY{=d8%;!Up>+-zrxYm*kileYLoyJHjOrKxJL}IZ-f`zL_r7xJi-)e2}tj7t(%t` zmpC^v+KLf9;OfW|PzXt-9Z0n!RB4eM0ocXR(9@GYtCxtj<_Swl^_zp^_}htR0N|q4u=a% zP!c%k=u4;M{X$I<5%MDme};9XD3wQal-$KSSFho3dYZtyJF~KR2FIY3<^+{2M|oat z>_82Tr)dS=QbtnwHBum7^q%7ziir1To5=h<*_!{-h1#WnGaEIhZFItxhK^^{7C1>wDwkEDu|1LUA^K;%w>m($yEp}a#mnQoV==5-(W6V?Zi zMYM3Dqx4h5$mrz_Ka@l*23Qo8SKDgZr1glW6J?}&;zDq`aD@`P%#?)1?OBr#5HW!< zVQ);n9y(5{!~k}+ikV_vlhoeRc8p^@rPMCjVQlMTbo3dA;$AFEim#+b!`m`2_FCaI zQrYsv@_ekmg8SZUUmkz0WWMu@T?tF7bOfZ+=ry6m0A{<@{;_{)QnFZKuyV3k6~Ult zHE7+)@D{br!@2~XFoY37G@c)XZZbQ^X7j^P_HJ>VIolHAtcKhA}tpa7}$J|1pgG(KtNp7O8w(EyBD471U$TD2J2U@1! zcjp|C&0SX{BL);w4xd`mW#w;%bxo_nbJW%PGpl~T%&Eg%M+a^54k>l}mb^`2gs8JT zT>dJGh}B1DKo~?aX-#bhm#(TNpsr%ETgR2~2zs`UG6|osWl0|N9qW?MGo%2`#Mj^F zRPt|0lZ6IsAkVIrJ}22U$&$q8_!CUPW)^(KhHuJ#=Xxu9)b$Rg>%ZZWqD)d1^6O<; z;~S9%sN2Q-1=PdQl+KG46-`Vt9p8T2F1v;gW{v(IW$zRuTi9;t)+*b!?W$F_ZQHhO z+qP}ncCE5)+g*R}?ufJdYM+~o85wz%G3Wfoi!mMm_DN1z<ZF<+AS@~W`N z3n~oJ3yVHMOv7MjM|Ksb)IH2IU%hzK!UEI@b3_6#aJQ2Ph4;_A;S-hRyaTxfg{KZ~ z$-L9Fs(ZSWc;BfdsVj-2$dRcyNR6VWbl;Aq1ot!#L02)Kl5ytpOMLlSPyHPp6!^6q z&~%h^Zb^$DI=F`yKrP4jvrRJO`?u?kpNOjyEQfveJ~iZJ=r7Aw4a99N*Ks{X)=Vu7 z*%J1$3JMJ0XN-Q+8%iLeAeq->MC12$$UiUO99^DdWdge#6s;DF>E zlM5IDYW^GO7*ME?=;h{m@AR=02$=QTDaM0MXcTX5k9X!jeaJUI+7KQN9xe`s0OEM$ z)(rj;yxj6Alqy%Z#WHn3po6t)x^(RkMXq<-QdGfV7y4l!7>3(!)a zDn=Ct9y0zB2=5iALmuE)j3J{Lb?){`sKhg~r!g%HLE)UGb)G69`9vZh3#V>Slw>KC z(WEef>xWCv^PU*sE8kIW&_Wb_-Oy|JAwqz`AH<$oNbrjB<>ISGwZO*7`!Q2^N(9DM zISwE_VRF{zM|s@;tRnlVKXKQv&+!LPWtgNTx}W<-*@b~IBa{;oqA$^-svAj|X2*1F zflup{mX-bw$KVrjZ*ZrJd#=^hf8sInbb1Ykh{CAbEZwp?PB#JZ$3gSWorFkW5 z5t#^H3z2|{;47;4t^ga&O}v5Y>zst&4US+Q{$Y)JS5j3tc(I425HkwQD53@bfNNJ< z>TDhc)Z1yvj)f{Uuaxx|&=Ukn^SiZRinCPWPAsNA`*P5D+Dc702q{Jor4j~GvX!~p zswqdo)7DS`U+MBGzWBT9`W&J(jTGkAkn;m+6P47{vEmzZh3YjnEw=fms!=uC-sj8V zDR?(HK5<F@M%L09yD>pAR-Kz9sromv%2Y&1nYHxPVA&>c7yo+C(Gf zyQC$HC(FHFGcGFJK%>4J`a@wP8$9-*&@%OZhC|*VDyJ8Dgow>qehS=C!)4eH$;yWh zm<*ZZ2+M*TNtig*@#de0;uPqwnzX4ExJ_~?lKReQ$d!{1M<-^zpm+WgwMGrS0M3oV zl^}-Z0kn-S&VTF)>PeZ^0`N_t!+&&6PI!0G6#a9>&>PWBov7ToV-pgdrOS}LjLjMr z*LL>n#e8P0w?_d5Hlw~<-*1Wfo37P!a`X2C=w=1SU&l!K$ILzC*3RWU$oCXckBU^^ zeF813scXQb0rIIVH*kH5Fu1|ICDehoGbm^zml9euzNq=zS9KmeFC?ustRfd57B;&t zp+n17?l{YoRWp@&1KS%D?7Wl5vRWo6esjp}Gq#&{3JiYYlRX4m82@0F9yGUawOHr5 z%^6z79io!mAv+QVvJEX&>36E4T-KzG+Kicy-q*kM2i~IU#KCj#m#ZvcMfL8_A-C&R zNUM_AGQpN(;mQWUR}$A4latDDb~?fIIuLF_Yo9I9oE4pt9}T5Agx7_;ZG;|PI%|vz z>R;U^Rmv!T`6q1492-gqFd-}{6#=qnCp;%*?1LZwv|G?oYE~B)5hdzxOc!;BBD)Z# zx+KpD2#YZ$0Vs}`lCwYwQOkQG&z0M~z}rm?&=y>o5U;FU%wQ8{UYA%rJPephbHc5r{nL2mGufJl#e(TqA$a%E-zFy;Yq7-0cR z3fVEP)9)zL&E^l^yLxhPQ__pr-v)X-6u-z9@-U)T;a9AG%WRhCgM6dO^)B{*)CE0)hG*JP48V9;_LAjpeE zCY&^q*-9;wXI>*l2UEHD>@~-Lxp}NAZbMv37z`aFD5QsMN56qbNtXLM13oQbB>*vZ; zxl+IsxT2JR*i0WIJWRFWqBXAo|8dgmMCCWTF|tYL)phSoqaNUVd*jO0>;An!g@*~% zHpqf+BmUWzu&0=&wcpY&XT9Ipl`wY+x5NxHN6T{i550xfepo7D+yP4|#NZ{3BgqC1 zD$FHZsv-Yyj?NZ{x_FJGMwsuACGFp5_JKdM`wJ#Rw<>(jA8Jb<3mt!JA5#OJj))si zQw1OMh0a^fTsqzNZ1H!O*LhXaW9b%}If~K=7w^TBN$4Z9k50#&1TpF*9b&&*218r5z% z?I(Yat(vv3FXg8X-yKt%>XT~bPdA6xho-KJ?!#m1mXd9&cf{7?$@TNpwok#$x4suJ z)gxP1D&1ehvIL8!r`!ma$GJKN=Lg29G4#*Xxf!s0sCHt0R-` ziHXkprj#{hVb}Yiqvy^4Ge0FF=Y-Ay1OV{=#o7JOtWb8p2yT5- zV;U0&eQV?YEm(?Ev>glRkwb1is1~%Ybof5=u4TZ|)(SSPOYvlix7I4N!Amn(K*@#_ zzQ0KQlq=hYbujk2Wou*B zB`6h2)CkGRjO+ptawz}uk`PZaa$c;5XT{ydNG(Ni29b$Fx>Oll;?=ueK@TWN`Hv{?v4h7c@kfoYdlb+y%Md;uff!X(%Wr&}w0?7t>BfH}%Al3O zA)G(TU3`=~I;>yi_mc}{t8{8&Qk1a0K5V)>h_~}NnC6iqI)UQGG{cF=6C#5j=@&Mx z<5Vs(rL!T;iUGdZRXD(&A@k`M-M6Dsx+xP&C`Oea;&NMU7%3Z}u|jgf#ZDOq5-G_B!FP^H;PD-FPVkEa)ivA^z zWiz3ZWJF@<0`1_Oc$A7{cTH)hb(+187l||QZC;Lwq1C+AHZ~!y%scUL!I&1sf7;fn zS4#KR9HAbzq7ae7M{;xpqJ3~3&?>l!7-_1--3_pHf=3S;SEgxc=tZ#ti|T3#i0VXnY@oai`sFgLqe68_=eD_zR zulV;o?|+x)I-=j~PrvfKoCE-X<$om4|L^_o=xlAR@8JGl5?x)}aia~zd!@FoVX=nW zFg~TaD0@g@G$nh>2vHM>YVDZ}I>+J2bMj)^o}dw4biD8@~-(YdxmLchdl9ToL7hP0}3^ zSak*WVMODEElH_^3!*PDJhu?UA)y(9?lZ0v$%#>b51H+%8vEn9Dz6jLmO zgsc0)jgTa~em5TJQ@lvPfrgMJt8vzAqVU%dtw?~P)tK_5`tz#zu|_aBsi##ZiO`UE za`|dYw?3hEonB^`83eS|cBNBDZDmR77$)Nv56 z)9Adwgv?c{{mJ31*%?WkRoSS_gx&=G41GhR&#nN`xc<~U6c5*YR z!>LK%HxNV>9A)(OHQDEEA&MTSXx<%RBOgt z15BsYR&!bpunc1h_pMBqO0hT`aB{N3R}M}|)u`)=7=`{m_(2*lm z`NBjxaN?MJ1krAmrt~Xgn}sSyIDyq)mh6j32Q4U&Fuxa79qLwYc&^WI%SjE-o1&y@ z!s_e|lj@ki(z$+oN+QZ6%Xfz>Q_9hujE@wxqL#R>xQCH97Nc!uH6mJC(8T`Vom)bh zNj?uvZ93TeKZj{%tavC^kQ4C%j~3J6((JJ6HmK+9IS~!D=UkeP8E%-*q6-k8r_IN`!ezgoJG{8V%;P@P%>35y!BL%K7$Bf6^)R-0;VPMgk) zJ-k>&%bt;haqVt^WfsfzKNGlM`G46a>P01&uzs=82;H$W%=JL%m@${k~E+I~%C@YYZtKBNO3yGwc19n(8Q+qi-&NYXd zV#v7EJ7Zm9(=hlA3ly>C9J+)vaf>X)v2{wBT9K<|oVg(@A}|=t@FFRSRdz(t&sf_8 zFtYpmgK)Fae%W0;i=~M&FKmdJB!YIb^>%w~m;8x_y!7OP_J_+c z@Sid;y8f!fl|i~P{a%9?*MGi|p73GFOfdL4WlC|K+*N2J1Beq!Y0$gpJC@M( zpsrv;Xn?soe4RB`@_<{mVQ6>w8`V%^tj;a^J>I2x0ck`j@Z<)d&7v59LVjJ^tw z3m!9%06YqFT}ghWed|Fc(jo5HF0`VpM?(nmjD)yet6F{n zqe7k@$Xl_c2k|r*OnHJ?bsXfJeITZV`;I*{QvTF*7_ek&(z`O@DXN1}uk%H*cgxj7 zLe|^39|!QD@ak#4ed^50@}AeDFlX8gZVUMYA(H>Ot=Y=&*Z`U>|r?mGfO!(ZEmk8|74_#1g%=^CZWuglsm+AU;=(QGY2BRM#!3?W9rIvAZU1( zC0}#gR!ftx*g=Ck>>vK0;WW9X!&4}uwZd2cg*O92JQukS?w@GBdW$4+oE^IMYVeIRy zpf=4y-p|TKD+ZlH=FRH$L9Tgvn3;j%(RPji%4}U54^r_RsmM-o@BrUryslc_5W#Qw zb!Um81;p)GhHWLaj4L62Bj{n8lFd1T&z9ME{a^+UYav} zwZ6XXe`^t^Cr!GXo%yR;tjC*kKsO@E>UJR7TE7*B!ora}Bh8E?n}h#HNNq)Iom7GV z0N7yva2LmTWku?VeefUPU_XIdm z0H(e>@gHN?_nnuChG>`)kii)%(+Q~nnu~4E>y>zFT3siPtKHgf7!NeBHo>ph71vhl z!#4CE_wA83B&W;364!M;f8uz9jPNkKDBBKLhY^j@T85n2lkLw-(ygbHFYsx~zO?$mu0u_V;tg+xgQQNQXQ4+?x5pCgy^a<9Y(AuvZTO~gcz^Uhg*k)R4%cpx0yf;mUFxF2R>-D+hT!n18pT!uhv4D zNugu#{?O<-1~dW&_uD9%SYg5UVpa~H@gyh%B>^8Vn$pK3hWnc0QO!o1>IPB8G&#b)V2sAA9kb0WzW7hy?LhK5UKb-p_lwVs0Nhtd=c#h z3!OtIjHeV~nO+}jVBEl}FMrOv`T1*OJ5+OPYOfnhSu*ngfUy@1F|`-lY4# z|HfNSxF6&QQ~`>oCQ6KiGN&*z|IUHjz^o7X}%HqWOzwKS~;$B>tA61N;H23*7{s4x%B4hUzA zkwHT1UtC280m@@}ebqRr=KTf9>?34(#2IjBlv*-uKUe)!n{gy!U?g48)3vwh>>enA zc~h|hepjS%4Z-wIT7-0S9V~C}(%FQKWyR-VfF2YXbp&rq`{0Q()>2lvBGEek(>#<# z-519M0P7`#eopl%Uvrug5Fchn+d5)p-fxW4JAryJlmMCJep#|nZc-14%M=NaJdU_J z);>K4(j+u8bQ&E;PV&cy5^*_3#MrZlN)@XIJW`?C6yv7~(aX|W96Ogp*;IXql-G!n z*-Yx2yV1zgJ-m3*_g+Yvhz=P^lN4E3uM8wud@2FK3fSKEHa#Ro>sw8`${|6Di+`6M zNcAu*2)Rbi1}8#xn0Vq1Ky4LTV<}GDZF$12Z2|^~$2Quv_RoVb)5Q!Se4Vw46}{yV zq)94vSM(Z-(gnc%`tg+0^vv<({l_4@Jg~f!3u<%zA&14_*&IT;gEz`(mN8bg_FmP+ zD%y;u8rX@^9UqC6FbrCzwS~1_@;W5^JVWMC3O`E4NHa*D0+hNO7IfnBxDAftLZc_T z#D3-{55Uo;Y~h%d=nF}zS50<>P?OZT9pk9 z9kgtq;9h2qXYE;&K2u}gWKQCnWAn|m^{Ja}Lp%8c#<45K|08Fbh2>tdpT@*_RJGrd zv}yOgsB)jEcX6uw--~E^Q=HXUPym2|-wd7q*^KM|7SYa*|L4`OrfIt_hU{~qn&n(< zO$fqCZ0?#2SP8t4iO2_WWszIa&*O)6P;W8<-*oP~<7g7)4`7aIjZ29_(VxTqGRYcR z{BogRK3f}yzS7o<)>&2^v_~tTLVd=)&HMJiw zaN-|z{8%iAVrrs{{c)|*O)6Rf_!u5qmpUeAhwHU`G-@8?y0y1GsiqnZgK|`=uM|V~ zl0WZfVU73QT=h8~L^$gOY0k43I#6c)=YCtQ#x@zhE+0CJ<}qpFo*|-iI@ahUx{+rY zepmwZu4=r=3~wq>dOX>!!&nxwg#<}Hj@}& zK*&rQAwgYKih@4!U$FrP{5e!L9VINbB7WLf@lkM3bmKXsc{+*)8ULc25fN0DrSt;M zgEx$BNc}>{=em+|`N{Zzh(TLaGexqK#9K{Pii>=^-B{q%aBs!KxKLU9toQ!k$SO5M zK!e6Enq~Q(nldi5iePbtP7z2r5|?p>3e7SG$ojeY(otuS$`Ro-3g~uj8RVm4vf~TyNLwPU&?R`GaC+f@M7@@ ztn*aHn6?!bk9y~~#-v6~q*(xhMG!E44fYOY1zz{y(hY+vvDNJjBg%f+UN~|6WLa<3 z%MdqRiqn!3thzA5c_XoZ%EsE<8Q?d072#WOk|gL1Xi}cwwl?ut?8yV%{5EXl3x@Ch z-geXgjcGJ!HBFqAc9U)VGTH7pA_JN5wjccc^8O0z`=HPP5r$&A!Fa3xFJ6`Ac9$o` zYCxvHF2(^Z``IOGBf!B+D6av`2nKiH91%5rr1LlR2(034rpw;ias!!6C$dKt z?@^MFA(knJw3XQ7{=MoQQjjNH&4|ycDLzFriQ8j0uDxS_rcFVz;yDeO)z3i!2L|a( zY94WDfGz=oB(X78q9jA=j@j7`I#4gBXtA+)Uk=FJ##c568zLYWfj+;8`%d)ZX99B$ zq{P)8+`f+h2JKE-;H^qDl_@4uioOsev{1H-yu#s(@<*3FGtCTRiZRS-j+Z9P8aslE z;b!bKJ#8nifFcDegluyujz_4Kv+s*R_#dqR*En{bf;Zn`k#)+I4d(;`2`RRmAOhh_ zBvTFU1b8N=RimfXzN^6NAqVwijYy@MN7+3jqN?t{PsMzLbj!IAma-~x1&i&(!>d0_ zhw$YF0)$M6hr3iEAyi9;l*45Mk$@hKOeF(TN!at^?f_;3CzKfjE(7~X)b)nnAvcrL zTk>rUJ&Jhvt-}1UK+H@-PJ<*XWRZQ`jx4UO18^;U7}aW45D*rVq`#0sri6e@BsM;= z;_y(0A6kr*BY#Dk^b?80(OK18!>sBKRch`-`1rmgvt3k%!UQGYJS!%5vSj<$n;iZ^ zxjppWlfJ~prOaHft~Mx!PTgQjtj=r-alY`hepi;`i9#Sv3P9fpgy!ri42Mh7FR?tP z)L0;uTP<__U*jNPxs5A_tM5;!chSSR#H+5Bw*z1;O_(!HQFTXK$G03!%|gFacvNg^ zxtIg>UHl2rA0%Sx@Rkk)>X3v!8T-$=Hz*rp0iA8YEZ2{(>!tU&E(QX)A_j+sC(1r+ z?q`;~JfDGJzk^R_vfB%pU9t`&HI`9XCmK2QFU2kBOQC$4RG`RCf9o+-KHWM#MHz|4 zd%XXCyxm4jA%FiKZQS600Ox?f==|(Ef1TZ%g^+_6-gxUl*?cl;XX< zRFU09C3uwbKoUbJeQ~+6kq3?34X?-8AfDdl{Lh-Um82e)A#ibNug89smUREF6?w<{e03_=c zAyv14?e74e?^w8-pXa?Tx-!?={8rvx_fg1u>79Wq&Xv3Da6@h8wYTwM><^BHCw7}9 zX!6Ps(wHm54cnzKtFP&e_YGaKZt6uJ0ooRgzq7jPtC{+jD{;LVzvi0W_v!GHrMao$ zMV0D#Gm&u2M^Ks?DlT`+l+gw)r>0mk&(MMxdOY49UhyY&42pJmF0;Ef< zqN)&ctk(-YkN3p0C4T5my~LCo+B;d(vL%5h*58Z~rKC?9Cp2bDfT&4!_T6MizVa^# z#c$^YD}}L)DlfK%?Atl|+>P8ko0qDQl2GWy#Zgo>h|4;M6k5#AnaI$s5#dDQ5HDbR zgB6wA(Cg+IIl;hKF|5sepOXh;Q}j(^l|bmZp#^R@IeS#^FN{9$DdUS-!!4ZtCqMNt zh`-JWVd?v#H{)9Wf{XG=b~%h5t#}le5$bEf+L`vA5JeOhpSSy%6>}9)N z477FF1uRssH(C!P1{!gL+Jc?@6SEqt$A`vCWP{<*n_UG@b7}MRt$1?*OqXVQqoIYT z#aqq>DvsJ*j$^jxYnq(k7U;F*Jv2wl82fSh8>0YWQ7rL{(?pP^ByNORonfQ?$oMSM zFk8FCE1}w&O+gWmSY)UUot?&rq0HFa8^=3)6ygL*gUnKtr#(AxcOuDwP7`;A+>Db2_K0QuX-uB@ zF})R>y4|#TG1RwlbbEhgY}fnJKee&)N23>`xnB`Ub0mz_o)5=C0=ixq{@oz&)nT90 zhIqRejcHu)gvT)`!S>7@@`i=EH(<}-+?u?nCX+w$NSP*XTf32uk)Oe4Cy!V228y*) znil_Mq5E6e3lHaU!Qj43_Ga7Z`et5P^de71p;ShMf3@10c^T9E!@_z28pJuH8%9G& z!VNh_W8e>9V*Jej--WeRY+tB506V@}BZm+d{S}6bv~DYzTX+7Rp!Na}b64?J8k^&? z+U_x(v>8HINe}&5GAYIi3)lhgZIE-cuP$Cu_22s#s4K3`oO2{84KEGQ@uFo|I4o-3 z_C4uZ4*q(CKZN&!HsFEO&{G=VJT~F?CDU?;IQ6Nfv>3|?c<3NaXp{BEGHL=zbIPD7 zh1@F;f5uqw?=Apo2c(eW0Su1)$w zQ(MGGgw3E0p`xsTiJ%{?I=&TJW#+VUr^;R_Y`9ov=C>fd)D0-Sz;;e!j?%@Z>a^(&H?j+%q?<72r7P%$!NU?^!) zeB{U&9NU`g!o;)ALgm8P00?UqAcVytG5A>c{RlJsJ~13IGFXnG$@)H#L(j}A{n!dHr#Js*7mr>DOu|-xJxJ66QN+(K|A1|J zftDhX?-Ux{-iF(SOPosX5Nu}24f16c((z>+wl)Pa`JLw=E)gwG)&p5RNSWoz%u(1w zW@NBS9O5BBWfkWpf^Iu7_E!b>)kt2M`bn$)OQE{VF06fo2Mcsr^PzoCt(m^+pEgna z<7uvba+;3I%SnnJ!lAf=JjI0u2lI01C)l|U-vXbKS5OYW|e)HGylXc z7j3ITycj>@hus=aEK{uVEL)KkU~sRN{}ZI9iIfrAa)%=X;>q6JnAxn1F5}xu`&vX7&F~!|z0z#8 zUpTW$k9CZ%Rj<#4-Kp*v3v0<-7oN`AlgC`i%o0cstefP@x($?*7SU+8&s z#%}eEkuX-gpJKp}MFiYI4+K_{rG|T(rn+tQ> z=f4Y!!V9VZ%c-LUdE;9<_eH2Q#O$PLdr$8=@XsU|-?DqN+q^Aqv!?gYHitzmIg3HM z!p?qYU8F7MXNNO0Mr_!SJgB!?LQx$6(XBsIl4gnj?z ztMj1wl>6puRSu5cwf5WtgAHp3CUZYDa|CSEnK(9_u3EvnWmYO}SLVHNHGm`qi4E!n zyy#ocHgukI_Q9dbC&_SQAEDPbMDKRd0T|gSG01SN{X@)L)R}?X&K*B%Z!~Ft!%dAF zN4C(+wBHLm_jd962b-cLjT>j`Ug}vdpE`dzZ4V9Vr#)y?doeuI^`zBD1)VxyKAsl z)qE~~=hU;VvBWnxqI{p1k5XFS#J4w(R&bzz)2qNTcT{&z!%6jrhEND_Go0;d&J*<+ zCPrA36>c}f?Ga#Z4Byc$SF5DubgDehGNZR2J3mm@Bw9(_QN?;GB%?aJ4)#!AGOMhz zMl?9u%7Vag*B$Q>W@mYlewV2)`w6AD8>wHN20hO*G-|P{y+&_2B9fAC4uu+=9aMoT z=*RS(cfW~+9T%mVwFmr@{%?dr{1H{)o#?IBqrGM^8{o5O-xiT)F89nLj87?r*s^sS zq)SwlI6jx0D_`ml3tn9VNRBm{Zwy2jOY8D7=Nsbq5VM^A0t(|O;rH!5d(`?;1=|kH z8^nWmj8vURu0C=??ejHj@4vb5cb&;=&-Sd71WU?Rtyp3ydQDhl=mRRKl{xg|P3an~Cxi&a(_v1SDbg8BxZui2{W11kS9>bH6~&hP=&Fme0;x`%G^0jrJ7RSoQ4C|$BKI9dPsLf}b}fo+zy}tE z+<6_SBNm}idbU!rJSWNsz5){E!xt&r$9Ks2lVqB6(&!oUCnu{ru5qMSiFVE>vY(Hm z2V7Q+f!TW=wn07g@eNc;bKQ0fW*6}qrJE^BH)g=BLCNFM8o^40)Mk44f-!`LZUu2F zR18E_yj-M+YDiFPPRzn|-QUp7>`yWr7g=)|+@bO3w6Q70%#r7{p1`yFyTC0v2Ez(7 zIvEM)tVMV4SCmW4!{d79vh_bV5ABeWY!B<5_TD(4Tbw3_mg$|@+cjiLRA{J7DbtaC zNi_+K4=vJq=B$i61@7EO*o`wN;!$+5&I|2>!4%D^_T ztv@O(i-fXLnbK}nxYS86ESULJGHU$krZHxBaBF1L`n_`h>tI*QsQKrsTJ#*l_OC~! zm53tiBscQz9i7Bkbc6qww;Q;-bh;j+L*<-EA}$jJ1J+to7x0$+ge&Uaw1DQdr*mr& z9L;L>gb~PNb5U~VrHpX{pojW8l%+*GVo&~ejTh}34DDatGo3Xb)$PkjSrL z?hZexc^0Ms-kSeUfIweJ)GY+!=%9=o2dkaKKQ5FRDv<5T^PYq`1RS*m9@a30Lpl(2 ziT*)G0rGKdIDYY`PXGz|XC-`ckn>5%d}1ddXp$?(K=Lc&8UR6a&c`Tt3C2-DIR$Q3 zzl(WTWW6?AdIF-T3xiQX2b{o9ZU^K+ap73(VE>7AB2h%q%Xp&VhfaH00vq|f$ST-P zP9BT+k#>G0L3AoV>8OA#1UC6Yx4CeS6Hq$Hhhl&Zas@S!T|)nZepK@;-*wW_4Mhgq zUu2$fcH6juXjc0;z#*tYm_ads5COc32mnaT@ z5g)zcG$Ts-f?)6Q%svbXB$lvHg$?iNf!{M-86}IHg8i8)a8NG%6Tjo{m+d zqYER_?c)+=wthe$q`V21H^0l@={cvgH`KTRQX_X*Ru=4W0u~EWrs|IsX`hS2nDQQG z#vuH_O92spTj3^$I3zvwo<%I?J2Xn{-|@jf6@gS1p)X(6f3=HZ@`k5T`5teMr>RMKiGw;Tf3P9!yg_y2Xyrw0b|{wh&ncbA?Mg8*H0Zu3K2UlY9 znkmYe?Rqi%xpBj-#!~Ftgl@yo45~CV6SZ)Mj;2vL`JD4Pha^LgEv(f-O|?{f<>lHWrA_ zm_D3OjMKqMAt>^u_+WQ&O<#(AZ4FP7dKoj!0So0MPv;3?=^D(Q;!BN^5c|Cu(@=~P zTxKI@u^)iqi9b3Uf1phuimh$#LG2>6{A>wZw}8~TuEKuTVNou0vYr3IONgg7OebP4 z2Wtvjm*S}L9BRqcMmW_nqlC+0H-ML50Wh*6`pl*jV+jdRPl^`RN~N+wgd- znx9;P=hTzJD4&j}q-|&IsE^M6d4}FhUMIAPRjLUSIKum#XIQI=tA{;9KWo5A)+~rP z2p!#HN{pDfs=={Hy@u%1s*$V=)Sa06p)*wy_UDV)RmJIJxYeOrX%$ZwyB!_2_%u)SZf;=%lm7S*Z{J?~ zNxs_Cm-IXYs!w`jwqfYkDd`k1*lY!ycEl_*QQfsl2F`tF3jPNdaV3u>1=nbsb*p@p zs1g_YpxHG$_=trvpbv?kM_n*J^9`3Ds%^UPQB8hlf+hNl_X&h4`rd*wQYaDnI5*s| zSv8~x<5_(@VMe(Q1oe_U#jM(l*-S$TPGB^B;|)s!&hx_v{SUTKC667ohbL%zn!kJN z-K8HMbHkJ)ebswG{pw@E4u?(Y@}J%f{q&POmGR#6x8-tU#Y!6@m30Sp<|?B$4R`xse(;R?U}sW>+`vPLURi zWWKlb6t}5=hd*EVSI_Ggy7~R0sML_!&P}iy8-+?@GA^As7Q}Bye2bv1Wjk+PYZ!zK z?PfTmFWK(8zNjp8%}*!w*D z2MVs8s}0-G3;>HHse?-L5X|o05ZkENuak_vw}(+g@p+6rkqoj`{rUnHa{Ya~zax@S zkuAXP_9H3FrUple`zl7{=XeO!Ha&yAJ|d>gYas=_^zY02J#s-h?FzKpJDr4 zT(3n5?Hcr(2U;JIO~y_BMC5|tt?F&xbMNe3kiCiP;ivq1DasiB1Yv6nu;uRt3Gpue zo@*Y1US+JenS$kSG0eYo{MAb1J%KDiiIm||h02RGkvb#WW`l1lN?{4h)}|5?dt7H~ zs6gZ;Y9pw&`x#JpZ$q~xMDSWj0C z-iN1km`h6GINsZGGF`W(>kX^MeumE0$$tJH(o6%DIWXt1wZe+ye`>Fgb7z(ib~7}# zb27KJ`LAVjO;6baF~px-*?G$%&A6~>%M%&uaZ32(5ND2W{ZC|dn$fqb z-GBBTHnO^GO|CC-llJ5RYo9WJuA7;nO&{m-qaX}E z>TR7_Vtmb-<#MNLZ-eqwpg<3?h3whPHHnj$N`OC1_IJd^*kZzCMW7x*cJy=I%`Rf> z{uIpTe5rcA9N(5&1M_a02Y3t#e&Eer@5I3Ra&Wx*&4kdd?`(0-1?=OO6gf7teLh`Q ziV}u@ug(WzjWQt`BJX3EYx00M#f=W61=(*TKx=q@RDZ!CV#5q*4nbIJ66(kimIMu< zMxY_Uj~rTFx(TS6O^ml$ueBt~HY|aV)L6qD)ft_j8;efq_W2GR@FUYTCQ4k$R}x7# z`0pY`&7&IQrckabq~HYcym6r0exZk<#v#DQn2&N1@Y)o*U^PFt%=@C0e+s1qny{=g zrjOHsD)loi&sx0aV#O3jotVMj(ip9&^0r-kLIQ+hy5Ps*fpZn?(%p^)kYGo%WkL;H z8H&n)@<<*o3|>73hSkM@#y(*hXwr1)&7mT>c#&vCFsG@ z_W?REk0Ovh)MFR_FjU+GmA5pvIEFpWoV;%JMiD`Wbl3DEGuF!RN-rO?FjPYptNyXFKq^_$6N8Ca57b^$bkEhlGNy$ z5+#I9Ot3nwsTXcRZ(84K(kw*GX8e@tOW9jT^M*vZH&w751>kU_BEqs6L#j>3Vf864 zk>S z{?gp!27Jf;l=<{|2;#_|_CLn-o5N7vxuv#E%LepOw2)t|p%g1Z!rLg0{Srvux>UwS z6oSH2X^Yoo{xUlMzKvQ}2G5^F*jhJrVW&O_d;@cJVABvOhKMGT=P#?U62J+Um&^X|4w**7k0T$E0#-MD4=;(tX<>^_x>(x%Yzki@wi(XE0(br4WQ&3R+x?TNGNrW8BjQF+j2R zC$AmDk)=~UV^fT5z2T;Ai-Oouy`^fVxS);fmGb;0&mP=N_~RJzJz}2dQIbLBK~tl0 z!nsBR5r)4WF!#yx>4~+vZ*uX$!`ovX2UfoJ!bRRwxj)4kc29)_tdp@&9VcB&L2B`f zdA594c({f50)Sb68u_&R*;29R?6j{(?^LdE9LLfCI24zn6mD_7!Vf{*%+snw5x|~@ zt!^ebu#hjBjWH*y{}k%Z>FT3+Gd4Ma?I3P+?;hP#$mEflyjlh~H$#`9v1Jct-kPJk&p0n#QBg#oLdCipTI8wi3D-{<4M? zvh{U+U4}?y1m0ZXj;Vp`1FpXzYym=fEtKDhT8Q+f;3a*`ud@#-!hKaUjAV(y-pS>s z36$Lc8b|0~L`DzrFp2SJDCduE8l7tCH2If#8a3XCvlzYJTnCK&!qXI~o!W0;Ex_62 z81sFLHNsETjts|2RGkFXc<+GS>B1_#O+_KWYAe|qCv4|dc~-RhZ1C`n4-^mx=zeyz zBoIOZu_Ow49pG3*Nn!X|WfrEk_`A7F_Vm?J*&0wTNM>lS@+~lN`ayBf#H$=5b)JNc zAm{zGbimjns+Ddq5d7MdYW|w+q`?r5t3WXrxfB~VDoA2IxO}~_^bCL?ik67Mh)N2>s8z13h?SL`^_nSY7{%vwLjMbZfUo9ox2TR8p~R z+qP}nwrwYsq+;7n#b(9Me#Us`UVE+iX|DYT^65Ex&VJpkx9gyC$6*^Nz^{r1mYSMd z!fwUXh8`G`8!5IYYOKD&7^=A#7LoS*3`$dtXw!-z907_0*0(y@t&ze`iK3uf2IB9J zn}t)LHgX9MI;FHJEhBgxqw^!SgSE)=g`6x{`CkSwz67RKqtTxzH#3c+J|Il0Dyt`p zMpkQdaJIJUcS$iigLc-qEy*9svBtd+L=Pg?U>TU7hEx6Z&!B!zILlM{**t9NK7+xp zCeMoOI`zqu>|26tyjSk#Yq^(b8ooWBVucO7V-DB9D@9xE-0Rmfo@*(@(F2rP>G5D& z=8rPpE$6D2Mo&^DeA({>T$jZ0O9x)y9A)Ezu7Sz2^#&m%*|i$l>}D=vLxUUh^w6w> z+MsR6m?8`QhVn-1_XZKf_ViV^NP1ds9Rw`gJ75{#tQ-NERs+wlTxAu?N%%ye&O;eV zHTzj$PA$@Zl6Szfqr)qvjjtlUSaRsY|5+4j5_o|jx+mX3I<&66A%dH#^3;=?c&Wj* z2g}_=_0=cwH7z)9nCw|N$k|MGST+y{O&HKzBxxJMo~( zHJYwgV3BFVT&z*SimU=N*@`kvN``e&DLh$j`A@WoHwW3qj65NvpZ%!|HK^)i`v%<> zK~l;!gv#j)#|;4%xnsv15!nGr_p5P`3-gEcMjMGpc|bLF7IhzbzfOt>6_IDVPL#UO z7Py_ePB0mLXUTojj{CJ4-p=rqp_JuP2fyA4?tNn376DzOz;L)fKzfPGdk5e{t0W04NS}F~H144c9}@>Z zA_tQLFAx}_5vkP(IPn@e%$he=!5&ISd#DY}`1eeKWnaH~`rh9`Zf;<^*Pp@fOm6Xi zvls~U_Oji6t@LUA!O_qpwyyHf8oH=zqMnTN>{a}Yw6Xs3-y-V>Y%fH(0F1c-F!n!N z)BX>}qySwV|5K2mlSO#gwQ}Vuj9%{&;BBsM6DN0V-}EdaZc1JXfEqqov7i6e@Bu)Ewtw1Vmnbm<1@$yN zCe7W&8lo57<>SH$!TOCsGXhe)IIT{a3yV%CDmzsWl=Tfxqz6KL_vI#pwZB z*G1Q*!{4C<9GXUGp#z)n`;FG2^W{2G#LD6P6jr+&l!PhjHeiXKFhpxCnpBWPv7Ib= zqaRQbkPL>ka0<+fvO>==Lc9k|JRoG015`+~6MEoiR~V4!x7}n*nQu}JE2Xo2K4CaM z!wmCmB$J*c}!pNeAaMoDAQ_|BoyMKYCt&!C+3P5D>4|6i#@VEf}sb23Dvn)rN7OEx?MO%{)Y2LqcKYu6!F zI~Mr=kJ!~YPmhW1S9?|BfB`lUzfRPT#xb0o!tQSPf zqloeR^0Fqz>YHn6ntx59NwW5ZnH=%qXIe_XTlErvfWpT765o7k& z5VXQ=8sMqiV2?hn$14i?PBNs8BqL+$qD4E}?jJ@Z<{g~)a zDrd%%yO_b7he|y_WE16@n^j9|2|me4@q!73|>2N{{aJ49B^DWyWgL?a45$$pT-+`1(C38O5KdJ8nqv}OIxuM6c!W)epO9- zN-G9?nr=jQ+|}YYz0zZp`2%cG^|ff5rheQ~H7_S`_4dBntUX#wwmtIvKB~jUAF3IK zb3HY4+G>m}*$4(<(Wg8J6MZwahIMYpxlC}@RYMC#W)#v5B3K+p_1XZHmg8>B28a2o z9tMvu#)gsLR7D_19-+qldN-sBS$rgQ@ywSruP(B8>dBf_lGiL@bJ?{#TV}s$k|z ztkym(&~>-IB)l&o=>m)0Cj~0G)2#ky4fPq&%c?pED7w#v(Jq-@q4@%0O7GPEm1_l! zi{Oj9319ZDN0P?X%;V!J;QFO9m7;xzeyU%AW<7~;-o%eBGZo!C^jzX>1==WDhoH-M zjmjwOC`&0jF1Sk^vN+mQ+0j^Kc$fSz5+OjW;|zTeYGW~IlGG$bePKcM37GFge}ssL zs*?;5&K8wwvxTWfF3GIhO6`%8{)A5BG=xH76=Pghk^mW9a>uKuBQ3#s@(kP3C>-WO z9I&oAJ{q$tm~ArXunyH7b6-?^cEHUCsL*_?znV6KAX;i2;1$f=5{XS>k zSH|7nBYy;6L;8u!bNC-{#g$S4z~wDr7jMXy1vK+aMCOa1HkT~E4iEZ|x2Q5zh&GpG z!w{i5B7CY7pCT#T{_&R15XsnkegkMM|Gr)hm9&{`IRpc9>m8ZLd4xcgva=yICY}%^ z`bd>G$`{jhqw1%)1U?i*8gtLE8@pO?zEonaSy z#QJnp?7wH&kwNM$HO0~6Go^T2E$fRfV4tPSRV!+%Q`LP>4l^gJ>_w)~meGe)e<|LB z5v~8kVV7u{4{CLF*0ZN0h0+NGV~B_ z4?^Hrx`$*;w$>g;vDr#(QW86#?Lf95S9{@^wFDQ!>iT2tr&kU#k4OHRfAUf8Z_Udb zw1jL~87_-Bu!(mg=M7fRk$VvQ zJ>`P7-%N2dDDbQJ>VgHdi_s;x!SjE`C6*CIQ*ijvdJyR1@{PfKB?%=B zL`$3H+>1G5{Qyel(|VtE2AaJRy4l_Bb#|ZiAen9+Z~f5BR)$oTBZUZ^=f&ZSj0!et z`iyAka`&Lz4nuyoxHu;HJr&Nqj*w*>q#pXsZXZ`~vE&H}PMGHRJc=|KDtbiJF<{Sx zq&M0#yHrq4Ktbop(eT1#_(iv;IqJ&2x)@K0K|paiEHo6nkPh-tD!K#9)ST|YCZO!s zjeq;st#5P__~!g7fJd;q7!wTcG~OMI?+>!ZBsYk2wV9-;Jgm27DSPP#vpr{(x-hP` zKk)w>kI%c?*Cbk767jQ>O=b6qqYRva_-;^uLS zj@)kpjlcZ^bi#z?z+;8*IA%Oog;pdYUWObgk|TLJn@HVGI)2Vb!~`2tMv%za=)b60 zUp$11fx3<#;_M(VS@Z?u1cTuHVWUTxz$OjyW&L|YMg|AFU)+!8JB+v%&(m@*fCV{! z+V@rc+u!?Rks*lino1CVzi<%I;@h5bls^~GSK!1K6aShJ&r&cbDxjsVe6-gsGDGC@ z>mys-sB);XxZXB5F0zkm$atZ}C&8MS5e6JubTP>X2QoZ%5D~>#V9kl10p0I z3K;`s5eo|Zl>#r`88#LRn;o#C;ZCxxvtWp=wg)fm60i84=&t;h@EI#sG%>XONHEPqEQn`3aj{V%tyrd~Nm%)f?G;K%T(IcywsPx*tGclDe4t z;lO6?OnG8>3nB<<$t~J#ue`Cip_Q5H26$M!bKItyWWLjZT@-og4%{U%uyfXmODmQr zC0K#GNluB-fHTjP9~{ zB90DXgLqcXoaqVjinG`d)bgoSu{lgWS~5!cKxnau%*%n*sP%$4c_VAYcQBk^^B%SVmPnB()SE zK674`e$rAzgh|MSVeK;`uv#Lq?csN%Btn|_kk_|YEh~}OM@VR_;rfv0RWR1VdKOA$ zwN0{g(wbB4^_rMbA^w9e(gcA=r;j0n&s!My^y7utIgnzS+lV(=Sja#eiVrt-&Z)E1 zS?iE_g(xQEB?J)LR+LPxxHPGaz8!~ODd;7UO_#76xsW1$pB%q$-D!|+CPZYyxltVi zeps>_(#f7R1IA%aHe-E`0p&zvI0BU&>X9GvFXA9_X=HH42_`PE;6(bOtShV1l}Nn4 zxg&C4C^?F}Z+Jry;| zK@Gj`_EnuA#8u)rnz(R=`g~3LJhP|ol0tA2Fd#OHo3zaeEmOCvA$D zTJ_lY5~RMVY~^soIDE|qsIM=r90c&c_PU~fjJJ|(C+nPp99Sl;AoC!r!(Kg0t8m3> zZ&P5$WR~IVFkm4e4pOQlCK+c*Q|*2WDRAp6#8QqUGSA&UHvqD5*|-vGqNPEhGDp(SXh7K=xS=#PPH`Qz{Te7DriE>$`X7buP7v4$2#u@ zdIkqwWh@<@xC&1DyHzIWJnl}`zqXKdNqSSH5>{D9T4SP0dXwk7B^1Y|O8XcQ{8^ zyE`5biKomk_#~Sa1KaJKRorMuW8U8?D`2*`T)BQI!sXM2;SBnc(#83EsXct+orMKvWs5$w5}1R<_sU=Pj^EAv zp>vbNNXX`u1t<*l`?VhDKMFg8m&yHK3PbJqM`4i@|0+yOg$LU!Fz%LFVN(w&v^6Q6 z#KUZtH!9*IaW~**s;d>^-?>-#4;5L3-xMl~HK^&AB;;|Ox#fNvCcYhF8Syhj5f}ea zSPU4x+{J$sh7k1jufixYqPYHFg+2ePFxr06`QbUr)qKL0>)h_^__{F4 zb*9WBpLc$G;v_>X!>Ni-#`6?#!lH(Suf8Q30y}K1yv>wSED_Hf9p{Di{Bd>NUO2kh z?fT_&p@vVh!C&{%c+&a1dYuvv zGfy2)L`dXdAsz9d)O3fRsaf3vTu`~EH}v)k1H{iCcCX7DmUtFrXFeJV;^Dmu>gI-| z=14bWVnI{s&*HhBTAl1~w<+z^bK9krm2aT`Q(+a<)0vm>KtRHN|I?-hB~uqyCp&o~ zD^p{a|9pbfy>Q+HY{shV1u-?Ndx`-({5!T&lExo)!*Tz>(H@uO_as7T!cs1QQBrbo z|J(a5_Ph6wb&3F(GOq|Wno_V8BneKL*xHrLxN@RM*yHz%K{*jDqK zkEgL_@vE}K(WK{(xNp?wTGX-Sv!4aTL;v}XRBwNA63=egua_gFyaYIPSn$ zH?xp10i3Js?;rX*gr7~(yCwWtMao9S!9PzQd;bu8-aXyIb9G%52d}<|2Pqti5uE)= zvXJ_`=r=bKA5H66;UY+eA|;HM9CZlD3%u-$k05k~Tj~AnKgZ`W*x*_B9YOV%OAL#i z2PFNqc$|Eo~^Fvl_(#1F2=NG~C$7Pu9sTkF1De64+t0$%_SHzD($%oVfKsuvGMzzqIzB?vqU&r*wh9aUVpSE?b(y*ws9R( zgBuHQFR}~&A}pfrd%m}W9Xbv$P}VBJbbET^pwLzuOYRz1Amq741B<30t#=@Xlh+2q zhs#hkuDjTJ_jO%5XX()2j9<0$x)?T%9r&)l=j5ik<00EGKuD+6^7mPmHL^XfeTQC| z!cpZQbO4{BsTJjB56=^-lVw3Z{Jo5pr7*f?E@}*WP2Qwy5eA=Z4?BTqICs%-$mEwSBeD}8!Yr+JQ z%_&be_x!*NkF$Atk;B_*zN<^P{I?&X>jVh>q#P6*%uMracf!0W0`bTXt_WY$pPVHo zsFJ%FpHcKg$Xs(?R+(XeYl2BVeMweJkK!Sn$WOLO`FAY%rmV|w}>nk0U|9 zu(dq)?`Sb5d23_j-{|v2j<)Os`tJv>h})p=`*R?0hfXQN&8E~>;MIi97k;P~n%IiE zO8;d-K)zQ!Mn8MmqAqEx`^{5C_h|AR47Hsr1DR(X9VUURsN%p=){;@+1Igv1-)}xV zouL>)hG+8Zv0z)6XXI`n_m@lrYr5eldBR$=woz+fV#W{!7h#cwhzaT#AUQ8v!5wxq z8*?exbr=%QPK*X_bc=7qm*IVi$Hb4+IeBYLCiJGqWfC3xDxm>!bzeR1fpU=WO4=M6 zz-j>WW%@0l0OAOd%^1mQ5N{G#B+!P7GWKShL^+}(FP#*?hPH)nyw0=`5HGLGf=bdz zTmbZhL32pmE9~`il*HT@99W)cN3uM4*9$Yn021MBbp z7p2U$0ge7=r z0T5(vFLXyV`a^kMdqns$MMas=hm%xXzo2<{l>VU;O-?8tVk1@(M#$le&|_i1bBJdQ znLu_#4dv|G!eV6jGDK1|u?D0@WSL^Bf@qB}dNjE!|xf<07?H)w}eSNZaK^%stP!idT<@rwc2jnk%ogT9ho#uX+=lnvo)MTl7=5Ur!- zFsCs-fia8n7`^Prf2F8YNaR+iG|2}W%zKPo4tS)`Ij#|78rQ((NF*`3pKTlw1g-+e zFC2iA>iZlG{gmR$5bJi=qQqYPFgxaC8^*YD%LLV)TER?l;sbak6uSe;DHE0ZNXr9q z+$YP`*SX~3`{-82IZ-)12XcrdpEah^_V1Lq>%CLKPvd|+G zvB(UCY)G!i-woFJu>^lWZ+|DOVGg^EmS(GS!XLA#^e}^HL|5j3F@5`HH1&0 zFQZtEKZs|El@Ga%uF+;n><1-emYE7SSDbBWQOTEJ!^ZGb@MD-_w?QXQbw0g=8DoHC_kir~6~3wgE{j&L-GK8}4t#1RiovxA9e) zO)y4oFzvzxZYA#A8MbW?T|QGk@Pj&WULfT#EVq}b0~qg!nsJ-E^wqR=xS&4LHT%4u zKM&<|u=wvBY_s%85uHZ$9f;-I6fu3GFVr9 z8UP`*DT3M@QDGT~Z&n17VWZoSm2XzSOALZxIyA&~M(n_$id!%C#UTFq@uARz+e8Wu z<};M8B6sN99}#l~e{OEOWxC@#M|U|8w(cWJpvQU_jO7S|DiEiHrZwpWudb%+)GzFENXbt!Oj7B!2)J~hnN zZQRO--3+G0oM%Lm*-WrvfICI`%^}b{u4~hZzqz^ks2}k@^ORcl9F>?_^2#YKtz{eh ztMO2OudxAY%n|arT`<@uU-|M!cxUa5>(ypmEOF0-;ty4wt3#Dx9F+#MzfK|7KT|bB zo|^>XIi7`aVs1R+-@6y9Y6c9ldz|2gMvzp%cU=l%9j%%)Onl|b^w;%vn`1*ac0C}& zZ0&g;L+i)%l`liHG*oblumE4B!x`|R%5hs2hkaKfqkmt`I@AQwUIXT(qO4LarX~3JKVhjvTBYay>N=EL)3nGLSaW8-%{XS zs{9Hn7my%vQdS?kVRyyA`;{G2fJMhY7)BPIx&z-~-v$P%+-Zgo(M@7Gi;uD%ji)O# zw1b|8q*j=_na3UL?rN2stI?y_iu4|!L7QlPBp!Le4`Y5+ zXwy~=*;9yACzwu5;s~UvR9GgS=>d9)7?e3ypn6K4Y7hoOJlIA`sA`Zy?v}5)UdZ;W zT-KfH#emA{%~ty}lIp{-SDn~E6uY%_A@hi|;bk!_*@9wvadaWED79i~8G5Y1GCZ!@ z5Lg7;aI%zCm!W#5aI;+n%z#u#Q`wYb4G!&fRs=^Q)efSp3vqxc^q6Xb>Ask+oh-7r z$|0D_N|m%yE=^^A;fy8DrYov~=qn8@!4Xa>n}%l7mBC9REOS*#jUQX5WFecX(y4@0 zB+RA@sDiAf5V8gPN@l3+o(NjN8!#Z&!yLj{ zwo+80IB4=Ju+Gu0#K5@484cnMFTi&gmC6eX(avhA6q-hyoJY_Iu2$?>h9!DU8ibbf zw83kWVn&D3kk`^KS&FqxvWSaTfMa!P6-bFKapjn4P|4Gw-(oLVCCanyw)B+|w;W}o z(&$Kb9JAXux|CsT!RmlIi3ZgO%z%ucpP&UYdIq+iZ<=MNVkR8@Q^p05R`?r#Ma=Ll`5 z*+nQ1ljfrOe6(6Is>5Stkv)#L%c9EIa`dm;PG7mtb{F6O=jCzsGJRjyZLk%ZLnDi>!wu9EX8W8WEKT57+c2vY`=mc;>%!HYQ=Shfzm;UCNBa81W~t zDBBo)s$G{}>h#2)Ob`K3&LK=l@7%#{rfN1^bMi)cvH zkPx9AW;^iA(Eo@X@2~{qR%M%(LCNbZ;ST}1)r{1*jNTyrt>|Eh!}W1=KuCi8(J^F~ z5~~;okXzkoFQwy#^1Z9&ulq#0w_+G<4w;s&pCieoZz#Ot$^AOqD6RH2boFDwe_|50awBYU+#w6i;vjFQ zV|y-Y6oRt#vz(4Bu((h1-D6y&Qo9iuNUZCjlkm-k$b6AT`{S`%TKn1KMWzlK_}_$h znj41dq6h%l)%!k6kB~tXqz*uK6?eoWFvjulUd>VTxj|p?j~mVl!){6&T0uVgOp|Y7 z-`@}cxJB;+Ww@ECpQrco7B|>hq&r|};C_n(DOH98;lrPrTEgu01Wk=-hEwZhD%GVz zW$fp2^C&O44DBo~KNWGF3|x7o4v!wsN9t__d`OH9nd;~T*KO8q1~gEl0g+Yo3tOll z?D)qJ&+>B0Vf$!BSr#$wlgwvxh$Ns#$XZS$_;p6G| z1Y)xXCj!)CI0atd{g}381V4Gm6m%W`Ggvh1T^!CZTISX%Ou77#paLAU$%%46`O#8 z{4Z+Y$*j(o5;RvjrpQ7*c69L0;V7u>8t6p%(1>sX^UC?C!l z63-_fT_-XukmdadBD^;Xf~@QWI@4i^4&M*~wN7l`4Xc-;P9lq0!<|ueOE#c;qM4sQ zz2J{I0aamBY89}{>Jgx0zDx+c6YTXgHZpx%tCyjD!0(@nvNvtf5c2NsSED;H@+xF& z;hs3dQtKwa-SYHKIbvisfubJMrll7+Y1@wpz+2ad9%9 zggDMAGW*sziNjULc`3>?C5rP88$lGjWH4BnfX)J6fp;U(u`r|XIBMl4UD`Wj%2p{Y zJ2mN;O5!l^DB+kwIixb93J3Iq7p**|DCrplK#`!zuoiM~fW`bWMTnWv;IGavQe7%1 zc&_2OuEPH`-2eKtXm$GObwQ!iGAEKP)uI)fAWb~u~H zDxWRo+8p=mb_uDi_dEBwyWHa~t~x#OrL=?RPlNgwo=E+9_cCtg=6U?E>v?ijJ+51eOcNOa-7gXAyQE`$Jc?9CA^j0qsC6iC4Kg{$-+>2-g3=y& z5fXY2yKkY$Tv*uEyIA)5<~17ERvm~S@T&s83i@vQmEfl8|GG} zg)W($9ko8jCYOW$DMPlC#u2LSuQ7I1nz;;w>$JG@0Hmd|9CWX$rot>nb=3>d@S+~T zD!dK`b!mFHu7ZHLLIEw~UK+0ZiPBE$uCo{YQ{mLhPbu4qT8i198Z_|wHi6wqN3izJ zQDLiXGt#n`-N}5;@Tsla_jhg+YN}ZT0;L!~34PrbPTg!N%*)T|O=hU=|I$6K$a(xy z{_LD2FWYj9^DVtf#1H=#qmX^P8*wlleFBt#khI!+`|JJX^ZESre0uxnyy^Vx+Q)rv zeLf+j0LxbO8!RKddc)4n{ET5-|8ev_YI8%_{M&!(sV08CDfaE>Ti$el-*?H?XL8%m zd&$j_81=-|0f5h!z0|=;!~4N5(dW&d&6J z9!y3dLuXU=AOD%Jt7|)Ni2YlKSpis3wtyqE6&HPq<$&UjXJAvB%jb#V0yG?5vaTzb zA@}7ydo85{*OAv_iUXg%eC~77vlIA~{ndc^$<`SdKVQy`<$L{S`>F39`~ALXRD=J# z3$k{%8#wW>QpzQ%RENop^|mkDI#-{$O{+F&Su|lBJNkNEhfyRtl1>?Gi0g@~^$B+; z9Jo%*Cv5k<#n43H)&;{W11FO6t^@5OfLf}KIqWY^&nNei=! z(2ix!RzJooQq8hWb8Nc&kelJ_Ktcg`EPByUpsv)os7YW-TbC+?+n9n%&cpAZ_S4Y4 zMbh07XYxUucK{2%;O|_0URGAuQUXFfpe&`Hg!GYIuawHTJIa_8->Ba}#EWnwJ)F2t z_L6e)7k5#{*`O4iG%;I*Ejx~C;l6c-jb9X@k#)q+s%nK*e3c035;==a>=)~`uCCRu zvz58ioS~F=vG7r)!_cV%b++rC8N1T*$0C*=(7nQ|YO>xABx_xMuCJ+v?wk;d4GY0ThgB=?Jvik7^lcik7+hI>1o!lh=ywOGZ$`dV*j zM)8oZ)ff6jqMiG$2I+Y5@mMan(#Qt)QbRB|GjZr>3l?;&%q*+ffUaim!lYe7#m{=EuRdIuO; zli?)`&eOPPn{qOI=QLJmk2cUUZ6CtD576zb`f-<&jbK>=cG-M9UErU?8sAWs$xU_j zU^~!@5CFf5BWUkm(l*-E{8RO0x_ETV54lOaITf(&Mow0P>b@($<*Je+!j2ocbC3$h zZv*j`sME2`wT*)dW2<;oE==i-9}Y77gPG88D*XGxXDzSg##$uB&tF}6pR5`m8*Mp< zW2A&Uwdw9V;*)>1@)Pv`yi?t-k-_l;>T@}9fq?k`FAoi2LmL|-zy>B^8%tBW|N1xw z=+fO}NBd`Aq-ue4fW%bo`kV$03gvIONPM#MFkbK!kxeTLMI!31k-6_4VWlK`Z9C_) zFz^awZWK>Do}Il>>F43~0?aR0*EqhAbj;K9#k1j20w_AB?>9IAMEt#LyrU z7FFt5sKJ-+8zS=>l`n_`K$-3|0uO^2NB4*^n_+l{BSU3+#SnB`C!j_B@$51x2GF8z zF~<^-XmiN@V-dwzDoN>BGZlCg+X$}A$kH}8Ok$bJ`rv}IZ6 z8jPJQr3!LNBJ^V}SdfFl6e~DNPjPIg_xtr%B0@&#IzCFuYn+O+*inYk5fR(0pyE6u z%%~EV1>NbKI}hUEj?fIN??~~@)~T5-ayn;(2TQm=3x;J~8vVJ>ni}O?=7!L%WO&7( zhs+O#JhzilO`%@6uv~qmvmhMwfb@D1LH~8Jp7^cbwh!%!%lSD_SUHf?3>0sLaOYjn z&T+Ak_1ak3JjS?KJf_`#G7FZs++e=O;Ym^{B}pQ11>m>G-AGg@J`|O=of@#gx#X_8 zaDI=1KljF6BOq=RI68zrllkT3>#Fr&*{}o}<1iP7@A)!?(rBnySz;qPLZ7Pe5>*F8 zU^O%JCc1vOr;Aue6yU?jE+$H{F04s6*rlrTZXp1c{lr*rFRQg1sKyjHBuc4rpRGRG zdeZMV9Tf}qaRLD)kEITq2n}s5-j6}o*;GeD1|N$(QiQ2>-PsQaS(?qcfdu;u?knw< zi#$*nd#|D;!L=@2x5B7FOHE1$dwiH>Ku?3mK3=+Go0O|?S8MlH+R*IamoZIGocvzs z6aRdY-DWoZ1-n?F@&Z&Zeb-^|wz_hvR5(t;61oCWay2@h}vZ6l&rNmCSzd_ zpCC3$Pqqz2jZ6xYVha0}8vv7vE9Q^*$tP}2!7duVFy!7>Vp(?A#`$*zvJJ_Vm2XK; z3P=vC`Eqf8Wx#zZF)3J7MEE#1h0$>w~Y%l)=Z6~i(Nx{B4+(HwyTUS}8rfPUmAlSw|RUG()x zGnxJ=>}@dv5*71?wA>H%0}R5M9r8-Dtxin8$6kvXtYjFrssA9R9ndY7lWj%iG>?Fj^< z>fDjxm#JjHg$jr?#N*YU>n5k~yeS}3-u5YbBXi2M;u`QHkAv}=Jxv*Bu zw4v6pr7>ww=3cR!!swuSWv+7NHwb#Wz4|y%e5c~=S8)Iy6cNgDJM0D#MQJ!X%3q(+ zTk<3Ym{Ct@a=-y!4MvVLb^X!!uphSo(hh+rr#speN0!!%N33qWD=mID%H!Mf4nAd# zU6$LVFzl+kH2b@(HvtP_EQ67Wc^|obr05f3NK45)uDlr3%4p-e9(#BG*BzX6Oc{pK zo0*JxIba!>JFvA#xG6%Q#K>=0(5IY>iG%e7S zbL=uC8C-rpXw2NdRTR#vA@L`)OKMFdJ;wroCmF4n`Z$8zjKO<Gy! zX0;oRLl=x7`0Aq@Sgy@oG20TawCV<{HXGQ(=&3Q(;pzmT+rIu=WH-=ps_Y5?+Ij$J zh5!HlZ?d7*8 zsPXf>6Ek+V3F3e8cICo*ce-W99t*Gpw*QW1>@Yn&(oyda=o>wMz7|v4%=PU4E-p<4H+=9n9Gpk@yq5lfgy0vzAUGi`Ten6Ur zIo-!PUv+rhls(vz)8-1~eZ~v!w|KJ+mS$$SB1ZkGaI9(4g|1jKV*7ckOdAb=xq($1zhq=E40L*m*{?_`3xt7krrN1(=vso&dla7D4|Jb34S@&VZ#M>@iKW3N1 zsO6`=6>rZK5zL|DLs2oJ*{7i;i!o_ap$ev89T{o{{V9uF$LJAbM@WL@6JWs<@^z(u zwHf}Xt?on;&={Gily>CXQy^12nRz!HF-fVV-%iefbTBfOdXD*+bNCf^RwmOTClYO9 zr3_#86}5){;*BVu3~=r{my4<;h7mp~@HJ}bPq{Z-zg45n4jtG`p@y6&f$(wlVV{{J z8m5zRYxwP|*l^@KvX6$iI79qkc=)6xwQcKlgf-)B=$>g)$j}O2BJwP+qSQ@V8S!U(LD5l z4_DvLMVl(Dbu(@?of9)E$41wU^59?Mo-l4U=Ikvyw#@V+jA%UyY7FHPcSkti!6kC4 zlq?Pu;{3s&S;}1jWC*-Pt&hhr81}yYgDj6zK)>+^zZRyM3mjYAY1BMtdeMIBBk;fmJm#6F> zqJ{P{`+i6}26NACG-0peJpA^f3Wrn)+;8$^VUiDk!4Xy?z$ zCSB*^%r$>39-Z++g3YwbE1{sWni0zUW{Y?x+ZKdqOD^S&DC#f&1f(GhQY>?))#?sjEe>+V8S7nZWugYG8Sv+enWYgCr8TcS) z?A(B*L0#PMDabECRv}l3CRELf-#y|zoecYd97&-pA~xhMHU{Jdp; zb%6|?1#Aq$hY}k3;=5s5k?s8I7j;n!7x^m#OL`Y0J!0nt45n-xMLU8KE!x5X7*raL z^JZsdb{nwFvWZnn2CLP+Qot34M+4^*g%I0nU?=co4*{F08Ab8bf_?YFq5sSnaDdi- z$tiz@k~^;Wks0RoVD0kc@%I|-lp-^2ZW#Fo!`@|kb+o(sNxH**g1|wI-!(k8Az2%r z^-@B7A?dFb1vEMR0jgtr6RndKY%k#1*0H(ao88Q0xAa_@kEe|SH&2dW^;x)JQ_AJ5>-91 zj?a>M9wZ_MY)ChJUbYKNn^i$Ohp^;Ku8SjTeo2^6`(4G$|AE0z!l9|sv{8w)mnV)R zuE`(hm0Q*WS4Yzkv1BlJQ{;RsDgnR2pK8Z3X4skH7O$q$X9TpPeM-jhzc_@1saWu( z{Bx49Frve$_RLijQ8W*PG?&_Clj!N*CbZn3p85xr@CWF<*Izt+o>ww*0{4x*$xu&5 z6XYwGsU1zd_>3)+Ue#})bfX-ojisMsev(BwNW17#Y)}^b%IH|@U(rYHRR8)jaG*hD zOI7Eq+-gRx&X+WhJ7;EF(mwJ`-}ZO1J%JvK_T@AZLAhdbJ*L{!WL9|&=SBAfk#xHF z{N42dik2|PnutaHvEYcy0Md2x)w+(0%Jh?fdVYyDoW3599j5V)z3`~Hu1tdpPm=da+8>44Sr$SO!gzvnMM)wzl<87Sk>=}`luGaU?) zC7uKIHv9SLwr~gGuGo*MyT(=}^dkCF^I}jlUrs%dj1c|8ElrSk!)YJ1-B^*XZoNNc zl;SyjtvQqfcbwgt!&w(25rx8;Z?27pYhn@@wi<_ zD{%cN{xw#kyp&Vg*2@OJIW}wJCf{lU&$I^=)SS&8q0v^fz2=SZ@JqSP>ifUZ1n=%_lMt|R z%or62NcR7dCjWJqh#R_?x*K{bn>x9fI>}f%yO`RUI{nuKP7!d4M)G^E@38920!qcA zv}A8d3mcN#gp0KO-*3^D|K6fWsmI+e!Q>j#LC}q|Np8Bh|KRiMan*YT+kMR5qVNs> zTxI(3ar9jVpQSv!NoFAa-Eo2;7hKFfl!QUE`iqK+^04V9@aKG|kNB8TBNPT@V1hBq zPEusvo$^4a0^OL)v1siK?M^tKHr!c8;LSluBCuSv<7wl7gbda+O@DliL6TLPU{eUA ziyUj>K!1*8Xjvit2L|4Deh)7gAb7)i+K3i6yLj|)tN&BesuRBJ!eeHb?`=tqcwMgpKP(qrM`i6g9K^qS|+1Rb(#{Y7nVd&QY3(k z?==##!}knYALv%kZW9S#7T!gP0w1=vQR1(2qg7d~yGM zjO(otCveR=ivi}VjJ47XNr@5-DvDqP=79=xcsm&K8p4j~S$kuERy+V&dDX`}X@9Ju z1yup6{7Vo_Il-4}Jas;V;gYA|8Gr9u83YR0>!P#Zr-yfU?u|- ze!J!|oaj#S#TnbR)5^XkIRUc`rF%?mIP9y{R!3poG^gDjI}Go_q8+jILW@Q1760H? zvz%2;Dh(&^ma|i{-)s7^*}$9TLorUg0jHf7T>bb;^D^Y`!YHW{FyG!q7Eh+*{S}Q{ z?d1|(qiH+s^vsk()9Zuu{YM7cqio0;zWVA-F#AR~fP84RzNL!))0$X_hK^ge zkeE@J$G<-21wk-|wuQ1D8N%8@M;3r?F|8QCW}5{~+edM!_u$>BAXrf z=&nbH5RTi{>RIECf_?7Ts1WB)pzs>}(&5p&*yRvc3%dv$369+4d&N3nx{Bt`I}Mx` z&My4PPY;7dNlDv3_l;5kuKB$`Z8BbQQHYu63*1AFMNSwkQ};{fc^xld)z;hrbxms` zAkltb!loSYSCdZ2x}Gh8BD%e zCzjv%(hHyD7)6vkXR?{nK0z%i8BnPhV!=~(9Z8rxQ<*P}&`X#NhPmBfxyum_Zc?+Q z6F7?na?nyahE>_APHKPM@1Sj~9~C#(u@lcxHKA)*yw5#@rtFP(D z{Y0f!^Q*HVsq1D$jU%h(C3?$i7<;PjeG6tV7r~|t$~nd)g@`rt*4j=-iLdvj^b7Ag zApC9Z_~XpQ->qYN#<>K*D|3Cz(o z=u zP&n5CK$%gsV9#Dnzm36a&N2G#uLF>CY*ZB08qQRUXui?|DOX=iRguL-mooC}OvFS8 z@+Tr-VJd;tWmc=V^OG5FKkyGl%vc0G#Lrz^+9Pdu@pFkBuG%*BcDfs&S4a*oebQBk zbw!^rC!&xi57zyHnq#6!rfEUbSq4=DXJ*M~rws8%gym96smLrUY5$wHY+;N0$w_7b za4yaqQhlISLsSM9YoK?laV0DW`wDlmkKQu9$sO}K&stP-uX<_93?2XB<;(69#Ezg+ z5|3Pmaa+1>j&4EjDrBe};v?cA%2huUrz>uqw%d0l(={jp15;BmlGL{^(ne_a-Z-)! ztkj43%EXQ2!;5zmy;Q!R>lBA@8BZS*YyP7dx)y7eanRz=C+Lda2DcbWXd+n-RuIf? z=2d|Ze*b6^Y+V#P^B8sg>D6ijW`|uKpjqjb--=VG-0^1r?hj3;bub*5Vm<xDl)4Ujm0ONiul_OlSr;Py~cx++rHRFPC8qpTZtX12wga}Rg zYFA~=3Db-Xcv!S#B|wTa&EAo{dGfaeP>dyeg0$S5*fkIu8d}WQd!pyHI_vLiPvJ+% zHJwDm^;R5`^}*jB>L9B$xTgNJLkcI4f&uH|tk6z6-^LHBFTHBE=lKp>4RnVgSpzS`0$mfk)a}bYcBL z0vE@i>aP@M8$TKq<+SVCm86b0vN+pNt^5@pzZw$g8x?BI4ysi}y=jA>L+0hQvSbBA z)5!Uaj$-kx2`rzP(Po?&Q}PD7byo7tVC~c)6R@ak#FSt5rDPugoU!D($%&2<&OPnQ znF9>8%#Hh&JxL|-`vac8xO4y!8xn?!4+waGi1#^?MmXf zqeNgMb|7Ks$I)TL-nDnPL~t)ytqd$Y*9SRLk)2rRwF2%`s9n9_q`UCJ&zLg!MeODZ zIB69<8}_p9Y?KT#lq#P1_P`F?``zB;3h*Mlb4>nP6uh*9Lu|a zjNNKtvqDqO)6%eeU?xsAM)uHb%a&X#Q*HX&M5(PWHZ*LoN*2v<&=2Ev-+197AA~iT z<(9o!mg0zdX<8%@Mz{h!E8$;+=5+wkmuh3W6ww0+AA?o0sCXtiK9poK7E&yVfOGIO z9)XScSP}!Q46KJ?-m_e?jz2T|)&~rp+!`>j;O-QV6F+I_Wa!Jz zSKsMfND6`j^%U+SMIfr~^u(L@SP`!;pCS-Z*fTDa_|ks9ioQm|7cM4oj z@fCE(Xo72*E+{j2{l>1Odbl%#2q>0(O-JI*0#V$EWo($>v)x-)yDH~FfT1K=U?3XWvcV?l zSZqv~$n_$x#FP?+q*&{uv^=uL@r46d5UJZ2%>i-_7cwfE{W0MLB2M3pv|`-CQ?Q)r zG<+RRx@xiX-puZ)Y=cgQBbvZIZ+~^e!gCXArj`s&J@pWaG2;4*)^isMhK5L^o2lGX z^3_#?uC-kCY*7kpK3PW!gegN2g}Y34+yuI-shyync|9_s8U<;UnKW$E}sle}WFn_z_~nhe}VkTcpE zbCmcnTTLk!?FtXo$I8DkF931ACTO6>m_-NzBJv+DOaFL8i~bvp`+t)e4O|1HW0Ki$ zwPi$6%F>7(gEY75%L0HtP6RnvGTT+Gmu+`R=^MG3&C>pLA_*gzrS@xp071@5lKlR9Y+c^Oqq_-TJ%z#=cYd_53Q}rIPeYxug&fc*L zjw})wF%(zNh$z2NcsfjhMZ(Eaw3{|izi~afi-M=R@zrEvi})%4%|C%KoRHK+fTl9xs7OU3MdbPHO63ZXO?LcD)j=M$u>!-biMQtKGPom4VL=+$_i&Vc z_I;ka7(j3P3Bht?e!9VUf+*)-P2e0=bWCR?d`x+$cum0KXo|+ii@*Irs7{lL#6zVB8xuci@0fP-4{VA;sSeJU z;;7G+pm>8V<~>kD;Cz>*L3q{HXkNr=T#zu}noX%qaiMu;7(8%mhi7r@ggGD9!9TnyAtCW^%AI(T6KSRm?P?7sYO6ek8&DBrkw#$wT&(# zM^6o$Ms{!HWmGa{GO@6(u(`?&tTn3KSvKAtU$N129JlDeeVE+cZxE>`eW}8NcKJ6M z=JuFbw;ALxo#bM@IaW`MPL}{uJQ=j-LRt)wn?vfYtk&z79I2_JgtQ&pm12lg25B;{ zttdCDwhn!6Bs^d>Qfp06DJ2Epuo@>-x9Bu>ULFpL1u%V!N}vasjS(ZQmO2b*1@+bKLmLSNbZsi^`kA9|T@92MY&G z2lwa+Cu(wIIC3Tj-`A5G1&P2V*~i;hEpvlbwr@o$bw6-UCWa9)R)u{%6m5@zKK?yO zl#J1hJg9@jXf932oCIzsB?U=ny`ytsHjp;wq|%Jz0mRL2Fr^2AT+9U;w1WwyM@FA6j>e-f|u zCMTJls z%4He3bKaCQ3nLPVMoE{3vIWWI!Yv7JTDL~F-^|80R}wbbISVJc{$`;^5gZTaib0iQ zxl^xcLeV5dWxY-Gl%t1v004^s3s>3G`KF2!)k+PF7~B$lC3O!2DulcF%ISYHbo=W; z&p39x0+$8+hiQoWnLIWrVc02B<*!I6P3%gX^r(Y0DS8n~CV2rA1T6_ys9z@8f@wn1 z2tiyOVux)P`S=2--i6Ib8t!;^0=PY^3WpyR_$5b1uz2*X;03DJp@Ai*o&nZKX zqmw5=qO*Aj%TDc#UgI}twZ+*#BJQ7ghtl-VNI2>DW1mBnSzbqQ~4jTCCMG3Cjbl1O@;ndlObZ5eN za~sqJuY*}T!t7H}&Cm4y>nixJA3J>nWcWRRdZEDox97}%v2p+O!Ts|rv=<>zDuf9! z)6klt0j!&S2ZOz+eW1p<0^NjfVP_31RQr6*z5WS^k}C48LW`dC@ab{RyA*nr^IwAd zz((3JzB(ot$?JS{=XU^%eDs_!uOPiV2v2s}2oZT!li-sxW5ytaA#%yoEaHxEWW~0& z-=MS`^ncMg`EJ3SNCw-j7%pG7;hmX68wTwLb5n_<#*3OsB_5>U>9Tx>6S|A}5VOw4?>42lrsrdnxJv_Z`&RrzQ!60=A$M~FAqHM#W zW7>MJF5)+30+r9V;~Jr6u_eSKTWO(KefBKs{XXU2o9=w^^VW=q`?F5bJ*>;6ATJ|^W8Fe!yuKOY%?%qS3o@K1b#;I!krl{mCmY9_s!STb$STbc~ zi6{jK^i=6qITjd|3>!4*y6G6EWA1<~^{YuZivDv^kfKNAm=UI~_bTsKMS}77!wrJmJeVKMnpzr3G+76} z>S`L+W^|6-oIJ&Q9>q6!)ie#8&4M>P2Uwgcnz~91n36uG8}Hr4AFAd-pUgh#eUV|# zJ)&qYh?peX689cHC-Q^q(H)K~oa#|`+pvZiWdqgp1JPrNcbM822wJc8#@w(DM00eV z8=dujBgb%}PV-$o1#gtN+hyHJJn?GY+|AxYb`?81EPrGplk%LB-X!eNVD+J6lCj*| zP@U+t>&bF1VNO#vVEdTABZj?Ok<{cXvnEGeOb6Hv$b=8Zg~xU|^Z9J3tW{+8PMYn1 zo$QqqjxS}3H`;@3HYuG#f>G8gIVSt_es&ZWWN>%Pb&{M=7P&HB=@jtu8)FV}=-nq4 zDaJF8lpf{yDH%7>{`>W7dzjSjqXN6ajI!>qBsHZpdN;<~M3g`KbKao*`M=GomLr-d zL_j*}i4p`v;{OkZ{41lcWvj5kgZ42~cg12!ixTOH`)Fm&Dh!9Kp@{7Ac;Y8w_qzq# z4sT7q(`wznchoJ3!&hYk)ZIe0iK}Jy`r6aeCtJmLxvjGGqgy~cPY39Wdzl;`_QdCA z82Q9oVzz6-aohQeT%s2?!mfxI4KegtcS%@Oq4A8rcHp`TgCfAx-R$V+UpD{S7l&Cn zcimc92f$=MzRGl=cG5kXb4Dh0Y(1FMoIO%PF+&xcPlqkEh*&4R`wlNT_C*z2k=W3( z>F|B`{1<-DhQ!s}NlCfzmnQ!Zw@)Acr;GRguRPth1@BwkUpf<_(271VfyOv|7Xn%u z9ger|qF=AI|k$H*-otC@N=cWZ}^^H2uQ zIv-yD-$YX~UbD7M-JW%cE%B2WzN#S~(Yjc!-c0oG1i#J`)n5_|5G*u(F6`vWFK)QY zR%2Tm659wBA7o;E2YhpG>#MP{<*qJ=K1`c_HikA$@ij4XQJ(YgMS-7~WB%6e=u&7B zNln;Tc(OddD}-$O!~Y_h?oD=Bg==+-Q~l>`PoGLdYiATsPjJ}dKLxmZ#=q5k0r`T) zL_%&6M=8ew61uM*JDx8$gnxG^A4T_-!Yiryr?n>``Nh!Efc6|x45ZSk1)e2mLf;mO zLY>2EWQb~g-C9XYCH15!I9Sb;QKDJ@w%58Q!>QCd1{ZNZe@H6rqF-J_Vp-ATb)9Wp zBGi|Ck(>6?CRr!u!ht6xo3qO&S9*uY=OF_5h;o&Y2Ro`GTt15)koHY!#Y~v3pabPd z(M)I@3l9hEh|EOxs9z;R6a<^{;QeXMk1UXfFP7A{@eMv3d*WHhrR1s6V+ZF`5fAe}m*(0kiYIGi#L~cW*O0kTcN(1nQPj;B z|5$`A7Ix=TRda_hFUsxfa+8?1a_63mbQNigPdQFuRc<6^nARw%gDLIgIkwN`>xn|( z3_vPiFHa}WdunPC-L*jWm8)O}@jun82lwC^O;+tz^j#++j*~SHV2i2b- z(@kVLV5UDZ3*dAf$4e;fx50R131XSb<-olac0-&xX@rXd^DVzDl0!@ct7ETW_4d%% z7|?g~dwe(yD}08UI&p|frfMOH(_J|7$1`y5T+Pre-yGPdJ91}zSulO~)kpN7C{IXDXn1=&PJ2}_*WK|E^&S3kfWVYM5%8dJO^LRe*rYp7on zCPUnfKH|Lk)5f(2mUL9A58?MC&-24T-3K&|jSTm=byKw1v3>bo3 zu(o0VJFBt{=1hYuxsyal1&{eAcm%BZ(BCAJea9_-27v*Vd@@!fOH`$N!0(GqvSMLg zGE*xEXRKewDs-e%bV)T

    &2p5W4`Enm$E)ZU zEleqgh7!g5IKh1v)3Zuin2NO#GLFSY(T!I{Qa%;CV!(Sqp^lS8Ry?1V+$O$BY(E)j zH#ZIzLW(s;j-dm59hsThL{PH95{^0T64#&KkpxX+DGNJ(!DpVfxK(08*p!+V>h-3& z%k18p8!x0>{0Zl=Z5D?{Xefz9P9bIDwB zYj`qp7HR+96^>KUC+gZ?AZ$&wkQR;z+P-S6rdGf0o5-rX0psMHiF(rUNez%YO-a&=46kK z0(b1DY2RHC1r~Lq;_$PYhEzRU9h}Ro%(I6*v<(+$H=HkEXh-L07;#3J7pBGNqB4ig zla~-%BF-^5XU_w|_jD+XnM;;81|^@{V2*Htj=TgoE^#^2#BS5e>8+FmNT$idswX^% zq36Px67{w?TnMatBpQWj&@atJZ?r6)997|Rtrx`=~x%Z#D zcm$EWSkH#Jo2-kTbpBmjw*ss@zyZZ|NtFLw(ven|Q@1pA|E~#N-Bx9T3mN#-{n^2~ z%A}$LPUS9Yye$Mqj}Hyswz)|u5s7|Qs7W9tHMV)zyDi<26T*eJuXA`Z9!+=N*txFD zzdn=o%arru^#ojC6!XyG{{HE$8ZK91Oy65q#LID({tSxOHiCQ_ND?Kg$d5^Fe_>qi z;36~O$bx9>78>(L(N9*5e-_vkp~4{Jxh|}k0cKeQvUKOYA$#IcB_O$`FGb_`B}h!! zs38P@WJ^_QfdzB>6scrJ5I2>|(y$?%?>rhD+Ir zXJNUyuY1oV%&?iyYYd}Be0AYggN=IrXx*7SAGRGa!~adz!ed-Y{l{fWgi-jE0A9RS zt5$T@;&!9+<09}n#r*k;ld=nOjI#=;T206eXBBcOfuNH1bz5_t>#t-U;4iX*ls zzm?MtDI*%-D z$d&efpmaWcVe8XR zru&z4ZnbNdA_bJrPfJ^PQ1zZ80&ZoG*Cl&bE>Em>)2eV$eU;~SzYyT+%Z(V*90qc8 z>$&Wk)d+$o;_jLPM`{+E_dvSV=X;W@Q*<>M0z#F)r1Ls#3CyaZHNbvJYRq?r5iXyV z-VpR8glXYN?y!=O`;#^E{+5+@r7)-}yjB?qbD(r?GvTpoyqgv<`_w_EK(YDO@f%Rd z#E$(F)_S6x)tP+F#jq*l zFR1lTxet-l5Q6rTb_cbML&To7RM~4=xNB^!humw$p^BTO}5Bx;1&fVAuSr zIkwwVYVe0X|BWEixS64!k`;UYJAh68Gr|R5{+s(LTF%8Xt|xu?icV=_h7Wgg7I~V~ z9oQ?nv(|u+q^$JOxj|Q=4_qc$#_{)u%}x=u2mDnTTMkQu`OE$}>e??YY!#AA-ua$E z`Udfup`N~b@Ro*QN>KOv^P#AIIgj}&l|N_~P}kr;`>}o}G6`tSo+K3oC*y65Z{Gyc z2&khJuS1N;XXZ84vh*uVcgHZ}z>1P`?TTZPj7wlAb5JlZz_|q+t88-X2Cp4yD#o|( zMM=oeIi^m2$W2P4{;i?!*Cey|7S1S=oh`3ENVjEE6egFIo~f&UvDE(l_A6Rom!7xi zHXpWRaaSF;yAouisukEn$5Dt{EAvfzL$8TLi(>KE#x^ZZox16w<0z5FX9(j5Hl6*N%RQ~iG>Nf;wE^LZ(Jlwnr-m@!rc z?9LMiWZ771hQIvO^EYf0%OT{gHdbagG2?uzSS1t)+RX=@Od$jeP(DG(VNHgnLN_)j zQ0ZEc)Hh9NZ(mqdHw|D1(S7#eat}zkwDT8Ca8Bmy=En!u!j1E@OnAZ>;_@94DSLIZ zecQPB!>c;MaVYOq?8l%(JB ze_Z?`qY(?r-EG6l#1t&dv!jraU%8NJ1IL!+&2Fy2vJzC+2>u%jp`%X9SZ1_X>hIQ% zr=#vU;(kS?A4Ww7Pajynu?Ak4-%j%%Xxa*QNG2|Q%rHw-7|nt1!mDlTn09#9OZ1us zpGMi?!h1?jpTj+59OZ&%MP;Q5n<}4LgYuga33T=hbP9eQhYiTk*W=f$eK5Gw_)4kB^sRTHZ(I)IO&U+!D_*c6~yA#pZ zVhdslhEW5!I;l=5!PYA#3~Agb>uuE8!} ze%OeuQ8kvEpU-~PIYTHh7JO9$=5@5*b_*kRaDvwan08;X_1rqB3|3yTPh^qRRkn3H zuTY98$%3KRzhfR2*&QQzQB<~))^u-tCHZ;p!-e6}H{sf?>R@}RHzcowL5v@Oa+Luz zK&}DS6}IzrbW0CKS54K_?#&=!F%=}dUC`WB34+wb4*;)QYV(Ez?IV?5PVB-)G}|Y6 zX!oT8$S?7I-FOdI;ct+MBcN#uwnctF`R!kc`6I)$1X|Q5mh`2Mv4JcJV_%hI;$51y z9i7x@y&)&;m2{zQi^l5lJZM=weRt&be3DZh<6ip1N=kbbk5dZ4*E;m1x-ms2G8#XZ zyQiFARO~${>HFAr4&F`LE!&xPxhl&7Ro?@2QMylDtEiiT#LnqN_D7t)=y9TPyjhn7 zZv}1u{f{40IUF?csjnm+;S`LR8Lq+Wj{g@k3X4t_-<^yy5!m-T_~V?_SF?iJ_!wjbf_HB+-AQPEAbsGhNsl zR6P*>D|!$r3mX3hqK7aLJ%C9q5C=~-D|#jdE(T`Czdu?5kJK>!^=4vk{P#DE|Kqq( zbh3A`H@3I=C&T)`u}|lK{EL0sE@VFuJZ0_Jf17mt0EH;|SjYTymt^vg*WDK?)4t>SJX6`D9HtTA}lHuI{Q;spfz)IF}_@`5Q zOzL5xDr)LxV?{tIE(Il8bL+%U$}ws7jAF^abpcjJ_OHE zClQDn4Sd6O@7LoSvf<#O;@_iUGxs)BB8;I=%dUWAdeu%;P&8HhZH8`&e0M_SPWKo8 zD&_;H$u60uiG8mOv*?w)KNrDBB#Q}VnMe@dlVMwuoFPQX({R+5Ow`lwZ2E1#3@t+- zjoO4{kz1Z77>D48XtOw;Z%19HT?5pRa$X zq8_4Cc=~%zfLhxGHO-)2ZMAqSI8Vm|&-|abf%!ciylt{c3-`F@{&i&iX&-mAG!Nb9 zP7I22=@#gmRx_EE|A%w>kCKNPe|!}kZ#dDUZfJ07)k9n@ecA@VRWag)U)ifd5OJew zlMVLA9dp5cG~!>z>9s}z*a;aJpmACsyweV86dS+BNtY2FXq?9Up@JqZS)&gIWLH!9 zvfy5iJDsbB?|hE5&lIJ)GXe*OM=atIcjG3E>qQf}SN+_{UD6$D9qZBoatY8AoX=3= zx)t$5?<5VBhDbhB#$yXyJ0fkiOYxnfHiGG~7U`2f-!#R$HuNkD)YQvTZxpnYf5rX` ziZi^mL=lKR_`0dLYgR6v;kZ{(!u1CLTAZ`E z)dR)mT)Cf?dpnL!`XT3Y+51{{`1umq)~~dIP5dein0sbCO|Mc5pG~+2dNHDwG1>Td zUq9xp%fzVP@q-6Q{b!=`8h?NC({dc1kK$mKYzr<8aZT7lg-Ngj9T}EV<<)QTzd!by z{}A=jtdtDI_25y=@qpDv2#e)6ZFL5JZzI)D)uNB9BTi zIG+s@Sz2)V{yS$gM^F&b2WAl)UK75Iap%u!+TpBzZg;@0Ep!>>7OSLy+oTgDC!evwLD0lsV>6lUI6Yfp&t==cOJ#!xi zTGt>!`#q$~EaohxvfTNpMA=!vzvrYfi^?@O;YLtC`#9FB}?YVn6~vO7nc3oJx>6UXknc9vcH&WT?u! z?QU|eUX5K3UqLl8RAUpBini5PCm413d7z4mufMb;(3&q&qsu6`C!)rjS6+t`IWhW> zhGGD+IQ4|p(*eAJhTrcHWY#L75b*MecwtFJvwDdZDf}`)y_tcIjYt{DJM%FH8WQ&*1tlwBo z$3SG&#l+R)bE;khH|I>XJ!Fj+^|!H(ud_-I^NaR0?0;w*_)hSLfAaVY8GaiS*d~_o z2|sApGP|*BJavn2MsbA`m{+g7(*M(Lnv2oM0RxptdQ#T}uxxlJFy!XjVKRTYc6w&u zUf;X>LD6pG)YSKM8+y5?Etiv3$uD|N(4dpCNRHz~Qv3%>G%4>oZ9fSgn$DviXA`kq zom?|{hQsjONeAL*@xrdfoP068YUL#v#T0}uY`7jG0+O|-`R&cQIEh84|Kj2b2CKiK zFrmF+Ci=>Ewc`w~I-D}s=R8Tyf@ZchnVn-T=WKi9pEY}nLiYGG&t8*eSo{Oj-}I}H z9RtU2a!{v0P+X*v$re#pGJMm@C%X3hN~XzB;a{*VCWd`Cqbuz^Ae$twFn^L_Sj+1_*J#QTVvlospv*n_w-d_*>EtpVm|<44L)iz2`_Y5Vg2`ayM;ucRQ6uo_w&uL9R#ZR2s@fGIwZBfQERAg z)Qs&5*%28{KxfX|{bU*#+wlNrt=Ir$*&~ov$RuJJ%^{r{&s|71xph|`+)bV`b$!%( z4xRxRziLD&=!Cm=x*m1uNqt5qZjt!Qo(q;f5URYcgLGN-URpcUy?xZ0J$9?Evh=Jj zdgFGhze!uYtVqG|`2SlG!~$&C@+oktD+W$=|EWpq|8SZRxEZcX{!cm%6R8&mqrt;7FEyFaA6K(ZJ!+ zDL#Bp^M&2U&U4JW>zY|Ls_SO3mxEHMk;j7YNnkkcNx&8h8<{UBgrHOc7>;9r5HyTRvW_v->~rtg zapVYUgpd|5Ex_qQVjwuZoC&lmJjUbU3R*`8__WpbcJ_1yU5J7=N1v3i7NgSFKg$&a z*}&WOe-G*^_eX*hf1ktzyA+6L;uL02Pp)Az`zvT02JDj|B#3GFJ7~M}vYe^k$szy@ z+Wy@q!vJ1MUHBk`$xUtLvtm=60;LkU2c+L5>S6%~d0Np0j|MFWB(#f6Q@?|HmLw_K ze@U<%J9O$Tl2w7ZkQ;NMu0>=~0qHl%1+2W+y~fta8z}_x?{rH}arU#s@cnhUgM#YI z0PZw2fRI-1>|cE{DcPRAIw^$cBQ`A=x3TJ$EqWv4g_cbO0Jg{=E+rqT@;=o8k}Mgl z37ZN{V*UBBc>CYsgF-VD9x2- z6PTh5gqHrxDLW}Z3B6Xix1x;fu~lqUtTm(=EJPm$Z+)b_{VaXuCQ9F(_MNXx?vq|bvgx4#=rqw4(Z&9 z;w$i@raB(NYf6@{4`JgqzE8rSxVN`_QdKA{CxT}~&QBU3?Q6pSccYBE!j7K@F0fH% zWJFpk43`)W6`$!e$`M5i#V^vye2h1Q{ZuJHH_&X2#%`~XR6Y-PKxV{Trc$&8UUXeY z37C#Es1P73orGWcJVlk_X0f`{3m?tSh>LIUPQ@A4;Az94U=0GRYM@?n6`OxwIMICz+!$F(J8;nTQ_YkC#tmd8+!aw-v~kvpn)C4u_= zqvT*pX`PqJNcii!}OR#B&#-}QP4h# zV6vgZG-1lp@Zs?E*Y)q-k5_%^OXJEC%_DcX!$I`-cD^~8zP84SG3Meq!|M!ospXCq zpe+@WEf}p(R;-op0WVq}FC$)*A-0EdhoPPB?ih+}?O-jXMspbvVH8*4lN2k^DiEC& zdd(sIJZfgnIvl3|o7G1*r!fYy`WoakR&Hfdg(*-OCX0Wu`u@1>eHUw}qV_ha2qYDJ z%wLaT{Z&bBmF-D!;@i|1Sw5^l{cfn6+9k`|#SP-%Zb(es(-t410ngHz_UU+ksqK?DAS=Q3#VZj4y`G3Mt)r(#2PKrUi#Z^$qwnT0M{LBWu5eo&8b>Q5VKA)|d&|_Df#a>@48LI#z zaaaOZ+%Le`ZJh?8T8r9_f-_zFNxKTD51W1biCIjVK&n{D6{vpepYiN0`?Lz3Z5tI4;BU$U51iI4A0wAQ|k8tNVzdAX!THc zb8517a5pc2 zmn2IV7MXxGHpe?uPxL*FP0pD$AHUN*RPh1-k(LeDqJEvRKY{h)OPqX$en6{3S-NvMMHet56S7MNxGIE8JXp+@I<3ZPhy6Bp*7J*dOQq4SQQG&wYCE=0 zBCRdC3RHO#C(>yP74S1^?l@OhiSsLUv@|uF%+y^sGB<}tXGHq3FTM40t-@5X%BIJ> z_t~UO2M%j{{a*HMtf&)9Eb4e|Cd`s^XtzZCkIi&DLXY_sYjn);6@ha2tzEEhKa}7` zB;*WBnH+WyjaV(#H@JHf*ByrrnmvDBJid1OM{{#JAFr9>>Fb>(9TGUp1O1UJT-hBp zWiw^p@7!a`#;l4(oXcEIQH zzxS!Av&>#A0KH8|93UY4{~<2_uV;9)PybrKfqA(g6(AlPG1r-i8e5N5*@g{qptKGO zKs1ScvoTR3rSAMS_v!mhIZ5B?H(wNYBXIj3@7%j45G2T{dUsjpv;17+5zSjq!z{ml ze)|0e&aD>n`IEQ*sma%h@?q;7e+h#uRhAie>SnJoEI3=S;t&5waDN~ESvz3!_X9ej z8sqxt*TbZaKv+Ecq6d}cC>)nLl2$aeDyDo(MX}!-9iql^UzJfzLI^EoA(YsJa)@7~ zU&wX@D(@;1>e<#E{T*BFAGU2tp5_o$T%>)QyaQVNeS1GI-hYAeblMedo#*g(CK%Ba z{xS!8nsi-IzEUf(Y<`~TDSeM>TYHxFRU1sTDOxFt%1^(u*n$y)fO_UrXH$3m7F~UD zrtXf^)bTa?G9h1SE71x|30t`muQ&{biRMMlP+uAIE8nx^wI%ZS!B8o-Nx4p2t2blnDAdc|`Mp(EJunvbN?kWH zrDsd*o~QD{rqCb0%8wjqW&$udtS=&EC?lm$aZ~>&%0>t*R=f))R7HH=HB_`$}spri0N}!eLcr9$y z`#Zb^NqOCE4^E&hOUt6sot_!${W3!^+GMUf3+Um?Vs{sDY!Xee(8v{xK-~Nu+rlM> z$moDz~G#0>E>CfMDcRE`#nv@b$sDiq5OEm3IrS|>DX*Bz#_l4 z2yE9N*!=5q{OI6qKFhabxmbs49qfSn=X|~JA9x&;;+>Eyx&bX@B%+#Q#~^sy*dcF- zy=+M?!XxRUO*>h_D|8@0mzzWtF#C%|;doc=s2x{G&Q&=}N;9?A6^+4%1q;9OXXeij zom1$#KYtM1WfmvjKtE8q{9&SNNQ(Mc)d|Suj<1Z%;Rdy#vxhGn>V8f)_PILU4lS}q z2Y2T-{DSb!Z<{})K5+nztK{7<@C!Hy0S8?SUCw={1IFgyZ!_5TBu&~{6CicczbU?i z@FnO(41vCpydaRUG9Z%Yk-f}U1$O=HY#LHnB&bNBD$i{q*IUU5lJ9_j}d4K(gU`9=gc`_u|ULgHN}K~+zdGp$&Dc@mn&lE2kWz~(Tf z9CVSkWf?5QW^AD~N6$lPCqt<~C2&N&p-G>gta|3l&sE*Q{VX{*7YjD)zgsu6PflG9 zmG|ML9-c<9fzV1CK1t*ZnEob`u=^w{UesA-4|~y!f?TGP-@WeA#qygD{4Ddk(fnDe z5!pDKoA0m2aL8dOEP2RCseviz6*$Ar1b*kTsv~L{r-J2zLK&aRX>E&B5A`VZaf#fK zw3@;j0{=j88IrYa7uH*0g4z*OtzxvVU=RnD63~E0&3G|qqannfS5yT)1t8ce$Jn4O z%HuDYI$N06pRlAu8OQ*@JVR(!mBM8xEH30Nb}9kGie|kcT})_S$mF4?SB_CU`zcl9 zI`L2z10H6>U9xkf=0Urq8Z?e+Dosnhe4Y2+4&eCG@GxwEYb-VKLvY76)KN%37PDr! z%rfoQb4kfl2a3O#2O!C0{wSTq_j~{K(@#yqBa31sRpoR!B*6j(o!t;?m=)uw`@=1- z5ysw+QQval2OQA;te1VP_?THfk z3LsmXtE4b5dx@d{=bS@fK_=RPkABbsy<%{m+sSUZfF1Q~E6Z$Pq`;Sr+Fjdx=RNDl z2)pMm=4Cy7Z#eq#dX)|_wRIcDf|4WE&5J$hS}yl!$XAaDCE z)bP}==hEwTs!{GPd-3zs@yv-JSUJ~yizg#EV?ez6q>*_XE?aiDnU0v(Vy?QGt z9@@Dy7DjVhf20{&L)LPQ+MT1>@BN!V4~raxyayy>G=OBx|GQ(w|0A*e`^^Jjo$qg8 zBj=$;wksirXFWP}mYcFHWuOgb4johEqI@0;7@-iZp*I=mq{~C^qa;h~aB)V@xhx;R zYPHpCLB+$mY1J^1MYQup-aD#p2vB zyO6EoM5*1%+!>^dQN&A(Vcd!|ul#pA5z#-}i9!v#zj82CH!7RV<|~c>OA#WyqAb9e zs)7Rkt({2yzuSq_Non^8500b$ZYMgkYVy456y7kafOLsu&K~m_vXvvTWpf1G|w`N@B3M%G#KZCW4lmpx>pvv~0k-f5fxSHvBEpUS(Vil8*Oyd#ePEcflQ0Vex_O$BtUPa$X8HDtd zwuFY!3s@+ooyNQuO{}5N)N3btMKn^Az&Nk)qJ_2_cT~#Pq!5K+^jjIeWz5}(W2(@h#hpbWk(-u= zxN>$f2I-E58w-zC0aj%oS|2XN&(KW2D!--$D7bWz&K@lIy?kt1vhWjjP*l-Wa#g4>!`yv%5o1A&$$} zs$DKdAvNXM2Ya-swRF-KD6`#0NY*6$m&|t6xHkfn*(!iC8{Mo!Fz?P%!}AK@mR|v1 zSsUi3`COJ>Yo;JA*3t~IsvQaEj@lQ)!ESY~pM=dd1y+Qz8RLNq2&mP~OPLf3*{-#lZo0c+{el4?@vF((QwhjZKd%+|II# zQ5^igRJJ>=2?z8qCj5exn$5*(>L2!wpYQ9<|9>Fc5iJCUYwr$Vcv2EM7ZQHizj&0kvZQJ(D z-gkGCFZr_B?EW~NbkdzvSEoBw_24)}NZW-=+lNcphDulkDtRO+xW>!7#L78E%Q{5L zIz~#_gebbkE4sxixW>skM5y_st9Yg;|GUg3R?R0()hk8CGg;F=OT#Zi-6vh$H(k>| zQzs}#%RfsiAWPFPLnkQPAiPjNG+!?yPcI}_Hz-FpI7cV&uVHwhVML)pSb;%!fqqDy zerTRySb<@9fk9}#L0G;~c%e~5A-9&Dh`ArPmMxctwWx(Zx2BD_RiL1un}C6#T0z~I4P z-2Lwef{wz*n`f+0wuoGBdK84QPTVTe_y z*aMA5B5<{GxHTG$(7`3cPxcBx!hmrk+w2oMn#>oAjQ#sce*94 zG2pb@><;@Q$k}GRJs$Qk8KW@>c{*HR=j+Yp=({_fuNTUdT6em?UayVm%hdcrqak33 z7Hat(eS5s#9R%4k7A!4lQ{e&WdT-upRYr-LO_Nc-^v+svq7}lHPya)wX8kJywWk= z2gT5JpBK4l-u>yk=?XINIvd26`@C;MrW--k{U?=p(&hFG!C2OF@WGguo9%vHbCdi2 zecX=v?S7dR$_GWcG71Pm)9jl4LA|cs{rSAFwc|s%8MOQT`JT-E1QYKA!n*T^kj?`_ zAL|3-ybC~x&I2JA?}yU73&fbp17jQOhx5D(B7n(<5ECCj%DfAvkj{rv8yi6DxC^0= z&WAA*AH>?Y3uT|mhjSYn#QQ)9eF7%E`9a#&Bb#nRILf1_b zTb7t{jPb%bpc)Ms16{BWwbnjHI%lU1x3P0~(%KvXYi-z$v5jjY**yT0kS3b`wbGx= zTq*;TtTCxhR)5@Bl`aQQKZY?EEzFn>WMx*LsY>NS-k^3fZA#R*GQuEUAK`qFN(`AY zB!4s+j*MU=C$ZLp7E(t7Vy*KQcr=*h#u$7#ruX~a(|RC6@0u&7bxn}^dkS^hxnxmG zL;q0iCl|aN1HP0*+Nyp;m*1R%<;BtPkp})(mu38<;{J@UW*)+Mrx^ z5nnDNf}&iIh*HjILKRa$rR+9=TnHml#vQ>dAH%sMj_R@?h*}|KXNa6J6-6Ktf?-AvqIeiGnAhOnG4;C3>O=9dTToEm0uW^=Szov|Lp5mkKGt-0(k2_&I81l;KAJ9ln(@HA6$x^;DB9> zqYp3;#wrN+zfY5YxVFgjqWJz^p0|z4Hu|R(l^>b59+; z8^SRb)ge;G`2aD4{((Z$Ad)035Ha*Fm%ZVFO}aE8Y?-g1 z|L`9Fvl#!a|11CTzX~i1TXPdf3+Mk-S5B|ajdjvTW_w2F`ucCbzkI;JL2w96{=yjP zB$^)_=r=m#`-;remxn2kE3eoMRXllt=#pApTbv(Eh&%;%s2} zUqKj~#v8Rw008|fHG27$p9TdhU!iI6W!N^=3Y%L+t<+LZS{!4TY&<_No7~hauFC)% zS?F9a|K85=EtdE-h};S{?#zpB7s_;mVL7|1n(JJ!cJU2A%?KxDeUGSl#qiS9d9sWi ztV=(ZGEL#jFm{S5pNpTE80hI0{Lc1a&j!&F@>dU2};xqlBg}2qzxi5Q!Fk`)}iU?qQUQn%z=vSr*bV0_VO5q{y=zA0IJdVVtXzO(U5r1bq8 zAXfUTprmjvAMus@)+<|PEB{r_X9ax2NWTTyjg5(hhlz{UkARwm;YQ9u$&8nt{ueFn zZ+tRd4svD+Jhi{sab5!DT~|7ixd;nJlD{~RNZ^be+-ok#69}BLllZHCEB0!ypG4*z=>81BZ7U8}PlgvFfGCniNnNIiiU#-MD zB6gbw;Wj`CuIzs(MZ45kc5KaVb+(ZJ;@*XRHwRT{t>BK^{SX!2zEpWTF#3@!Eom5V zB#IR76V19P)6191L4IBbehd1%W=NL-lvj4Rcg*CbCa57RF>a#ximg>)D{=0ky{ZG( zMEudZ0_twaxEM*Ldo89Gxd*aGtrU9iplHO?KK@W2pRF;=LZbpaytSK=?vLIs!(ZOw zY`1-x_A!II6>En-<#qUb`;-Y6GHH0dx{>sIH!3TMYyDglT8wtzhHuNtT-5NKG2f2s zyn@+i`>E|=OC!ergHUtiT|0x7TKY53{2=$a=dot?byB`90|aLESkg5HFd4=>g>(%c zq@k(Q>s1dGP%=OfdX+RI_maFw{Kl>JCWzs)j_=vUxp)ZOO#|bH^Nqhd4!;*g+*-vx zyq1#7fWdWuGo0o`ojWTLJe7J~^(ak1t-Cy)NOw=TrB8U&xHBOL)Ecbm8ktIdZc!1E zBMB#dTrHqFU}wK0N1;`@p;+cr(oW#Qgx!z1X62rW zN(cJ6p%&{BwPDnT$(|#b&U7MD_3yzQoCtAu*}z7Nz&g^>`SZ;|=szu#5At=oJ!Gf= zXiP~{9dG;}8(sWkO}lDb<{zzB;guH?o;Ze9cG0!U$|hNMsEZ{BGAG&{Vg(Lc#+{L1hBCY3c=%dNkwyz2RiO6fH; zOkHUX)}HIO;FfJjJlf?NAS2(o@-o;xnAs_rC^?Ty0qy@nv4h_t{GOwAz-`Jgbt;$f zl5yi>ogBMlJ+hycr+dp$yDp5_GwCLr9B!lMRKAznnW#d`_b^%wFN0(}H?X3uKUY}% znXRtqq1>_r{WIWHiTJzXRhm*j+PXNS;#R*xX+hh%&3EGEz&@ep-L5fe8p}7PG}n`N zpGWm z^~!T>4`imG4MENsBw=41kkiN)Psbi~|LI7KE@iPc@=jA&bMZONc5jfIw-_dIGhO*q zEHL}?=|CsOe_!&vJIC{Nm7SY7q1^g8IK}H-ddXuKH*O-@$e+ET3b;5#2pjv^Ho01H== z`^qrjwy>!qQC3jVdN@femxq&i@^7M(h~t~H92YOhjx0aEF`wdrJj?a6L`3~J>BD!E zREQ`W;HQDY(qQ4_lZJ<7%kb?fLuh6u&9TGSNk|%mV(a(wko&-JqP+(e|E@GTPN;73 zg$Ag4I_W?tRq9YDNM|*6Hes>!Bwf=vpkL}=gUMe=aie0u+rLN)n~9FQIhxP!$bS*b z@!v#NIhwY2$phSy9`s=K`FxyY0^^M7s1;@TL{57qFwA``Sas)em6a354OT6YNZVK ztg=qkuAi;qwtZ#unYM6Vzf@S|xtYl7o5Im11rO|}X$&qr zlR!p7SuH&B`!O*)odbI~h9qiZbT0-SE6#pMrb&hL@F+)##g!1`dltp6g{f4Hb6Dxi z>fYN>j;z^n!DQ1=deHlts+jC_!)eyhV6b>I?*Y2+rB>7E(Wk+9`G!X<(u>8|Gh3-* z1-|ZZ*ny$Z7|gPYdkfB6)5ww1K~UZIao|(l=;Nfp*a)l9=qct*BNaTaqLb-1eM`#R zLyq32LD_`A_a-+^onzxT?Pr91QiHoYVdcJmm6Fh5@ZcWTx->FFNfPe0AP-h7tYk<^ z{(OXvfh|@xN|)Ted2B&xdl4^Wi^o3e1YaU$(e||ByGf*ES8*86=cb2_PzF8fmT)bn zuUhTgnunS$rq~P6vsUR%q?&gu@Y7H4ns~_TXUxjpl?Vd0U6O~|(U0Bh5#spVx?~c+ zi#;sw1@%tv!RliMkGTC+$Fb+N3iHDQ?pV5=pqi{Ygi>_qwc2jNoYIxG7e9VC+h{b zf(Qp)lAlHQFe7e56Q+(b8yXR`I>h|d_7Zm*q`4WQ$Z*v7QMqRI5&9;PLt?O; zqf!%=;(}jm*aFMVn>87byX2;GBRJS4K^R0)*L1ikFxmSoB)YWnpfVzeLM&Qj%fn_*)!Qn|JVbStgEWD;I~7JiQxa*rKyZy3bB z9{d3j*j)f{b`3a085qA3Eb=eNgA^#Q7Nj>Ql(&x{Bnm9N=LQ}L0qRpItvC>J(Hh>+ z2;SEYW4axwI47U_G{GCPS2!7&d=Ra;3X3`rieVlWkU6{p0a#{$;7%iW_fYUyTNo8$ z5KFP3bwqwip8RMM!e`2!9eMsO%z*Wp5RGhL(UwpT7RY;Uutq1i#{y^vR&Z$GfG-q? z#c7y&OPDulXs6LYHmIO4au9P7(NDlYlJcMdc|JnN7=M7cx-ZBYBasw=bQsS-8ZAu_ zEm10u^b{i!6>4fGeuQ~kbY7ICd0ZJp90YNwNT(QzR0uePut$hk*rLelf{0sN7$9iD zm0UsBATgI35z%&`gyKHWd&165Lik<$y{n{jqq1_6vayt}J+_xQzSk2PgDGY1#WT$H zrCgtn)LsPO9RLXT007-)fY}W{Re#$e>YfVM-4;(QCy;wQ{-`^YaY0Wqknwt#g z4X+rtm;iT{{+CSbo0sK1_W~pi4rEj4eN~Y=pmi67*j8h#d;j8FY2|$_EOH3Z!L2jN z-8^~E5E<{exbn~lWG>kpB$;CknXj^xa)abitazSY%x_=98wZiZI=zM*y)5ych+YB( z0ZxqoVujq~ycikQMA6SC`57FGb@>H6(%m-qkd7?k-U(7dg&IQ#5k)4`>g<=)W z`iV*c7r=4KVu}s;l>2rdXP9CQ=scwJnO{ONKQY8(h$m-Kzq*NVZf)Dav6#Y}l_raD5W6cH!ft&*$Fv-_vxpJjPI zA=z~VcuwJHKNCDxyc|;zh2Z)iYTQz7`{Ku8XWnU(-^m$;8kxZBQYk%BO{}s-WM){v z%zij0eTkC?u_2LD#v3&!5yPigK~FuPO|+PfI*1y38IO9KW_8f6?3@*PQ39v5(X4s?hBMqNO^d z;Y}5Y^DK~CiKN4jG#`;Vsg4T6jFiKaEMJWR?;{NpEQTplB+F9Nr91M|F1|V}YaAou z%tE3Shm5mN2zNjnDLcwx__*xM z*u_r6ok+eeN0&I5M2nU*8%O;VG{I3Q=8PW8nQzt*0GO$A`66R%3a>Ng4=`5inqEw-?Y&SH{-PQFCd5B}3E8=86@{HfqFh zAb^#I0!b?ifM*{~aoD$tc3|g5D+mz5G`f^$Cy&X5qVq0g=yAc#f-kqg8GSvl>T7sI zr1MU)wrNze+4Q`oFtRBY{nJuu3Z&Obugeo&k$qaB6Nuota!+~~%wZ(><33$vy{+)i zRFrt1AFX#)C5=McFSNap5mvZvOxJcBtT-;@avz9|x32=*pooRxZW($A>sCySUy=Ct8^`ox zcF8GQ7OHe6>Tza@OlAsm)~*{H^(wPfewIr9bcf>~y@IS~vdm_y%;m1E7&^@-5zU_Q zX3g*>Z_!qajh0M7tr%9#=uR!z_RXKQ7WU<)pLO%)S6e3jRu0eBa_J@w`sU5?=HEIS zrfzfSN{gsDYo>i$*m-MsMvJJFOZzelH@0(eMvK;Ma|!>J;w9&y^rj64m!2$h=q2mf ztTTW}=g`d-v&rUn-KG@#)-*9URY5L*ZJY8l)hSV{7B{hM8?9z>H{=c`-$IVsPNu_x z;10!sChXAx7>E_P2OBu<4IFP6 z?7d)Eh;+CJbU4Wk*obhr@%~U0_t0HnFcYIt-3+)1;845_IBF*N@?n^W{?MCg7@u^U zL}pxHFbsw^oF5@}2qIc#eVR5JaJ8CX&*?@ll>*h!4Wb z`v_ZRb+Kk`Q_0FkViWg-`yvsb4@df3iqy02>ih1uBpGgr$sQ6^jM-&%#42{OCo8fr z{bV0x;HPAx=vQPCQ%ZmR=V=$q$8*Lj2Fhb5C3te~ar=g5)5jg>BP7?(D6I>+|+sbA5^mBPBS;c0X{KT$z)vf=PzWOps)tITq3@L9{dwDbz zaiUUsj1+lX(0pvT3z1WUuIO@-)O5l!jM9rjOP)q^)<6RgiF&y6_^uWq{nQBB5s~^V z&ZUvkd~t%iBX!#K2-|tbr*Wd~nM^dCtk8tcEltJsF>Bv<%H50F>6tW8YV_Nmd@-fg zv{UnjjAFG@L%vEQZc3f@NHd&Ly*5<)zEIS*&h${&d0cX7RLyo#@1a*PGqV^!O(m>THfAsyZA}U20lM@tyw4*34_BkXyre-nR7Y>?m$to@!8; z?$l6gYIkjJ*J>JmNYx~W6(z_H6*Ul}&pzS1#tlfs^&FaE3B&E^$;RH~iazLc_T{Qt zG(DGiW4$WtFyaBH2jSn z$PwSpp}G9|zV>N3?@rtJeD(i?pMoD!e?_@~%L&Kl{_(99 z(^qOwLFa4!vnM{ge@A^VXZRt%aXS~?@Vogn=J+v(R$p6G=jZgX7d4R3{dOPMVMH2Y zl-4r{u$Rwyu+|alSf=7_9nIcQ$#u`l*MP>ipUu1P!VXz*#nFH_oRuqVcN@(0myA%B zq&1GOF$e4&SAdcnVUIHMS-$3t=B3{wc0eQJ#lvvhg7>&==yoghIo{`^P4Tk{&LQ<< zZ%zBF3dUUkq4%Evq!=~#GXwueiT#>!7vG*;szh7AL@tZM>t?Z32Oh4^H?6E=Qt zb)`{b1zKT(wCASct3QPAz%k?5vGw*Vman;vuNeY0I~>AGU)@h6I!)^vZ##yju2Y=*G#ma-UfKaX}ixNUAF8h40!G}?1Y$2xfWl& zqqI%5C;RonbEyyobA(a&*7vC_fQ*gS-lx22io2DePskoY64;dq{Brp6D$ujx zGtLa+C0&STSzoOcEm6^y9nXsYy_f6AG8hC93|lm-KquO0C_$xgjFmLy3q74$7Ns_Q zDjC{=;u9?Tb6G@~mZMk8Jy-Sggy(Uk9`5;NiE03sBCP@#@7jIvUg=RVz+)_o6ciqwcZxyo-?bUwX~1#wzG?;w~0$H<)ZsQ8Gc>v3}Y=uTW{7CHuinvoTl~ca*zVLcT(2gSQ`beZx8_VX{Fb z28*#Ta@+0Kac7U@U+PuGiNA-w-}bSglpBe32j#*NJ|If0C*alUm^!M#2#I6p#L-5- zn8XMX9ys@$$Hu%L)7OjbZRlG2a)zY3iM}Jyi|RZP3+{57;-*KbKd6Pi-GfB^Q{tRl z658v;)EeX~hi;kox68Q*Mr!?A5Ty-)Tf$kg9en)6-GO@dJ=LXh`!`B!eD+D3$zwQe zQL+NO?D}+myivJ*1}UibpEU4j8m^hy)T41M}eoiOaW8suuv}dKwQIIHJ^cr+$`w*h7qd4EXSSL_x$IP3|@)Bf* zPm^Z`o+4UV-`Sp&haMC@690x-aw^6yVuLxvkoxw%-O;sp^m)0s#H z8CxR2{;0NpD%J@Yp!gef24dDK|d__DqBCa%!Wv{jHK&-@^VHg10|k zT8pE_M_Ouj!t~YpbGl)529`wNM`2chm1IL*MZh7l+!2dSveNUn&ZwH_%C*0c+U)Bz zL#L!?_uM7>4zA;i%PxQDK=_4!Tng^&E(>=WoE>2+O_9=naXk56qBY05_0RZvuv*W} zq(ivic(Z))`ae-<%XQ3b#_t0xwP55OhU<;@sCV@VmEEtU z`7Oukn({xewI+ixt>=-}v$((*JyTnIXy$pyQP9T0GjqHIRE(DjI6-T_S;Z<1-lJC@ zO!2)Z*n>(hBb_3-g2H5`7}F83Rn_15vpL3dlx3{j^Y18Q9Dn=GY395iUv>$`;de8V z&XaS-lY_fGAI`|)`oZCL-sq2^dN$)<+<>e|1x)Ag-Kas6MOt#eeQa_}1~pml(7$6^ zPI9_;_kQlz75t(bMbPZ1L^WuSJ=z%9@JG$$3t_sLCk%{oUgK{HiQ1Sq62}jIh~r!% zinrIw_Nus|uBdJ`?Lq?B7%@ZFmX~s2Nx=L(i0tj23pW<337#&mMmrWNCNxbXP|?@9 z`FNI`-;j_9@f|^$IF?ugg(iBw^{7{@`M#XV$wZHTgK$}|Ds~L(Y~jY^#dc))Hd3xom7VBJckVXj=O&> zT@0Gd$~nMDXb2_&VU;B00-NZX^>(P`9ba((bpCRR4UJwT)lH!zOmTiI&*fH}f}!v9deWf)bj4wj=E9Vwc}6YDES zFZOqThb=^CunE{-oU)M>`b|Jz{u*OcY;)1vZ=-zZn-r~1sHGr- z6-@7#W2~HS(>YW(u_*D!yqx2h1RS$X8P7%pv8OdmNm&Vbf*r=E6&~l)vbLZQ8jjhm zLOro0XVXr!`OP+eh`~*W_AXLXD;3O@%-vKr-mCX*5w2J(t#OO5xZ`=_bD3IN<Dar7fvHa_v2fk;{9O#QjRD|t_wZt5yVDka|)j@!bL zW|bV`6$3UWK_C}+PnO&VS4^#Ls#X`rm8&@wZL_cB zb4s`li=Re+&VNmGac9yCH;Zd-W z7L%Sq9r>`o0Qi)>4A85J23%;Z1S!hWHcvF_DB5|uaXK_HH<5_5bmT^>VVD$UrJ|QM zokxe4R7LaN95gyItkKdWX9u0jYHP#ntQ*p;mo3(iU&VDYuDY_OrnacJ4${8wpaUWS zjrui{-tfxYU|Gos+;ZpxlY{G;LbZU_f*JQ*C;NFnLecLmIajZ^0 z;uRkc_5HjSSGY*Dl94d3d}zbavLk2BmsIpI8v(xxJ~zN)PmH^@UlG#FIUuMQMZ@Ap z{xV{{Pb6R8`@5N|{-H|qhd*&=R?wkpveCVV2KAlP$F|MZtCwg$_T24}IH~&9=2K!- z*6&Cy%hqDcoyT>0Up)wY2RolGX(_*XK_X!n=FKOe?aE3az`YsRzF61!l4mA2_X_sy zS|oGZ5=XpyLq(-58(RPVj%SgyxpTva4|`KaeuO>Dc8kKY^L2$G(#DV>-?150{>+&A zTy67$p!xK){WS56wH=*W<5b>j-;T+b6W4Y5?_z7K!;dRq?y2isp3PBF90vX)F+#_M z_d#XlOm0~hhh(sa3T`)#jnyO9htVqX#ah#so78%($5Mdg^6Nca*gESi?04^@K@WJj z-@2lWT`oKA@y8C=y-s;kz7lL-3`c!6yZzB8gxV*s6}-0L=a>DknGjK(2p|6v;~D7v zZ6iHj0C#WqVXq;WuMpdmog&BJsBch%4s=j<)K6RCQ};jZo^)DrdT7w~=+=+K512}4 zrDky6wswpxHxZiP)4SiOxH~3~Q;fmmoJ?_WU|jYe*`ISQ&yQ13GgB{59%!LXk1qfZ zW96_Pn@_UxS86W~_mdzsUoD86=1-AWr+Wwca^G}1vMoJHrr5KG_U~1_xmmlLe8|as zUyQgX<+R|hyGQbzd%dmf^DcfMT?jWhxt3kRJhymc0O`JVc6|L0@;5Jcdt=u-jaWsP zkv~g36Qs=lxMPJ23#ey!Ii{E;$yk_H2>$>OQ!-4>GOQTKIQzvom|6H~n79UD5wp3F z1dswxnnD{;L7E$}d#cf3&5<*zQL_5cA7ltO2smh~7z_vao!L0YzP7Vs&8j)TQMH1o zBExbRBa4NlxgiWA%DCgCHEhuO{rzNfQK39Rtt)bQoYwu;uaq_VZ~0x5yn*L3rXar@ z0<#B+F;a#|BsdTp4cgQ9))p)XOp=zJyHwX#zFN3sB#0C8kI7sz+p_crf1sUebv@)e z)tfhHDT#uXug`ntvJ5_td`=1f?Og3^xpAg-OK2<^mkts1FI$!j)nx#FJ7Om!;aNOu zO5CmtEcJVult+t+Sjh>0-3sB%TQdtQwD{%cB$Aq>KH?T-(vM`nP(8tac$SdB<;Qq0 zccP5ica`|ht3ZU)M*vs#PeP!D?$fy2b5LHO#4SA~Rb>RtYG3CEB^(|lrD{3xAB8() zbP~UVN1zn4pf;~k$h$B0XmHYpO64yj>w!gP2w_n7Mk3BBqMo?2`R)n{glHdr{7Dxm z%>D4FnPuw{PhJUVLP@a>GE(zqK({~0oC^oo)&A>O@@&g(ot2Q2i1IZ+=IUN;;^4-! z#J00M2GhU^)^|C}z-PpFHvWgF@nX!bpg&K(oFT#L*W#-VGH!3koKyQ91h}n5Xe=3M zBO%aQ48nE)!Eb@-e&G@|Wu#PA3{rs3%Rw>KK{3in+nqpW%8E?1<-ArClowM2wL$qD zW|IYceTY$<3t&EMAXFcwjP2x7PlAQZKpUF_c!(1d)sz~S9CsE>__UzM2u;BC7nu_% zAN1=ijaN+6;bKUJ2`dP~=(s!E2tJdfRgWxPn^atBC|MR&eu+7TgPEirNJi6%*;F=W z6*%Qs8RvGGF5Asw(G|({5NKkdBS%3|9Q}pt0fLYG2{_e1wW|6AP8BH?~R2g3MYZnh>q<=)*~b0HzBD%nd*;KQSy9 zcwQFZhQjdX!`z+}ptMM%Fh&MxFX3o!bz< z{pMx-#q6P}CVs|$yQHqwb`ZaT_O9V;=`!Gua_D-BuyzbK!-Tmc=LJ$knF@-Z9Ep1X zSzUCldK9dhwDU;e<)buI?82E9rg&l7LL}RSMEyOK`Fr`!*KFBd^db zp-6+1VTrXwp|a*E+lk|UPGX$dR3Ow@M$UFPeB!e+n2%aHvsV_(C1-Z-(p2H_PJvrg z^LLr2<0cr@#qy9ZoHLi*%HcdI$io;5x8fKqk0y1tH{5O6KVgTKs7$BC$?+DYq|#6u zphrWh@;ez51^ulI*k@p0PbAOL#k!1j9(jj}Rt+dZ(o!CF!KMe$D=|y@>~EV8AKVye z+j6X)cx__MZ`8-h{hl7t1%W+v1bU=92ThR3U}0Xh5i0OOs*IsQt25d1Yyc zm5in5%k=6dJB3xtAt_>hTFU>jD8{MaAFnh7Q~Y%QyJs)k7FE;fp&&7;Gv1{z+M!Tk z0n-yNV_;CA+i3BY|F0^0UynfaI0AC`Qi_Q$2Xrw&TB%Hm*-toRFGBd)25VR(gR{~` z4iyw}7ICX(V}tk=axj2sSZjDLVRV%7b9adXUr}x(1w);cj>JjIXLiD30WQ$;Y`<&9 zx)Bl%zY(qEP|70R9!0$zprSKL%Cux^H!M=C#76m?&6W#Qepp_ji)Qou%gWmgz%$3}ftX!7xeP$!Mq9{f7xl&$ zNydMK5)EJ|z^$18mqFG{r4cz#DYH5ajj&t~Uu^ z9|3pA2=9aj*Ikb5slr}1#8lMJ^rhe|-fyeUV|rHe6^@}FEaVKL&sk>3W!BmIsaEj^W$MFHo`UFjH>Bcdo&-oDJ zF*IqrL}$8I8vm9OhCwUG+=b%+hA`+Y$|4d`?n+$mufV2M@Jm9OkU;`Ke7=s4#VuIe8y1;fQ9< zO%}Rj%!#(ol-kLm%EO_{xQ@>MZ>0JA-)Nd`2`Y;hKmY&={{)b1|Nk^iJ(K?>VftSr zTNKkdefN?7$#4Cw>wDPtj`~!R;iGBt?CpD)cW#V}s@k1t?N~Oes};4fYhKyGs@_(~ z?5tpJO|h`1UTxd9bFZAeQ|;Z)Q0@N5f+f#%foD6zDO=pptL*NT_0o%ZxuL4YJgJ^nR?1U*|B7VvAEZvW(sTxU(l9 zNtHe$_~@E`>iXZVIm=ldss#5Der_Fy-|Kj@Y-zn~n$M5d zLA$xSfZbE{?EVLGXrf@D1Yo!b;3(uUSHIZ8!R?yDZNeLwnm*sanB8R3f0@_UC&>L% zQi28}TyDBDoJ<=uKkMtKna#ACJvlC^$W%@3)MjjIxLj?Ix9X=uCI<$$x2F%z>Vw2$ zJ>0|HH9d8^oX-bDCZPvbSX*suHFi6lCg((lR>oFRzmKETX7zX)Q5k=%ryiakTk&@_ zf~BvdG1Aw6(&fyp5)a0HkmnTQ-PI@Z_a_2Dj^WfSAP?ztL_*0mhn}-J-PvQbxyh;D z-z4a1eK#1k2_ED_OcH#I*#>#(+l7*mQwj;@MSff77SMM5#>>7z_f9zI~_g zU_#ev7cwMNOT%IK~>Iw^6L<|PUN2PG<<9EapwTSrTX z$jwf%$o-BWHy>Ms5 z`;DGYf2-bolvR4a>u&13*P`KT)3Ey0Os6rq6{45jMlsF)_+C`K|N6M@QfqU1qi|zk zv@yd`h1bzDs007E&&TC75-jhZ!=|Yr0Kh+@^Z%Ki?&SEtt$76voJ?4mY5!x_d#!Dy zwkh874bxi?+F245#ITkxLB?&1eMpi?$ZC{TaHm-;%Y&=dpj#0As>+`~)QXGaq zbbdbP(FbmwRw0}Hx~@WHVa#=V?L;Zc#+Bw(S-E46A{x=wRC_hSe*Y!O6M2<|q1J-O ztBndTqN6>2CJJ-Kj_A45z+Sv4 z_G0_wCiV`90pqi$4wpPyo6Ei^-F-adjN2-F<7Fl;<=fX;{CnDr3!%2V50CKghxL%Da>S*AfMV3NrhPXQeV&msNJL2-chFA!*QM(Ndj0kUI z2qlVcW!?a}yQNj+mn^{{f^wM^|Jc~1UVS2dH1mae0Omf|L#N~}kGDP=zPsLR5{WkVz z5t3p?>`lGLz1Vd=2?yZKd|TIn-e}ikSt881{?(9a+MP7>ym9T9>#1lF(Xst!(8QB# zE}wiFGV~lLLX<3sg3AOL!&}K276&?~qd&OvEq-{Sci;K;LYUz22tEl-#mEB0!%Z-#^*dF6)l9k5Y=n3RlH!71 z1=u0;v?UCY`fu3X5$Ny@MTazSUq+|&uM{$|#vvH!kCHeyzk}9cRWJ5$l=zWA+4TMTC*ob7m9H4fJUe ztgb*1FJaw_4kjZ;sW)j@iFC56~ zV6Xew6cCG&=<5sPJ46x{yOF`lXP_+bNCxlUMJlL_v|JVM=)!P}eu8y4B+#PR)iyuB z2UXL@davxB;hiej5}1Q4gK39eiHdTLOpBP9AFC8YXGp4SPyRXsG(2EY88=wB(U3_n zj1Q+r?}yFp`kM&Iw;%dnY{}5dH7Ew%y#SO1LuUl3pKg(C9V{Jq&bb8BNENAf-oYW| z4^f4^#{$GERlAPx*hM3ctYEoc7o9r?HMtiwy?RPS@ zk<1w^2Q1_$L+8}uzYwM?7#p$WcATL7!5ihSzXArbV(6+m%QDW$*I34q_ zz<@)V7;#Mr56s$DASG&52eDK&N1X83tuAKIn#Jlm0%Z#F0oo8w#q?-|R}6$)7#h;F zhfBJ{vMO(4NmU!;b-GBk&9b1OrL#6@IBa8uFBy2IxS}gE!PaDIy7~FrP+I zvq)RT?fi%|aTGv=CK@bkyD)Y%92li4!3+4W4TP6uzzFLAlTNP2R>yW=7bi2ezZO&n z7Z1aMIOKk)3fCr=mqk&^k^F9c9k__J{nWU1>0U&6^WCuMZHUbpJvf7P}DxbhU2INdq+xSM4O1tx8jc-}by_oSw`uHZ3yeL#E5h9)bR zthXd5TQX$RYsOvBpoBt4{u~YSCLFx;2-ffQ#$mrD{i`mNKVx zbF}Kkw()VolDX6CplI7nCa`no2M%GKUQzdX( zPe^D^0JktsWAz$s62_#N%PyRjGDt=NtyYhD9xki3$-P@Pducud2)o-`ZVJlh#c?1~ z_3=I8xoh2$`P?!KqPwL-CdH9UHSI&)@?q6w%3JiHalS9>BlGc?F-ui_R`~lZMP#aS zMi`cMzPaf$ui0zGJ-<13WLT~wZK*#>_Ltpb-7@#5aq&94+w=P|>$mYUmKyvNedpNA4PMp44WD$%X=c)1K@P^BIb-V8_a^t%UhI?19 zbx$};rJz-i-x`-VIBb4XoyX;Vy-W>-Xh-OgcjKSI&%yZZG8x5Gmi02Uq& zZ*JD+>&wQ)#l%XA`%Bs{vQ43h_2D!damhk=sv=c2IN;eeZeZKiwzFX142UZAbOYaA z28p=2u$wL;)-ARn&_LM*viCy8MDNcZ3_Pdhg3A&Tlr&I>5Xk%9+%g*q!?iTO(;rVv z(WR~IGMAZ_1MoZp90gQV|I-MKbXg?@scHSIPg9BUp!&qVhxm~P!PrWq$HMX#NBuNN zrRyWy_(=D1Dl)960jo|T-Bac!cgx13tQ9)#wfoVoWPi5QC89e1$!7@UwWh?|-EiDK z;YuQEeME-fVOlfQNoAY0#1CxdG9W;XcEs57x~UGEe^b^1aFZ|iKE5d-D>a7v zVuh>kEH$La(8XQgf1wr$(CZQHiZKaEP;&Pw~% z?LN^rx+8A%dEIYoKg>DSm~)Mf`VE>E-Xhi8Q2$;{mvrVk|1~h3s0HPu)@7n|sAftL zlt85&ETTn8f08YODXp-xD>=rp(hjW1s$=NEe0%vIKg|u1vf#szA&xp#%Cc;}YDj8h zN{X7&@F|z>@Q>J`M4e?c_`XZc;yZEu#@hURwSMb7%rG)VaOMp11zbM}nuoNTk2E=88Z`g?+7sBW2aWKe?MSpPlX5 zXz4RqVJ^iE2}y%h4eo52J5Li~31A++7jC3RKpZVb*AAIr+qqCcHCn(#xL;&6sdw;0SP{!zDzFYNg(M z*=p_CZ6;Q#FV#OENw*r426O}?dNjVKzV@qpRNsY0u!VbcPhrSp{k|MvTwpHzz?eXz zn;T8$EZ{LM_An-CUC6tDdf+&i?*RQZ5)KNty^DFdLNWhBX4zlDCC#5!_2>A6oLv!G z?cH(Z1d=*zV}ik6HwzJ>bQR@@j9QJAHB0kSv)c1MgQ7G>n^}yJ$vmQ*b7>2kCNOT= z2qK&vzh4dj)c^PNaLLkY*Yj&-Iv>u(Ofs|SH=|{9cXpHvc0v=p6U8LiH+#JJs51Im z&P~UIOBgG*)rYPY$PM*}y7IbZ?Kj@C6V-up^^5QNxAGJl%qD?qMgJKWZ@Y~DGakdW zf&LW2T23)>`vGEE7{C?hJ0f!f1t`Q`zfNWss~sg7f1IRmyAAh0N&G)`=ODGJDsIB#`UIh>g?(Lsxx;yrTizqCBtW z3qT}FBVv)9q-a*AS?FZ1_K2;cRGc=xz07j^)}6AST$T>8Jqct(3T{n{etWpq9@Fj^ z^z9NMC5#hWN3c=+ z+_pneo2uGUDh9&0hkAL6o4MX>v+8!qu-&E0ujIOPs!mC{C~;a(a;W}i=||}XHN6av zzgLW8_!H&8!vXP>Gom3Aje<8N?bV9YI)yk-X-S?4WWFU;^mDz0d&7MLu@lBFHduq!RrjQjE<=#pU(J-u_uH%~MFx`WhQ zu5%Mu0HU6WFRG-hHbkC~PUaNT9w>_wQ7Vdj=u-a_6?I-<5u)!uftv7Flt{gns8OjL z#57P924q~8#iFHK*8|(Dp`W3Z4V+4g)wvi91W`X15h<-;*D=9>QXxo;<7@=LgYogp z77N(GI4lM~X=-Ibq+CITr%cBL#Ji{30Cig&a<-2gQjqFLTB)fHUjD`!{K6Q9I%x9f z)_`2SXz3~I`C=>p#fzU=LhOU+dFSBfHBnyLuL;czHHa%N78Bj$itH9O;zf%`b1S6I z2k$vL2xIfoc#`<07ZIO5Nb1_RGVz-z(o`%?kPGf$DaSQ zQ#tHo99-i-Smn=58Rq*Vu#9G+$VRpECo_~UlcM3NIy=6_(L=C6z1yVxacVq~^pq1d z;p!ya%eRDK2iF=6SwE2S&C``#c*J6jCqU00(hm9ZtG_?fRa`)3TcZ=w=tbZP7TBo& zMQ-;G{0JW?NO!p2%lsw2*S2uI_gO3Mqmt=QoI!e5zY}72pT%112<@0Xm`moS?=(r> z@T%*1{O%Xi@-ffF4=o%u(wtzU4)^b-$8(1(HUT$S(r#}BGZEh&?Jm3w z#_pHBtGWSAx~eliZQz2LZZ#lFXRfPR0&m3C8-$vf#;JVJwdH6%s0PtzQvY%#&+0%& z=bicCxzp)29`5=$e|kUVmBkOb^JG;Zj=O*mJ^RPRVX=1nGr9Xn&PpYZ#b};-YVoDR zO*mwQEHu6)Qzc^o=OPo_d2PRr4B~S*g(jwoF)VoPEO z+{I1cy2wQ2SHq^1SvWzVTFC?HuU}QfBbD;TPQ9%wNoW?#zw|&3DTR9Pp!lN^9iN8h z(1UvlngUhh?8FzauUCN0xXklE&jy2p>B$oi&R2`IMiF80 z{4An^7MRnQSEyDC^ionQ<}ekUj$VPxZ&vi1+CUtAiqIMooR=fo-|BpX%yv&dJ~i=i z&J$$$r~+6Z0&p;C3hduSJC;@R^s?QbpN+NC_>p|F`Jsc{05^H!R%p=&5Kg0J6tA!G|bNyQC zE@_V&%agxF&CgJ$9M`d-ss^Viz@{}o8?7@`wl)3!pA;@rk;npT7$6{P0w5s9|JCLb zHng$%e-plhZ7fagT>iuE8`ZRR-eg1hrul(U$v>)&liQfx(#dg=l5Cm5$y`iS(ea;B zASEpZlT69kdMdLcQ3i5rq(^+#oLBdMGxn#$`kos*x66-wA~z4d zOZvJynb{4$dwaf4hz+9Iz00tFu|cWvM1h(FaG14dHN{Bh@!n4e(8BTS;6rDc8)+U9 z%W5Qu)Fk9YhD^3h2+5I1CxWf#)h^mlhhJ$P;iQ>8a??63H_4JTq8J8nh0#oKY~I^Y z|IHD%NiUTr4hr`{J-Qndl*cWfBi3wYkR=WjN(yP?@h{i?ybLr=Dy?2DVTd)bC33!_ zKpap}`9-rtkwZn2T+#{khV*ZW8_Gy2j5kA@d^Ui;#ux+Ic%13Dh_F_V9KZN)`a(6( z##~mw)pDXOYq&8?2_H=>?-a#E1zJcwAq+2LjyyGa{! z9vyj`Gc1M0KC}5rbEU8SM*|=q-ceu0jb_q}{c6W)-2H9^cU^hYzl7G_W_R$~&VWC8 zM)ptpcy#4?)7wz)9M59dx2%ml59_tJwNU_k1AQ}}R@__tZ3ns4xCpWRF@+Ph< zZ7V?sccZTLRk8ll5>^qEnp`*_TiY0iphln8VbX;lydVu`8gO6ZPiKG!XpZ!>-w8?% zRv!ZW{fl1b5}!7Iy8z#w@vqOO{wqdz?hHAa`qPE8>65GA(7JjS1 zn!&F;aoD*rAf;uzU8n8Qt#Db*8t0nLN?&VEV_0+ZinGDz8tGKt&0*HIjnBc5T+A~* zQ+ONYcz6cH-|c`t!;_=p*s~jHRyk8J3~&x;0esofa52gNte^s7@BF;;WZ!>sU z5mXsyn&4ov4Sn|w6NM`|`vI`*JXA*bsh7NjtQ8HXr9Ks<(IQ8u>Z6vgn!l7^KeJrv z!cS=={-T07cIgQ-vX^~Ej-Q_Fv(FJ?s*l`XechOYVBqZZk!c|(ul zVu+`U$dlrfOF$38bc8)NCYVPB5p^xA_Vg>Y9ro+9l)AW(c|iv>YK#rQ4!A1ecYWMO z=jG`EACAcozk~unnLxuSR%1Ie-tz&IAyyxdtNtIuyD{u{yPJF?O^`YdV0Bi8z%Y!T z`gD`@})FA5#NpM!$?o{Kq`b&7s*yKQmWO3#1JkJZtdohmn-Egkd#;d8j?g zk&41bFaAJ&ZEHN}@WOBGvrvv<_Ua*~a}0b2Pp7S#W{vSk0GaNTO%y+J0+&)-{N|{= zm=MWf`|DuOonJ7-HNdV=#s4N?P55(|Z9FWGXE65Uo=|NCj19+F8R7$Sj2na3a~%P* z-hwWP=g9LmemUjk(4(drId$lU21yo2hn&Z&IgmIQ^X#?uhUGHAZ&5Ia)e_kuBWvX_ z=mYAM5wnoD=onzJ_&&5J5{|=GQN?taUOG$TOWpx^;r|-me{;_AtxJPLn%dig|q%m z^+CLDPo45>#PzoiVk>r?r-Ld(mRIYNZq*Z7e4nbnb#SqTi!FHcNZT4)6t$v^toexe zCGi09!~#CjPT=t@PE;qCz0IbUnt_wz!A1Yj z6;ix|J1llFf{*;Z-Im2YW-zf z2C&84quiN4cB~wM>R5cWfhHNYT+89E`D67pW1YO9Nb{=YUjCjpz;Y4w` zE<)?vlpD;FyX$n};ZiX?B9r)~yq#58KPGW-@=og|Y;e}#Kll)Q6uq}^OUTYZ0IM+J z6giBLMSPknqHW65+{5F78@`_5zR((hImia3g_uX$zI9@3K;3r*La0|s1i9i{ilqQISxCKy!w>sAjIm4T2qO5s#z8mv9CaQ11Diz>B*;lk3aU=N=2vTN zCjuY>zh&SQt|0Ui&*D;uZ%SS<`71^^RLbB%C3vU}7P75hpYlZZ&OdIcEpn#Y@xDKlIY zXkC!Gd|9zWmQ(ro!?l7SR8n1Q1RpuozZ?kH4ZsSHw!bXPCbdc*H{XA2R)Y zKMP&Wt$)cCp%v;~v2Nfzg!5Tgl)AsRh&4+=FAS;mt5@|=`#|c6J)!xp|)Y!$+-tIrtYpwe3KlKXd*Y*RGlYgYD+NDd-R<Ze^;7Lh;_I`}-DDf-~`uw0D~|U4sP}gn%y~!0GQ&{1-OpR}e+}i>zn2 zn!&O1>+yE?uFdD?n>mh|X7OqfO15`~d3(|96vkE+NbR-|iPpkaW*bCaR)D%4t( znKBg_%o^L@IvR(<)yyHpK({KXo2hpUCrwN;JgcX|SOtQ2q+LG2)gYZ~qT?ok?K)Mg z_$eeR>>0ZwyvS$3vQlbigQ8lSNrRBFiSbzzUEZExyk}}E&H@Mhfq*|K-!lipRFSYv zy#SGO$jOgTAm5PC$+pRD8o_P1o4`8#x{bHkM9%;4x7G&v>B!N>P$YYp()q=_E=Zk%Lz*_Ah&=qc4|{O- z3Rwb37&1o#E&HW3ECZasaO50^qA1m#*9^SPVzF@G~`amHt)E9k^i zBWjAI^ugaJdFEhTF%Ktfoo$FUcQu9x0=}gb5sw+%>9?5S7T*zgGl=P9*T9JoF-<7s z%pHf4SqvR8!klNk0Y%a`jjw2*46A@r#^wddgCy^!pq4_cKg}#BA_Zo=5*lcw(;?x5 zXqp{DQ3;@!@^h@~h6ToAZ9`{SS18zAm~1+^%v<9nkiH(0M<{rVC}(8CvX4W=kf&&2a80ip8PeX=3-hju z_kD5=L6zgo^TJ`)FsaLv+S}LM@$|PMFIYMGpjTf-;*DQBzH;%M8>hOYfb)jWyKfgi zRxc)J%BnwrBe5>DRMu7*`&84E^GoCMJ*1nKFOal&i}>@fE+1dSck}bH?|Wg)+PW(n zCJ**3wC|d#x&~FeQMQkN&GsMOLvi2X&BAmfHlb+NW`CX=_=gV527?;yX)x^(H2LD= zr^-OQQ-%c%cMT<4QtfFBlIqPx2!kK)k^)gZye|8@+l4YwUvk)qqcY*gij`pO0h-H{ zZP)KyeTFv1?fEfp(IRnUHxS0tL*2wketE^wWytedf9d%HwG&kQ^7VF!NtMgAcN|cP z+!JVjuif%t&6BB72}+}r4~K-k>Q2U*OKf7_yz~!%?(yW1Ves;PEn8-Fy#?D})>XAk zdBqo|gVW)?;}5jr5*$jL3@j#CoxjlgcR)8=4lG_*L#C_wVR~!BmOfy;ARqWu;cOtr zZ4!$k1gpm2fx@M5VM$}DTu)2*o8A9FC2L5_blw@b#W#)4&OpC>pKdOH&YBCL6l#`+e7Pn9gci=b}@>((ea%br{uc?^K&m z(~6Y_y=6vhFWvF2#dBu~v^MOJ!sNxFf?7oCrW*CdtdWvxN!&KhwHHnul;VHW2#EfD zeh_k9`$yLWLU;@+*&3iF(VFj=uL@J*+g1eaC{b;3=CMV;7{{(JP;hH2VSXfBpY%Vz zhDqen?fr6P%GWavuOr##@*A7?Jzf`f75BRS!K7Ey=IO0nIWmHetgjlpyJJ7V|Gk?C zg+bx({M!`N_y;!n-@A#3shOdxjf!J;eU0LR8>XiVm2f{gnpybyaTw_`s>~n zOYTv^%?ksEM2Xp@b5ID*g&H>3>znpnzwR1yc6R(N0gDag zasqr>opjXKlk3e}pUJ?8pvkn|Wv*9nlo{L43iwda{!>ds??3~0A7K`P6(9ZRQ%u4akTASaX{ z?vXVaJMEOYKaeG4_zORnNGs7cuw&0e4pxGF_e6dY5f0r%n3w}i;DIvcIg(7~$KM%3 znu_$mntn#Eoc|YPZI^^dcOc7jA4WfzGrn`HD+QJgRjPf!2iPV^<}X8hFdUm_n`EeP z@jEfy5XYW1hHN6}3Ao!h3bYaW$UWwCDU2REx4Qu&;t3R2I~;b)26?IHd-pG|k_4{-Z&RP|vZ|5guKnxQSCCr%Y*x zX_nZWn;=tn9+Lx%0~L;Ar$HYXVo)i0N=lrpd!9M)fB*C%(nK@Oo>V>=y2>FQr4RyG zOkD1Y#gVmG%d(A32W9waDEtF`*xSa-iB8@Oc#=qAKta#TNNrhJ4-FPtd(3AtDIt-S ztiYRY&EyIn^JsYd=mpqpw$kSh>k%71eb~BtFL+-7Vb`IEp9?ul4)oyMLvTAWqPqFh zd3T1V-alrjW45rj*USFugUfc-Y*+g^3je3EBTCCEcgxHcKUm@S{I^KF#C7z(L7~Ow zu%~!@cLz<@bp2<%U^)RJMFY&ArIW{-^h(f7PBgA>pB*Iq*Jx~BGd=Dr2FSgWi;vTh zLD3z4-rFcBJ!Bni7tz@0pThE&On29~8_UtuI}?}CO$qitAnoc8cbm7f?m>j=>h)D& zem1w-7>#nj!28>Xm9 zWE=EE67@gi52dtAO&tvk8eV8q!wv~2;Pu@L=+EBeiN#cS zZRqqT4ByVy=jZp~OgN%&hRK+nYN?wzR1>7CxhbrkYM}O3kYX)Ud=(ZllrM^EN-gWL zm%X&^KM~qskDkKX;$`o2Y_@8en@M8Rvfe7pk>Uos(V^el9@4ZV+WGq-h9N8g8$6b& zdr?bO!1&Qvb8oK)ig7n6*vmCmyKik`ygHebrl=D?$?^BS|(HV20>bPty88s$ZU!v+{(HfGUm8< zytb5tcoCenXgIK^rGjfAJU(0u@w=Qhc&@=EfVP1NpL)%pCVA~5mUq7Fc^qs*?AS$u zp}u6v_{?K@ThMaTS$Owl@SgE?v@R928@~BvcT>uSbW>hb^5UemH;(q=#}~ zU4q;vX$p&C#}kBk7a4c7#{&Fnn#oW|AO}dFMmWwxgYcesG7}3V{*wXTgJoDcMJmDw znn>Rva@vHaqzml%MJA%W!61@iha}zpQcyDi66cfEX}ehCRCWj z7Z}s;>>eCeYy$Waq}w+OuV2`-#4 z4RFe0ppPZ>H)#dQP}e$=WYACA5pRs>4r2#SRPQ^Sw?ot~s{x!bzC46vF)~4?ENPG? z4)nC!U=r^o^A*gHCdawbK$wP@mr9<}0w>IoM-rR@H1{&mL^6qvEWVi9Wgnkb2$2VX zkhAW1#8Ry7*hH+uG<-dZbaM)K>bO*_mnXy1BwP^K$D<#qx*+dKjzP=@`xYb0S8O38 z@S@XDZTHeqz}iFYcWZc@a@4IWY`1-P#elfI1Ck$vx#dW_OH0negBtX53TZBSG3Whq zZWIFK=B3+ZK(nxl}j!9d8Sg4WkE;ZL6p8>93u7 zub#%6X;8#%DlSML2wB5csf7#d!SeXk2`O?YI@6`5YH@*}eW7@HSUCjsS>+=70C*yQ zf?tZR!*&^Ih&}7#0(llQZs6}s3+DB&j!qj}b5Z$W@x^yI zDuz?heO!F!1ABPzJzywov-}`dd{7MbXd0xqE!U!N_R>BU_W77bDcS=#IA1qba#_n) z1!HWc><>UZ6jmC)kgpp#WefyiuIuCAg%r)Q+9#;Ir3HI-QF(xrQIMnCfFA`3%k5 zv6k1@a(1=GMoWk3X3j<0ZOUfIU%8xV4sG@`UL9)IGdyJ+SjuMXGQTbD&f8{WqOsK< zDuH);^Ky*MN*) zv$Y1Eb{xz`?g)OrvaQpMV=$Fxj2=Q+*=+;G-euvuc~)YmeVKyb_NE`LeNPxZl(OZx zSIFS4rt+X0DhqRET_+JQf943~ijKT*F#ruF@SQ zaE%zkZqAj5+!(%P%g!{h0K?$`<8mqJx`uLP#k85n=t{@JxiPNTEHD@2wReHKJVVr2 zqRd^3y&l4+(JDcTZ-N_pqO2c1gG#c_qK{)JJ_$);+ z^`1%uB4bcw2WY=bi~$0Rj?Ep`8-dvBH*JU2Gu;-6guOhH2@hh2Ux%gHJMO_M6GTho zcVbzuoHbfl^?xMVihuh>78DSYU}BVRxr`=``jB`<_*d_-RMvbI;{&IG-qzNvJ-&QB zoA0w-fn>fTbGjXhY0Rhac1T2WI*~JR%kL7cn@!r`Ln^X~7-~9F9tFZ#a_IR%@U7g_ z$P?d0N1-sM&$uVh7v=QGZOtUy15n={zqBV?hIR`dW2Tadl1ho^f|37r%REX`VL)eu zaG!V?NP`=677Hbm&`uy$8M(CJMIBu)u<3_Md?XnczWc|;c~OEPdYDq9EhQL=?ru^l zE&K|kT>i|Qs)gh&w!7WvP@KMbtu*l{b!jPjzAq=N7@~HtZ0Q!VU+1Ww8HE zpyfhPUtb3*@a}VbVpygpuXRa55)u0MRHZk5l_y!=xAl=^AL`LJY_^ymOQkT)5!Z=5 z$an)=2d2(_O3lKugD@$s#NnNm1b$Do7vGiO%L+`?6D#RmUk`|p$0QBKTSZB@V*_RK z-UvpFS}fJrm85D^$^Ofj)r$wrvz}f4fadwYfn{N?b#%ECbUW9H$XLljbymh)*Eb3v0-|#g~){*;K4X#A2B5 zARjR*>V3!8PO;a1KMx?*{p(#L6S6^w>#;;M5mYpvHXw?BC_ORjjR1eQXnZDp6mwzj zfdzFF_sv$N-Ccbs*vwq2u1hbp#g&^q2fg`H{{jo$zQzEpFJdw|%YBu?y~7?)d=$mt z(O1SrU>T&WZ(>TElKGCGNtsg^ub3qCyEbX%cdZ5TMGIs)3WUc_D&Xsis~V+sDYcCb z=6a`4`-Th>3xn?&guU8s3<-xaGbK%f&{rPW7|6BxLf`XI8;xHKe(tyImx>E3Pq@s3 zNC+$ATw`+ZJvvj;yd_|@0@i=vbycW+?Pm+x~E9y~Wes|F}%YeKAzK<7Q|;lzDw zp;9M=y`cU>j~XqQXE%_VZ4%mbX26@l>85Rv-07GsF|yZq2qRh%m zQG&#Slp5=M$zP`;n{te=Ze`8Fl4lmAk_XZ?Fl@JF7`=U}EIo91){?16HBbI-@}eQa zo6ufrrJTGr07sLHx#7%BovccBx?`W%PCcV-Bdk&@WheatwS?uz!iYT>jE2p> zchBPpHK!&-uLy#0`f9g&^}7^Fg8OL2EU)c0?pZ-aNIt@xf!{G11s;JZMyRjw$7rwe zQdE>7=bkNMQNK+rr>!3BJO8rz+wBY9gG6I!pW0Je4HhZyWJ&1fwyN6bt8_or#UtGtR`qM78)5O=X2&C^mj`XbyZ1w6HNu7cJ0#apT3_*Z~y z2yWqh@VXBhpaYeAEmE-ysv@a$wrq$lOZ(MkfeD2du?2r5C{}suyCj0Cu3C5%f9h2- z{Y~B)#4zU`y;`h*q{iajR<~nuMp3dwZom{FaFwyG;M$|N`HaIWRt;`&IwxDC?`JCH zWrKm16bXuL(m46m$b9u=B~I@m9>AVZ^-Io=k*@PGlvN5q)m$_~bxI|i#3UTLmA5tD zqxnwFcn|I!3g|Zfdb=r)C)kEm$J_+j@bx{JdbC;yQnT?*tLpIVn!}h23C<%$Th}$j zc(o0bGHeTFSocNYKWG)xUR;@+(6-|6tDKFcV0Z{0!kOZ{h|!)uz4 zQ!9P^@Qf`>sIr_Vw5IhI>)*u5VJDW}=>KMbZPdmI5v$&`4r`*yFS0}yZe1VVpkfVf zQ5)@Y5`iS|=C zo3ia+bd+K3;>SXF-pD>dY3-}_M8G?T)HcOJ*cN!w5*uWn6MBg~uy&Y^&I}fc##^*z zIF6kaHEp^{lln5_vK3v`K<;nkZqp((EpMFq>I zG$L+*&P-V?^K|mXJzGKodidy7a#ZM7a-Fb>jDpXLOce7$U8-M@;)4R;=(8%VtMxs# zJMPLy`f1O=K>#u}a@Yr~)P24$z9QZftW0Cp!A}^P__b=r7_W#|zPaD0hK_*C7NqLc zrC>nGZYAB!R68E!8vWe?X`NHCvb~~J*mgU_JD}0-_N0u_dFms>@QJwU?569~ediPo z2h&C>n_m0}{J;BxJ+UjUfPYtfNL(Nwmj7FIE&ShyOr;vy&e;S(vSuLoqY zPcQo9-%c=+UqiS3-|zCMKi|*U(CRWNb3Dh@v@z>R<}|+tY{@7^HN06Z=)L&*3 z^^6%&RV?Bs6V{fb@}AN^tB8|HOP3sx<&lO#$tYq-T3F2-g@h;^qcok2g1zh^s%St@ z+(3F{uAA>b(3Zf>eCXc**%;-BuR@=b!Zt?4q4_J? zSy*zx6lgRx&m>Wm_(8%38NS+nW@FbMc;cgdUQfQu}hvEi+-eKM1{653z-%!69fhQ=PvS!ao z>at!o9|k-NyR+o4@c~Z00k9=3f9M;1R8b{qe37bALW_5NjuwlEb5JqW<=eBAroFOY8gu_)4rq}^e0_+}R8;t6l` zHQy~r9>dumM2kJ?a~4nY{zFQUp$}e&Gb;>f$74@(h|lre%k2~-o#7D}v&G{eqk^(g zl2>A|c88T;e-bDSoT8W?7I!;mUU(@Rjui?ChdeeQcGC#g#-7S&+`{vdc%;N&LMP+V z;d;kQ?pw7D>zh4w2fR{jrjO}AZ@|L*QKh0l_#2Wj53+DRy%K)!wf3D(s$<`zodccS z8e74rIcDp3s8%Uo4gbP8-gHH~1=NAUY>E$Vz;W1Nf)T^!0ot_12`#wxq~(l&blU(q z^82R6c;10b%B4_HHKtb3L8GYAWGo5O5Pb^~kttx3ilL*mMaE}P@?Q)p6M0q0Kw*7m zxRZ=uhk{uz}AYt3Xg~Z^8H@v=hiQPOV_2zgCrPLH&H?lFy={w)9f8?#t2Y z8l)iL*3*gD#)B}uxDCd;bJ~(|-3pja> z_mVfmk0BUO-dwZ(g%fTJ3mGx?1(^`1&ISqZyn4~2jSFGsrjM3g7-H^8c!G|Yt~eIm zX9)9c(Ji56x+dNU8=@W|)w(0PdmE$e{1jc3h!ksa&*;%1^WEa`a{hdWtPLocAp`GL zqJOi5)ZOJh>1*r7BB&Vk8>s%h2xAQL`d#(?k~6|+gsUX#(K{?HOH$RTjPByNSWlFQ z0*!KhG4^c2+TIH)`73L#*g<=O4_Dj8?ccDy;Cj3Odh?+Q-PD-UTW0(QsGV|5>s4|5 z4;GO0ljTkEhQe%yEO2e9l2ws+;ldrI5o!~a>`d?~A3Q3I`ZwaVj8G+eP)U-CxY77M zhEwiSSA|0ywL}q+#uS=Jvl&Fi3(OGf)b((;V13-GL$J-|@84j(v>?V27WJ5yqI9H` zh$s6`^gmc|;oQp0E}ROL7Rx?UwR#5$;D4hxl1fjN>1CI+S*s?} z4y?)Jem*LyrB&TCJ3VWbl>FHMkT+VGTk|}56>jWH+4jF%A;te)7&1dET}ZKFhS2{T zVph1HA^a6uSU2MKq$Z%&!E07sU5Y|Jl3A{aaZ^zx>jv)f_FlB6QKwOMcS+ohg)5T8J z`#edIY}O5NL&1PZukfm2U==5>X@41O_?e^w0OkL4`wTU zuWtc770g=99mv+gX2*Z)4ek-4#1A5!h55VBUmameh+|xE^XBDN1BwDmd$eGZNNZGM zE(y~95(iTqCrwm`)v!K%9K%zgUGA$30mdjmkR=0@R`EKJ9NMqDNaeO#pO}9r;N7i& zWnwHX@9M?5zC1E@a^|42Ly!AX<^H8%xQb;}Jwdn2RRH3S-Eyw-FR#S)AF9E{;WKWb zqalvY!R!#!H?bXB4?Lh_>0~%OaMgBKBfcTH)?>xIn?1FQ1+X!>EEZ+%SZekld&)La zWLPbBP>R`?#ZF$N%V5)I-YQ_h|op z1=ZwMe1n1IApK_Jdhl46uNIOdR*!0&7_R;H+O%k(wF{#{+2z{Bl^g?C!G&brSZ7_; zmosbPu|XNTt7|LmC@fIFje0grPkl{~(bIw4(H7E*w{B>Y+33OGP}sjX#w?6h5Gt9{a0q;U>$tSYP$WJ`c6#yG^qd z_U>WsPWiHTv|r64mhqJf%WYLN-F50U&<10IA=B{`{u{%VTZ7*^!o?t3R0iMeL4OnwdFj^szzJwrH`prpLdb z4F?fwwA5JEbhJQH`BL=bJI6^%rW8LmDbr3lk7j|J$YY;}`8%J_(`h8H!LMhNVsM7s zc(rNT&+GlM-|OyYT{AxSL$Yn3X!mA)pnpG+pacLdEY_@?c9%lY?qdpy2F7w+{m+5j zT@yJ74u*k8eS3qk#wVEYO8eF#JWci2tXWjM{D1>QkfDyp5wo9vRSLMg4;qee=5 zhz^)pg6o*{Aw&;+|8q}^@VoP$2TS9LI2MEk90mkuct%;Q8j3&A0rzDvjoO`xFi<`z zya23I=TvYFTSCZDa#C2rFu6~qccP7NN@@xQ@h=q^kr9sE=F*%8`jdecTrR38^%p3a zu`^cWC#|?r#1-yiWWcioWA}*#w167Gqa_Np0R{_vZ;C9#+1r`ZJpFKgAa^!{`PabGg3m=R{fV)M!xg~bm@#OW(fl#5^Cw4Udp#z$9MGKw z#~T0%RW@sJ)7gywAojIyv=_TG@U010x8N}szMaY*Zh7kn_dwS0CBxBgpmyKr58=;s zwm+NMMV*4R=M!?l445-SHRpPHk1k|3O&)b&%0p&3vfrC|7as3+f@GTq^2q#H#hY2b zTEs@cSrvpIO?{oMOr0#D01|G*C* zYiGir+4|8O;K4EnJ)QCdmC<^MfK*x5m)WK+@$m_CFpGXz-^!*xq4Q?ON0hi>c7I>J z>~21_;mAuo!eDNPlsG{s5~n0&!wQ=_jm``QA+3x*xpC>r0QBBhz$$q_5`$oKGt6;4&i__}9&W)ez2gj?v(B+K27nxG%dq zy2J$Rcrv$Bx17#6#8D*lI|Ds{tW(qi+a45~0eBZ_ALufeJuZpvjhKW$vokyw{r1jn z7Wz{b$9)~WTv{H|2?n#9x4+Z*^9b<3H+6GNt`L7#+k@Ah$8O5WcK4tC^zgXn{#fVT zt)zT3$XgHVwM;i_raAa;vXC5L%j{=1YY84nTn018kWbR+`5kEvTTDNQJ~Bq#7EN=^ zXi-Ayqi@J1y?zjJ)Q2_lc}Vq<-mWujG(qv)Fv+7h6tRiTJ3b=dBLATLvK55_lsqGS z^PB9X44;m~H>H6a`Z$C7?co2_#`m+t^($6ehHC*DXxk2Ykq?A@2O2IXX7p(E7PUj` zWf+NP0YAOg59cvW=IuCu!h4ZDuvdP({C0wtGUfX~pRYpB?%fQqentl1KvB7W>rrAK zC}h1MgK<9IVIN`ptW*c;z9U1lLAeA|GBkX?KS$}_BJV6B3L9KtD|$(lSc&ux418aR z?U~R=Qb`7`_&Q1z&rlPkqyj12d`RMotVb)qe@0ykAQ7j$q30#&M7Ri{cme*=ZPuU- zz=5^abk=5oesSzg%c2$dJoo-eB)sK-dCBOx9@qh$rywT~VZFjjln^yyFBmqc z>)*1NLlOv++774<8908zclf+yJv%67pp2~7r&(yx=lj&j3XsyU^MyL&To;BAGjseH zyJ=JcPkAE6)kUHMg%_Ajqd-PN*8c%ESgMow`h@z_<~EJWn1hm6G*mZv(<)?UBqk`K zUR%S^7)|yx6uCiPhBV<$333JlniA5lUa!i3TJnebn-5QEtpz3~TDy8<|GTK?Klp*YqJZ8evgoXgk0!mbD&5yirM3$qmf}4R)J`5TBJe>Oj(;aN0_w6sIJXW2eVJ zJ6BK_(pcw#cJ5WV1|=jTd4u4zlM4>pZXB5l&g11U1{p0~xZwl@x%(11jjC&{Ndfzf z+-fA|S8}R>#_)yAT2CZ>7!D*ZSl<_dYZ9D$-S39ZrF$+@3m`uE^rZT;M`Wh4kA1V8U9p~8k1nErJ3ZE6wBh( z>ZpvnD6?N5_GJK$Icen)5%r%+IVanyn$$}PFSO-?36{5($&3t> z^ne93v#g9o&hxfaYzaTvwT$|gS?^CqjxM31jLM)yQ0CN{qnyF)bwT5o(c=|5ZJmgV z&?;TS*!+4^q8dQ;=}j$Q)uMPoJyj`*;=iG<)Rbqiw**7QS_&hLjtdCoBA)`OA&ru**OXF{WD0`i!-tjxkK-#DpB%#@Wg3TCJO^Cu2 z!-%WJ_DTI(;OcgbEHq%bQLX%cC_AU<${KB3$F`G-ZQDu3HY&Dl+qP4&ZQHhuifvWy z`u92QwzJR6zV~%K&4;=A9O(TUJBEk=_{2;}_!Ofa_uxRy^2RgEtn6(8r$7&i_ay5r` z?Jv6yY$~*7w)EZKkn=Esf|f2&dl0?4kQ3bY!;<6uhvK#y1)jH!?Z^RKi>=B?5aMo+dlYSA)Yi{Je z*QKyf1iTD=F zxLTcn#@A3l4o_`)-T!fHw*^&9K6(AAHy2kQvdm}+OQ5;zQOtLte5`!IVENU#dFFQ` zw;Y`tyWvkPTE9Fd?w~}BBt+QyS5X377NORqM2$&$L0JwM?${g_-DEA-xP zI7sSsf`a)*4P>MdqhpQ)jDeC)-jY-Y?`jH$eSp2^SPh=pQY8ED9JYq|YuoIBF~!o| zv0?;r21;qHgP4WO^i4g- zvkZw6PWyt3f$npIFiG0GQkGcZTM~1CYeNpo2lL$`ym5%cJ;gK>Li~!1b{qFrD49^a z;bU!?KjoPh5$o;fA^QjXs zcc3IyvTEj81z7M)y1)hdkJE%cbIzsM^1XR3Flc26pDDvEv0Gqk60}sy_nb@qd{3)o zvm-syRl9-^;1gy9`Ju}Py`$-p5Z1`?GBuCH>>)3R zvI+(u_2tk}(egOhAe9{9UOAnc+n^c^N7&AcBE3DX^rpL?@Ixy< zfB@nfHdUVE6<#rtXKO;q8^zM2w%e3v+Hko@^CZv1NcQo*y`jYsPIH$qsX)x`7gzay z>ST$@N1wLYGy3C2`VmIIw4s}Fs;24kLmDhk6&D%puZ4s_x%wLArE_sk!xdm?o)(x2 zwruJasL%m2A$D~L8|atr5m^gDEK?*$GNG-)kp$t6hz9^}9d3(kuS1mJ*Fc#DOLOYv zDr#NslBkD@k+8hzO>Kl#a}Q+)!ICp&7IcH*>IXE`_Y)QdMSrBa!Z9@_O|;}w^l5Rr zH-&5kkxu@Jr(2OZp-b>wPLCbq`&)acD{ zCbqcz-PPgB)<+fg@mq|pFEjjG+O3xadGy>O3;k!4^QVlNE+rqHrbu6S@n*PohRu#* zi%bu%dC+QJyUGdtI=TyXpb@<;+ZLAO=+Z!bRPn$)nK_RGy1mmQoc_e&y?X7=-=mV|yP_*)9@w#8irg3IhffUdh)aT9C^z3Xg zm8TNX&0TJ1;#d|EIj)s6%hMi9u_b=X#=}{+XseTVGulvsIph%nYz*1!3)vO2KknZl zHQq+LZg-89;^Jc^_JvqHIv9=&)Pf-{E@3&R=E$}alf&dX$YM+6W;`pRQHYiHMG@RS%mUbbI0#yAPLq3je_F0&=zdHHJEbZLCqI z>|1#axi9k*YLe&2f* z!u{7a&_pkXw>)@dL=Q~04^M#=7_2evAwc8bB4(>Tbc z??`*;dj4Eb+Q7;@iAtrT)){P0A^Sm8#qv~S&!d`QeT4opsChN+A7B1=fq&opIj$V! zX#otr&j30C=l|bq=ii*YSE`D3n}AS2jo)y-yky>pMSU9GFr#LH4M@XLgo={D1OU*H zA{vmGggyB8DIdC~c2eJQY4o$N#GuFuQ%9cNDmPCq(W375uK-PK+}TQDLAhd|GV z{lnjG$fPY-z;|XFc;HiLx!tzEmfC)XPGgz+qin(oBwxNlYTH$<_9cg%?Z^fCLoJ{t zpUq2)vbAx=#TK!S$#vE9wy2X1>%w=sha@?N>PZZf%fLNe3p}QQXWOMBcqCI?=b&Ko z51ud8P&xHO(FG)>jE*JuQpZo8Q|WKI^|pANnmo!N#+#~ImGm_FZj|@5|ICRd#C-o z;L18!F@Atnhxf4L1oH3CL}(crThpxNRm<12WnD)^ayLOY#ZZ??-e#9v*Dv9_>{8lz zO-RqQ10~2DcmHPYdt#Qob{`W?ubLYplt`^ip8bRJM-r_;#h)Oo?Z;%YJafk0h?d^o zcq^DVyMQk6N0UqjQhZS#=43btK7_=Vh9f!eo&%BzQJk531T(Y)G0_1m{k$wW{xDfx zt-LUE2)J|**WV`!-UG*DVY&8bsx7p`x4K8=@gUlJWew9aW@Lnzd)Rg;;@=D!?lHKG z^mRNK(nwAMk-D;J%$(A$2YtlX^295ngU&y58w8bf_0)hX!f8nmhZ4a7$uiWz^n#n}Pus8iq6|KDtv%)2K_0myFEio6TqN&Pf-Q7SMuMf%C?@=v%&ymH61>bo z#NO>#vBSS)uWO%|ZGwsxxbO-`Y^ORw3_xFAx$G4q&;c)4wXE;}a>8U}nr|q}A+%km zr^wC9c3<2fegc0l9O!jFJ^^9TwF)j*M_7smR1TTsO7D-e^?q%Edht@1Kz0QGDlF=BaNh>cUnt{ zvk~$W&3%Ouy;yxKhD#;FSDuS}rA}rG>ZXr+X6gIEeTVoDMEjGqFfr6MU`Ix_Zd|qF z;=$3kH#i4x9Lw41)Befpi%ebUEk}K`ia{J7{?<{4Dmk!#F2e47<`OOn4*a2uMfYyb zZ{1uYA?;rXhpan(>a`CI0&WNTP3KuHMnT`Zz9$uMHh~qNT<@^|E?7Q|fMF~I5YRi) z|BDe=N=;VH!o=-gW9VjeS-VY6K=`27ke!0!mio$&VY5ioL@PpbEKxl<>5mE{3SMOj znGgX&wCMN8OnjoOAL}-Liy&S`w#UOC2Tl?_9o^?W6fuu_s-zPW_^(rtzwQrzYJq)w ze0coxDKCm;jx9lr?r2#ORP;-!C@)~7G{+t3jG`1y{Rj_>DkhK0%K;9BgyA_Altj3S zE-I38wcJFR)dn~lqa zu}6-GqraD=s6SO8aws-S6e5(t5GDG!RO(7xgudX@P4m_;4#!>%mS>g28>osW3wH++=Zbtn^<|@XZ-#*URd0jenfv+Gy0#ra7b&qW4Qc4e`2=Y(mmYVBmNM%`$ExYH6Tq3 z`w2QgU#_8kYNd1cn+NJ$5jknTmI?1P%oW}|LFx(-wq7#QdlRC-^@%ksIk`1AtsKMiHlxZ$*_)XM*Fo@gHZgX@WItz*CIW_$7ucYkwCl9qG-(Iw3 zJU>510a|6Zq2k4eK+SWD4G^OILZ{M;<L z@yN~U7H9j|^HJ{V&RthA%D3ABjlELp$J=n9#jq2f};Ig)bN? z(4+S1b+NR)v@Pg2pZ>wWq>K5QBM>1^YnDye&P!+6Dry!OfkYlfsx)v>qvY6=MAe9J zu|uG8z3eCqW_NjP&!X)JnzwwY?QZ=_PZ_TK0 zfqW=vZmm$u;u9H;v4q(cQdr$nbDOJ>YvxgX3l|p5sBkvZ^4Zq1!J*8Ad6x^Y&13P= zqULa_&MMd*+-S<03G%YB>uI+Bv^c0n<{weg*^i+*0>i zj8s}P-EC`|$0}QFp?tz_llvl#m#sjE>&D^Z?mhZVgy?xl{ehHu6R11Hlm?J zjcC+C(uV^mrx{I@eMTTzQeE;nDxSWRA^-5)9wtCmRCyjfga0dLONEYwu2yo%QBgl* z0x~E|p#d4xkHH?IT1|g`y>72g+4>;Z*T@)=P_jl@aA5wTArG>FYSgCYj1r#;ysT0( ztQ?>Swu@D%KWCNGj7~m60&W!z55v>g@}e?-7cJPTN|uAPReE;Xl52%tbiFfLdFi17GT4{eyLT^YE3-U8tl;5$htB+oTSco== z(^rw6>Q&@&D`1#6ebjm^%#kFwULmpOL@}Hm5VQrYHzXa75wsVgNwG^pzfGZ95j{yp{ zf~T<1CS&xW2R$eDskS`nWP>-EM<#9|Y??W;C(_>aSVj)H97{)vsd})qY1=P%`*vf_ zI8k{q4`$dggaC-pdwVidQGw$C4T&|)saXva=ExE-=W%cz{EAWV>iwVhlRzvdq`Ru{ z?eK#^gO}|aAbNsGI`+f@wEv3A#r+eN!>1R2IqP=co*j4AN_KN+>vR3S8eKe7v1=)5 zzO7$F?_u}V$dW@}ZRY+WU7);x?cn}pFu`M@7v94dM_2l6<>VAZHyyVux2zX=sN6hu zYTS0@yY^Qi4tLeO6Vf_N=Vd9W&ZX^dRK|_07bFCQw%Mh3>x=Q4jGsVE&Q(aIgLYu< z<0@NADUmp{zLMdq4R?+{;54LH2;1`)4)=K<8Ju-KKK{+rjW9EGK zKv_3eZ*QrwpBu=IX3pTfkH6MEZzdSMs3Dkth>uEGFxR_(&p8#WzB2SJR@CnNTwHys zGiCOGd9`I7%;x;?*F*Ggb@6~@dA2HVxxOYypa>$an3z(X(%b7i?VkruE5Npcg4B8w z$G;}G4)%Th(SU%s@GZDblHdmzk$}0hc2`gN3})Hbi5oBkFYfeUecyb#r~gIxckSZV zkP{C=1On<41_Gk_kJ_c+Xyd&Y6=gAaXVCLL?Em zepAOQWQ~-9U{LBah)C1*8%9XHzM-KeG0GwbnN+7>p>8Cw-AINaBy?ij&)bS z1T1+baW!-Ff&C{1k~`Yz_5|cUgkdMKaKVGXKs+_{(7M3#>OW|)(@*#=ws4(gye$j! zIZ}<-hb}b{CNu`#et`1EIlOwW5VAXzlQ|{L($&v|&v!R5x)VjO5K`bu|3Tbx8Oec- z;P1f#{3%ZL_ut&=_~VDqOA$vQA^W}i50braue;UyehqSlFL2z3F1)&bWq^i|cC0@$ zJ{DxD$(Rv6EvBMxv(yu|>3aWEekhd3d^L-hVKgOBA3jl7vemlUD}fz0x(@Lu!)j9+ zI$+Wk%7|zSfu)Qhf^f4ZO`S>_rqNZeqZg8tMJZEnBrJet@KhL8`qRi*>B65in}$qp zh9;%|E|@PCo{g+_UHLmyScBv^oo;NZ@5A>eULWv)^icJmHpR+bz`rZkR`# zEy~vD{ewB2DSa74T~}L$(|*D|ek^e!0g}QTo;-YK4oht_ylxy)5EyQn(*4#W<#{Z) zKHhVF79JNh*^&GAoJ9+@0v0d<)ll>^h9K@QZ=vvZCk8a%<4ihrUFZNDfk;}^Hsl&4 zsxhd8VnaTO`WkU8hKh&$eqs{h5HhqCPz|L-8%}IwI9p(v&}ca~kX8)1(uiqY43)I} zS3HQ?Xqf~C8x#=&Osj*Sl#y{{2`UROBC)8&g9Kx)^@^{Ymz+5>~JdxJt zx3G|Zd$Gfm8CIJ~jxtX}fuNhfy>!LI>iZ`MsLn`2HUn2RvU)_S!5Zny=X)+VWBuzd zF&fF$U@@#MMW#`ZSRd3>)jBW%G?~Qg1C+ zcqK5k$YWBl74j|w)%o|`exa+=@HX|Ii(F-r*k$DkKTwWJ`Q>2g9c70y2 z+e9!h-<+J1B_WF!_`K|MBc7#1H(Rz@VuZ-~s8X&sjPevSZkx2RG*os^O%}G5ER1n~ zElHN&j`6aHOYkzkwc#R7Ik9<#t(&j6LAMC>L< zB93(9?w6@JyC3VW1zGJS*ge#lXt6#p!azRsNeKvB$!Nc*gD(ehXzxh2yp^t?P76lt&EVOy40&=`kgrV^InGA zrv$uru$47Xp=MJ7nGaP^odZHSyy@$GirGu6%7n#Rp>jD=?U=Ts&I5Zx0csL5)^Sz) zdc(Fi6o6=(4I6w^_;sFXHqv&IxSHnUXpvzjH7&Vsg+!nEkvGLv1Dv(mMSPX+Vj(Xv z$YY+XX*WOPrw8*>aKy@yZgAK3QWW|oZ?Aw&6S(;PU`>mt*`$t?>`_nP%=;%Lfuzmf zLrl5oxl|SrK1>a$H)XoG=e1*8TL=bo3V}6m)^atin~`si+mstosKB#eU*{v4}bH`{6P*2=ivBOOQz zF4bz8uzC+lj3wjV_y<;ZNXBNg(LI{d8$zGoA0&8>hzCGw12%!aJuNuR^zw+G>ltr43|W zP0vcQV2x-tTRs@X4TC>1jPS`$GCyAb`G>_jhRLut3selTNs6b<*PP>_j~ zLG%~>HbA4;7ti^|vS}`4xnr9y_MM;D2g3-f9S0IV(`A_VTE{N7Viy(FoAIJ%;)dqk zoiS5u#=kTFh^1NR=zL(QqI@OKbeW`2h^*hW*mRjqp3)m4&PzK|GUq=>TZ=>_eV0$1 zD_7N%@B?X@8s2?Os{NqlTo8nXfpcBCIQP-j;!4I6SsWi#w6k7{OhmtYu>QL&-2s$> z=y`FjYOd{5F1kG4=jD@0x-*?NS0GEu*kXgIa?)GeZM%6C9OZx$Om|mZste`7;PwjJ z=XSSzZ!exwQAq)VKJ**h3NTIenDfK*OruS=o!0Kw>PQ)L_!9}P;Bpi%wqhbWzTxa? zvX8^>UsIc-39qRfFpb9<$Oar90+qQcsLwcsopgGp1@=%urPH>ThW0xf3PrZ94)-G$Sd?&+epB^1RM=c-+o?}@jBII z$IvBI^EBMaJeV4|)!W5ScNzF%7Z=CEaEz(*0-H;wB?pdJZg_6c;n+4yJR1UuuWKY!fvG&#-N$aJxPeY?YsN02K7rb|l z^7hcOH$PfVK&{Rij7NC=)%Av+ll3wHTwm;z!N>o~U^wxvP4~v(N?rvGzx$$w7MB{~)vB(Tk z02=H{Utg^`D;C>I8gRtbgig_HGY+rceTbLR1v%QkO@tY<3b8Fs>d~@=oR1@!O=(FiW2)cv&R2I< z?T{sZahQ}=<8;jajE~JsN;IH?qXIK&1@*5^1HJMRC)KHtn3l#!kn73bEP^Yb|C>^F(VtMFRa zH*>^-8`^NZJl2iPXs84gPP|2I>z{fpP(X(8swzTt}fEwohoE9{Tlr?9@b{ zP$+(b@q6*_V!S@c`5#rcBOQ+8L_m14Ilulf+$<^fSJ;op6%tL z^2icjGv5H>-#LKz_n)&H3LdOdqV7f}_WuL+CksHwBJpYU8d6Yf4r<(spO?t-6iY}5 zhY_m+H5HPG3S($R&iNFZUsD*vi)>LsNW4nK0&E>G)XuYOS>xnHj9S*Hle?u z*|gFjVfX-Nw_B!~!|EG|{vx5~H02R*-*3TWr81@l|?N7-5 zt^|Lk!XV$RaoOhn4GqWfbVmvYL08qNw#i(XX*Rj7*q#FkjcZ)OsA6U%`z!mJ4xftX zN`je|6&>`PIqzaXgZ3X{*G3agN3fQeZNQXO|o+OtA%mOF48JVNycJ12`z|tcc^}PmBTkSRn`@<}6tLT*`D(#j#*80R#tjwLnDW zl)WJyy@MGyFmWbv?Ts_6ra0@5cOJeG6l&%!kGefjnc4wHfjEY4zt4~ zD7hV>RFM;X%G6JkV^qHUJR7W$JI0i3IOm$tNa%zaH2DKm=Y0eKuFHE1kGnBx?Ks+& z%uue)IFtt#@tPd&Z-vjfIW}>$f?)yuxQ!)C z|4 zCEWA;7vkT=#QfGK1{ko^K!^H&!36vxCQ^V?$^TnW6rDCW0S~KJzgS9o7)qWdmn41j zCz?(W&q!HXZlbVW^ft$aDV4~C`|GBaW-z3s-T7Tmp{dfY_p9k0D?8_DAC9+shi!PU zA|&Y&jeb{Ko0Cx{%e7ca^{P*vKqUtQVygix%Pdb(j7=-$_o4@2J;@-mi54ZRb&9jV z9Tuagu%!Q3GM>DxiRQ9#sb8Ko!Fj9EL^BujwUU*pK%Ij*Nh}RL7|^6xrzWe~R6Sv^ zeQJ|%SfEo(_g67oIrY81jDT>_MR$(K2vN1->r3$R$!!Vl^f*Ad#7Sl+9pl478Q>5) z6Skvro-10hNuYu>3L)x40jxCIOmqCip=eA^eyLP&eftq>+)5;XB44uX61HQuA>q!y zsnXZkF|DqbD7jd}{>K<)&&U{gnqrIY4Bx7@g-dg&(T`%J6D7v5Dc-U4mF}-u8665Xi_Xdi zA0xgEAIUV@brcMXG%vB!$porBdvF^2F%`Ct8wA&7&K~#XPHcbz&L0KdmLJ!dAN|Id zfNb042%hULYLDhL$WtLyi$VPW#wto2X+eszBo7Nj)%1j{EjAMStMu3x%+ zvOM0qHA2&#}g~a*DuHP zJFj@q)9vZSrdQz97i_2YyNKbLOWDe5AO06V-&YDT59E^`5P?aj<)e{LT~Zb8oJ>{L zM(d03oWcA_Z8tjagI9w*_Ty@nXv7V+>;zW&RSmzlMC%zU@)P#1+m2|iYwzO+U%?JZ zGJgN#98$}vj{evhvqnK{s)A*cpb`9)9NQ5h^vgb?yN?fGn)K%!fasmf%8GJZUA?LC zg;j{%$(=^zm6NMCy}6La2I)qwRGq-Ood5GO1(9=hROj*`4RwkEFk(?E9-+}aVoR72 zykXzNN{j;OeSCxbyQEwnlW8CTB!v+mDIEWK*8q@||K64UZ&m4c{HIKj)~ioS$}y-~ zljC;Hag|Dl4@MBJ05caCj|`?|N6QD6;K`Kx`%jsorp#f1kZ5puaOH& z&1R(g?V@4~dN62< za*e97dP6Oxj^#a{+F{=I9-|lSV5!-+D$|S7Sy|0_tmCKF^50L9%W3r`+S4c0QsK1p zt#yp2CN4*^CGTPa%Eu#_!p#Z={1Iwtf3`s65r^rL&qyRS+VidrYNzL2k-EJE5_teu zbe*W(&Yp;~;Gs}^kFb78v{hC1AFe1b>i|GmSnH;>Oe(rsW>Y&W9Vz0#aH*28EbprU z-b1%Abtwt20yvs~yoao1B}))LKnufrma7AG?EzfTdgd1NDy9|bCeLmDg<|{-uuD1g zB~tgAf@`KFeA!*f75s-p7J5NaRF9KVIQwpiBrW^~q!DR2W~jtr=HtbTU`><|TJ*h9 zqU#an<3kzpwz~;T_j~ky#i_WoK7M*7axhY=+c#oBl>)9LkeUsR=#DF*Id(|JUmzo# zEfM~jEiIE2A?^Zwxl2Km(U)D^5BL2v3h*ADNy)WEmv4bxyw%O<^74Efw@K(MHZexAj&X72L6r2p^`qu@MO zc5$#1`#yI(1TGn zy50i`2%TS@0(PN;ir>n&1}`*U2ESpOok+I!#!5?j`$toL7H){!@CPtM5a9Jl10dX1 z_DXHp6(tmMi>CE{FP&Zi1Y&%|KE6C~!I-|P2vmjpHCoHB6%Hzz7YYYGM=T^y*aV*I zkiXr&aS$Mj3m+b-TR0jQpvS}t#BLImT=ZJIo54c@`V!akbDC;($dn@SJO_^ zZ+7*xznpzsd>wp|^BICb^V7<*vklk#({4ziaZ@y$Dyu~6ugCgaIm_g7TWo#Nz)nU- zPDi~%d@;EYmN6B&e;)by`+;6!9Zufb{kz1xh36zW0OZAk5D19jKT3>}iL;BNt-PTn z;C{w`c`*Q|-wMT0{#t)W)*MAO&5w^S`?Paw{^3+sT(twd<=?s+ttt%@1 zy;eHUPG656WIQe>5Xo}B;d~;giOD`;+MzAm&OwL`TDV{_w5wy=l3uT zMQ)(W-;iwc%m_=Mfk6(AWdS4H>7R{G>t}eEu5cPRdQe39VBAcDnk$q(Tt|aNijC%- zCN9&ZNv4^8V=(<7%M^H0Z8CUhoFhVH5KTCRZLQfOIgvPAH~prJa`^KN zEAN z?w%oH9o$ev*oW2+_46rX7%=NLN)8PG;vSVOhJ0xt6He(k9UV&zW zf5lEq13n8DDu(Hff*ZnRJ&<@TnT|nsd{7o#oampZ>0dy*#TY3%)ICT-R>T(qM+_b%=;5Spn`DO+d@h=n?XD=13u$BgsQ zLU7dWjo$V;)%$byGyG>^XSvpz9^%OvIkyOBbN$U3}?(>dR!XhP|5$Fyjm`c_*pMF7M9PoMB_1X9gOD?`!#^ z=5HCmL);7IY7TvkJ|%1C8T8Fco@xXBGH{%N&UthV^Mzf<*wvRwZ!;J)yr9+ABUt!p znkvu?Z)H0(f(7~x{;=WC`L!2bsPZ&UVv{-C5L*@7qY5#dVgjl7I zoo;{C!P4Jp=h?cl^vKZf>eIiqz#)E0$~eZCPiR9qax%Wq8ZOx1ftsG%stuNU0hbWM zQv}7uIOj<|=8BiV(s1ng5-e>T-iXz|9TRbLH|oq7a;u#{6iqReybsoug+V7g#?V?ME;r4QV1H+L7>q_bG8+tKGG?t zPNk;ehTfQ1N>0_E*^_Lh3MNSSbLyG3t7s?lm^MmdmugyIAZf;)v_QK+G`txF|2~*c zGO&PAmI;yh_1Me%oi^6UJErl4Ys>GgPYU}(@DGo-g&8A+6K&cilG#B`*;v{nqv-`{ z(jbQ^ne4a>eVSvh4Ma=k7gmkCocU#`eWUDMGzy=nJa2@!?6u*8c?PBfl75TyBJ$pI znbTipDqCb@|1-&3kO^E8&P3hP*P}DEK%LxTuL7(W!fOg`@`KFyhwVWUw5C7A8exj# z4$EpwNZR#L_(Zgs`ti;`Ub42fu542Zw6?lL=ixuo z9v&K{ehsf68=;BihIz?i1)3qlqK(g5hbx$w;T?9pe?nnd?+i10yP1!x&{gX!4q`4OCxx)>^v% zNjaPUixz#ZB04cDU$R2~W-+E-?j9WdwAeSiV|MP)MyI3Y#{?2*f z=x?24zI29@Pm@|vtw~0ak($VUDqCuqmn*xNBMx_t&58EiigX- za~nL~YO36&z;8t!3sno2;ezttf1Z zt>rM1c8cnvyJgiF-FT|byJBkXx&&HS;Lzt9I`Vc15gJJ17@Db|k-q zYU%uNE-NNYRi4h&B`7NhrXjki<5wODBq!oDHVn%QmTEYyS-@*~XyTq_VrZg@)p>Jn z&u=C+a4HNkVYq245Yq2OuQfcD;|tn4npXlUdce=9YeMrpFI4m(1$~3&lJ#xqOUC=* z#WdVF8fQ`?TeT+fPL{9q)aThm{CLmYk^|o-QnWaRECpbRter7@_PJ!>Q+tf3p!=Cbwrm+<;&fp(gQ7#6*;Sif3^?0QRufA#!}NdU){&dPx( zk(}yTSVK>U_Nu6)WQ^=a+4Do+MVR^&&fEoGu{I{wCL5jWqNzlRVB-%T`I>I}X4^^a z_EVZ4|2nyVAiXk7!~d_73x!jcI`|qlf^mF^mkp=!o9bmqxb<(nD*C5&a~`dVE>^05 zlMCDb>qKGCaNrR!CPTXSP*gCZDk5dXV@dlG&b;RD1fiRnYJi5pe3Y(DqwBw z)$=g>IXYD5UN(POz|ux*!K|EK@K`@~_~fDEJwvf-r7R-!cO|yO=65T;bQW14n2&>F z5+1O49FxJ_`eqEJ@#+Ilsh=9}jZUf#%wnQoIAGO@_+x^b#~2Tslt<4qK7*OYLdSaR zHi0~wWH1sD%K86~n+x^<;=bt+yxgNN6e&oQkGe;c@h{d#6pfN`W(*Twga)9N4ZQ;;E9W zpup~Y_{EE!4c1g4o=7Iyk<1%gQZ|4t6^!Gaj;B$7IQATEIXoT-2s(6P&}-9SZ8*#q z=JRE^|1);+@xc1m*kz7ZDUivNak$A`(#4PF%iRb~X&mO9vtXm8yMW?u;iF z+2588=)n}JBK&>{F|B@{Kf+GBB46yOQm!$$-|jipZydpulSfS1`N#$htv;g_8ub7R zF@7PEM>AV@jVguyYmPl2R%`TI3U66_(d{9wX8xmK!72C6GS?&JkfD>aqk)n0&;R=2 zU#(g7|Do&~yDMwdY%7(DZQHh0v2EM7ZQH8Ywr$&X#ddP{+xL#Xr_ZO;e_@Zc*Ie_N zQUxqgi2!yDeSGi7Zy&MsY;GZmRDp;6zQ~0ObFqaX_ra! z6peJc|B*PeY_eP(FCM>h&LOojln75~Otwg5i=H&ADEH3UD|Q#d^b%jCi1^DucTG_> zPHkGGL1k@pUp(}<1yP?-?hHmz#dxu`1e0%$_}jQ(OuNZcJ~g%8;!a71V8W7Btrvac zw1c_?$vC@DxMFH)QZScmw5@wb9xsbpr!=b^$C( zPz0+uk#Y|5Bg0<1oJIp$FGDSQ0^^!kNWT8HoGh!8-k=j%I)tBcJavPFKT2HxOR)uU zx*JyZI&=&_g@&LBVP%E${V0kxhrpvs@g)4|krd9*COM4c-*zynZbSs(K4GteAlf}2 z3{bqme(l){RDsmvzB^XrR8DutL_=-=* zZ#&Tjj=(|trW+Y#*_mcozTmF}g08raTVq@ujS$GYD1;?lX?u^iGhYkpy_B;&18{CZEPCygl zbgUT7?iatP^=uAk`J~{Gklb+?G z!!+`IJj}^dOo2uh`f%Z%WLg(tzNK9tan)R`*>t5ynB+;>_3w0(cK50K@KgVMu2HgS zJAyVo6$rj0PsZkMEJ8D?p+;N!9H@X^0&(UiN{()QoZ zbOMKIg$1nD_>1LD%am%qd2-|CEwF{5eRfF`Zn=(KKoLoy^UO6B$F><9zT$S6M~AD& zQNW-a-&wwIbf-qQHOSzvVV=4%W8UI6n@f{gJ$curN)V*p*dcCok259sjybJ}opX>o z)dE#9w7)VMlL?BP6Ob(ZQkGvblm^!MllSz>gUf&>l4bkl5erG0x!kBOpTUU^x^;6B zUU3K-OBZ!h*6o&4A@OU3Vv7N)(c}3Hed3IsIm<~FY&bo7KAA;o$payU)8bVF(bH#+|9ooSyPG8%?UXza0NzlUUCiM$5CX5u$!B?`xtC*~J-iChOc2x|; zF<1HH(wX^BIcBX!4Q#CpgxvlZNI_!{M+@tGhM1TWnlZ+6_w}m04EM>#jX;LEG`_m? z<_phZv}L??np(@tQcDQY77xhA3c(gdk2dS{Bp9vIFSuD-Rb;IKP`aoLM6x^9 z9_qd{rk6_GLet+d-Ogtawp(Yw>LUyIp_>Ck^D|OUAUS0v!*lB7ie+gw6eFR#~+a;9Cd(6gCT4g(x15K$&H%e~*t#yGTw^hUl z;K*=5`%hXI{yzpfIOx0oD*!I3S_8I?5dVo=_;&}j-<&?o(p4SgBR4fa|Bj~=Ut@^C z|5KVgnY91+#XuD1V~49jQX`3m#D(^BFG{G;#U;mm%T#pa`=#4g^!Imf{JC^Itt};e z0?^0#RA#Q!$HyKRQi#3$R+v;JD}6*NVeA6p1vPL}N02~Lek+P&q?{k`PX^hEaxqiq z;p+>B-IxJc(RE@?F>4C_*ZU{A_a;np+`rgeLg&Ql>n_qS`vpC&-40z zw{moCl36!WTk%B`(p2RPa3o`MaX*Xz1d$aZUf)AF4XJu|?eKoiQ?T{(!-7ZPf~=L( z4KytR_PgVv>fcCvTBMH>hsooN{(18Z%YcDjKV_|O!srHvOgm-rqBRITEVZBnZXE~* z;WlkU9tbl*+yN;$vOk)LLcFOUE+7&JUL;gNoZAvbE-@i>WqSUMIQkT5OE}7nX9(mg zgn6+1T}c`xf3`=RQl5yyJpT&1|?+~#1ZU76v*uD-pH}{d?v|g@* z0q|C~K@~d4H+#%?_Ib9UrE6sD32WOdwXTVN!2~ZrH@AaFVWFTuHzL`gq<3=;WJEW) zMv$1Jd+&+tUeiQdY3`)!CG})k`uGAFpt7JoEa2KE3M`H9qC|1ixdc5~r7rOIrx%-m z)1(5!@hF)iYzGBLnqJeQsULtYe|8w@{Mrk?Z1)5#{JI3-P|~Mv2&S5QqF;r9O@@lz37bkWbX4O4UFP$oS+xpCO_zfXPNg7AGZ05 zu%77T)33?CAtgr0VaNTLKor`e=09`g@N=r zA=>X?>7iEB3#S2NHHlKLb(a_nTfO$N>?trR(MYwn>$S!RKWE(J_ZB+^rTr%RdnHtn zHdBg3?RK$wbTa9PF76HZ<|n{9{d*htWM};`E;6AAsjCd*y-<^F!m4cTpX<-let6iq zEoHWfL}2>{raDmOIA@eQJ@ov^#j7zghUdo7>v4Y|N)=y$f++6p;FMIoTwl6t=!K+j%NQons$3f6eqoSHzP7Uvj` z#*e-mw^{0V3AAS&>_ew>%apr(-p|RyQ9PoK;i0-w8UIlPPWQ=T-HG++%4tGQ{xWf@ zqcM&N!qovC*0U2xTTe`&%`D`Hy@l;F3;(F*WB+oC3cU_Ajm6M78~GHp7?ajwmN=n) z^IN=8dtJI)8dG&10B%2WbkZiy=~`c7YSccyTtbd$lYU2*`$(#K6s8ZPM(#^yiTFIO)f}^;#js<-2N2jK;E)a&=t44Iibpm1$ z&g|8^&;G!>U`(+K!$4P#XfrM(zpAZ^+ky+4R&l&>DR;<1G#o zb0j}B^LVwAlzQ%1yHkD|$+B}v)3+}VEVM`=dtC81c!yf~ZJCu^l1RpZRl=9JzRj~{ zqe0`{&W@`P_H(WP?Uf&K<28>p5hp!3mGD_6l2@m4?aR|!)3m)U*x~a7PjG~-fq3{C zx~OWj5=PI+Hz#I&DWYWFsh_#-A9dsUM;GR=clj=g zt(n*1th41hWpU;k^H{n+Am=ou2zy?PyA|wPdo9RYH{EegDCJW3p7v+^_}^yuxIq_b zB#ked&v?RPd}4C>b;lyX{Yl;N>rQ2X+cRVtlBBsXTn2tgAJvWU@$-a3KgG=|VSOrZ zoT`~tv-SL?CO?|b1NLeDB}#kkFx3HXgHr_hjq#?D>E?wc|e8~yL_j^HUA&MPCUZJ5v$|GU@`d@3KjL&nTd366)? ztFoOJ^dZfsld{#L8I$lP=bN9nD-K5$dFw%U8PA)^z8C$?<>YHABV5UzZN(&$qQsHl zNy?HmTY9y|S@A=rV_9F1CgH^Q0je#uNp8@ZZQSsBdNUufUzGBCWU~2G4V#J2x~o%}Ka6ZVa49IPud&GNm&CdM_!Vs7{{H zT z^$4^0-zOQ=dapl#B^u=fHX}3;^Xm|cwNVc$rbzi0q;YE7bxvIu>uO4g@~CzZ1s(Z< zcFtrE<8)Q=&)-zasZ(iJ_fIiORk`Js7)46z-Qx>E48uu_39R+B_sz)|))q}uOr4Fc z;*q|H?813h_ZHG+<@H%?EFOv!8?ha|+q_U!1WoHhN#)Sf+6#oato)Tx8PrQD(p<)r zOUGr{11W6KYCobFo;WHl)PL(qMH15)TYqxG!(UOQ;;|cTOwxi5 zlZ>xqR4$;`1Mw)+5GUPJiOlrJFczj-m5PYgXnOW4^-h;-h^IhuVs7FBUicOQ|);4UzYn*oVob6;<= z6Z*-43y?B^=xH9StOMK+WSi7!8CP4o6KSN4+ItUDb{U*q2KiA-oZl+XF@92gd3s(YWYc z;Sj9C?=;pz70ewcotWkF(@9?aNR#r>K53i^DF(g41R zSGH-F%ifshjGPk6dN=Nnw`}P|Ju#5T;EKyR?_Q;mT)zKc_qaEZOPAhAkz}o^+IN>J z#m~ylKy+>8Sj|F4MgciuB-@r_hV?MxrEYgEpzGIP-~ytvZbnTnH(A&IA9VTr}V_LhG6EA&)4aCkEfA(fk!sMAv&+16cZw9@ug z+;aE_tUnHA(N;$}Uhbw!yrj6Gc@@pTQf!yCPK&m}AD$9d_aet#jJ$G40^pupMB^Oe zV4Jhp_$cZ|kGMTLe*^J?@*t1}ds~+UBZ`Hf(4Le79i9^=ruD*d)1SkFbUFh}hcYZR zkgm<#v4Q3{w--k4G7X0vUS+yveo>w73C%Mg@y^k61AiGbz6wu@Ybt>izPzqZR;l<;iZxNuMbc`e9px17z=jnrSuy8LcM$nhY7rIwz>JOoZ)Lb%n**`j93n+a)rxqnMs)cGWWq-`3aivX=)d9n#g%Hs~Eu z!|5=?$z*m?y4&;xveb(OD0MrZwOT51Xx#C8K|5YZm2EZDeAJ$1KEDa~`k=O;8J;7` z?7I|X4eO|X6Y2mR8$xipEZ#^4JWQG=1`<;mx7JpnBc!uk2d`oy4;uvj+oscf^Rat$ zQx*A^SB+++k-34s|AreDkeMAcfU74c?2jKz|M}JPe`1EbW2OkaPqWXE%$5l=TyHJXP{UBO%ws=tXs@QZnVWGNTdl7gi~8 zl~X#cMK&tO7Zi%0t0nI zQGOLATiG&Yp0)aB2?XzhhQ$Z@u9143uJ1y&>#+t&pxY@#Bp`r~X6 zMxee6LZyow{=ogva4#8W|Bx!gF!ORN{@oq5RbR~K_r3=!N^f9b?-Su(2r9%(`ZQU8 z#~?|D=;14Z^iWe+EzKRx(}doPY=SS~(z&a(*Yl(%CG2FXr4u%4=!$k@x-bb#xHnni z^1H#E{$4O}_=A+tskph-g-mepN6+DQtB%lKGEa85SSki~JPuZES?!2o%Pm+t5<;uY zU?*-5wh@B73=O1dHPOq!=S4sQ&ZfE_%lf`&-unvnyIx7Sx3zA-;aN{17>UENzvoo^ zJ6aUd^oTxjzWd>P(Y379wjf(8LJU^U!XljX@o}DIpSV$jrwISL-te;xZ2iB}oocv3#as~a_^SDSpRD#mI3eBBLwV9?NaSu+h5=+r%+9z@;#d#lL-K{gI^WyO9qe!@9)DeX(^zC@A$O@ah#D-HohJo#DYtRLWK|P@`E}GwMdX@W9chss(Getla=DTQMzmpk$ig0Ax~nYzjJ8z5{->U$nzO5L+2;W$eP=R~xW16H{U zeP(h-ywNn>&Xl64crhjsJJ8W8KSwQ_h9z@vJeN%WhfgH@?&e2IDD6Svr}altIKDg} zb(QhhBCZRFN$0ZENfk-wBNZVt4xy_@?#Za??TaymgGY#z- zq-)}GpT*L}i8j5soQ0uU8If;@W`^b=e{He}rPBsGS4Rl{oLY$4&_&Dkxkb9`m)WNE zw_}`{!A|2UM;!G2W~&+uyAYtQHx-J1&)2EyfW)JkvgBm}89ZAXK;a`F1A6LP;82lG>k3o{`VGPk_S#Cyi zIYXuqm7!y^c!q2;1C+gGXO^zm9L4u%&RVd+-TZ>Mu!Pn11vk_Ex9+ofTGC_o_>zTWfl*lmamDDEBVz{@rE<}DbOxMX|L?pE zQ)O+FhT>@_esPA9Vud6WrL$tGWc74oVFerWyb0x{lK0APBJGS48w(NjGxz4v`5=b3 zq@;-DaxG)$s^do5N7BrLqHalx55?XwNAFw|NeO3$8MQJ-2`c5Dt=Gp9q8ZBh$S%pU z979{hrLba|UCKaGrY`-toK%4o_42nkV-(=yABIuQV+LQNcxqDj#F}LeH~DzAEBl7m zVEI{WHQO#d`dHkpaQZnR~XVtErk>V>Yi^R#Im(6%^skah(L{71E=aLz2+$B zz|6C#QmBSMHP+%JSrO5%C@j~o8B%4;CvOe2&60!PCeyd05j7AR&;$o*>pNsk`oFrH zSP%lo7kGsa_V^w%Y^~kC&@8t;dkV)FP54ZroDhJ%jNJ|<_6GXT-WpQm^NR00789N6 z?te%b{zG3lQW%SxD)OE*jdPRvpdugX)}00IHi2 zrcA)-AA2t%jJ6!25d&sFxLV15jTZ)C)tD0=ELu@J}IC*slabD(kALV%#~zP`8yMD?F0h^w*>_M-)uJ5LCQB6EdeCD zS{`=_FV%YHfR_*>VhO7!<37(kW{0Y*{z9OBgIcM= zlwk61P;L+LKYE)8HSjHuvZW2j2%}S;*KQsWPea!E5#p5XZdvrWFX>34^_6Ry7->

    Uwj`pliC;Crs0oMb zdBQyLOq;GFm^2)wi4MsFL^TtphP(1!)XMu+6mEJ|JvT-83B$19P;?5v){m#f(SKYab-A+3`yNjFWJ(5@Om#=CaPjH(4Bvr!`9QIhQ$`2ixUjMr*eII_g{A z!JQ?a2nXC?(p@%BOAD5o3_2U-82HcS<(_2AMW|)AXF)!Eta=aBp1}8~UF154($1!* z?)1iyRN~{QQw5{BAs+Z0vNH=7mglCwP^EJ+^9Nw!o?URqQrUk$kGbeo@fo))xc^=# zUEunQL&>k+;kkTsnvTof+KbIBynN<2pp+h&@|Nf7o0V{(0SpJ_QotgePv}7f4vh~@ti5F3CcKV?w*{G}Z4DTf$6>L89osu!l zVamgL>}QKZLR7{X_w9L5Sp;8(MYZLzl$nK0ZW@S$~Y>8rQFFmz<{>w`x)Di#F{+G7i#j-bK z*b3^#-KgY2VB%S75?18Q!+x`iBe3|}gqN3_Uh0lGQ;KnoW+H0G5`iY2LEfkGyrt94 z0N^(`#|{~hNgY}z*ReZOV~ln`A$k`N5~B!Y!hsjy{7zPnB?AmixSucS;cXI2qC zZFXL#jcISq{g-Z;MGah~ z9e`XEhsDq`2gLz7$W4hv4UjF>c>D9?D(b=D%GyH({+h~D z+|^SapAkwM+|325L8?KuNhAYg%L#+T>09Bg5y)@wheYEcL4a&Yapx`}QpBnO@z_25 zI000T7%_yZpfD-lLWyEhU=SoBbD&;FFqe|rDQ8aTK`trjke0`EHwT?Q19zi0NF2G4 z9QP!Sjb76?f(mBBQ3JmLB{zzb_gW+OXWCAOcuC?=qCskU_Vnku+#k7qN)>@(dzCAQ zmEif5=wDCa!sKHI$@t`$F2Rq)qOn2fCIOs4`Y8B!?gVvIauL(2!H9Zy${Fh%pu;Hv zAmdUY#&vRVN^|X^f3zZlIDcI#Sm20v#b?q|On_FNW&lJ>)wwF9{}L^CZV0dhzF~u* zLB9@ZcMHzJtG!)E1K_c3!%6_6)}Nk$%`tzN#8CeAE)=#-Kk7y=YG8C zUNwCVFcMkVi{4*OAgSWMLAWNiaof=W|MMFLto1OUgP$Gx0M)YTpj~vK2wG$8sfj6- z%R4pQEe((^wSt$|d2a!#4Vv_ny+PFCTbz^@TpqxW7u*|znh!DX zx|S7``DO9nswI~ws!crMVK4uA9_hxj)AVQx-e$Zmq2NkBSj!6ZOx>wYwhl78kZ9O1 zKyL@Bh%j$~Nd0bI{DkNv7Jb6o;b}_f#imkhXyJhLjV2h8EQPoBKc`Tvsm+BZ5Fe1~ zn-C<0Z?n6z+;h(RBfG!17QC!mf2DPPL0#lrOCXE*rz&=}lLC3L!b3Mcm3J8m zCcn953ED8#r%&bcZ9G($N}q&E19U2-4zlqYC}H=$&(M!)XjR{=(#u!AEgLY`8&C=% zX!k7tJaoqU7dhW67fu{cilkQ~7ViyCYzUrCp33`s$vg8n^i+~9d59plGre-vthRv} zU|;TG^Un(o!w~stRMQ$pth!Ssn?%#&iIwzFvPnrp9VB}-eoj$duI&1L z&q)DHPMr47W+LGWy~>lbf_)^;k5k*ABsaxtFWZPDO0onAh>GcgJ?Iu&OJW349uK98 zC+5FUDx;2_(Z;uO0+!G>jH@%nGJjx69)+y9j(`u9Q?##!j9E4&q|7pLX%UUiNKXr) zYM(5cUpo5fh?mbKaKe#}SANPAStS4cCqyxvu_w&L`gUfi8UUtci1Zl%}p^c-AomnSjL zj?TbRSShLcj5aAkt@KYDK%!~xqIZfd5f&Y}KqH4;#9vH4zn*;iw5hlCu3U82T zOkBq0n(6>6^bI;M4p{;>z0j2?&i1!T<8f|Dv3}Co7B$bItmt-V0mt&3S-!h$J8KTK z_J~u{^sw+5$20S>=z>YUKfLlhiqmEdk>c7^?ok#}bgR<)3T?_n{=OE({2CQ}qi(u; z;C{*wcDuCO+SO_-=wVlv-TL>DTC-FX8Y=)OnFB!Se_<;XGq$pm(>FE#S6sRTxM~iB z;J>myBINlv;I)2x<@?!XSvQc4z&1M;<;uh~hHyE5-I(G!(gLcjav6kTNyMD& zp`}V!m;0^~wCzl5{0Q@!wQ1PQDWn?FtU1Az_^(@8A*}<0h`KzIUbtS+9GiL--fJN0 zJtLKiKpWkg|C5K8laF4~IjFbi5J~`%?N>R~tVnB$djnE3D>V{2Wws$MF(k?bBaKpg z@?*SRB6-0Y)GoSOL;~dogu_tlmL4y}>2KTt2|1E~%rs3YkQG0Rz`GMQ{GwH{ELP0W zB_UbK8xWQq3a%*%tfPN-p>ZgK|3y*ajIb!ES5{czX_?26#Sf=lu%t8byN+b{m?*J= zOvFNFyeSxxo~$vUn9SE3-~ZKV7g3^*c|XP&eX~0$DR&>WD|Q^r*6X7)NG*;Z;Xi{0 zB+AVU@2Nko&IpS8*5@!jKy{I845^*_IAuFRyK&VrqDeZ_sNf>ea`OJQ9=)lC<1l*+y$ujbbT=$FD4N+CY5}8eLk97+;RQC zQ(aWh^RM7MX@tHQ4IJ*c+K4jsS;Q2wsT6?~bn9kKiKD?!Gsvur8b*-X2dZu@;b`b6yb@U+ zrn;XZ+^rm-J`g+4G-akYCW1m(;%}OLZkW~I8dALfM*2RTc(u;l>DsaMe%(xYvZZ$5 zz$UyIuv5EumN9qETrsx9T@|ib1tFfR8LjT2(S_I4rRE~shkd3*@vf<9Lx)d(>SW@WaZ#?yWb3)&ls3*zOi9=90yyKy6QC%wa40n*n>*sAmA@7nW8G z$?BkhIo?i7?pD?8)0BJA1*YdfpWbg*0WHaCv!HbJ#w)eOj1*m12pgtD_u+scF20Nz zd&o|xE)GF1(-za*d=%;JT{W*z$G;GMAH+YfH^lEMeYRJ8k&BoJi$R?OBWL<*a_l z@ky>N+0R(Ca7T1lzgmiu%sZ-ob%&NyOhgv6+Ng-8vs}&-kyGxaZNf&tu#Z?)zclsH z(N9}qV5)c)^uxH>GCQResU=5xPcUC+G|IfVIV(s1Yj2x(phmyEU(@JL`;VhEYl6TB42+Wc1j&huIxoW?=WTnz>v9~kK;!O?;73hv3@!5Wcktf{1gzj<|1 zJmw!NHjJP2li?9NzGR@ZOw=4z*%!9yWF+Qpka2=^t>=C3MqsYb~j{Qr*FK{ z7ceIT9u=!1jvLcJ7S|&aqt^G*p)vNZn$0HKREw$dj9`bZvGX+ zrc|{Z3s{l8U3`YJJCSX*>n|V77cP+vN#F z`;jX(f_*#7zw3rzy7!~?B*v8JN_+n+U3m8uA?h_qw3Ye}>@c-gM~jaRLh0{qXANCkMTAKv+jZ9a66os??`kfxH(HZo6Tq?CPip>?zu6mODa_>a?2| zy;j5@qzk#g=r$&7rUSSEoLj#(grO!+M~28z2W~&RaV;6T3$_pfj=rfM`c0MpSVFV+ zII}|Ine1s9#Lk({`ViY1{G`L$eEHe;tp2ft3PaNZETQ+f^?K9Vs5t|xaL&gj1XzC( zM+u&T0?Y@@z3`?5WFQvRp9xWaHJA^2*Tkx~4kPQsVmIQR8NKJkU^E+ZfC*mk`IAs7>=mK|N;A6*}-mD;fZ=?e}^J`+^9l-uc8G>ab^#d&{T;s(2^J5I(I(W(VnTvo1e!*=?3^KNr5d2hMcK`b2yJ#PiA2BY; zsI!(amYye&tBWbjXGda|qDQwFiERv65i4IerTY?!mbrba?L8opS#bbhP|PhP3(KlEb8bdYyTgsX8U!W^aqvIH}Vjq<(*L@r3f@mGk!~Ha8xuhKlvX z@bHf`lf}nBVb9RAcrr}{tbENxDZV`n{Jd}?{t?Ba1FiR}{8I##AqXswp4*{qwh~;L;6_6YthZno z#eb%9_NWu3P}>y(jJz+;x3jRp>K2&={ik=TfMDI=N4IgmV#XzkN?@6ErfUac zM2U)F$I7Hwe5F2>0`ujHg|;o*{BpG~dxiPfNaY+Ar=eIZRDZ1C}tQt9)+R1SC35_^xYVss+Z7#jApvw$M9Zf>lcdjj`d!uH?&Y zN2-3h#x(RrdRQix{T))thD+KkW{IiaZjQ9=-%_7zo;5DyY`op>Desonewu4fm~FJG zV5iH*j$b8unaMnm{G*F@ON3jfwl~vSpaZ*}rr6$q$#<;QD8DXTRH;ZrS@q-wR&rzA zg1f%_;9rHeJa=ZSqxzhCK2bTEw6s}~!?tw=UHjcMt61|b z(T!@H9@%OZKz1kj!9v@uw&voZ(J;9}m)tfm{4H?b?6JZNG#1}oCvDt3j{J4J!-uDD zmY~Pt_4Y}GGjEkd)OBk=GFghq@5TB0Z!>V;%3rGTfKrzp-2asi^PkPie}%~c7^3E>5jpdu*Tmh=~JHJQJ;| zDhz0uJ|CUCZL$J$yDw$Ye17t1Qwuw>P4Gj!-CW(=qyl~Ma$@$D(cNLhzIs6je#emc zB>AI`;JcdQyA2QVz$pt6;D?6Fr$nAMj}|Zk@j>K`vI{{NnG^h^-Psfgr%R!i9^%I8 z6-=7JN4d*D<;a^{pwAHTmqiD@0SN^`yCK&woQib!K7)3C5^m-xACQyfH$|U=l6!j+3qo|>@M(o}* zpUAysPRxds0tjDmtBOZj(ZYv6qs9`r5frXm@E;I?oWM`n5MEFDV>k>?~Fe_XB3Vr-%3LOY2&R)c|4 znGZE<9rz+`hMvRmR&%=6^rGv4~J2=%z%$%XSL(rljZ=OH1j0cyTY6#oP~Ho z7hjH#N^2SVNpMFmB0+}0q~3lx?K49DiE2N>h+)F;oNRtW52k`f82&AImIvKzEuPDu z(d;PQiVU3|38bZs{(_rmqkQEaBeG}5;McCY6RyFKqcf#m8|9dojg?|ETQ2=ub&}=H z_$f`&Q?2YnLU0PnLDvMvn#$I4h&!$g#07NyCL#Xz$V6Zu!%kh9?`1jr6ixZzOv`H{ zH}I+-&Szx#N955=?h=0XOuShAfel**okIM8>G!s3ff5f*u_CS8sU4o|i~YtOO3qP; zu?d#$&u;wjH620J2xVz-UvoklmddkHf9|->)!KDJ1*GcyxzpFwUm*UgXku5cNjNISJl=YP>AXKTZG9>wYr|n8$ zG5!_f2l;DIz&|5>4459{!Q2Dts%wn*f9YynAw`+=k{@q%1GSCgbqU@vvWY=toiFtxL ze;Twv&1vS;R!%W=LmFvsYg~hWu90buNGekt@rLTsX<0by+@c<5CD^TX zQwbLPIYm*44f(;|4Z)h_8mM?f4U=sBCY+Er;pdfTAH@dxY=4S)CJ4$holL8Q8D!L8 zaesTv!@ns+;Gq~$;lxl7M0Mxp3JdF(wD6)DQ-SycEu%#dd}8hC7qRQZ9_**rP3j;P zNjhcnG-D;ZU}Ac<8KlG%9y^D72=w?x90E&?oN>s7w;(y91m!9B&)73dz)C_i{kjsk z&x}2+PhJYCj`;%pb5AF7m%oR`TtOj7m;3kIsN7eCm;7M}e3@EqB4VB-`O78R&R=iebe#?p9%((T-_l?yz1t$_8V`p)^RmMAA zTFox-&$->tVYxS8YO!jJ8={C})fGPI7WUM-J&85v2QnzPfAFZLz2$*`YnE%hhS7BvSE!Zz6b$e$r0v3wFh8 zsTbH+n@^U`pqA%rBy*#s$n)cBuui=(=WfOV2~Wm3t`%vUjZ z-w+SfW0(wCXBWBdSIpzmsKUNC9k)zKiX)6PN^xt3a=F_`S?;lCTvX7?FO?WI!NjwX zo24IWFfVdkR46mcSkXWp3x6oQ9EaC}@OcCW9GBIx2w0B0mH$!CH-@jEY*H~&P9zkd z=}ET}I4Uhw#jVpg^Yz1x$E1!F1=hL{0i*QnLdGP$b!Pc zOpLHkmJJ0i)e-t8bBFldn08T1E9oG!Q&MA_qNE4m#qB>(?jbS6Kly2KSc3-Xy+CCo z78HVwb8tjN?B@RN!eW-ZudJ)xrim5Xj+Im8-Q$(nSuHy3oFh@uyuKo%$e%t1{gUw% zU)=%N6a1#TcV`*x9y}p`b$$PU!```ed}|ijvlfX%*F3qb-Rl%zdqekZ{IYVpB!35- znVTq*HFx5DQ9`*#rBARW3JK+<&&R8;9ZHI_1H6KWsa$~^3flF?o_gGle?Lk~;d&04 z0(@840N)kIf6nKnjje4R{zuTWN@WKS^aL~=d_xk-x|C2{&X$tJ0$AyF35 zE+DTdozu>yY-pIQDBPZxSTQyPY_@MFR8J|o(r8jRr_|gr>tmskH04K^Pjp%u`_Q}YfzISRUmRA6J4u;8-%e`9$9`BKiV9u6s70Q{iE;CT)8X^AF&l#0n zt2Le!&0FVCLe+Qb2ZBV^MGK}XKmCH^E*4HmEpOHx1o8#*y0FI|uLN$s{;lsAb^I*= z@LQ$JbSNlS>8D5YpCP-?!Y?b4R+Zlun&CpQg)?S=0da$(3M!Bk8IXgzgnHHI(GV3E z?MwWimM$N6iZ1s<9+upq{Ci*x(}aOtP-q6Cj;CLsddXwzq)N1Cr8JK6vnxxo*9(=#d+``cO&A+yOWbpDk@C=x(2}c%nBmf}U_B*I*@9kOL4G zVU@`O2OL&ojAMKS_Yb&{<%Rgg99W-;|8LC!a=h@oYuAv{8(J%0WZDNlsqC~-l%c&| zoi6KB6)DIGGg6*sc+a#Dz;DGsJ7av&(Bb&vXm$ngTcH8`R$d6xfum{%FU5=wJfhQs zi=dz{B=#g9Y59^P_c<`BMC_0*22t+w>2CSYp>`#V$TpMqJk?sY;7K{cb6TwlX!`}f z0UZa1eU;`^{&?YhsO(@I(%I>b{`jKb0&oh4v;qucHl(h%zR{a3@m&vpD^kNPlTBr~;VLxhk>cWPi?Xw+(c1a`YWJS;|F-QgHk zAbT(y#*9+5)a~8KtD^It+HUrz^f(Crhq8D6&$R8byeqEQwo|cf+qP|AF)Frgvtm0H zr()YSE1v7_e&(I&=l!93{)h8(oO|uP*7|;T-5*T4f#Kf|HXOsm%2==$^aK4?AUeIgROS;0sRvwK^E6{SZ-ch~@>_9ex1|C7R{HNN)2EI} z#s0>uGv;qkjX7rxqNG++p@Ur;CmkizV|vENYq_pfDV1s%mEauj7v#5WS9dJN*x?Kn zYmO6KsRgj;?gh&>u3YPbNh!~iaK(Tcot(413r~-?r)RLF!FfNF&yJTr5AdvY8Psmw zZf^lp9E0}3SD6g3SJiBr#=Sd79r9=7EZ-pH9 z>#U~xwVvoiK!-fsu{5o!lvK>13!i!)IZvEXGCO9^TVKZ# z>1{R$9{*;w%%o(hWH>3rhI~hX=a*Xl_}m`c{g(5Dz!IpAr4I2Zsxm9Eb^W<+v-ngs zn_#rgiHiUU#=mPK89ltu!oFs5YRAqSsT|3u(|0O6>G;Lp-ftVK^D^dP4(caAp7d49 zfxboA6yxyFDDpuXU&OSB2bYPUVhZjW3^JYTE5u@XA`lEiNr>=wwSTnFuL?A2cGl^% zi2W~*>Ls(NbOVAVFWv6rZgbEh;9Vk&{x^P>qyMel|7&4GHwAm#plN zJ(9~o;}Uxb%9K`2R8eC!qdges9V>E{n<$*D1mPol#78gb!j$_%rByGPDuu^198jn{ zBb}_JylgB~)zGGZ;aPrNh=U1|%}Pg*_@25)Vn8;jDpe}GAYj$4e`%?%RBx*m83kdP zRT;|W2Cjz9nu+<`wfIWTa7Km!c0rB~RtogZ+9kXgOAty%&r^+wS`jRO-L<^Wo`KL{ z!n=BlqlcQaysoh z4AqL);P=J^ZI;rHf>=PT(aG+_lt^gmU~j6+A%1wztMB$x|)uDRUb~lX5)ZfGzR^rX9z#A|v&e-aRaT;TBpw%XJim)28aku1&KQHRno5!4i*ezufq;}1=AL~BJgra2Syr00B zx|r7d$O%7l(g3ebs*F-(2m?v>&AiC^GUtSM3;y&c4F(N8*Jd~CxWgZ1S{3BwBLKZslvFe!s{7fUth*D{3YjM|MmzGf+IizazfH z_67HEd#J4`JgQgXpm$FrZ_%9~(tz$Gbrhy8Z&9JnbeP4SQk$PBCJ@?5KmKqkx!-&l zxLvQZbXlU9+Oa4U6fXG0?d~zKCPSF#aCzyWEnmL&zI_m6PVprdkiWW1Qxj6DwR*g^ z0%)usO?g2$N@o0iEt^nd^38Md^P3r#Bn>R`M_(5v(qG&RPJaA(tldPU3MCBPG7Zvx z>~E%E7xi2Ac_w-LotwS-sefggrE~uy{+lm>)9vLw(>Oo%5&g9ifDDkZz=%A%Aty?` z{?jM<<~QLm)5hAl8hns_A96Q=$45(Yoj=iO07CvJ+=)TAw^Hh8h{TU)u>mDCCly$c z_4<(TrTGd8NtYgYPU=l%vNBjD0@CyGy@J9iB{9kkGzzN_SI@=C^`C&U1i7A=UFiK4 zQ=BSxeHoo*`x4$G+}bXwm-r{IvBYS*E{|#RU}rN2On3nMguvmD2!DdboT9W81b>nw zQw;vz`e2`Rg%3vpWnpCusai=(_B$&6tU!mll=-2Y%2ssn`b1O3l_`B4W34J5-UdfH zGeGf(IQ7vv_($?$#j|EEq&82t&D=V^Hdkv)6IM@BQ<3zNL3^ zgga>!=|WOC-|Dv2n00x>0E8+%P9h-xQFXz`IrSk1MQYnm#?FHyi8|hhEfsEtUW01z z_5DG$x5W6IhZ!JR`(^lTuIOVw~E@K>UA{80Ji;F|22?6t*%miHoI#k z*2=x4FvZL^u>488OGQ*fGd-GxGyXiO-_>r~Q6w69UJN`3-Gl0$tw(9ZFl&Yg8(FR6~VZCmN`9kR_-{oHwW*7)6j{tv)MRn#-4`PwL|^05KVTyj}|> zU!ETWine!MxNz-$j2O2`nq8((QMp4r#z?k2E$7Q=s15snFp7RL}`x_fASAq=H zF_%Upb9}b9w!7TDYejs)-tncv_dEUU!tP482+F_Xc!G!d;d;NezYJRDp$ttKeZEp8V3cI8O52HcVIofHZAq;B<@H5iVF6ptAfduCLZq<6KhYK zVsihiyYMyihqi;z`|C4>bUQ-R{aOzRe>?%#mTY$!gV|6Iuew$TWfqh->nv)lycA$z zqmZ8=)93c^W%aAUg{?WXcNgE=rJ-j_X~vz`9}NMnUn+M&{OxI>bj5s+{m_r{@!_Or zzPlsXmnYlz>feh&-~aTz&I4`Szk$V|lK=DRM8(q0)zrbmUc$)L%)8%1~Lhq?Wd>tetA&(!wy)zP40b%o;8n zDeNw*Wbaj5W(KzEMX4o}L>nn-+pL{=sWjD0S?OpXip7RK@!Q>&1UBkx#EAhUwA79u zc-V%miq;Lh^#yzfs|&2%@{`rLuH5CKY}e3Kv03%h5pB6ol-2DLtYBLTjr!R7u=Y7k zGyDgkkQ?2!ZC&mB)1db7Pn~uGY3~s>^z#QCCy&0A3-|}S#yeu@b&8mS`%gjL$_=Zn zl3F_G=iKhe+I}S8@=EXgsK-iW-n1+=w)7~c1iMUUqGR&_wNxvf=&GDWuLJc_Y0i9l zX%Sz;3!GJM3bM;HQmI*!sI}Y%>P@#{e9+z?owB1Xs%O19R*9y)3f1C)D$iZfpc~$} z0h?3kxM2=P80WzkIWo%m^wNUs2+0IgPfqwf$=D>$-~H`HJAf%iTbCeQ6vE>K#mIs1 z1KTT^9b&-)@Ordy66`j=y)na}XUd2l-%aigWry1^!Z1e6>ioTwJ;hj%+g0Arqe~d} zaZ&H2he5(Rg?Yn{;9k@c^bK zGl<4^dq;y&+NH?MCwaS!@lvAIX!pr5SfNFxPijS7>GTTd#}HGL=#>V9<5zZXqCr;; zhwnL+5+PVCq3Xj6O$jC#v4lb($bujdy7)-lvC^{()Uh@h>7$ISjb@Fh9b*fje7&3w z>hNGdJ8o~~L_>0Kl^W2(BN!-293gZ>AhjjQ#ox}PhCB}V{|*Qp{G8`d18e=*S7OmA zZ6W(LdRt4Lv~6I{`tUQ+_iIO${HjSKZ##N3ovI_d-F-R`Xu8!KJ+tjFg5tezLd={Z8lD0K@%rA4zAFtD_+<7X2}o9+3; zG!xMXught!fR!FnW&dYBzPr~eQLBti_(oIrnwHwuE=y}WPDJh=Ka>cAI<&zq-Qc3) ze*4YujK8tvjiy|qRoBGT7@zgeMEluvdLunp6ernT zgxybAm4`5*nGmq3#ZFu>po3^0UIN`9B;1~?~D2!HqH(JV}?+s9U5 zVW!oriDyl$J1_pc=JU`9u!YTD0MN1otax9y=XtwxdEUAh$ax8wCvKVV?p5YLHHq|g z`Q3L+ zHO^8-vVey1Qso-mP^b9KIo`2UK$`u)USy)IFJ)DbQtB9OcG!FXQwN%9qH+e^U|~Fq zbmIYc_HIEN42+NDj;N9|>M(33#VxIAkuEl!nUm)5{Kd^HzfA4GZ7|oYDy!Bg>(xG zV4tLtmdXJ0{&gscx3pu7=<%RF8EA>FJt5SWaIs9vtYNYi>6G_-p*-4C-w!LY?|$~t zAIy$_i{(C%{me>TeI&dS*gZz}|_bbi3ad>IJ$2`dbprC4v-UjYE$ZTC$vA@54f3w(2 zy8O$ELOBqYdF{H&5X{#?W+6$LYD2JJHj7ERn@5u-vb%m^|J@%l27Vdy$keorHMh-V zL4n)uKRA z8BQgkcQ>5+<8S|~Eiw5Z-!-WEY@~_z(lzRdkUZG_Z%*OFOUm(Jj11Ie?%mNG;(gDg zo?}@5JssBrPB1l(X=jBHE5NnofoG*N>h#$_ghL%Ln!idk?Bc%%O`l)caj8?wgZ(y8 z&r?2t4QyM+sv{^j=4lznUUc4Y89p>F*LOX=SQ+5Xv`;&IsqH_!Q52m2Wj#Q1cB+v` zzRrykK{~-sm&y(2O2J!I)Je&^guM2W0Wys$Z{P-Be>yVG=%MBy1viG34TGRh!-8Wz zfO`0DxR{kbV)h6yc;8qSVu3I9-?hwtc)HBvQ*4diV$0fPmTL(5gsTlE2q!r+b0+?} ziUG4-pxvUTDw3TdP3B3DSCZ~xJEOnsAY6cZ8Gs&Uq`y2yduGZ$w= z^7C&pq*mR5xZj7ip5$XdWy$yiO^_|!x!R+PtgO-g%6wFtP<7K|{}wMsgQkLlx6j)X zhYEahWNGKpk4-nOs0p$NQzVB?H%6Nn@%9`^JNgCCI(mT?nRSHY%^A-RHLP9Bl8L)F zu%WxqtFblh-h~qgtH`2`%{@uz)^_!7O$SIIy}8E21u@WR!7gR@NCWD*!j_!Jt(2~( zth(Y*>ZV@ZC)^;;#}nCv>`WBbg+cc>i1v^5^~hz3ml}-nj$da7&+eLZHxVQic&)^f zi1s3F$zy}p_e0rbrP3zJJ`!O~Qoq0@0vE7SP=#N(ZQvxN5_zoq?QwRq*D{x!bh1hM zo*?s8= zxQvxGV*Up1pAa~?T8*6EiyV*BhZaSt+qJg5t0)8Te~Za{+-~}Qvs0QRI}&)tljT!= zw;L(AIbggh8^_P>%c7WIjW+FE^ezzT zAjRI@2BbvJ%r>iRTz^{rfb3he_9YEkYS`@OUnSXg4q3poK9FD3?o;k4Bjt5gMj?l2l4xrdjOaL5J`QEO1kDGBv2(DY z^7Z!Tf}Wv8s)&b8*|_2UX_94d^7ZfIuaDC~o4STa_PA?8=@|^LzaIyi5%T`qr;Kp- zcY%~X%^^yj)ANkYY+!rJ1?Yha1X+{*cLy2T7hQ;iDU~W*xf;3 zQr>gI-_y}H`ly}Uprs+kAMUZNCm)!lh}|;l|2z;0V?3@qdZDJ5A?W9xCfNT@I38B_ zTLABaOzH`(?741qj*6m47^}wfMQGQnGpstmR!d0R6#2|P5=iJk!h?n z`w+`Y7?=l)LKX)__{=d+z?_{J$}mkX?(|-tCmlaEUbb!Ii69;fHvGoYpFfN)QxUbWUFdH=G9Y%^K%!TFLX=~) zaSxGOgr)B2_x*mVrg>k+YhZ%47aN7e;6%Fu-OpfT>kiOrtUv8{J(ANi@GN>4pKKkb z$XX!}Z&;f#rc(q1J3h~Z&6zrmA zgFl9blVz{fC0dQE+hQHQxJ3AX4WZfTY3apJpnRGnHb<+d!dLq7g#PRgU>K92xb%7+ zWS-A<*O^)^{xBxXn3I2ebVi5*rT&$*Bbf3L2Zve8>3iy0Wi_m25yY^M2WIU=&l8B& z-9&*|yYtONVAgIyI4p|>7nrq+m1ek+EgX~pt;w&6P)eB2Pod>}I!xDFi#1DHAeZ2n zY=vs@nxw?12;NS|0*IeRRT8tmtXP}tEnuubq!-zU@02hD26JyR&IoxQEtXP@^R}W4 z-dr{e!lHVCsRpmN&#b_rEXuMEzb z@akM2>@LeIC|v-(7UFT4Wf|MN{ig%$>i;a8d-?vQ1%)a`TU>Rl% z`a$Z=HOQDMLitP1!ow+`89zF6_XfWgNmg+&YNNIKoFr-_kS|qGT9Z4;K%z$CcyURobofWy?fcg_)-v z!gb!;iIIWa26=x(_yzC`)(M5}0n1n{vl}h$(cb^t|3(DQ!)&$M`i-MZd|=+Y3psZX z+9$|b6sGnE5(rI`xNm@2ijV5g1(p2e%s-I$HkH?AYzW1b^SKG*!W_xv8RFL-A23#I zgx;cOphdOP@Uknoq=0dZN6-?_A%;@k=Iqw*~RK{dR^%A z+zBooWsBcePFbj-O88W51AD6)&X*Ebt``TJqHlR3 z{L(Xt&N6Ht)zl4)LcI~zyZ=4@J2|1=~n z*pwTq=WxK*mo|0>ub(QiXn2=(Px8Q{k64;4pI&5M7Pn(IkYN0&>fd0mapC+D`jQ(i)>)C%zqfyp8p+x?E6G0EnzHP#lX^yXh{xZrERIF#E!d9pdmA6h zeFDRI8c_l6_-igJlIk|A?>u z=O6VSGX^`bi0vy0{8r|h{> zItl|wGCtoFF4A7)t@H^61}mz%Eag;exm8YJ=BUGJ)f81{&T4@q<5e`ZwF{%FEWPtS z6(a&cj;9J95h|V>%=Lefj1{ZQHL!}D%W*$6584l^IM^$Gp1%+ZQ#{Sl#8YFPsIfiPxEC;QazPNMwjUvyeG@&V+xHalo!;-QB66A>;$d(-ZL*QWbmtg?kF8g z2d2gHoW}NXiFY;vfe+NKhWQLBnx@&_+bKMcM1daex%0}AJ^DQLGn9|`?~Fhj z4T=L-xx)3(^oMTjIqMthFUa!e>)t6x*Qz^fUV8UtdpI9C!g;L>_2))$JDHS>)j~xx zKj%83;Fv9)_FrnoFhOk>*QMKM+o$y;hM9Cfe8d%3P^O%UO=AWEL)g2jtjiOoCWrN( z`X8RHup}8aEQ4WF=j^1jsX2eFV=TP^e~UVb{=2SYZ#gMzc8>g?d<1cj*L1PhDB3v1 zt6DRqG~*lelpmpHyhEe2-x)G%EAo{U$^s&qr>|0=og(E+5mfcGxE(hrtSD5X31Uvl z($qT!WEyze2=RSGAXqsc`6~Ax<4+eoA7ak>05WHfA>Ty`3>nuKZN$5*PfAt)EJ}r{ z5C@~`K~50|;ea^c9pi1ifYfS~T{^}qyxtJUmdBpZrz;Wp;GMvGk9e|XE1gH3ufk_Y z{tO;G=xp>Pc>dv~Di@5o4U1IR_DeZhIVonTMEbTD_kz~jp3nT6y9_~4IcxW2mf){P z%UCiR9y~!~Q*He$#Rx&2nE;D^A5@#?n4_m#EZu^QPvO!r&KVy#?j~xctO08Eb*D=W z>O=Uy(d2g9B zE-)9n^>o))-kDeC59d*747`5+r?bosZ{8{yuL%S1Q>=j11Ov5$1|DlUi!~^fN#N0- z;>UQc!^CAhgPjOKWu~4*m3?8=q?5(oOotutCh5Jtlp~Qh{+Vlg&VN~Nk-Ua+*t2mM zy{oi^!=2tj|F(tZYI!N5wNubL?cIgFqnh%e%I?wBD5vDCbR20t|HXdxdWdU=zxshG z#iK*?yz+QbI!&A+bYJ7UGlOY>v!2GYPps|^rwJ0xH1#8E50}qyLk*yn<^CDR-RE&M zf{B09z$v`mD`~{3e(zs0vtnfAF|x+m?UK>2*wn|b39eIlztv=>??ySeDc(Nb2q_<9 zC7RjAPCGQlZwy?6C7{nQ-ADz{%9Q@FjZz7u>a$tL#d*EVX@&@QDz=PUnK80>{-E5N zy^U7%VC3N^E~j}pY4Tyu+)Z}f%%JT~j8t{5aClcm%H^B#&R!U6Nbs&M}MJo2IE>R>H;|y<5 z9M<^zX_~nwszSD`Oe#l7aL>H*oRX|Nr(N6+^Yto&E#-So#ZVHPafRZ;Z}+Y`K@|0# z%&yfTl`OLVDO?dWnr4GMRRGcjJQ-9=)x1-^8g*{;Yy6?d*ChKeBZK5`!W~xy&wS(N zgK$KqCXHJV2TvsY4HcOB{;&jAjY}g{{hJW`4ykO$;7!BQ`IYd=Fe-=^-9|lEmpOJk zRaz#;N=O4Z|81v8{xTz4u!%%D%$`WT5iImAwLn`5Wz0>>NVLs&;aeh+Ym7;i;thu7 zsQXUvw;Um7RqDbkR7c8dqRlpR9X8Hh9rS$V!P8cujDNkr&@g|Rr5{G%QL7Oxs!CEn zf)fc(4(mjmnhDokfqE(d>?H>AD0ejy^i)z2xjJ9I(hM2=-K;Gb!S&mpjedGa-A`{0giOK>RTCI8f>!iW zv7VuwaSuK&{WabmuU>-W*vbDr8eUyOhpn}(=G z@sec@bnJVA52yF5m4n`ES)Ad@j)V+ddrLHf30`@bhK6WhVY+mx^5%FkA{ykKSz>C6 zw8c3!opa><&!v)L*0Ki8s`drSM%K8+P>c}mREo`#MR;^+e~j};rBty*`Z>TP7R8YC zoOGp`wi-UM%*n6c{#iiXorwWI&0UIU_jyOcY zAn*tpGyE7MY8Xey*P`RE=-ivEL_uVr5yacCYeFCk~MdPGM{E1O!fi?=sk{hN(^iKslE_V;1oXBoF zU%4Z6M1Ek%Ndm^Yi`yT?iThtzX+tyJjHh=S{al$>?1v#eL(bqxGYIRsqcD7f>h%ZQ z7A8(dZ7qZ9ae?$=I}c}95dprUIq!4AN8uC8dSI=u^=y-+S9B374dR$VW6M*~p&8#r zSp+!YN#44YfXB}5H3<5DA>H$-OxtyKr$tNBRg&Ve@F)VnF$NbGa|(jDg+g56V` z{CW<+W8EU(SnS81S(NFHIGclTmDN1{Bh90RsOag$U;@}H%;rULcuqUGW|_RIJwA*o zn1etlZxUj1VfREEvn6tXFuRPsYX3m6cfrZvtlPTEzCGnnR`7EG!nl!WdgMXABgcCQ ztSW0=5f()i@AU&}ZPq+=$sH1`VOe_jbL{M`K=+p?!QZOiQB)nTNfvVnUgfu1n7S!G z^T5omJtUij{n=kE+)p0>+;x z;w*0p#=M_|`!xG0{KKts%vk&o zcel{|{Prx}B|zc*CoA)oxNjGS`N_&x!9TsG(5+S^oHQx6X*X9!H-47vQ+DKw;W+j$ z;s?PV-|akikc!2Q^r;Kf=K|y*XPk2D#WI+(K!IKb6zGSnZ^|<$ zIpvE{LQtcd8|=ttJW}UPE6azY?xPTtr_; z#e-te-6BI{+}2PntHEiNSqd0zOigniKeAn^9-Vdb`uwh^KI0~Xld!~T^9W6Lsm z)z&Kv`=w=8hc8ZhdX=Pb5oYHu0T9o?ie)o#)p;2+K%+@V+Ei`$8% zt3~8xK#@*yH~x=ECy;=>Vj&z}wlv)lyFX7ikSCWd>a(c-#_{09k<3I$$vhBKRx3DN z1}?$Q(FOVBN5De-Pkql3u)YTlsk7ifma|WULDGjnfA5+y0ZAMMlz$&3Zfvxm4g6^Z zwGetz^Fr=+7XLnA!UorQz)W(N0&6;?9arXxJP>z~H{v*<&N4Y~Is~)LXZW4ixv#Ac z11Qof!JF~$TXz`olS@+%ng%0ydX)< zLzI3zI1)V=1u81sOea2ha7vJxvjS76>`C&t0TI<*7;<5F-G`@|6(vg-@#ez6nLzBV z4lGn{Dcp1Y$u2t|cIQ9XTfTOGl%m31X9lhBt7d%+S|>~m;^3}B5Ih%EBM>>ezBe?O zq?E}$H8{@aY2In8_016cAj<}hB-?}(tX-%<4%diRk-xL-7t6MX;y&=;m#9B}0_zU4 z(g?%Ef5GG^TW}J>-8t)JVenV-u>H1If(!0?EpDu^wzshM{ams4fRgx7^y=)XgP>XF z^mC7i?1%~yq`3gp%R6Fv~_e%ma)q$ICNS-E$B69(Yxd^xjj#4X0H6%dQJb7hw<=J+ie1L9iwHhG>}N1 zm*nOt2P_J9#@`Te!qMBHxaRU#|IXM_D>1RM=I#ttU0Ul-V?-ACUGTA&%9GH|HugK* zNYN};hau&V_-BoR(mt=Ht%reOY`Xh^uPeZh*^7l6Kc|b6i;s)P*X`-`;zF$sO1Qq- zkmO4xZyR;3#K|$7HIKI-*7qiTo|wepoKJY(h*W^n@3PO*^VK>;WZ51SL37EiKrLj5&dH~NWL|Efa>*7!aj`(}(zUgPpS1_X71BT*33eLtNJa$KUYJnL)A1q#n(WYxZ;8O@vdd8++5 zJEPt4^XbhG@*#Dy%ab;#(@k#8SyG_e`#CE1n$Um23NPhRyVUt!Qr(rZk*Z=;Qblusd2p(1^%jSV}Eztgs@9w}>jLqJoSZTMeK>Uy`Xsrlrb1N6o;IB9AE5UD~DE z3s7Mhr-(~ZYly}YDJxG_9d%AxXpXL+x_=`Z&exY<>mhTTWA&G8;L$$P%DyR-rdu0* z^&L4612acItfDJ1BOq<9ur9A4RWzHiOxSj@JeNsb9#cLG|5x0=Z;Y0ICpdC~Rhqp4 zuX`Nq(aX;kAZWG`0Hl92;xz*4-{mS;#}nqHjoR&#hMKOizN5&rR;Y^&vV~1+*tP7^ zH5bq;(6Gp3g)yeok?z4Cruh=)van;o5!X3UD?5!7Jsi|0X*p55ySaf~+z9W7Ht`a5 z^_|*BAJ1A2awSar0cwyNG+|ESwD?kPV&&N^3?)f^UniUc+%`qDMUMv^?Rw!YT@ZW5 z*2NO0Mp8UQuF0yOV5p7J61*={ZWsZDe9Si*1?*UpP#`Y4EW-f`1i&A0I*HsT0iEmi|LC8nL2P;?l2t*OiyowRdt#SN=2{O|fA6p+CrOtYIfK}VbGys} zi>3MB0>GVIS91kFWtUXD8dJW@obhUf|>h=eu&m0&vN z;~qY4%@OvkuNJoM-*F!++9bestjnP^^1ebn(b{14iEJ;ECAvQz{arCVx!FP7W z_Wy_&SQOlB5bL%Dirw_}S<7Txwo9{Y+`a1zo3SkDpSZHrGsVey@Bj)N>hM9}SO@(m zDhHD;gpfm@6s@vp4c8s-*+Qus9wRo{!?yJFnLTIdgO8rfaz0_=Y zXUAN$D+^x8BajY%bX4q;cMa^-%tnl~e8D7uSLOYD2dIv#QKxNS(?bT|Kb_v$+-A*5 zszjXQ>NIGlN-0%E;i9LUVW>H$iMgp%J|R1ui&I!_D7s$RZls%4Wa=ZQq>Fi1hLS?~ zZ|&W0kJO&y0c_oV90$h2Axz(K~ifaza|Q`!awerk=UFnWL5rdY{4uL_`Hz1l+f_hFK*J6AlGW+ zAw02lE#0Gg6q(ch)#InBCZKHdJ%f@`E!e#ho`Lazf>*iwL!_)Q%R5vkpRzdW3JLj2 z=5%i)N93wdYEBa`#=5)FF-U=?as>ZGfuZ{gXAf556-bfd3*+`!%VruB{wRD4;kyfb zGQJ?X>x@=XMe}^WRE@qey)2ApSCuC$Hu8>>V`((Gw^+-_9L@rFYe2K`>K*AO-JWW{ zY{@m3%gnvoOEMMH?!171FznEAo150y#|JduMD!w1@%^$=(Vu;G6`i=Lc0*t=lgqH9 zBUl7X=&xTrNL=ful5Z*IGW3y5t6-avec9iD76GEBPb8BW_*;cF2|o)EYq($ULbl2y z$g0k$bfjJL2YFa8cuz*F188*ictS_w8lCMIlxCYS`y!WVG#9f9uZ9Mkw=(YUCg}1GGQ|P8zBiB{q z54%50|DrZ2KXN7MTZOC;{O_q>^}jlg3cv{#6xdDj-&axo{~mHFdv^z$e{EGSwd`E7 zB!Pg}kH}a7+X8r{i>i;NNS82j+eXS@gm`=n>J(v1+61DRTjLRr^DnLQ2kud4Ne604qYlwkW`>!6jE2Rmn#HZGYPX5SVrb0x$A#Hw z!{yPg9-a`$TJXwtuM*n#yjJb_E&I$c?>0VJb*jw_zcQ!zNg1IcR@yP87po zb`NnDb7u5YP-$X7IBg+KfemR;Be2H4a3+D{iYsznpzKRUUO_X_Vwr&sNC%Qnunt!D zX{q)|!9osf2dQ+54yG$R2D=Ol@^=5YqAFyTAlhz-X9iyuOXkKt6enpQn?~3CY(ES< z&VJPhwE^Xy);*rfOh@HxAC^KLp1N3To|^LI^e9E0f9$KHVIc+k(&k^fjBp2Z&oD4afDxQl~sV6AW%3)o!}q*V0rF$_es~rOpP<{DN52) zZs|nHR%v70yyfy^lLy+h3|x}RQXP`oRhlgNCLMikc=K@RCy%ZsZ`;`K9E<_`6-IjL zNyNs2ore0EEr?s~OgUF>X}p$P-UBIebo-{(Fy1Fuzn%ZoEBLV~nEZnRa=S4)i*%^OO~jWaWAG98UY9PT^S$ z0G?%^=Ww#NzL&<1WMp}l#)|s{PbOg$!Xz`0WJOLAQCY7kqLP-&g-jx=`(4ULzY~oi|I{qLh!_ zYKd4(Bh>;y^no_cl-B5dTyVn0IZpzTxi&;t!ZoO=&NY zAXof3`V(6hznMQ%W%SX?XlT{-Gga3qhv!P-V;9+nlhs&+h{t~BMw_pjJti) z;RP2p!iK^=hF5rW_LqXEh)8sfGx)p@Ss? z_RcZ|Tr8vB_q#&kf|qVVnNXZ+u+7!7&1%+1iehfrFR+ zR`e~VzWrV* zjNUV|qxT>7$f1q?5qFU;Ccx;GJT@BEm!LhzVC@A`cHuB_*vK;FThNxU6@Cyug zBh3gz`c~{0-6MC?0<_@vX#i+91>d;cLZ+C}`i5Y@>nY_5>5K9Rb=(0rNR0df{xTUz zrMBMYnL+@eN%zoapjjFG7ex74$rrpXWno`;z=HH0molP1?ZJn&&2pvwY)>`Z&A{Dj z8$DG~tyX)84iBg8^_Bf&b(94a8b7O?bn28l>b{t&-)!-l0ObPB8~Ahm)cX2p8& z5u#gm64K{N8O6rjRVYaTDr%RlE_&%F8u+OTG=TYUSJ})RecDZiGdQBz+~oWlGgVVs z!mGA&!|rTUI)#!iL=O0N^I>_AZjn9N*Rf1FW#Pr8V=fg}-YR;sV@3NalmTCh3$?so zU|MtxBwq?A$6B8Sx~URd)FBHxefzTI$~9yflmrY9mC0>+;WA9W_x3gY%F3fCYYuPY zx0Ay$66=^~Zd|$7Rx^a%ydB(vdean?Y2gt}&mw!@=yfYMTQV(hHuo>i+*m~+4-YXG zZw2RIrwR(C4wMINNM|qH*lZtpw>>&9J1>RG{60910X71)jR$di7mhL>VdkQ!L}x6K z<8g&W=dp^)i}|&&keP)+T}Kmlg`I{1jbwDWYj9YMKS_P>KG<4QA7ZGL z7CkYi=(X~;S45P97V|JAao0`Yqp%1;4u!k+^z5HMi3wSlC031`=(`Co|K?#2f97|o zsXlGX!1k+Fs8fAzqqGyt(*`z}L^t^9$IiW@w zvlBhv=XqnXw*Sho^Ug>ix9U_kKEy)#93PfoF!{|Rj_ab?K^U2sAQB~fx6?GfFw9Q9 zP5fGy{8{WJ74B+5Yx=$|0zzH0xLs3}4sGN`>P>`xDYi{y!Be9OFYA4+{;Rj}ojInm z`_q+PGUri-=1~^Nl>ePK0aYGk6-@kQ5fRm-(?rH?;)WtDi1_jKZ%jsa(0pVIU=V~8 z@!L1S|Ceo}A^<43?G>FJJOPqM_NKP~`vk6G^ber%gZ3-3hF=OkWnR8_L7rRa`+AyC zK4`Tipw+H{n?b~NI(if>rO^aAZ zTU9yc^J!WdgFnm7+i@N|j<}c`B}@Cp9F_Jcue1zv<3CrW5L&bPgs}jnaQ_!(sfkKS z;8{tHOAH`eN=7S(jr`L!UQ5b)s}!vYyyv_@ zm{08|Fgll60iUI9?yOc)&uSjfxsXg4RPp0k|Z*OlL@Z>_5OEM-0W(h4~wtmvrYzjGyP{cUH-Y<9q0 zX{1|(iSbvd%FV{uZ#Ct#_yw3cF~DF6c_*V+&LjV2rv+=B6lZ6^GWnOBoi8BUcOms8 z#zv565#MMD&~@j85kPdB2W092k{Zu;l0NMY{&nGR%;yMpgVe{lt+*B_JkXMJCe@z4 zCcK!S%Sa{}BM#Lz$I8VJ%%3a010HL;+c{9qjTQ%G?>YvH+=+Jua{DbS!;h-E!sLxD z^9_}8&Qf<*3M|bes(~Me2y%%0eZx%@ruN$?w>E=Dy#|1~$~}O`MP*v!#Gw6Ff^@m}3A~y%4lIb<>LxK3pQrl=whkHL) zzYA6fNlqAtp4X1lJ;B;nOZ~-O4j$v&lpN@$P{Q01;D!zm)A;Uy14W_O^tbO;pak^Y z0-Ei>;(L8V7l{;q2TO*N_x9N7G8w}*FS@$aThS6IT%835}Fq{LUh?$-?IqW z%@&(hFE|%ckS+sv1+WX!^F2-q5`_+*ZT2Jep90!bVSvdJEQ{o7P;y-^HvD%Ir0KVQ zOQx1uoF`8MpWXb$WHmJZfqkK>%Q`~H&>RaEb2BbMZs~MdHPr{^4Y6Rl3CtQ-qPy72 zU!O3Q$>Rqfp1Tc0c?wVXMEvVSD$(F2wtIWyP&MH*gHeq(VMEn)p(DB1>kNgq&+~5n z4$41dhG{@&Dg?04me1X{ak6X@g%QGMH!1XjsI1t7(`p;3R*Ep$<4Vj4xo@9PH(IHR z%%mMlTw6oFF1ykmjlx_UmPquVxO&8M1Hxfuu?ZW;i{1c0{1HaQS)4#%9VB%r0UpEY z*B?2y)axQiE%KGK6?EU>+ma8=?^nTmnRVd5&$rUlq8+ma8oHdjSQ%i5YPy?ss<>o6wHM}HrC*C(ib zuJ8`TGTf9bp4DeZv(~O&3yRwE+|iuAAkRuRlZSkgvx2xLr--NX}R3BiI~Po zAQ~Xhe!M1Oy#3<4^Eq-MyGUrPVN!W0l%5Z)eF4f(U;o_KeT%mUk?t>KN63HSGQl3V zP4IH+@={Mw0&VPsn0<`_T-?>l-U$i=w+TYsr3qiK1N z;!n4@$2m!K8055cIlWikdoOjlyJFtc2T0(L1K#uXe3OoaXN&b+*f?IrxQrgVVuH?a zGKu!LnkF%Wrsj4rM6-Pa-L$n2ium#5@+WnOc+*v`~oeDaN7)WMG|cg^QQMI7hJ z7wiUE5W&{d$FN)k5{GXm28ut#*H5G*J9diO0}Xk;XKe2bDq`xIVP}oe@ue-4$?HS& ze0rH7u3CkCCr`bc{66uE{}TJRlTN&Fvv%Ob4(t2<4|~)*2Q1kJ?#dPQ7+)^tw*kptEVWZ53&O-8tz>){;m z>j@X^noH;A-8NrOTRj3QRQYR=H{|)VH$=VCzg$yl%^DdL@=%9I5W%;9G^;%P8j{D+F0#dF{E?KR zR}6(fWo^b>D3)3RfZcsv1L7}|WMxv!NWr7_u%yHsEl%#HFLc!uNtU^`jYsaQp1pH{|D za83>K0BNpb1O3iIs48Ce#nn4nikB{)6AFCYr=3AE2$eyMsCHzmya-7Kkw`e>S;Tn` ze22#8#NRhD4&$;}2-7HPn^y&sj3Iwu$bJ$XqbOlse+Kn|G-y0<>^G7;7r%hw;DmOm z3P%NhZ)mnZ%UvmAUDuaCW*Ljq#5^hm)M7k>namaVBS6hiAf9z5qz7A=7q}lu*#_qO zzyMpGIpV1jHYuIn9S|wlQz#Y>^vH;am~9Fjqm|`) zdtgDNJuJ{!mTiP4i0!W*7Vr#vA2EC!_53SswRDJMd|km#iR;Jm=J`#ui)Z=+8i!No zIy>}{d~MypQM{0z5V$yT7Os-MgGPgitj&4{VNZ{AZJV6cv`yu~2t?_btH*sNSEcjU z_UxGSKd`(drry#VS2#oYnA$A3%~wUcCdwHKptPTOr|`4%UQ1z1gx$PCzN$Ub)zMqzH}c&i zKa!|j2PV!7JV=D?gP6SW;c%h{OHd^AC_9V^%QHhn01m|sfMQ}YbdT?Ag*0U?Fyn0- z!|W$Aa9Nu!g}&}T1GUOK!Ga2h1#;t0MJWNv%QL3K&A{~iJ%HyTr`wV^ANwmSe84*` znk@~<&m`aoIZHtwbbF{6WeWd}zG)_NzN=NyQ#fXeh(cF(1hLyya$7c1Anz}|W=a38 z-+~I>rsL}@Iq$}52y_aL^3sh9Fe@1mczy0Ac#ZDzQR7S+GRSAEb0>~8gC+~%S{HUD zxU+0D`HT{6U6U4a4O=x8(vJ=a^ObUu`+k-d3h_qO+ZBvg{rt)w?^ZKNNt3LCR519c z@H4|7!iSof9|is0rYJwj9m^=V0j7)I3BcpaWY*IP0!i z)Ig+f!=SH`4n&dj#3WDA>TaEJRf7kb4+Z6R7cz6RjSVshtjL%>jz~C}jLNGcs%-AY z|I$rXxkNJ%1Szed57B%J{-?uJTo1Q^fH27qaK)%KG;7l#FcCTw!s{dV&p3+%=_=R- zX}_*zYvOsJiDWkq4`mokn^Ksk7t{;0xU<-te*Ze#Do7HH4`Bw&@JbPPjJ7~m=tfiIp@nA0*5p~bWUQxdyC`C1-O`;c+O&nhaJPWhzOwY|VC znXs+YwAMO~4c0dy`cc*@BVuOL`?~Lx3F`)^{uEJH^R^SlNmj5Gev6c(io9S_D45C8 z%^i>utGZr8v@b!S-^&V(MyE`Y93jndM4LFwsSg5fEXS{5#tWEzoTjM(iurs@SG-gO zZc=q6jHK zU{I#H8@8y)GA%4dx59?()W*Qr4jeO%B8+ySMevZ1t%Tla3u?)WKC-fc7g16q35BIs z91`zYc_pO95@TN=-<_|lRI6tF0NrDl-e4)C{&*q>TkVBRBoLuii z`X*kb!f=M{;;?>EqQ|Y?sFNHk(h$%km}UWm9VX=L1439ngqx@TGmU~|sI*{rFy6Bd z{g%FDP6k+*ph`!IW(uZmf>aeztfL7h6gN_`M)onlhh;iN4mz857NnUcQhlf>qCwt{ zFsjllx-#qjqY&&2`rGNsx1h+1h*LS0%_R2|Q7D}Co=3Y@QBj&46wIO{X-DP`(0H1C+U z>0mbKrBK$5_Csq$-f&QMSsr--Up0FvBxU0UTT4?Vhseost7~v`9=X$Dej)`KRf<7D z`BnW$$R?A7O*Wbs_QYsN&a>dkJ>~=~eIwHM5@R=YXYG;gN5;ad7C@Gcg z{BNoR@J6%|q5k@;XvEpg3~!|O$~Y!9sbeB0nKUWLmm#|pJ*dB{GI z!t**mC$6_2nDCPba4tpPe4w|f;y{lUG>o~w!8=BSLw8=_S4}K=_XuCq24Pz@cpCrK zamTxYS04VO;)u9@`swj;mVeK$zpwVes3(Y?ZR}Wadm)vOPF(LRiXxxFXg`5D@s5@AnmlHB z$u7TiP(ga`%=g=`R}HJuoy{zyA?ckMzD$~X-dJ0Wr1Mh9xLS?ZVaY_!Up=G85p)<6 zU$sU&X{?T4sAJJ#2Qrx(ZJ?zRuzo^dZUpHe+KyY>;^yU*U-XKhQ9%#Lzu zm<3{nOIVsyU#@vOrQI0wH#oG$rE9FJH~8zuZ#KH4x9~o`@b5GEHZ{3_1o?a4XShgf zvG1AO_i-Yan`x!QfQ84#iGWSznM~WMxwjB@rGA!eqD01J(de^C9BVamWAm%wwaO=R zZe7c8@+{tJeYnaiSCnlYX50-fBSj1IxSP~CAe8#99>dcXKfqJaQmtDr@ zi;&CeMa}+NQKm1_P%K+@NI4qHrTA3Okmj9rdHrUJb9iDk^_+0zK{r89UPuScUUBEiV9$+2ge>KHZ^>hHl z$=Lz0N-qCt3~f3A0@HoN>P{z6ZICZa@6qbo)TIWd(&!pG*Y>H|BY;z&B2z)NgEUb{DSE>%@YQq6;BYL&T%+$#4^-_c${M^emVdM(F5CfEje;_``{AO{05sYggPT z(==u1z#~fNWP{P1m`a0^nHC@NTf%Bq9XLKTbZz`d!jXpO68~*n$a^N^I0|43IVrD` zFc!dE0Cq7nqRzc&uT$#(wF(&o9CfYb@Lru0?vJIBWm2u|29$2F+3^~(nB91xG^SO!|#1zd926IXP&2qwx zdWZ3nzArAd1^?YPep~{qVgUVX6$2Ed_Dt;WDu!Rtzf2+BXe9GO&b@ZHr(#M^lg?IO z=b*X~DK3(nvjh`>DP$Mp0U-x3h-Q9-B_K!Ye-4fl`@2cZkSDCYwT9S9NVn16hO@-b zm}@5amlHVTa2#6(X-uNY>aAiexz8qCN9|}TG)7`6D}eZz9fbKriA#?CWdo$i|92PT zPn2P}Z)42&Lm2TV{X$537Si3`)DzL>k3@!>gK!+C<#ACT z0g;rYgNFpL?b?VJ8h8-&Qgv71<@-DvTHYokY+C^o9@b4O+j?HrcL<$4K@(jH(Pv|! zHdNf%d6zb7#s~N^QlLQdwN^3J8|>C7^3`6aq14MPj%$D=MEj2=WCO5-+8{_Ib$hRH zoT}34nGs#=rDMkH?Z7Ava9MHKkRv1nJ#fU1ON{bo0jFa*u5Bz|QB{Gh?}!Frq1t6M z$Kww3mwC%=llGgtuCZuWZ4*;XylRH+H{A#5W$1FxT&q~JYoi_m5eX0F-N_q9QYLtg zkMUbJDsYAsu+^8KF~n1?u>{f1j{%wrnGhuUlOiK{b>(4I#8=tu6ykJ?DSGTUSCOte zx>TNNSJER58zc?ay=MMDV7aSk_wxm_UY~}**E0e#xA}$>2KMMDix2)>y75jwp&Sa9 zCP6w*EguT`gf$aQ0^yX(dc9vAc$!Q`JE#hV@0>IoViQlz8q|lqt+Qp+b1{<^z7N|11}F zepd*vi!9w$9#FD(%;3TIQ#!@Z!J9eSbe0dN@eaWf5D-(h#cyrse#M0A2=`q4-g|t{ z$gX&g&(2P})F!J^TcC1)PoaOv|4!vWcPrtN@JG_TAQt!o){lcR<-_ZC&zWq5u-l8D zR&u4X?jDz6QCFRlWt-ZUDsFz`+#5v3G*!EzsGo)z$Y9jCzXyhWSL58XL^P5g*xXnd z^87k?;viha(*sxS`h&VVt=137`Ob!YOQyTvt`f4^D0P)nuFfB9B{I*ATx?&k1xcJf;2|5=Xjv_CQ)1mV&_%g4&x_Tp-N*EfO80gI6fe^hX~FijKu%8pW~s#^@tql zSVAuMmEhFFAUz_*rb6O5k7l?^2W*RiDJEWY<@*qzGr?e#@+$VnuKwbjw)a&OwkGIZbOlS#;^_~(W^$QYa+@0JXCnk~t3$q$G|*5q zyd7;|x!3I#`xzm8JBe$$$eux3e+qFkR2VIbUVHyb|K9i_L>z z0WvV&!V(Q%(|9vftYWZMRD23T#m#sd?nC+z+Ro$V=tMYA<#+9mBKlPzdXYHKe%t?I z86flaQVVM8sofCPdV2!4XPE!+@ck1d#fIO6z(L8Kg`-xdmkOwf%#^x?#w+TkK>lZTw z#jp3?T3bo@4Y$%b;x2_5+wVJ1+@=9h9lRK>%x_U$%x%MDxH>KhqETO-tda&7I~nL{ z1=j^}r7d;3(j41*lV?R5L!Ge;IDid0cdiSxIbg<|!oSineu~`bVpj41j#(n2uu&O7 ztlmhzN^J(%ANlXjKJ89Dj1KGx%g4{N+TSdIADLKl@Cn`~)zB!9n%?)U6X-$Y;!WIH z10Eb(&El7BbUSPuFKb&b)|%tiFbEz^xu9F7T3#HSeGeB$n+6Un@82O0vn4f@yG$J2 z^XhhPzXeCaDANU}%v=wnlNtXJ=p)0s56Q4m&lnkm#4(8qipk(p9@8#{Cu>Qtzn_X- z5(scG7C+>R<3{!IVZ67V9;PGeBmVsk&3quuITC*bk8_MMThiK!RbwFqa$Q)ZE4w0u6unj{qP62Zn5EpC>geI~IP%dIE zga?5dP+oK`g>(^BfnCHvM7afXcPK6>*Vm+Bej{DtB%*#Z7}^H7;%e|gQ?~g;niIN~ zFftYezF|6pC#+zG8x+NdGaO|$Ymre!RZ2+z zBpDAMxb%2j?=^*?d9Pj~8~EX__C~5?RaVs_wNgsgdSrGrv2n%_%RQ)xIgewn1R`zA zO2fsS___+HrTY;oey$%qeLP}lyjhb>cA@=Dv880#WKG{*f)p9gL%d#FeSW4D3kL$g zZ2H$72|p@Z25%?RQ4j_03sq)agu(m-ng)njdr`NEd_)6M6-D3ZEhsa$76^fy>-sFMuz&MK7O2pD_uYL$sagJs_fCScRN1R)V4xX5$ z-ZDT2Zi&dR&k<)YRLxVt*>#KB+5pfaI>NzIFvP)PUw8Xwj_|paaEwBP`yQUSrnv*T zA}gIIgJiZtgs|&A#_5NY2~Ja^s>_4y+j0(UJ$>8t7uKF}uI7}36lPC%+^L4fM&k*^ z|A7AF`YlWh0`m*z41^(Wi4PN&8E-$r*D|(@txN^w#egdR0ZK-*{UBmY40;(-pTc#a z5TnQv+>c=mN){pecuQE6ASgQF@QyyvL=2npd=O2n(DAx&hIs}S^p(QqxBA9 zm|YX4w4qZY+7$$^fPrs$%iEJ6l-6_BxGZBG#}RVnrZGsbI^2f>7on81JgU=zSNYZd z3z+f7ued%)Zg{8lW&cf1ny7SI4{dfl4d;S;-EWRi{QC+E|8`)nuUTxaMeYcCRa3uP ztispqM9+HQ*Kn>0UsX1rzUv0M!j8f!qyLJ3pDX*)WG6I&YbE>57jzaowjJj38>nnu z`~*Gz%!*g*8o|ZVNy+!_HP6;4;n@3xP#l>k8kzN+NER?PYm8j7+dInv&ta5~2FM8y zQ%n(CLL~;q1)7qCw<`7sCuuq6qz$AiN&Ty+r+}&6Fx#DAeimTXm?Xbugz_`XmQxU= z#-BAYIP=K#WMzCrz_244V|bD(NmSM4x@5$Lwo1zdlHp_A%uA&Y$c&BtvW&WrSks8* z583SBKPx{sC{DCet0E3d2M<$ZCrhh4=5b?7_`1VOm1`GNr$(KSD!$O7OMPWg>_cD_ z^&dYogoh4M{BK{`x32ztBk)}JwmYDgqCJ$|q&Bh;9!#MFbp_%2`B4J|as(Mw`3GHy zBHRcF=tD^-k6T8k5jM#!h@;@o2~yuAn#Z@467_9xiQva#3*cY6(VOO@^q@-lX=X$U|Edo#uDAY{3qYgvwnLq_DFLVp&awY7}FGf z*ZyG}9vM!QPqHLwG;>EqcgoieC1ui+RRy@GIYU({SyA~e*Q9YIONmt0>Lh%yHj9pge4OBw}_@Q45d zYhyW7?i?vP=-C)RfNz(xU^eu;{+8#sPL2$=>BUM--L@+;fZ*?|9A;F(XecL77>Y$r zhN5Od(Ln`VdXw+Lav&L+9fGNNbDe{>Z+N*(TR}?smi`CY-T@el@J7gWn+e8u7Jew= z2c7|5UGI@+#0PanoJ!PLlR273UISa)U858-WkrT()F21hRw`(l%bi+=LuDo$gubgW z#3S(xf+j6_3T|#G1{jaY5JoQeYZ*5>Y(zF#~Xt0l{UKPv*FJ;m@Z>`RMc>Q>D; zQ~QNzZ&klX`pBz`KZ)-TqjOw_UQT$|NbFe7=VDvbU03SD);(oun%7mj{mM4BG5Ol+ z>WJyMICJ}z<6o#_rsF5PKBU=P*>yMFgHzi`-*aknPmy1Ib}?Dug$wB#sq-8y8fOCq6do+h$DlV=&sM4745pLdOO0e)c(66% z`Y$E0NuHdo8Sjb?)H=_&Et>K<0>`-QB6Xs^Y1PiRNOCCHH+G5H1Ra!P0~T77xy&Uk zD~Oiiu!647Ooj5ZXW4E4>Ep<|YXZ0Q$6)2^l^dPT4c%+iT`uaPS=U-E6_u9_?dUX0 zLhy030<`WwE-(M1jrm^CGBO4V1T+UA{{iN@Kn|X4*7QsaTnx;N|N7I~$jyk+#mw1- z(Sey!*iOON8W1@0&+Q|C-hIe{#6SAeI3c-5F=EV!C-~ok%t^ z8sYl2KRejkT3ubALV=I{W!$XxJKlIoZG?Uo79fN~jiYVyg8aN0>a6X5Jf4{guDgJa z%}(;viZ?hTCMnHiwnf+2_L{s;RqHx2)pm(sfPYo4^!$GCbJ)P4l7knm6T&^coKu^) zzG$KD;^bPZy#)1GzQVRS@xR=IDs5>IIISd$0h*BSS8}a!lV1izPe<1}0?tRn4PYFz zMtZI9!BW{<%8GU{TP8kP?%vLD&H2{`c}pi&GI$>lmfKM(P%Jyz7}30*hc(M-(G9;n zgRK!^8R@pNJ#zZgn=Gevs%=jT-u3xm`gPw6r9K91eTBmGBWM4S1(yhIsR>iZKGCCK zqU#)XuE)A4pwB&bS%0t?{1kqjfCaWNLLn`)L07tMhU8NK@gg0m*IA-AHEM5x^Ad$eGHWkI6C^`&-fQ7kj@F-8c6> z0b3^uFjMZEM~eN3aQHi>)SO5l4;*swsYP-D!O#%49*21Ec!(saFEgeR${l;OeFEqc_|up&v@trx z`0fNbxIZ%QvmrF%2{iAcXb1DG7Mb*_CwgJ@7z?zJjpmS5zW{Rz+Hn_<(Pv9iY0#&- zA;AbS!sJl2(^4k zrj;l2RG^^F9xT+Mr2LV>ct{v4@e3D#Nfc$t-l;GPxOUugTN* zeV$!q`|WQo@HyFG`px#be1mHa?PuozU{eP!qtJ?A#HL&MMgo`jgYOd|9Jf60X2+X4 zTR1VHj@$&2u-%_aBga%x&tih&?$3``tUR5~9}5XNs1@$&wYX@F1IJ5$CXW>U-HXQl zdFC$((1S_9wj%BSs~)iYryy){A@u+fEWNB@Y!@Ed`R7q=Q4_=&rH3Dt(oREa$Z$|n zZ2)@E%i0|5@!q0Yz%#}cet+DQSx3vzD_ zh1lA%&1$;bb4kky@?9sQToUWTGF);~mlOt=db6Dw&Q1{g#QVn^MA>}MZ=;7r#m)96 zcVBkC!G6T;jJ{-8#J%vB-{nG7x%ONA%jiaAitre5Zl}6Xt$~MognGfzH;UlhT0>Sk ze~#X~fB5cl$>5kMTMS-X=>PN`wgL0^C{%rin3XpXVlNb_1k=9!Bn{Q~IrDXBkz+cR z0I@-iVPcSdn%3~CH{MpSZ5n(;^fc^&71Z}Am@pTy6D$;Q7y>~21Dgg$R#hzCkpP15 zt+QXi0}f$MtskM*dQ@Z3q}cd4G(f`$g4B0es|dwPVGSS%6eIO*e+41<5?63f)ex

    m>LT?M_FsjB*LserqSCs;K1x) z>e6%~{wDkjLo>cVjGAp`#sb4`3F5IgU84C`YGMVDL>>BPQzuriXlqFAOrcvpPu~I3~n&AiA z^U}2w0_PtG97-o*ksI4Z{-p(dOuW(NlO)y>Z6zDX44KH@iG6Ovj)d9zIlr)n#}Hy6 z%8u{I)Wrp-Lp7;hSTJSvOLB^m0$N4xt5lvt%2*FneJ1Ojr0+~t|B(Xe3;GD~`U`oa z*F__d!OXb>KEd{?Nf%eQ{Jdxb@9n8LwNG$Cc!%y!u5A2r-{@KL-tQ70iVseI)>rmH z*9320ZS>vu>eer=*Z)U-CYNxx-vfM)nSlEI|NK3&{>S(DU+}>EB$0;PR;^8RPzu>* zIUx-GM459*ts+#nMoYz{V4KrFy}TR3-B*|pONcPVhrZ9gySZbxzpvkM2fjyKe2r(m zha)gd$a=7LX$;79747 zHlX4v!n@W?@Xf96rBC)8Wz<`!HZ_|qzOXaGeTypc&QXZ&xG_MBSe_7m&~Ql*l5Nuu2<6BBZUzNpJ2 zGOst)YVPwNq*iFxeotbpRi@z_^dhbv_;yRBFaW3S*^07IoY?kYkPVEJ7dJdrL>B{N8B5|2N{Z_$#eGeErWa zHoUCuolb;av~Xk73#m8Jp$~N%>)3hUpB1-N!^uB}zn36qGj?9=fKcqi2U{q?YVEQ$ zWApRNhVF(B_G4XF0#_D~dT{kA{0Rt0Yjz1rtsfR`4%;};T78?+xjZEs^Mn)ZU!Ix+@oJ+ZJzaQ?6`Z}Eg@g6+Q-?!PbjITe*3(pzNW z?$P{ne`_~+dP&Y3m8?sVo-~^n&$?m5A72Yg$XmCc^+K)*RtT?v+AVs1xBcuZaf4vo z2_v2f}I-4&y-{1;DZuDNoEuvxg&G+X#MIIf_%`zHj(jX_M zdREa@4}0r1_@{;n$2~%sc*bBZ+wb{VUj#IptMIl?Mk*E`@sj%2KaGz2>ea8cCg_6K3r+$CK7076?o-AV>90t7ReJK69Zve0P z|E3gp=qs%o?q$#YW*~!d>6(zy{lJ|I*@e_P&(uCYc`P@ec1TWwE){&vLze*uo*rfAE44wZ8sy(;jhp2;rXbr2P z{X;l|<5fsn9xOOb{y8rN_nrs!eX5iPNHRMF_t2{o{jg1~QC>N8I*ex8o=sV41{%+3 zBDm&dM_4`;(;U4@k^V(n58QX$lg1#eQ&^8TTi{ z9tC&?)_If(Pitj;)#xhUP&On3rNMAWzY-Hp$b}p=Q0UbaW9f;dS%F!Pn+St`r52M> ztRfEpHkxPA{wVM9#E1@H*o6J7d}|i9C-?|Y-Ba)XFfMAI3q59M<%eY>-p4}+g~VlgQZ9|k_=@l0ka%I_StA>mew#+)yjTih9>6E!NIu#yt3208r59gY=~ zOi&fR13!#%8ZUGsC<8=RJ!p&&0~O~f1ZDKAUrTN1zWCjRq4(OEdn0w3@x>}&`{mxU zdDC8-FWqC63sApbk1bSQahZM|yjB}n)E1t(y{PrSj|y*X?4)F`^w}5Buao;F%_xHP zFi2W>3=rNjfJkDhf$MbB>h!@s?+t*ruz=#fh>+c&ZuxZgemz+Eyx+R{I6Hf@f8B^i z<@gbP{*RA5$;zA-AFwIH57-nT|NncPiyHmYs1K|E11ZA$6P5jQ>5PriEy>na$*@su z6wGuSwY22*@AoZ_0p3H#9`^OTt<-o};<+9L0*e&Bzs>)=HF!8LY43lbfVdM>8YP#h zzq`Z~foSC8^D)ymJ3kg6t-8X=lKdNfZN6oyxEX|XQpj6FaY81ojRl(o1{iQERMq({ zT`sp0uZSR=hfSVk8B0G*@{lSz*FnReQsg{NUJKYuG<{X~hd z9it*Km+`$pg^-3((SbSry>`_%LoAn&lY%+T0iH^;ObIPl#&!WJ{0@yiUn5}cJ4^b= zsIu)nPn5YZtU+aEN+No276sKvy+Cy^gDPzi9s8P6IVR>-wCcGPb)Oe`Foz2mR@do3W=*@jHU zJ{Q|9SI~Tgfxf}mUTUUB1qV~sZCb;83;|RSKEfBDeFwm2-g^d-0}a~#ox5;)+qqlN z2&fh_VDN-bsI8BciZRd^7kUTnc&g3(rH*?a0kPI)f*Mg*hI815g6^B(p89Eq0G){c zK6I?a_t!qa75-AmqMB>Iv@y*#s%tN(ziboLVq_xH0lLKA4GZAj<@(mNg(USf+bBu1+r0{?PUPlC=mgW zMtyr*_ck?MhF{=q881L_l+8|sh7c6c-(>y+Qv?l83EXmS*XJXs3%2Eqs1z!j0>#e6 zys={JU0aL2eD^^+K+gQ2h2_9)(jFgv<@O7_SK0{*Z4Ty%pPBX?`vM)7f!FvPGX|7S@JNsi+nh>a63}|DLs{ zQpCS^t$$r`FMW@LWD7@V^s(>PRm6=x2qS3*0Wyj>!)9FvR0q@fB%Z?lsNfs*BFZKn zh^XR&Ce*1!lZ|M)MPu#M^Ea<-r z{i&38w;BQMWW)O7gyZ5N$@OHyGGB^6d zeKCJ)m`Y97;f0Q56g+u9d+L>-)Y9Z|@X!J)2O;NAc>f_YGtavl zRBL=#p9M+rF==zS!thh6hl|IH95c47Lxg04f%pHs)2t?ICq)EM;c#L8AG$3E7JwmY zWoz`$CP$6>f7+pYDvgS{hy~#sr^0EL*#UGNB&pQ^RQsq6`Fz4Dm~g|N9@dnF_H`bK zK=y=?Ow;7FTj1 z88@!_ny>1(HAO1qD%v{7pa+Ay)#^2DYiiWUFLds!Wp&r|>~7OVaw=IEJ3WI2*hzIbll&OnC+Y+SQ6gwL7%2#4NA}W^68Db5EH_=PuQR0r;aViSp8Q%Qxz?PEu{Be((rW3c}A601?~ju1^$W zz$I1~Lby@8+^098UaBae6y$?zXj`+)Cu%!0#>!w~%{k~N zQ2n8L2LKYI0b)62LPSY@i7|zb?#W>($AI5KybT+`8g7bD1%D+4`A6pR)JME8Mm&Bh z+Ks7TNG&|}!T`E0Y)>L}y1iyY18kYQ#g_2;D{X1xe!p2JK)3ZK>ksI*X4irx(M+jz zQ0`47xJa^L#?2_Or;^EBdSTN9ZUQ2`BT)8pCc*COBOfaMc}6|5q2|E3)Q%>VTD2ze zMb^!QN;X6B-!$;Jm{T?^#@LacWjTz)xnmIUG5obeM}zq?-~L*npq@GZvP8)+j5*?8 za)yD6Ek%Ulb?d7DnDzp^9Zvs;24{)8?F!;HKTo!)JYJnW)|){{`J;DJfDeM?lR>zJ zu2qkQ^iI-u;4V0O+=IFRrg7N%gJ)rzPJLbUu0oZD@Q{Z{N;tw&zcI>h|j9 zPeJmxExOHa1R8}NfA(Ka^Itx{f?Q*Dw=+3~7v8>5v8iq;j?YTI^S}H;A2863Se>c# z3R)co<7IEPW*GGycsciTxxMUaEHrYj{dh9@Y?>$UwV9d}M9M4vp+Ks{@#zzSlLC)& zkcXJ@T|K!yXngCo9)>+z3qgF>Vyp>Q4lm>a`7>hyDrE{p429$LUktxw6J-R zX#5yaZ4dX+8Sb!Az8Eez&p$57WW@^3$8QFBPf*(aaaY10^xvDa)@~lnK!C1P16V!m z|MzzrVE)1=;%aVg=JX%Asj7hC%Rg=`!GF&?$_3Q>TX~I&i7WI4Xun{ zCpS-bcao4l>uT{bENZsLWmdd3O%;^QN`dPt3!XXVWJotHwAuj@(?BO?V>xOD-^Fw`)_F-sY}?!|DVdtySn^~(Qmo92b*VVEi&z9k?Ux)(9MsvOub zAabHdpL(fA!35XYCt%6es%W9!YNno9yysi|HkJ$$7Yg0{qdHBjw0zr05ZY1!Kxbb7 zC=9$ZdgpM35N*)vh78tuOHVU6#Ykch{YDr}?9K+}`6(BvAx`FEh@q}@@#2Eb4E?}H!~?!(fV@P1jlj|T>WPQDKovmG}oV^#%ueOI>&nPHt5l_p*}Zh!F&o?5$Tq#`I{F=o%2(%Nc_ zwfHUneiD{1BVM}>>(*#lxQQpplT;iS_``4orMWo&uyg}bqgtN^-2=83DM;rbG%sQ2 zDC*t5=p8R2Yq_c)z}MlRQwUo>1Nb`7I|pf~ul<>G4iqQ?Gl*9d6+a1<%byQ6{TRyR z(*DG0fU><`?;0qSn{A(B7%1^*I{A#|Z$0PI)cnTorm4<;M0}DAx0rlP&uP=r!Y7V5 zCywR6QZd%>8#^yc8p(Cvtg+Eh9_MK46mKi@u13P26*MtqJG^~Lx`c-O%8{r1IcLp- z&7bcY35k9MH`9o+sLB(KCa8*R*+@j?oK%rs=@~0ReVk|8JAyznUg) zVregGV`XOd|B2au&wgfVMm?HaIt?VzxC3FUxEqX90t1$-rK2FzAWv5hB>|{emy5VR zm3ZI_krMCU0W$B_<2oSQhBJ_JZ{o6-{+bh}>=kY&Ki|Lk=6;(O_E@WmqX^}Zp!-6f zO=~?+MXjKT*HWBv15#-Xpk|3w6Y?#;^y*~Rg;vn!+(9NFc-GrG=4jo`l3TnrCtI)R z)>5rO8<%$*HA0jpi*(aHC9p6|YXWL4e=Acf^|huVngt?SCsThQMX0DbX>dFzOqNkq z{8$ukQT&O}y(3_<5~wSvJYg2t;ajd4JCQ&WXzqR&C=%bxrkvVQ&!p}KrU__CV{|x% zSq0XZ>lzy|YJ5{5pKxDq$3WFE_?_N?|AU(g2Lqe=++vyHmf2T-3im8*Tp!A#QbgZW z$%|FXDWeewh8IGPW!0#Sgf7=L`uVSD8(!h$fM&9YvIP3S{&aVmOu&3qa|<{JK+f1l zVg}@Em*d=AGLTPONy?%7JCpwPV44CXI7FyBjhjRaiaQ;O!QFVGhpuj!5KP%KkM+T8 zv>_YlQ#fG15kwn1fFNoFlO5c%K|Ag*MK;`}>`@(qwe<Iy!3)DJyV$`L`K-n^nKFwsNPpUWCD(h}?Tp+|24!fmEHW|MWco%)jlHt6f~ zL1`&v+Oyx}!K&D_LmMPWeF@_AcK0==ni2tXWVh)cumjoAXMs zRI7)`&ZrBn9!~S(re``xbs3XqfxgC69C_kV^GcwD&9J;So-@Q(x6E$AUyQ1AWlhYDl7|YPLV{Y2u<}o=rpARX zy%OrFWHg`=gRwguf-wpq5!AgmE|})B=4+Tb$#q#ouS3B()8SSf7lVcMQMgSdV<0-m zxlp@`^2epnK-D5Qo$2}*usnj*y4H)MZ z8^s;@UopyS+EcK@Eiw9GB(xYkg4|u%W1?j7i;1<+``BxNbLoQM zlP#D=;h}jfXfCeaiW)nds$E0JyU(~c!g1g>qGyHTvb_3Ld}RWv1>BMrp(BrKs}5sR zx6)?E2O6m?36c z977Lk>B~qCBW_#$0<40l(}}*f#6*L9n-2k!9#0RC4D!h4?yMjm%jBW`8@xLFGQe@F%Ur{d;car1v`1pi6s0dh7)_*VmqyY%~O!Tv7xLw5D(i35SIVbxTw_=q2Hp~?mTPc;?&P9AfWk~JQczpY?6_1ILyp^Tm6=0Cs_ z8UUV~&8>AT9Dl)c<)rBfw}oL6-PtO)jU!0$Jbxnz0G{y1RRP(S=X01UJ=Ke@CjKDx zldwKifL_pBkG(nWyqMBL!3<+@?1SmvY{^2NIu^Q&#k|%IA4HhtNx)fZGJd{cm z$fVOi*qD6xLjxp(IgAv$4MC6`si-cM|Lh(E*~dEtpl3a;$H9}p-()Z^1~g0-TIa3h zawb&(XD?P$3y&R5w~(%2ldiHUWUocpen;sqVu(y9qe_!+Xz{Er2fT)2)S|EeB_S1P z3B75k25=p>vO$G?#ip_G|C_NH*hJJXYd68C;(UR0;zC^l-}{rSQC6yR7FK8O=l6yK zEDXZEqK*e-a86x{TQCImZ;>>t%s2awDt&5Z$Wwca`r+RgpzYZcp!5ovhr&_hcE(Tu z$zT{dK%ULAX^4mvJ=4oX0Mqd}xA?!N<0JyoAox}Ge+rTF)rCiEo}J4C5!><0-g8)kc`ed_9m zihoVV`Tomvyh8%O=2e}=8~;eCCW|fLDz^_u&I8Aot4uK#*&jXXI=)q{w-;x&?6^mX)<(W_ z?Lo`1-6n#!687@>O*x~jDZxc9{&dQ`9hY5Q9k-y*$Q5(*!d3pZwQ@y-)W%aepSNv{ zXZ?9JW7XIhLo4f8i|{c)GM69$vDD)1M|F2u98ylY4g~GCWjFP0Eh)_hPqyq={;V@| z+vX{<9DCO7&-v0U_@PzYPjZw%2p&EP$4D>e=8B}X$*r$94nw|Qtk4zDFX+` zt3a=RPy~8%uUiJBQsD_uwD9 zZCQg9``-p&>-;^utp+9u%la5#5|`MV6A`^D#5&p>TFTLv99?uW#oOcIQ+Z$A%zisng|1L3LGiUfCs32GT^kur5({PDJY9Ji42QSo8zr zNH4KOvI@%2?(QHKDyCdmi*x4V1EDF)6(FW;%I7hPnkJ>|3%~_PP=I0ipskHWAvLuQ z*?p{-a@-1-7BPy^In~6a2ZFCyEWr+|Ip@&-tHH~emvOA}|J`?NEBdqdRZ3>k(F9L0 z`{&JCZG=Jx<2Hrs^U6iE0n~kz7G+>F5a%~EZH>yK9N&=7`iNcxLv=R{-6%^3MzU_u zrv>Byr51aUV_dz<^yZsAn$x?5NN-2ih%amPHG>PTieS7xM>P6CiF^ewCk99KCfD65 zx&uF--p`9SCeh<3LkCvv2z|s0=>R%j&nEk10a(|o;A#{l?a#4ZTH^6R)cm4zw+4G( zM|orsR=~ShR~_U^ghEf&+9jPGsVuLX@V3>Y_l{+EK31k+k9GXX4Tg% zq;QZYbCDb2YkUmEcye6b4YT(VeW~ps8w-jwM`asYH?qAszE5EtisY=7;UiY(5Z&T5 z#~qXWj{tjePrfy>Q#yROTF;mH8j1piu1yu~o|;h|!^31Yt0k);4fHxtoc9B+Te7Ab zq~91VeMnZp8?35fa9uci&apR~m8j-!u4hNg zP1q)HQJ373P&s$O25eFP)D zA^beh@oUz^F0L-gf+zmKxk9Y|1Fv@iCnM#8@SS<~G#!Nw{m!#rbYqT%A`Pv-C$VGq zFy~=(#O*@ns+?D(x-%L-x30u!cutrfe&H?V$pE4A`IPfYuO7!#Zl6)&RH=u(49~E> zAPv=aoNP<~asT&vu}Wt~18ioUbTte)n!@}GlORis8DHm#w!2c3c}U(f%)%wg@63Eo z1?V#rlZ!Cr9kK5~SQ$)pvksG=rj#T`-3y2;raOCGak%&PpmrGOTrqMXH0o=?Tr;9y z=}<+?;p&mQS@qLk69e@h4b=&!F13I#E3fW<+O`)(ZVt1vN8+!LPh7CY_RMz(MkdV` zoqo5*SRO^FLW;lMc9d_2ra^weO-^w78lP)<^d#6mH5qMbP;`ME_K&K zmQ)t-PSUxpKm?E5F=Qk69~QWXY!(fKDyD)g74yG#YY9W6s*Ica^)o{6tGa(sK{2zFKra_%laN|W{B$osw9C?|G#=%boP0VXAR97a+tN^Yw2-MoPy{Rs}#PkPQ$D{QpJ||Lc(P&m#K2h77g; zHDq*2)3cJ%al`5`>emgDg5r)Q9u)Q}q5@W0(qwbVr7eK;;VzyZ;fJI;%S7Ua^BwJg zY$GB$ey_*68lsMXa9uko_j+diA`&+z=ZC4G=j9q3Oo}3naBWnrn*^LZWDWn!;~8@` z`3T7*S7Il`U_asb(Xk{)1)MT8QtH4qWI{CMHhQJm3)(`-F*@3x8p`a}g&O|!nQ}Tf z!c@-WsmwBv{UL>@wdf{9(MVX8Qr;9S+G#3=H!_5>Ip@*J9C4P(KbXt03xw0Qxpf5u z3|0Mgx0ED*hP3@OVZ=@(Q~{Pje~S@)c2`&%QC^`;L<2eksDivxsXYe(1=O7wMuB_# z!2iqRwdL4f!IaJM9fKP5yR}aev@?<^cNI*YbfO8cNP$RYptu2)n4u_DV#gX;2bHqX zDvUmcdK7XpYCtJ#gB3s&WE0Lr3;GjyZjMm7Sn2{giP$YK+K*j#zK~sH7$6D~)xa?* zN;MThyYv7pZ9nx;5iX^iw2D)9k8l0 zA~$Ww2~&&@Lt^iZ;QGmVuxQKk-=O*O?nk8$k%^pF3TQdpjd^^2THq z54VvkeT@Gscl$A6bQ6eK{E;*Evf|%9A1w&!>vWRz26 zWN;M1lsi#k3t8{Z36l>Vy?5`%n%?%2t;e??NkNoGw_vFfIrsjG`*<=JNG8* zQ7fA0J~9>VZZFI^>xDbWn==e68`xuYhyVK~Ku+BmcAn6ZnIkqWTGlXi0|mMmZm~F%64~8yoTT%=p z3TOCx#tGUz)^h+@Zno?z)jdU2ArO$xU0C$m*=?A!+xf?w*t8RLcNw;$RcfbcQ|215 z0MK&#j%b$(IqOAunCoX?`>dDal1c8=;CL5&=sD>>B97LZn}_@JXqVasNF*_&BFsL9 zTC0*-W3+Zl8D;0ZYNy$@=!UnIbjg{wg|^6D!$}%hs#B#DA})|)J1JucB3m_=$(f5J zBIHR|S+Ip{@RhrO?y`!Ihr*rTex%`*;2o+fH*{BHaD@b<+Nc0Ysd486X(XV2;lo4t z&RBAuEL=4V&>*>zYf{C}MO#M8+c7e7BAqxP*2I|}DYqh8oli1+FECo$B;o;^qLo^7 z-;&H(wsYsGZacjGH2}WGutcT%oN2sH+AeLTx%hSHNWo+frXjP>LS)1?Xr>onF6fOMtbmPD=A~p@Hr(>DDW_1Gd4;*G} z2ub?eQy>Uqg`K#HEO^piKbqswVtyv6V&`Jm3#$70&}0;ko9fTf%AkNXXMi7z3-mZs ziWl%lPfSV{)ATqr_B?<9;KQi_#M4pjppP|pwVVo4EeEviW;KAO1P_o`Zzs`b9pnldss=`x$n-0ZlkbTcC% zogwewD&EdjH(r&_Pj8w(KJ?P+zU;E}a_pARd1~;?yKA0Lc+q%mUwe0GavTF8Rt)tN zf7|+Wg*{Q@Lqg=T%nt1oM&9bShUVWcU?JSvz5IOnxA|#KoyBYv0R01q|Ix1TzqXA3 zAM{^$To;G+PU!-kOk7Yn94AGn%9N(Gw6O^#S8+5Mj%964NmJDR+1gOya(4}uk|fI2 zMSkl>iVcJN4MN?I|K)Y~%h}(jSEZtH>Iep;vk+l8G_)K3=KjR|TY-9J@Wi1b+`b!q zBqvSps8j74nYo6fH_bdz-Z$?c=1Iwd`tvd4a=#XdJZdsyu%z<2@P>r*%CJXl+a%o- zke2ZY;H0MQa^|2qWeb-?GA099!X;!?MFtilV`!Qx+j>-j31apaA|#6GMIF@%A_cLG zcm<2-l(~S23Dhy;aNHdYRb>4b{h(PF zjl$RILu|xh}nqJm>J+$-CMxMl|0R`r|IMA$E#nfdfb&M&0@gJlzuulYT zd_^LA(7lyp10ArWL|!2yzbOn$1!(UL)BpsNrqX~tT^R}^Em)mI@=$e;5~vu}w0QRcI;czXgDrK06MGU#@>A zAa)f~T+IzY&=;!_31mX1fo`P*BXNFV7Ii_snv(;gb~q3>c!}6MAHAsc@XLsxiy;Q} zOqbw)2aHeLT|2bc-=1~s*pI%jwtl651Nd^@I?>C^fP&mf>Fa3;+e07II#axquRanYenEP73h{7* zH#*+n>L>LC#D0)Dni27oj07t6g_{;g*fXKPP6g;&yT_`#_3EH!`(cQtv}z21pNMiE z>$=A?q4$emWmSM zNZYnuNFf}05*8<@)&Q8A?x-|J1E!~VeTKYJXDtK;Q+W+P+dO!Q@pO5*ooDW&=9*Rd zSI&l2SrDvk{Br|Xx^_^#s-(?VP>}52Ac|%uvRN1Tt>3tl!3m{YhHhq zklU0kaIav_GN;6%L6rzd6|0HO$ezmKTM;XZ-Qh#Mvn6O(Q+D4H0BG=EEE27X9I-I$ zagH&^JX?e`!z*QncO>&TxuegC5-Me;HQDO+`BwSDWRGEXpk>i#GOM@0cBPz39`r79 zMXN7t87zgeFP2tTG#jm5!S8a;rVxVLqa78}l{e9T%CB>3DWYD#<(th)Jdf`$mYoMQ zEFy)ajM&P(%g@t3EfCO*vHaXPKrT5&x7s(8%M<(4q-~gIk_A7mJ*IN~vT5*#ikr?e zHvU;B4gvyE!I^n5ywYTWaDQc1(*oCByh`;+KH1bsb3K~6O{-;SR%G_#%N^~!g%SDr ztx?+0gKFTZ_fa-}vBoM~v%0W1HF?)3m}3I&_d+u5sn5KX=4t`7TY1`Z%$s-S+<6py z_vD$@WY?4ydPhEFyD+!`|aw{Hyp^Un6)6)2nPHcI>A z$X|fPLrk~m)_wT5%nC_~C)pG*)#P~GA0hI|41uge{gT*K;sI_-AJ5LQq%(%LtqhmMxhwh1Q4@zFSqV`g999J!`zor8s7o|oj(=tmcvDqQ|Qr*M?sI6Ed zkj9yvhS~vHVz_!_T6`>2ZGuSexU@2w+%L+*XbF3H8}(oU+|RGNcW&+BlwodrjJz;ZSg(+rPb=E6vDao8BP%L^bHjur9o_o=h6--L8>-0)aRw%WM4Nwvz ztzK*snFz`13N32Ob@E@(_)3YtvVUDsmGRvbDVkiz^EjC@ zb_(W!G><^4U&pL`;Ntn80?0mGrW-)>ae0z3o zwVx#bhooh2%(e%vHe*m_S?1{#Yl!H%CsA)Hu?@0}7szWluwF*~Fyj0P>>1OospPsV zj=_EK8^;~>3Ugnhwjp4oQ4NO1?~q#kE$>PEb#h}ySN*Dtg#`&@2<&*`mKgGQ$?(>= zEvc7cRJO4`*h&UN76oBGyVc+5d(v|Mh&VdZp&{_@+;1Ih&Tj6s1T<~wnW`-zgWkR8mLieg z-k#p9xDl0cWb`y^aE98cYDYh_BgYz0ZEurDA5!o0hTwKj#*+%_ju^_T(52IxISg4y6B{VBTO) zw%`aXkSqLfO+lklE1rkFKrMUih)7gBbfFTHjgNR^7F#l84n*0Avs+CO>>=pi{l?LS zlXZ5qTmH)L5qHrt;N5)eeUnSB#A?2`ArbGlw^+_NglWj=CLfwhbllvIAI&s`(~zfC zgdTURFC*!SDzEfq(GgMaNN#t}ZhStboY^M%;A|!pF2JsmOp#h?tYL+>_dCcen?K0s z3X>_a?e3+@*L`xoJanvMk9GUVIO-=gP|X>$owJaxF2Tb`*oS!6@em{fI!^jUEIBu# z&yq!X^n1lX&pFDfQj!VP&0D~C%}9d}VL*y3=c3S;mHX`6OPPXqXmbJ0BKo;c-p{_` z{6d2|QYML#aN8(i?(o~tMwaa;-SO34J~Px`!&o-xA;nUZV_#ws*d9~2?7z-yx}ftn zI`-5$Sf7U5>&lX-%vygBZSB+?UfytRt?A-Sa&6u3Xs1{(zny!V*{Q#xJ>vBmFkN8| zHQzSZ6kS{zjm>!2AxH&rh&!+89&!D6q-(LTUL^Hu?`p~$mV z`-ZyUg-f*8Ztnlx1rn!&S|n;DxwV~SXF^{e^K)O{E0*BYitF<1IM43DL3-b)ci#99 z8X51)cFk_LuK9bjFGwDox87{|iOc@WV*B+}f}7~^?zZ5uFidbN@+5wmX0$uD7hP`K=73wjqD+v1_!(Q8-US(s-}^k8sV z$J_@ z!Z{J!MiS@Z)@58h2 z;BbNG4;QPt3F@u=YypWvvis)xn1=CoUDqL&?_mDPqjT84oBOX@s2K#JZc2b!lNxYz z%lJQ6NEW8UMUT>j0S*X!X=S4kg=m+oOzob7Z5Va436 zc-4Yig@9>OC1Ra*1dN5Eak$fUr?XXALDXe3Ts*{F@CDyIB`yQzlLaW@ZFk*J`1`O+ znNjXv1Gf@AOJ6~EJkv2|?>NjxU(9hM6)5zO_pcmUp(Az1N@hpW)(3H6mr8Z{Uvp3* z++;Mlot#Q7kQgqI1gLJ>sh!+gS<-17KW8X+#HFymNT4qIbPd^CGJ)Ojg5u=N zMJh)u1?n+yC8_s+&!o#k4P||U(jgRyX^^vP!$5S)YH6x3?8+j5+M}cekP?m30^+KPP(>)-@vqDR4@I0uSWZSR;DGTZeNYJ60N`Q-UgFuGb%_PenD_ zb*M3rj0dDHC7MMT`n6!@Z6&=D!eW9wk-<=fBg4BplWK-*x&s0c!rZ{Rm=wl- z0XO7r;|39FZXKUz8}I-0OXFL;Gt?*XY986F-DsfY!_}@_Q3V|9d3bxrVkJ>qJD4vt z7F9^m;}FOoVZm>vIhm=D*GIN`4*3R%Mg|esoz&9Y#XPYIPZLJO;z%sxgiFUd|6*qd z%L9bDORJEzdd8Yf6@+p~1$a`!n1!4Wj57l2!a__rW$DJ z1Vg79ApwVsd47B}(V#W*ax}@bSmmxDm$GI-3U9e=u>8OzIOcr|hMzq;P8;p96|pe;Eq|uu)SeFHRtQEc=f6Ch zJkDs)JY^|qfG#ldtnV2HpD&}|ma#P_WY{7({Oy1wX$(VYajSKj zsoS_0?v@XkCai}y(Jm`K4aaA^FdUcDpYulPE*DOwR`P`)pViLIFeLICA%L)6X^l~i z_Ap=HA07^J-F|u?2kNGQY+_uMj;pkk9G>0X;%-$Kl^EQ$On$yOb=t^Nw7@gZw&U zt+P+@u^4hK8|pReCcSd4k8=2Q`&eSrYI3%VMSM3*#2eZ$jo#DlBFM0Qmqgr>lP_tK zrHP6p9B~N*@{4CBNzZ^s>>A&j>~>pWH*WCP?%MOiag8%@N!ZIfO8U4HcJZ*HfoX~$ zUz5&MN zIYRy^;0bZ5eA3R%u0G@iCWcG;`$@e&ma*qXcUvCmKISUg08yyg# z3t`RDI{AqGeJb*twz5^E1)B4tjp<{?w4@vwmGf$=A%r)D&?8{F_j&=s` zThin<8(!%jrTBC zsI$~lC#~Flf$-5P*{0s2<9sEaL+5cIZmMW1Chd_vB+1{3%A%FsaAl zL54W}4%Imzu43RqX-7!zwk>GKV+P5Hgk!%9wJc3g>Tx>ZPi*hui($R;_UFQM+cdiI zS1twZ4<0Vrw9Q;vSn=%8;It-~e;T`n2keurF z?=R2fIKHaD{tPA>^pP|yT87feC6Dcw1z!mG^F>7o3T3H7T{ykl zo1Y)oLQS=7UFq|hkH8n@7()voG^MROVAWfE2jxYRvns?hpSpb<>!tUodHV2X(Y(Lw z{i(3$_D*b1ps#~BXnkIBw6J^DoIUrxz&L%!tUG(3nC`ef<`+An^M8Gs^bN^R9(Pd4 z7MU`$=nghXk(w`)zF$9R)=nAcL+`S`FJs;;Ng4N0;cA*9Og(609#yp_&WKI6c**S7 z&M_4c4C1TE_{t#uw9S327mL}tI$#bj)6AQRXiMRlM;tznkru&b&E;sl8Zn-Aaw(a%+_c7#u z?_&z{Wn!85CYHXIZL1NXj@Z3*w z)U#yn6Lei}1I<8yyZ=rZeN+_k^a0&WOw8qZ`MAjg`IIOxk{Gmv>p#&nR!|JcF8_!OnEOk6|CDTStn6#5I?T6>fbI(N0j=; zQ-i9Zap+JdafGybB1j5`DqM|RBnlfmVV8)hx4^9Uf@n@5bKg$d9^#%e2xUY4Sez{1 zy+HKU5?UGL_1m9y+GwFlGWfO>GB08Fgu0t-HYA-^ZB*M?J|)SRQOwA=;FLM?`R3QK zJ2n{&Q5A;6uWoPx99&EjZK5W%UvXT*_j-APcm`S170Dw>rs+|;#YPh%F~U$45>V-) z7OiC36q480ptW$cqCe29K!s*&X(lMJG*Jv&kd>hzX&~YUTL%EjA8frtNxyJ43ri#C zC^0aYq;$w}ue>YDf}hcX8ev>2kXKTr&ZLXtrsajSIJJF92!uDX{ppu1J^ z!he8+VZpZ!`i{*C5XEy?jD5Z}@eHVNJh=N}d$ib_L$ut-O~g;QF!*_=ztBSV-22B; zz<>8V>1p1KtN#VEfyX&dRSs%`R;%a1dB#^v?K?P{bw5E+Pc*n8{&lb0Y8un2W4dB2bcbha9+6;uWZ- znrN<#E1rp$e*4f3v%t8$wJx|gG0{_J!n9hT5{#!{1BoWzS)uy2-)^=&Us?G8cEUXN zk_EQz6PzzCzjwjRNcbu{AzOb9K^R|F1l%l#W1$0LLf{w#{h{X;cyfbPAI}+i?I5S} zFFEo2#k%vCoPbLQq#_bMZ;%)pgrE{ZuwUU@*72zgM_Idnfvu7;JJWMB;xK6s7qC->wf1eilkTR-p?-TYia%yhRw zb^3mc{zA3V?y+6vW2>$s3myz z>Mf;ojjc)!ScY1Nr=03%uqMmfC_>aVc?F%KFvNEojXu$%XDj3$0SE}X3$^AIja^s* z*IDjgw^X}pUe8T;qu4k)>=G!}GxBNIA!%k|)<~0JbI{P9=Ds3S1?%K8M6vew`S29}Z@Q>QQs(vVt&|UqB5C?seZqar zXq1&R@`k8GTm3*jakSanI@L*G##!4YY&x-osmaXa?VR^}GHC>IUBE}@ubQ+C`7y!N z)&fb2V6z^iH_-B-C_5`d?LCr7WOysn_MF`l3nr>dmZ_`xFEgJj1&P+T9zV2ALnljF z9PASIVSrh-u(eW6$4%JBO|Y*os2xrUs4bhUp1FriIm}V8?|jqgnyYy-@0xEov4;0C z;4Vm3HakFG(l>whWLCAMR3Xc`Q#YJs8&BUPFozpAP;dWeSNQ50xhCd?M)6M^C5>jN z2MxCHQ9KT&ZhqW6D1M(hV5D-=I4)w+k>0NxII^|`wL6qqe;aGq{1aOeWV&DPe?@lo z6un`df179>0+QA;8N@QMyg?`PJX=py8Ug&rh}JBALOvwN*tujWm=Ym2XLH$Rxf`0B z@a5WExIb>I9q(73jgJ@UJ>>Ajdgts5s$zk;N6ko^)~-rU32~p#=Sr(Uf8WNB9LyEg zjg0t@v!Mmee&(wolk@ltRVmAW=QzF&^A213nwArGZ|C*U%*09rX47Y*?{bI3QnNKf z{dSiH7&p$YKXU~4ZX7h6CXN506ci6z-ZNR*Hu6ILm=%eJCMfRLjr4$pHTdXdKGCK zGTJic-L~s@s%YHyOK?<7Mv^$ptxczAT!I9T+l%3^5J6x4(apRu+h*t}W7rzKu8*Gq z(`q&3jFB{rXswj+`oqonQT5mpybzZt5*q9^H@l~ENS*2XFeC@W@`Fm-cwtCYf2#u+wsq~uXh1Id2br4iCp@qN_^t{M>_ zjnQcQpnC_c8)k=eq;=9P(B(hni5Bee26JA<){ z%A_-dlO$c!dl*l^sslxhz{Dy=lLy^DSL{3Gx$C~~#H)v;Fw=u3=a1Xk^Ff^mqV-1; zlId$8i>8l3s{lMT7zzQitV&xW6}h-VFSrd|zozX%j%vd^Fj$*FQ-be+l% zo#!#=R9?TH%uu!B>2JeU+wBMJ^#(4{;@~OGT~DuuSbcL&)<+QfZR}HRdDCVJC%Xe9 zu#AqPe=Rkv^skrsVSsPEb}AbCkq}T`ig6s#rTyAdP*yi|cHkp~|CX7qJ)*YYt)M&P z6$?@~Cv~Q6&AslUC?4L9C8}dyXR1xy`}A7&&p54kqqo|ak46l0);+WP9!Kde!zWa$ zo{k>+bR6MYR=i_c9YcFjmtugW#uUI(L!^!UIUNnTW;i}rXMj2z3e?Kf0lW4YmMZ=0 z5!B$Zcdr%boljVxA<(9J0xW^GYaU;lnC5OZg8KO=2t~A>@W23-S=}wC>K&i)Pl6V0 z%eP?|=i_`o8O9&xiXOtfO?g%udTQgW( z#_wyOQZw?F4T069Lhf5mY`-%+$XI^&gKOyPqA(rw3E`{TsSEDMm}tb*-~(>k1Hx&p z7zVl9wYj4~VfikqSA)H@1n+5>bJ7UL4>8|$pzL)nZC)Dg=la&WN?4bo^K7B_v9cYx zOE1g6clfQnAHZAn8>n6w7ww36VaTy?WwT{Qmx`W6VsbnUE>Ebn1W%{Aw^}LxqDI2d z1G>R}96ZF8phC@uG%2RiReLOB`bAR-ZG5Lt%CcnZ(_&@o$ZLOtSY@S7aDz)-#oK?6 z!U$?7fxEcFVS=@N0Q`jw51})VJ#ia0$W<9HYX~ZTE(ti#uf@GsALQWk&dVtOsJ5cw zZ~i6s!SoG5iFZ6t4&{uY0UxLy)iU_GGlXeK!hn%-5P$lowTBiPhCp~# zMjeczTk5RmjETRiAm=A=)N*%+`+bT&cwlG(zE^vxMHZ>y@)0&{j0&jAyQP1u4Y*A9 z$Pgi7Wod91TKTa-)bQ{RO{`O`L((5s^FL&hT@#9~@5m;Vfb>IVOKrs3(QFyJ!E6cDTOABYx}oE)nPK zO4_AUwzE>1QYPU|P$LO6inrJ|V)1D~?I|ac%D@ExS7M^WfpQ#Qz%gbm*3`|Ur=9r2 zD2-fAx3YdGx{zrr6R0(G_wF~4Ap2^F1?cNsZqR$j<2k;0N)|;30hvaT|Nt5B;V>4 zr(wy6D?tN?6tM1392kvlvNBK%`_sKbg0NPaX7SG<=>4ko}j2 zs!67gF*TR5mCsTsvi&;k^kC0Z(~~SRA;*mxUHevzG@G-g)3z-fTO1QP7to7f zho~eAB3Lx3_FfDJ~2*=o1K%IRWpqB!-F#WCEcF@h_=I6f-bwX!9%C-}I+ud0grDJPRC(OD5zY0O z&=YNtGjJcK!CEj~G6AE_P4b1vrLRNiN_x;!Y>U`IvrPFCyAlruQ_2G^p@nex#!8OQ z(e^}x0I)Oj2eFprwM0LF5&B`<;f%PXOQSF{q6c+kfa~RYBrAXLh~-%;ri+}IZh#*R z2QMWarIce2oJe_BbsaU}xy?tC1=lmyp%ponYMH%iRSbUYXW_gHp`R zNrbDqfy}E7j_bi#Z`-#_*7pzrxZ2gPr^lx4)Io8Tmxzo=vVPYOhnGgGBK$VCFXa;h z%kGhFxGlQ(3vXpWR7-TL8Q>%RtluQ6$r z8>f7^-QU=bDMuumu__9H5$2~(b#`PtWp6Ei0zEID(OtGcWRpTDd32)aR=UVxJVeC_ zrWh)RKWKSNO1tM}+*xzbi*U!z?>YnWO%jyCD$YRCD!iYFE}$x=r^k^J1#Bn47%3*z zRcO2+f0_?xV%fE(I$sD!W9T#V;x2bv@{-sgqng0EKP7DoRYSAom zL~dDrZPU+j_su4#Cii)^==cqa{dPV{y z!%|~b+#-$vO`^m?;+U3@4o-?d7FKcga{9zuRo)Y(jNscWv$sG1KLL$j!O1x8;QyNs z89KedJO>9l2`gs4H0@qgB54$px^sxTaAZOsA3A4nmsSsi9)^sK2C5PTCN43^Nj7h! z7N!zFP$&*C?kzxJszUQ(m0-DmInKWDqTwoc2@CUx53eXQ3gwJ3%2Dj4P-{Lx9>Cx$ zg#OZ%u1Zv}BQ2M@>p)huH+|h{FX;{5W$JnCahk58KNmhmPKT9twLjJ9tp7Hp-9FAs z09z8NOmR~=#b_Td&*8ODnaH(=HAPnY3=;dw--56H>2H(A=e&a@tIwndGR|~qu-*6f&(dbF(V}#C6q&0P~cN zB^nfk0ZJRV4d*E=J6C>;ZV1kkbov2DUum1nm}< zdNiQlqiNA7W;EA} zT&jsMqu^8X6j}}{NeOE^=LXA#(Jxl;BYk;Hc^8lL!v#z*aA})x)tp^!37ebUfgHL) zWB@r~P8zKWt_$A?g;f$JK_NhaTGc@@qFLpMd?H#=ZH$VNeZuPnR4;GKw)q@;L90p? z@9K(dy@<4!Y;X;`0aeRC`a&fqQke5Ym1tewHp8m-sUX27p`(C%B)d)nytltHbBjDP zdFK9HIEu`@NXn+IDkBmfqsVlX+#vuc?K!vjBfI4s>yim2)SOTiwo!dq zGLP7vMn%k+xOckI=uCUl8bg~za&H~N^|zX-SvJ;{)G&qvu`C*1OYLE}cu2jS3HUn3 zz4BB`R=vf$D+TkX6CA zvKAj(A?qXFf@Qs0XovUUwv7qS7+k1Lr(QNbQ_u?>FDZSbX0BF#o(=vesXUm$JA4d}52;2IfpD&HziMIQeWgth!Lz_^ zncrlB+p%#Dy*AZUgQD#ehWA%lg*4o&Y%;O_-1p;=K?f}}kRYp!6(`A0*jR5C<>Vjw zGuGRW5f*r~|IvRm>hx4)w?BS>VifNp{ut*myGuFO{~e*s?DC~w$7W-zalbiut1_6s zf#3UC!VTLt{J9tU`soh~q>Dk+biwy{Kh9HBV^*l|`8<8T$eqNNALY%xXXmwO(haeszVKUbq3#l8JSaZRXkFj_ozU zzqb5FSeFj20K%UE|Jyg-|BC`EVqk6k@AdT(P+tN3{}*hx;Q+=pYGB|?vsY>T0*I^8p7`2rnrGiQ-mg1QW&(BsWH6hx6;KQ z)T!E}f{BoKQ9yEpDDK$Iolb)&8P)p4a~ewlWU0%9ik~!&WgQcu0~EZ4W8Y!S$MmP> zuu>Dm807{7zPH3JEAL^52sqIGKa{;wkfaN@uG?L<%`Tf=wr$(CZQC}wY+GHnZQEv_ z?74RAeb(G3;;jE7BeLr1%ZSXucRX(jSGoXsLKsz{6!v7AmnvX@)iuG0kmxLtA122- zOxRm+AOt?*PwUY^z98rXD!jw8b!}t&;dP@%%1WGYYepEqb-%7HTHmSMt+X@i0T*3^d zV%bTfV`Yg$qXK3>s$#aEIi#LAu>#;~so*|64hE`IK@nSD&x#@~uaYk?;bWnc%M6Xb``?_BecVV5m_Y%0Dh zBvY6&K$v7J(@VwCCc-UlE9 z*!N=az%K7Z@P{+Z)000z%((>NQ%0~n zp4!GXxRrfA^oQZa=wYnb95<_vHe0%&4OYJ(AGCFU0vMKPV31O%I_~^ya#4WvN*2L4 z*^a!D!-@kk;G8k?8-hXrPO9Xd zx;%rP0Po5Com95fDif3Er%~n^{`)je=)`CV1-SWCj&imfo^9dw45+o^?A1Xcq@?-* zL&W{zf#DvaOoHeD&s2xe8{)b^YqORlmAiB~ZOc1u2y!krA%q%fOAJ$$DO*gclXHE; zsLiPrUPGldIy99?Mvh7ya(Xpb02yU3S5HKIpIo}X@cs&AM9E(N_M-=E$vF^kKd>Rr zt|l*b6Qg)qb>gNN$Y#7)s@uEoeij{5LHKJpQ2r zI~yiM)vD%6JfGko8hjw842(P#Ac|xOx1pu>@`WKEDPtZwYRR}FW6-ad*2%gJkyAcs z38x}$R5$(B$_Q8YpAwo?s&W1v=NCt9PAhX=?SR$D53@qFtJIh1V#aPol(XVZV>!kQ zmqA7f@r|ACDdS%)%gyD!SD7Yr%g0Sjw3b826XR=ki!bNdJW34G#fNoRWYAd06WLM7 zK6Y+QGAnH6WP>aZQb7*fNY|h>NJrD`Ckac%ts;|mKJI$Q(R>e zj_bsuk_A(-w`)x9?0l8VzuRyb)v$4{LfE>B?6b@#TGn!Wc?Uk?GClF%?k3`-nqGLC zMIn1FwxZ2`FZ$EaK3jSe?DRC3 zvvEFh@J@IRgnpdihkh44&j7t3N#=8+meSyub&)(Yn4?CGp@cjzy8m)0<md@T_X@Ie)>UVUs6zVrdr7MWDQ>%(yk~KK%KWl|*j_XZ{&3m04jmo?6 z&OSrL=D91*rodWl??7_P-?fcx6}x+nEJ7Gv$SrTKJ^!E_h$(A`c9q=q$^F*ytLs7H zZQeJniEyfA)vDu6^pM_n(s4&pWjC2MeBXPUxI5hdra_v9i}e2CG_}rjdXUH@G+tWq zcBuMa+(#Fm!4cUPp*F2WPjjU*43;Me5G}R6NGCdwCAhN5_Hm@)^1E%$4n1t$gstqW za+Pv?tNNe!lU~~}!0zuy?>iaB!}qSo&0jvI{cMS6F*Pq9;6BX< z0GQ5BwTfP+WnAacoj*EUfmvP}x^b0}`+BK@Qk*g<6LV;NfCdxe@zFtZC1YD)aNPM? zh;`W&pV&PWR+0Qt0Xe#HjyViAsSy6h)W7&jmhCZH5g8Zb-G1Z#t*~drAP{&$58M z*()WHwLieXSkJ#XMV|)EqJnd65@G>6=Nz)HRUtNJp+UY$QDM2OekF-CPAJvtY6#@U z15I>!fs7~0c1El>(u9>ETMAREu$Rmp$mV3ufnj3K6^RsB{QAmt3zVvZxZQ_EPZwv2#3=T6wk4+e zlZ}9rMen=w@OSfj0`1(P0mF>t+c81Zy0P96BN480x)Jbzq>Yy$5w!-D0(yd;qLWaU zn@uX$0OG3YkwBWexiqcQu|mn{#9#YC@+pjehg;ECkl{6Dq6yFt1q0-%qyt056N{(J zP-7t`op>dXi$GY5f+Blt*$i00L`wQG0Ok*^$h*h|Ia8K=6?yk;s|L(GH1?5V!12mM zZQ-zi>UeyfAKtmAfyzSSG$lo8KN~T^E`Ixk%=|*_##A)q`EQ_~dOEpc; zRcw>>2ecEC)-UG8!Imj3SX09M&5s@P1dn+cZ*N_%?dkP~zx2mgu$2p+c2|cwoqfVG zx~ZwLM-j13)yI(0TN7Q5XK7{1;i}v8{H%$OI9@C*Ms(ey38Q}A^Y)RkMTU3pr#SW^ zyP-*gJ`y^EoL!$bw@I#H>AL{-K5tbT-EiyZ_mQq9)#U>UU1-p(5sJk}S6`M#^Ocyv)R$d2{FqndN%i3-9!No)PYsrbR?t zO%?b8>bxV$O*NuP13PLoAO8s3-1|C_G}D>IO?GFiN<*jZ(Z18f^3}$8+;rMI_H|^s zk^wwT+!zKYqxKt~X%@S6*4!~8=eo6P@ksrUOC{*RpQn-5i=*~>cFU!|ajp%Ww|QZ! zlw+sbs8VqyG)cN8t*{PpPvCX4BxR>pha-F|Lo(pb%zwZI+=%9|MrigsnC048*|MFN zO1S0)f(QCs&5m4=4w18*(4kwlChp-RuD{pWCz;(=#@qV>c`Z*|_Y%gkZ>O=UZq4D? zfgjf>98Kvnj=QfX_-`>@9?qwhKVM}oP;LK_MSXZBv{dmX=n)Z0eCny(*m*Hqrvo(| zRd({6b~J_^e6a42|ua%x!J{9j=3wq%HU95x6;f1a8HPg-f`t z)mEf6*XC)NDB2DR){sTZh%n7jSgBvHsU+`In~U`5Co)-muA?h6j%__Sp*^nb>E^WZEB20Hc zPXExr;e*|Io#;oQOX?NU?Irjr*NXszJ zL!saY*iBKfIu&iW#RxoM{wb4!c?G2=480@tk{NCpjfi`^W0lWbe-L#15CvY?9-(_B zCqgCzSCceR%VkI7(FaS(7D8*cYIN3`p^0tX&0)-X^>KzpEi3XXKR)7CC3KN;*xN-N z$g>8s&T&IBc?^B+JXtBFFpGq$@R_%^*c8Jy@)|25#J1SSgHp*sF(*!C^9yzcZw>fJ zyZ{D0_Gx_eZnbV8ww>TyRZx`_8B_;NFXsW8f3dR%CnU>MUTmL2hA6v-d2`iAR!k`DMj~;D% z_4@O)o<#WxO;etCN(%@B)syP(rX%(s!PG`{)M$o|W5##z|K91!QhSl~0KQ|QsQ=k0 zS;XAf%IM$Uu|w5$n{^Qc@01>cJTgk-^p#7+XrQ9frY1kB^3D~8IP?syQ77?g3EC=% zU;sQ-a;#uaG7h@D;4yIWVj?Tu(Zpq>)BE1rH)aQx&7rBR{;&#&2(xdrW8anZlZQng zNA*nQpkSq!SSNyU%A)ojr_mKQxm4JhP61jTVg%h>rGm1>VKh0HIxK{JV1B|B<=r zPMG4{()-(X5SaQyZaACF26Y^}`4%8((FI}l>;Uu!IpP2!G(dk~YS-sw8u6!b*lVcY zGW#wnl(vYo--~6!n)g74slhu<^lJl}3F|iVrZ;`{BS4zE^7}ndhmj3L-$Kr0PLU+c z(?|^SO4Ry3rsX}sCggKEG7{iGJCM>K8L+=&cVy%Kt3X)&Y6`FQr~;_md}3$dFX`*( zE*y!?y+rV=;*no30w|xSAjrao_(%H5jqI-6k*hp=KP0becTh}p@{%KgObilcivJ4< zojWYwE6!Kdj?K6odjt@!S)IafX}aec$t#aliS5Fl!%I| z*Cn8CDX-B(>O2)WFDIqfUtn!xjzv7;KbZzf9z8ftMY4^%#<|{lTN%$`u|@{}BUDHo zI+Z-4(t@8(1J*J)ybE4gYK`8|@N}kmi|e?r>t&|RVF)Tv;rvmx$bPlr+{wPliGsgx zkQ}*Ya2c3bC|A=_tDv0OHX$KVYhmM{M4wO3HZGLIVJ%tu?d9CekLn%|xj#{FW1KTf zw53{1%_h>j@u4sO+;h6Krh9Al*Ggem4gG@ZbSD4cq5?uqpKVp+$t+`MsybAY-0TTy>e?CTY0mFah@iZ2As$xGBua$Uqz>9H zlsrg1)=K5EEp<$MxgkpyTTkZ#D69h_hE84M64cY~3s$Wka&Wt$)kQD#A9A!}{rm7m ze2JuC+x$T{YQ&+4v#Xmp~gH>)=Vt$l5G5|FK9-!0fv^*x_` z_0Mkq_skIK{RbfqkQqt=EhFE5P=SaUTiMC!n;QRL$wB4+og9?YTgCnIMEn#M)-;qz zm31mHZlI;BiQ5kn0BG%?-vA*riLrc)Be23let>?Ji_6pm`%j#!adh4%MPu z6ycs*&L^p3}hN3VM9iW|;A zu>ilP!RY+M!sED|5fk?1ULI<$qxT(;UOIHg7&naGhAh9IzAzhg$Gb_K=r12Ze^-o- zKBK@~Xsw13l6tuw0iJNF6Ah*>^BFhn@YT*Ej(BV+S2%zx?b=Si_1%p-Jwy-aok}(gUHCfTsIZj>sE-YA z&?k->_3$wi>O0RFexKF--7pdjIGTl)Ih$+$5Z|W)Kw>Xu3{#WZJZ0E?*lvOGg1!er zFK1h9dLL=qXTF-!yLwk?ois3jK~n!OOe{B~w+xWQVz!^VPAf#~d6Zl*i9`^V+6nif zK}XC{lttSXGr%yQ9nDZ}3ggOfHakXVN1Zq{7HEzC53z_acOag+-O6B=F8#2Hku#*( zs5gs6#jrO=|N4$3G*PIKt}3kKUvLxNIRM-QZs{>d@%lucuP^)3d5*vFDCoIV_K#b0 zCl^KGi>wzjUz?|+>5y>F{*SV08JW3GYDzB^^WOt6wKbC+BE@5k7G9?0Kj(BWYQRD# zR$Fl4%0;{$PbeiC1Rpw1co^q@XE9YImkRBLLmZE<9#2SS5l^RiXSq6jVs0vo$g?@(jhSTLRfE1{J|DZVsjoG(u6?ql-4qd2wB_deY@t5xGW)#Mg+qE^n-%9!uva$ zyhaAo%H-gi~|&9*#CQzu@JZ7K*{Ez=0T?gGro zPPW%Vl^#MP2VvG^fecuuuwMG-Ver4ot`-c%lf%l*>yop}V+E|Rl20zX=pGjomo?%k zwMl9CR7aFjqU-TNanM*DAb6c02M2FN#;~Qj5LuD_gk(@{#X}g_VStImJ`oR|$7(GP zL4?Nriq!h-Siz(@5UY4pgTIv@h(;Q+`-y`uzQMCMluy11#tskmSt(j*zdT6eBwG1P z{KaRxub6VpyO6UHaP1)7`8#c8Rl#`fN9fWDe%R3I`m)fQ>v_Ce$>GNXbIptZ5}VfsuZ_L4I#Ip@&nh@swv%%9%$FMvq@h$ zw#=iC)JW8AvCpwF!nP|NjX%#N7mw?%*I(HN+*$v|f0qd>Fa-iOUecJ~zVZGC(OyDD zTE*Ph_20q%QhnQT4RG`_`0%-VW`YnXL9=kxn9Hao} z*LH^|lE}&mY531{lQG~dTic$^>y9pN3U&agFBjMi)PzyDX!k)wfXM9k=x{iMZ#SL!Ya(&Ec`P)Pf2Nw9RR$oA~(cZzh*AMm1;Hcv%P*itX)cC=J_SW;MumPNAzW)NCWRSjRSQIk-~?Ua^p_I0y*!D! ziugF^F>UKpnX5h;jH!lfC87`e=j0yi**ej{ z42byf+UgckA}t+>vneKMHjN#XVWCc(8~Vkfoh(GifQa9L)#`2H@Q)%H zbE+t$w!CU3W&8_ghoez&_yp5bto}BNt5x`@SD$-qgNFpib;gDJ3YH7Zw0Tp$y<-B? zA+e3-WUX4kW`_|r&rogC3ZFV9sp+*&i@ zXSp$FYLKKsY1hooUe+#`O}p~kJx<_(ojl)sMfE^wX^AQUG#cmm-A};%QYFiGZM#(F z$jyD)s$Jn5Xka4vuEK?8yQ3_EvqUbb_%iF*aW)eGG_ zJ6GC!pJ*qRUbly1A^50ONHnn}E>2K`d%~W25ghW4e(o8oJC4NW$jww-dyUVEE;~#5 zl24?pec6g*RqD|+UG|H9)p3(S*A2>LWPmnS zD+2gk$h0n71Q4~|2`5q|dK=ohSdb zsrAh5P5Fzv9ns%vA+fNXqmxcz#|R$?VERnltR=%F9D?LpcW&>@s>eCgaAV7lXXgen z{9pp)DxBYEAHRAO7^lv{eyFl!_z5BxX zxb$|LAzPX>P=#Mz6|-p4>^05a!NR&w{Bd;l2V3ZtP!stLhdCc4AXeuLyufY0dbZ~o zrC-y7wVV3|{l9lJW<4CadH~5JBI$po=#VmZbo#%No0_%bnivv5(NPd78%I*_ue3?h zy{SQ9?*ry^Rmc_()RDHKy<4GLV+listd111Q&zE!M0|2^Tr6ZEr zc3`icg|W(`{Q|)t#L!RD7WvZPIm5Mmg6yQ%MgpiN zX(S=3ksalc+>!sPCVv8Ww38GDt-F6!lQ3L&`yd@aMG!pUprKs41Z9u200Y}7`4rQ% zAw!132sCb>)1=tHJi@#s@lw`;PRY4OloZcV9J@-#XNe}~-eZ4jZzJw5@J&mUvpn@c z%S>&pb~|%S?oAgyX3tiOc9wHEZD@3h^1u{k0MOk+C-BQ$?>q)L1-6ox9%C;bSd&@1 zm{VG>*aXINqO>Lp`f+w$BYyQ8#&Ys{At zMrreSV}AY`J;oaFc)VGXZY+C7etq|rB0b=)^G$Tqsjp25*3b~Zosy{6hBTpXvOHA) z<5?OXd4z4Ukz;;daL_u2R>PsEEGBkxwtvQ(w>yG`9Eu%<#B2$XRZCn!f&yUfJg%W| zL_j3x#TDyMb~}vYRzILs#Sk6x3Y=vVtkW4yjmGN%S2rVZO=iCri(<~zoEpTYo0JRs z`GJ^QD#LZ+Wm}S{)82ynXq^=2#VrtLP#qscbhwY=;Ks9c%TCy^T=3s{c|6;i%rgRkQP5)N5rOZ>|5xlq=dk*m7a%ariz9V-6-l+7O?rH(^N? z+XpmAa85vkeyY4C2>Z(|piIN$yWL6tkqyb5SmimaLl2T1q1);z0jZmt95T$Y+djgx z!7^PRP;KS5Lf~u-7jDtCa@-{axMK(iOC#CZ?C`Ymr_j+YNtQ*8tBpU>4rk}uBD8kG z5TADSTDu=jPOG532C}Aya4A20@W%=<;t%i(EzMt$3W?a?%PQv0Gcon7n6wmQKXMCpZ z%Q#|Ws#Nir!W3!BihO6gAviQ(t744{kSG$v^a%=^MiW{%eT6K16xp{j&)4fAB^1Qb zpl+qmlc=l;Oa5l5N%!qn-jAYMs{3T@sj)rf_VZxtz`M$kOup);l|1Nj2l6>NuI-v- zji#pH=X2=n$KWT2{LcaDzPCAPO{{nYTFNxfbaGo@r}iH{8IXz7m6>#2#Ailo3YOP% zmWg|+XHe|1719K)sSkVVjce!<9;uMJv~*VH9!t`SyE|fN#lr7kh+ui#Pr_wnD1q!( zrr^_I({7#KOO-zOvH))L+VU=V=_ge=fke@rAj9`ptYPDHG9C2MaYNZ#aGo)cn&sab zjz!XUmsQ7x zkrJUw5lin-Zio1r{Mx=4w`E@bJM5XL)jMILzY*P*TEkiQ+;f#rv_|lIuBs-8&Zv@0 zy)Ur1FDPfJ5-jgMW1e68&6-ZP4o|k-?^|41d?S@OIF;9mURRg}s#Njw3&!ZO3IxZz zANbd_uT>9P@o~f~bJKmJASE=13m<}M-nDFEx2lKNp5M3n{iDirzkY1KeYqudH)`gN zUgkDyHnBy*b!J;G&2Sxf$YgE=tdQAih??!^eV%D?t~zDhFm>G}KmhD3_9h3s#^yB) zMJWiKyv=B&q`7}#Ft04%k&!)QvWox_SOi^eH?!$?HgKRC_;F#sN_uB^W5-@MrK#L1 zXhyQ?F#cK6!k{w0(e1vieygVNC0?zb8AV-NXGY^WA$+T@lbgQod5uLnXHs1@^U$wr zQSoeA>2c=2ZD?qyoGw2Zat-LQtvtCcZp{q_pR)5c!(r^>H@%+;!3BZWyaD6bUK|04 zSJPL?esx-Nn+`lJ8FXRTYHlAYWp{5WS>t8y+_dR$TiESP#=cHmaD(}fU9ZW48#&%9 z8cd(Re`+HeFJqj$-R1OtA-^0> zuCtPQ_b}^cES&M%7~6xeu8uH{V%N54&TPRh)W<*Sn4lFP$dNa6$~*gPmv>SUQ2aE* zhIEINUR)|$;CyyZUl3JOb)HSL8(9QaYWmV_ija$-9A9?NKnQ^pr_m=nCsdU%sjaB+ z&e;{h-5`6`7j?=q3azfGTM2v*`@pv7Pu+lK`Toz2>SI|U z&ooWhTysRhJQ)%uP{$CTl4(hfSYpY&-g+wFT;z1*yZ=uH{}e_F;=dQNb-5r;zan;Z z=7{ywQAZdnEV7^)3Eku=AaQkziYDn9Vo)k)ml&+dn>ge|-!c77$P_aWqQ=dgEHa|& zGZTt2lSt6G0?5{8q4y99}V$<#K3y}A)H%XrhOT~$@5zfsSq>NJ?AXbRTn<^S(dQaj= z^Ur=tzuV7D-*YR{DZCGKwG8aOmHSaG75=oH%P8iD&T;bJ^;8bzYy>;NbDmpJ!o|&$ zu(DePh}Xb(2q0OpyDZ`sO&DT5=tE<-A4ySirzgrH&k+fTRHIJ=_pfjU(1ZJN(X<26 zg*4jO3`j{4S&IH?Mq6`+5dV74ckAh6)PTT$K$hD@{A}Z1k!i3ojytKX|_1%(ug{-$0yMZIQJ>;jBpp4@C0(6OOSHm z?mz?MqB($1(I#%sc!-D3Bc)#%C|opG)hab>6&8n;lT%T;EU#P0;dy>!fTh%zo)ykN z#_#OrRv>OQcm@Se7u+)P0&{mflJp)ajWrws;l7z;uFwrGvT{P+S>Z!LOkR$g$ccBw z4_G~Tmf(0e;XEtn@DnL_{v;K$a;nDn2~@!2DKV3x?<*~t<9q~(c6^u_{q0_%+;L9m znc#g8{0$vWB?gw+1H$^`^)X)XOnXP)tXpa7#IQ-14E!zotY0@@$yyuiIhIiw7|ZDH zFdw}yP3i%RXUTbpOhIY-q$p26RD7? zx14uf#TSlfvR>Echwm^?49T3IxqcTJoC#=niweR}_*expb2il?MD5`nhlzN}B^Cb` zs>m-d!lVHPhjrNBz6tyX;aS1h#LC$4|4XT|sJ3piCx+lnswin&24U>(S6p$PrMS3x7?AB+^S&0~s%Mm_ zju$;m!W4*h;l1)XQ~bQi;^amGZ1hi;I#k)=;hjdPJDu4jRz{;360UQ7CXU zt4>i!4Vn`ohXz!D5>Yt;oNqE!kaYYL!`NM+Q6Fp6am3*dI;;o4?~1|a>O>58n^?^v0*K65zLg* z%kKaUfkh4_=ov&mH92y;MDpJ)5$;APOf);ORs`3ceVthA8u?)xN-rTjJPm~`h`4;c za_sIhJN22qXa+7T_?Ivv&qMR=NrPVzP&nOoW+=8`RKap zop1`SE1RL{E;~hc+rX)*3c`*+UxWe1hM^ukzNW1xeG;{}pi8WeGP%lL7&gmJX}j`T zk@5cOLdl;jt%4?LOnTcGA-_1utb>$|2b;B?T4K*<;IzH-p6N%9h)(-x7`;Ma#oE02 z*s8bfx##b>!MNA2$~!wI*coAut^~JTl#jepo8kxN!*k^5t<8q)hh3#5^2XW^8HR5| zbIdhqCVn#%LpbIYx~h__H!Dq~(X6u*Xi0>uMY7=FGiD7-x18N@iIzCErt$S_>V#LO z$=Xt=)vDFlKBXDc*RO#%S+!RxU*??7VKor)F2%2w+$>h-H(RtirL9}o_Hljd@-#|# zv0%vGO}CqT``D>G#}0LVI@RgCvNW~)9hm!K<~MpV4^K0a00J#u*VBYbl1M$=A%RNT zJI=EN2oS`~UA?9u3$1T@f0ERUp71rf_T)SLKKkfd(Tp`paH5511w`$rvEHB4W9zs! z^j+%0iuOIdsP;D;ZM5f@HA1u|TtQOo?L+&JgSEbuX+72s$-G+m8Y^hs86iaq7%uu1 zG3LTHkds`CyQ3L;)Fz;V@`#JnL6kyZE6;@keQjFk$#?RZmEP)e9k(@aI0Soarz};P z$S$fyysUL+5s+8^Tzn~%_)c3npkpKt5ghmRYn9MBS_AF_t}Jazn8ncFI;spn?q z#_6D52q&Wwb(y|}P(%Hfa4YB}mI$UrU(>S#b@IN`YG+1-$>n;1Sq-^?GlmLqO2F}^ zRYvttCF*ez%;JO3rDUedd{dmy+fQvb)w`QMZVF@EEn0~V)P^O)tuotB{+*mxM+;QU&i>DbEWhug(7tI+h1q8>J`%{!YJ zoo6a+E0lzmS@H>JjIlQAPA`$uRLq+n2rq@n`|)Di%e?o<%IM|1Z!aG$zHvngS`yZU z%&)sH`;P)|e~b=xzm6u24(?!?qN}jUvhjB_}jOAz)+Fz|3B{;yXjloSs4Rvu#Jq&|7JaYJ3A|LL;b&R|8a<{Qtq?a zXF=c|)@4{poG%}>sY_)!@lP?!^rLPgOsziVEP!_I|wfn#DLvl=U=wn^}ukPYJx zgbRdh$8*r!+K`pBHlhf~VqX5Tpk?PW>_eb8BP`s9;o)sDqG?M5_&hhCb;r;QAv#*q z+8?)smre8uoR!O;MQbc+9jJ#qwW2KT;Yz2f?Y=iU{Z_*$g&T29gLdJJrQn(o|b z7RNd;@Ytv-IgoCM#q@K^Riu>uPk7IXelY_WSIhuwtNJ_ut6=3XBx1<}wonJZ(z=$O z1=Wgyh0@989KNxz1n>Luc?Ji_GNwhgzegc<$3WXp4GRk55wGeiwZ8Ae=Y&0|kg|Hs zeLq6kbZw#P)F{=C%zr~eQcK#gCal^7G&gLd;51yHNTMI_6m72k!OA+2ny3h;!Y((RFa8UKujYJwy=7BE(_ ziFw4RRL>XEDz5Z%RK(`!hI zeU!5ygu7TI5oj?IT?m7?O2*u`p00}LP-3hS%=W5fn;IxT zC}xKTN6{(N?Wj*sWFXIE$9*_#a1&67Mh#wLsL+c0tS}$2z)fecLwQ0Qz)`oM3i(I@ ze5@cTWIh8M>Z4N0o=>}8+Ls@H)tqhL?tlBd-M3YJi!Hx&UE7o*Exx?m89oKYBMUtv zKXqi#tK8nczIhxZ8r?-N!kngAdhVDngZGsoMK2+qcQ2KpP<(4(4!oC7bIp+Hl9cLc z?Vb1JbuvWh<_@+YMbFjE!^w7G=2D;E=`(6C8wjlQd^uG8(BW!DJfdMy-tL#3Iqlcf z0w{A1_L2qc_6@b}DVc;TYxShs1SA6C5k~Is59t2;?tudNumlnSkg6> zHZd`F5H+?jcF+eL;QX&JAs@TPg5aa$Gia8(lT`1jtm{$16Exsg5jV54%B}QBfY4t| z!VrN=UVhcH9fjOj+tpm|#04@8IJsfnmbwUa-F{;Ab457mZ=-OfLkr17^!oGl8H>lm z>vf}-!1^S7sq7p=zg@;b2mA8Q&1MT6%~;M}ZV*Kdl2l&6)J(BlMdJ+XuFYqOlEjR$ z$%1|8vSggiYPxb(%!rpI@eb5(|EIkvNa3_#BPBD7o@)df6ns&ssHxmfurSuxANoB( z(~Epy_4=`_(zSNhSzaN|ogedAf$@w+}-elwAzFuMS1>aq-n~Iy3t9GnF*yfKw@J`s|pCTad;NY#s z$-`21Vb({a7iS@a6l7llD?7xb@TzzqjY`8| z3N*}!GWRp-dv&V$-GCt4Z=+xlF#dY2A}^Lydf9@wBA^epflakyuY^$$)+=xS_-DZ6 zf>-DJd!tl3Lfo%j^zkqvJW{bwH3w3z9eV_0f>_g+Pa~u~A%OvOt%6J`t}#hH-`F4{ zDCiV0N3TaRu04lC1(~)eicOUJ*V+fg(Lm}uMKzOCMnt&iGw3!5!dyBvHW9cq^cCbt zlCU;>VO^S3%2o=Oy4ALMid!2BSa#-@9;qai8`(ge6=m zaq1O1%(r}80pH!}+L2vfz_ATfeG*NWJ0l5BymZh3V zfP0$ITT+gk91GLFM*^e_5XYk7=p2)#M9u(_(k>>60IXLOT=t6ig@s5{f|D>nReK%` zKJ6ItzVgsYCtJ)jN+8$Er}OvJMo|Y9njurzYiQ3eo|z!ely;@`91s1VzvEz<$7HJ1 zYnl0AiKZrG)v*hd2b9-qPpI1wAgYDzdw=cIjt9Nh@qVN2zP#4A8)5a`S^O4y58o5J znCtGmLUqrEk+jIq+Pt2#Yws{N9Y7liW*zF%H9UoEY)3A+YHSIyk*(K)=OfYB|?5TjX=GNb4MaP*M=;x+&nD#C-GAz*VYUR% z;D`ITc|cpsYj!i?@w+8&?)Qg5jusJetSmc-j1~(osaSz5HN#ekU|bse8?0>r_L@o7 z8yH|MnF06QO5VWpW_^$JZ|0@e8>?;IfGXr!kd^di?>F)bfzLYyg2XX)i>tFR2BcD8 za3C%RdVuu~{fp`H4k4HFcV6nf#wuU?PojlxTO3@h{rstr>*>z!uc%cBpFa|1Pq0Ta zhJ4KV0_7kYmmfcab-Yi#?V6Sv}%~PMDonJMSi=8rOHzpo7s;mx#8?p)goN6`+ zW&BgnCvhGTLr8X!IG+Zd)%MHx5}-netyjI>U#vow*wj%(pq^3LC^0_xI;DljnEtnV zmKg|K$7W7o8PQ)%B>Qgs^ABFqK9*_C@}v zm{({b;qK-ic7i2E3qnz@4Rpz{R(~w4aUwlHx{O6ZlxZ#% zGW$rX(%mHxrAl0FskJn7zB})PP^a#mLNG(1*`R8 z8M8EH%|VhT13JVvwR^3x(sWaIQ3gh#BEX7{>3-4lZe=oBTT#ChQPua9M( z0*WzgsJ?n?ZN#s2mLr#qoft>NPiO-`YA$qP9~9{TL{%f(H2l>WqjvUYiRZ0x?)qwe zxo=+&7#mK{=j+gyhI8mrjBCTH{#uipuy(9%f_E@o!z;sz{2oaakj~n+Nkh#P1t-kM z3v!na-~?A)^_b=jtLECIXZq`3zhFpAEh%=uFZb{3e?Xc0-$jD3jk7i5KcxYcnADVv zl&ms&DbwhjWR=(i&4@Tn^U)q9x%Akml#19S)ugf=D3S=&qE?nn3_k~y&DYnGR+HAh z9wd92qBj`u5S4%z8{lQ<&SXJDPs>KjK=;pE3w;-TIzZHL`sYu&|Mnol|JCLEj~-`A zoSIg}4)9RU!6Ot(j`sEfB#QpOp}LcQsOQm>m1OqxFr%B&2z(c6G1bwri2s=esQ{WR zIX)pdAx};#PW{~>Kr({yeh$@(Xg~Y&Ge8jF+Y*Ihg_^{rICIKH4NG zGaeV95F-KAKDAw|MfuSijf<+~y3pj)$&BY^ScpvS+@vFX4ope)Wk060RWt0_L8E59URGR`@DYe(AN;D51wor#A zIpr*@;45bH4~lMO7jTXCwp}ejb0_s{O2RT2iX+hhGU>(jD#prNc)f$#VB(pHlgWKdYLR^NTSm zJb)?zfpW^S`H(fJ)orsLEv|)p&?b?ojt>E6btykWlEYiDBI?w;%<*e4e%DoNG-Ge3g)iQxfO2=?W4r5>2Mw zvwl=OW!M9|*_}L|)MvwxFCZXpnI9*n8yUKOW8`C8KTw{jz!wC~>fve+O*we{aQ;VC zPU0u)h3_j2JltanO-)@Gn;{QBh7g7WgG~f3cb>Y{V!_JtUK|;X^W>8;d~_v4&M#+8 zWW9#dE0y2QOchx(*ql?e^$}c$5&YBIyB95;BfTfG91yB}&gR_P`tP@ekJaXyHpaV} z1dr}zS+zA!MZ6F!i%(s&yiOl40rw+guTCLM!j)uldV9BX=h=}5TPsK9d5IO;WqBDg zj?`QJtBv(2*UW}GG4&M>X`&vn;|zlBBB&((~3GlKLI+g@8dCr-VrbYe++-I(H_=cfHgQ=qRteFisg1 z_`QkZ_S~KqThl4yL@wz%;>2BXxumOe_jCp^O^hS!6ay@=iuKg6VQ1K+RETGLM2Toyk9D7|!;P~@Rmt473YLp;*2jR8h!6%KIcx%yycWWS46{N?7&FlqCI zzhGj6rus$=&CAP&BC@Hu;{-j*mHPwxfNuBMxnqznRExzk%XCp`v*BH1~sWj#UD8+0_5!IdeL_Z+;+OdBA(SRH}OKwk}G{IVGZH_D|>rzA)1OwdhZ4NFsa zt=A%W&NN%p3NK4m8cIV8*G|a{h{&afm1J6_hSS@#wzG_RV-)1C@107)JWw@!fpe=C z-XAkU(Laxb988Df$$DYV>TcPAEjY(>pl=>-=TnU)PfT}{>Luy?dISY#B*@H%V2no& zRB+rlsp-OY*J9uz7bKYg$;!asPSNjdsm5Ho*Q^3?Tn}lR+HxAZMoTx1UVAPOaPpu| z-Ijf*+_%*I{^Jw0-V>Jbv2%pklIJzi$9NAzA!WPC;KrHqPqh|9(!9#SYVm+(5!k@B zJFMOQqgjke225bCkKraC&xTy(EUkh@_}CWo8oM~pqCIrGqeQfCSc;rTr=i)Rtb%K; z0+pQ;wBl_yHNtPd%M;5&+tJpxBB=Y>ub3gh^-#eR4kg6t$%R#+?Z!P)n^9%Bi97DS7iJ#B;@X16wATs$L;>MVokMujsZm0(qeB0(blDh+ z=c-LQuiEjn0ZnfaRIPeGi!m46Q&==jG;^6 z)_%XCsh9lUkJnhagUJsUO8($}&H)UTP3o84Hi^0jA;1QlJ{Dk!C_=&mR|Mrr+eWxw@pBo^!L_0bH`jscXFqWUGJz>WveQG=ASx#_Fs4PTjuB*NbNDB}aB%ddFga>1dZ#ye@kkWt5< zld=hNY55*`+tk`O)wqYfYHgmh^y}Z+&aDt_g6b?qGopjPBtm6{*u)nXlq)eJ?8SNa zH}=*; z2m;brY4^{Rc}H@wQj=K~XVCdZ#8OerG1Fj`*;N#1uY9x9Lx?tB*Aw^RXK?sDpq1W3 zctI-xLyH%4j_wJ#HO%hmXAE1xM<)nbiVTa>iFq_VAvrkkd*j3MheE%%O>L_Hh~MAb zUYSB?8_!K&Op3KOR9jhsB{Qz@H$vZTKQ|z)o<}L1VbkzMey9n6OQ6B2D#6{M&}Ds| ztmnsPsA_=4srh_~eFLc{+DT(+0`}pz6ex~1=d}o^6B`7{1O3M(@Tx;+8@t&2Je9f* zkp%@y&$~&2O9pXBWdp9!u%j~CTii2F4Wu%{QutiT+(&xJw6-vnrCOxfJMgApM~WLvbM)fvKP0M^B9;S-0sH(&9vzpmDg=qMPU}|qXhS-$ zhlkv#4}s(dJY7v4SDw|4-B9z)wG689)qk4)$Q2D9LzDn za8qZh70mIg>8$LlJS^%&hQZRw*YB=NQD7s_ZjkeI&&-rlB%T;wfaOjm;g^DI$%s)I z#SL2WPXUq41dZj#K*2r_L{5I|E~eVq>-TbJz$1(IQu^&z<+zyT{q(l(S=s7kF(4X= z-$=`+{8))8x_Q8H@n%c)2Db+Le73{x}93qE!>&-9P|@si)Jo+%`ydX$mj!eq=*UjS_;XQ)dPK1DU(5KG9-eg`}8^L_8bo6sc? zauYxN{n1jdk{2Amcl!(G>gR#x!vi5kqepjJ=1dx$zjtoF#$liCl9BUF<9j!s~c`>CBo>=F$Ym#2ES z#RDn;%D*hbBWEg|zT>8mjd9>q%$L?5X~1o`^C=ubxqM~B*|_;h6i@YHEQTGgxR#Tk z$tOp^wr~IUH^2Cvfw_>0TfSx~-4w?NFI!C9?6P(&%em-_;1H3ZF=n8P9u^S*l!ZvA z_;mT94#gt%OiYo#C;b8Ck}e8=0F+a1pLMMQg6#4)kqqK{rg@)Ljt|X5bXU{U$Utvs zq9(r7P0K1UMVr*$exs1<*R)C5^*w2z!4n!$NU>rZBBY$xphTC18zqu#QS7=#b#b8W zFaHH*t;rOWn*5;)+W7gXOID(yBKMhRK=t(k$L*=xq6G}amrDtEc0CmR`i_CkRD+Bi zb3{}Kn6$GZLv>KKOQpFYPY;tiw4&g|;3~3rC?C`v&qo(eWrRBURtbX!kOip};}~nU zkKjIJL44QBue54OZxx!~E`t`;K8=E=8+igKIR!$uT=9h$s;>Y_4n08qM(9sUZrCca zdP|+szTb!T3U`3eh`RAhtjcdaghmQVQXQlleHo6@AC$C?C7d+jL~xzdbofF&bG(K@ z&=ZN_gQ`Nzf60Qp)QKOmV9WluBn#T`S8T1tY~e$o-cyzot=+H8tx7|*TX9%97>t%l zxDPc)PvV(Lk*9K|y1A3GAF5zhSkMo8rkYK+9!hn@DG>^9UME=!F5<#%oZv*sIQ-V& z)xy0chQXXBg8i-;h_Aavy@A2ZIeli{wknCk*SmZj7#j}`IA>+gF0U58Z&yVd)?@fe zO*Z4M_`)x}mHjHHRT*}md}lV7V3``TY?mgeT0dkK*LSqm&Mudqz|PM%K%lFz+S>{j z=UDeOH8($dVY)YKVz)2r?gTfsw8zl#iB2CJ{!Py7nZj~20YDoO0Bz=fK#++8HX~Tt z{Hsl~0o=^qnLhnFvcuV=XVu9KvVW57#DJy038Y+l)M3FwI!8BfFq(;Ua8v9cGGcDD%mC_YSU#h z!F*$LsTWgDw)i`hpR&epu^bZUT0O_6> zlfOZ_u4iB&Wa6H%^-oA&en8s)v*uhheej@2{}>DqeGe=E(se?lGyj6Lg&6?S5G(B7 zU4KBDPnF{cZh9FUW=F0Y-hezPZzG8O_|Odi>6H&i2>BHIJhg^V|7-SUxWIGYzL;TCY; z{8TIL{ajVQ)bv1Bq96*VC8tvCKY}NMLz@v`Na|gt#6r!HR{u2d|Ir;+=IF=2q`2E- zsXv^`0G61p#|utf7%4XTgHkoooYYB(*ei9)Y@#pDgnbC;1o-Dp5KxD%+xrJ?q|+xu ztV*}Ya!}+>!HyDM6zteh{tZAI?L!@+$%4^7dH`AH%;8P1`^Kl6Mlc zC6P!?Y-krnS3XU2>#OICKAt4}E7DrL;X~)YC-S@V-{E8H;232NkNu1dCpEq&ULNk9 z22&$<=|;ETF2*TL3RDrazfg1zC2i7I@|dQ#7q>fDN!T=dt)CLVsDa3=yOcq2{U@UP z-2jMAKH5G7AR4gZgL_HkPW1KQ)60$xzV&HQ=}MEcCwuBC!V0>M{KAfOZP_Ig3QWWT zl4s_tBfMF=$fY%QnL(tcmcjgkJ|qa;lLJnScdF;#uH^V%OcpEvynZn3?EeI}){aK@ z#(D-uU;n`F{}0VRG)65>D?h_L`{l)Zb?tUA+CSt=%_7dqycY774Z8{B-Vew9T^*I zqksI5d%eE-2Kb0LS~zmd{}McA6eM4f9TZqu{8}yk{%Hvz71})k?j~}8!tqbUn~yP9 zIY4d^;7kAOf?D}-qj^OGRAAYA0Vn4OJ}Y;>KceL9g{jo?4gs-pPzsVPLL()ANE@7? zbsaff*wxZ@!6p+wu;8jTeAXr**98sXi)1$VIWSdF9`Abx5rE#jVRe~KnnqK>8VOVV&4nT?>UuxtGWv|LZx#S5r7jfNWE z{SidU?+dII+22I*FfRmRFz`~0)`w1WNqC2(tRe@{n$lgW= z>>=oyoF3U#8elS^#SvEIIm0^<|3a=0t0A1>TMt{@$oAR!!(_rJ)BU~bW@a7}Qv=%B z6o;wa_#m@5_W%#bLk`UeGIRD9*XKJCZpW|&jQZU;7{n#b^&n0V&8nNzq(PeCCGaTN zT+5E(Cw)r2R8q!Kut2lOwvw@@^6yyI?J!`Oo}#vfFtc#glP5o*fYjhU8)P`P^?Z4x zNKJ@7Wr~O+-8lMP$PyBclaP4!leP}iWH*D}FT{Tz*XzGF4U5@|B-ga|9O6u-3`#d> zN1WHAb5kVIgFhkCG3m*Z_h!V9N4`djC>jFV0d6xY0j+_J-?f=7fL$lyzR`t3FoD9^ z)@x;)R3o9RT2Kn4A{r(6RAY=(1okq zOJqW=pE(XCiposbLPHWDoYb%F(tWLg$7ly4PS$Q8qzeQdOZexmhUQE(|9D^RJH=Wa z`uRAMBT%Jg!7#Z1MA<>@Q<^)#?l!efLQUk}1tQ-z(q1nbcXt`teZmiyiI}##z9PV7 zLIOjnLE_mx6B^4(oXt(QXjuH=G6~sqArzsb@X`Ue(P-$?I4=R&aUl$ydqUo}A8xen z$+v*E+|VSGyAL;7*2#2Exp!|F5PZ=EUeWPGE$3-s2Ue7$14Bl@;I*oZo$_fB)I`*R z*F|R+A58e+s93zHkD{x0dAg{qpg=L z@7o<)|92IY0KNGOXGqm! z&oo2r$(V3c^z$b=0)6n_p#HmyRsBV(WePY>NC2fSE8yJwUv-K9Z%0YKT>M&YG8?&i zt`%_YVVq(+9~E+jt;(O>>)Ri`)-`Vs3s9zUtWLL!ak-Ya`x9^DFFxO&d(CWPWPvJ= zEgs+BUhq*oy&u1KwPW|hTUojIm%G<{?TP4G!7zFRc1JtAHjcyPeB%AXQ1W04JMkcQ z2Zt*PQwa#G3t1k>fLKuJg}V1X901kUbOq+`v+;oqoOv2J(*pQff#-C$bVDYBba6Of z-c)g;N~YSwA0N}XHRAxHmb~a#7nX<-TW#pNkyVFV!%d7nLbQB8foB}Yu%Rtgg?U1 zCLmzq*AXiKzE*g&Q${vQ+&8{UQ=LgeY1&OVFr*D>F?t&vb>uy1?Z1wX-0zuQNuGZy@3JtJZ+L+vPRS6QG6Okzs^} zfT2=HZgK(1=ewnb>4&iwv&e~ow5Y-an2C@nCU?DiBJTpvYZg$y4(}6T7-LA%1yUBT z0-rE0G8hS|?@M(TD&mw7V;_wM$~chC^R#`1SH*bbrH4I48@>6NAPN0W;EdO8-@VV7 zb)2&*MUja;bJr;G;c6w%X6Hex-^AyX&^J#eu=K1M61d5d9P8yUv>Z>OxIWf7?M5#s z*tXE8!wAe9ZDH$42=4thLi1eN(uzU3;PPGaa|Si5Y;~xfVy4^_8|;p|w)t0Zkp-CE zL@IWXp>06-n)}hc&Z@C!pvyD2lFmij4o5CVfHqP`1H5Y6G?Nq}&ZaC3tjVdWXm2bS z`Z=4Unlx^)_Tdu1%h2OhAGI#0K2@jDUxpGZb!}J&WPqg=hz1aG;4Sf?KG06w1hlnE zshB(f_CKyoo}J3gWnE=~;L7lOw$}VqT+3A#@+pS9eY(#9X{ zYV_}L!K~e{4sn%Ui?#6INe9RXBs0bKAcA>^(07XCg#vQpv4!6s*5oC2SWa5uywda* zQ7_KhF8DH|2RFP{CLs@Da@NnK8(JDRD+Vv0cXvhcePm_5QqTw1OVwFDLc^oXBT56Z zE4$rXbQO!nGN{M{fp6nWYLClHl({vI(8C)8wTiOMb7r~glmQWo4M9u2QEbu&y?iPUo{`8s|7ejZ&!nsCuYJ4jtQD0U-QJ^={_!@~| zthpdON5T9=gfQlkJqU2A*RB#a{rrQ5+W6*|)AOy$CCdCy=PBmo*SzU33aUn>x#yXn zE)aV}2+*!nA4LK7hx7DvCOObq7-Kmqz@Pqy;oM!saU~M9in~%^`G>g|5mZEEM1w*hhUVhrV7=weMmRfe1WM zJ3QONup8N$0&x{-nr#JfS;3X_Bd2l}~^9_4A|*luw52iSQWY ziIZd+_hjl^c)11IM-^n2tbYp@MgZBdtzMev<%gwybRux;M*=J4) zj4Y=F(z5R!cssKD%GWOIjitN)W5lH>c=dVjEAz{_rC*jHqLwYu1~&@!ec-0x@6F1g z-lyyjI_lnmXBqv3|I*&~?0x(t>{*UwMcv6HPkzp_q2{u)nVO7)`dD07swT$H^+!j& zmmYU-PJTK$L$X0|-BITvOsfa%a$}oPRNbcojfJn_g4xx})tQi71B|A6NSfW;9K4*IoWxCdCJjx;O&fYzzwS_z*NAf;PN8m{BR54w13c!e?HD=& zVtgO=E%3cBx1tbiw*0=wTpBN>q8pICjeK>ZGxlWTa(5k(5qKTbC}SIW1s<5=cf#l+ z$D^mXrGlFn%29flx^N5hzQwIuW2H7~?Y@}J@w7iQ0Iw34Ptr!WNprUh2vH=#iP{eMdKn$cxm3vPQ|dK-L5<>+%wFyniW}^uk>L z>*vH0a|Oc@zY0?tT_}bKyR_O7{;hyWwN!MCg2AjkKTmRZTtLO*iD0dht-nQ+?jsrW z41L43VxwfFo=A)7ubqd$8kyEuBHt=Dy03QGC~vhZ4v@LTRZAVqXcCj|@!smcK#Zo$ zk7R;Qm5M95RoGGKf;~2}g}1s(Q(Hy%aVR=MXsGBZY@~{=Out9U)hVR%5{6QrPEX%k zDy~tQrJ;PBbz!DdHft$J^P?QCTYNWWwf?cO7_=&kS>vSRd0i$o1njppI3p>t$XI$i ziPVTsBRy*AIF+FtS_4v$;riPVMqPd(pfxxW6M=#}xBh(xTd=ZnM3j*vV4mHitA@La z?dYdfv8d8!-TF4sK5I}2&6*9AMVi5EYp73}>a8`q+Q`j}wDGQ3Vd&%Y2M$|cbUF+^ zu<%2Yr@=vSeXDT(7^4knKhU_%x`EGSfamuFTliv9$*r^|sS&9BjW)llGR3Tbd5p@% z;Z)&vH3pDEgT0XY;h*ld!MFqyh4>I7sPB?8YN+l{|Jn{UBqB);Gh~S%`Z=dn{TS>a zrKx}zrraXL03n`lF{^<(1kOAkFJw!tTjoxohD$CvoM#J)W=)91VvBUvq%W$c!Tc+H zJF&k(zrMrv(0QUdrZB|IjmfzJ7Z}+6`WK2?P7x1fk^mdbV`LCs-TV;nntiv3)Ts7)KV_1g6=1Gkk;nC$SKO*82Rr^X*P`u`-ST;f4JI-fMANfR+L?8^oA(h=4pz<|5q256b>F9+ zVWO0+cj|VLjNPjnG)Iw&Xn#Z7PnDINl^nH)Th~_WI(Z&r>pk?4Lg>-9!nSLBYwauU zeze36XE6AtU@<>OiDrC(j{!240}(j=S;S2|>T@X;ctuXoOOkhP+32Vw0h~L9#$-%8 zlX>3lWc~^h*%_&AAU+<3;)MuXcQ$pS3%FqcUdakhbfq5uVEO$@!;JLBL;hL3AjYgm z)GQCabC>Ba2#_K|*O>Qs*$$ior$IaiS63FR1T5`nT&}m@=D&h=me)>>O?xH==_N@M z-+fH)=I$I3UvaG;)?Y!QUUr`4FgPnua2^rP&+mBC7kOa$U%DSvv394Wnd=>vwE~Pnp7A_TxUb@ z@$YJMQ^q5H=GN1h`KpSWUV3q!fHyI%88d&_rCry7O19&!e>c^6E>h9$qDy*Vs55+rv5-x4e z`^-DDz0?`Fi$p>?NO66ULAs~z9FGo0go*6c#?8&_vI?*y81SRGOZr*hX?VD7%sRTr z@c;DqJ+X7~fH!{A_>;Y$1dnp;LMme=`n5Knuq|)iv5+@K9~>22LRzn_vw{6(y*2H+ zHQUQ%X#Q7=R~vKhNL0nmeXO;Xh7%s|;(Mkh_iL-yBt;_iX~o)J1NxLw2>W#G8w*#Wvn3@vJOg8rXyvepUIR z`Ec%BC0fq|leEPFMBbqVKU&~{G$5BjE|6t|h=M|lZAFnzB@N(~4Off>^s6pE~g0Y3}}O{#Rk461TkrEzYVkWhZTxd7Ko(D z@SKn8{w~iq{Av*kPW^f^X+(18Y})djYu_2>_Qq_dNF7{lx2S%2MmiUqWgbHUO{h~d z!WJCs{5g2{n1Vpvi^Nm>K8v%Pl2_(yjyEpVEk*Ww@Nfaare`pMF z+^gdWkTWK=czGW_)-__$l+KG}=#Tdh-bH{_wm&VE9A-d%Nz<%^nbtIwWNWhx`6v-K zur;4TKENRr886K=YlRuwQV?_UMoOH`)|PjkNV+a6N!$}2qf!rv=TiPNyfOJzHkLK% zQIg$Kq#iv_t2R`*!G}{e20wq*l$6w;F?>3iE*!Dxpa;~|tNcM>^OHVnWZfHJB8cEMx)0=u|G~U=pVY_3X{Fv(- zZ}!pTFqZZW2spCcOciu*QRmBN2Uz0L2}$kzjbb*0+tn>f3wQXbI^yGQHBEw-b8jw@ zFnYGgE{AhF=WzDX2>o!$OUf;K`};er1J$)=W3Hs_?Zu`$7?HYp#gvs>*O-la*M!uhmfsMzLTot$FYXvp@!KlO5_sG$VBP? z_AsGnv74bsRmHbVV+Ob(0CUnWTPNORmq0{y{CDmTr@HEET}}A>Ua*HR3ZHIjT_#KA z&!D#Li(0>FK&FFxHU?>{&EYHY3Zep!C{w)SX0G~TTz;Y1cCw(CK8QowK-;k? zV8pSZkmw{reXkSW&FdyQ73{{bP%#zT0bxD|TZKpom==FLR=`AuYsawwIMPhg7w&_Z z9=Rym)uCdbDwlRLRXS)N1q~K%a)k#E z;H@uE=%B-SF>mIl&~YQ&R7&X+Q)vt@c3pLGH_L7ff5__JvN`+QPf54*4ROX zbD4aerV{6ruDDTK7RnhbvLcb;)!`gI@ zH+kSOW*ln*cx4s`*P&LUT8shP*Tdbm{Y)qS$CNa2c=(TsFt_6w54NKkNZf55x;p{p?X! z_sc@0sn%1n*)W(t;uFAe|8O-mE1x+gSSy(A7u!De;caS11n|;ip2IVt@+`#J5Oj;< zw(cK}`!xQ*#TJ66_g~*vssS@fU0Jl3V1ZY>nbu~;)w-t-vFSP2q`|bq*UpQwuGZ@61n+Q` zs&U??eh1g+XnMFRUQ>C5aUL6fEtxOmtO%CzyId7 zPH?fT<^jE)CP1Y8Cq3UsllP&({e6hXDyGA{j{z~z;!MGkvh9-fC^{e?+6<89N7yg2 zgh`r2qwL{>>-{j41ceJ4*f-&+Z;aA{e~dMIx7D@-6U1V>4*d9CWSWXU{yMc1A_+zE zYUaw|5`OIq=MJ^WT1;%#H!7kMeB$*_W6+-!Jx}FTMmmLzo`?o-0=+xadvNOfv0RkS z+~;}08pj)(_aIN?iIVJ%{fR7>(P0}OQ^Y6;#Ip|EZZI+$>g`;y-^jtQr{_bP9)c=L zT*~w+C*g^+t|n^ht|vnwQJG0jm+y=bD`;29*C6KWs z`a?pcW{i2*;MK*?USi_ArMLxCop(quCC}*+u_E+Xn*V9 zbgo$R1X6@-4aN~$I-qOt^Tq&w?w-7?g*gE63?W{87SkPeEB!+CbvA|9+o#0n{@7jJ z7qMHB@Ehhc{L=X-^?6E(8KDsR2|5F5dbS9Lbv7}=b6J~wlXG(^KG~vz7CFZ+sAVUA z{-bO!+Uo@IN>{V?_NjbbaNw_k<8%F)WC6_5Xu{gmvmmrbP&C~@i>(UAs?1_MIuie>dI;sTpqjsmU2y zl^bu1S=j9$+*%dPou1#PQrL6eI`7uyd^Ervd!_iQQ%EfigCxlWG6T>Cvl|?SR3~c5 z(NjZneF1^gn#P{WVwup8!l1$vzA!_xMCr`5sdh-l7OMGbaX6P_dm#C#?@}-&a&au% zFnkDT!9aQgy8X3*lEXx*Mn@cXuyCIaMRC2fM zpAvkHJPaenU#xB92~W%4+}*JbZT0T5$ktKotWlua!=!Us-~cWe?e@VLAWG|ep*18>?q)!2{`AqMy_5P z?hu9O{S^^p1u6xtvyopbflPsSG;dF%tS-v!Mu_ow%-7h1r@#wTl49 zWF~;Lwg?Z4df!@S>v6Uce@Yh{5wqD^_F#>CowgzDHV6Qb_w<0wKYRwtfRs-r^1|}c`|&MU zGs|aS5lXS?9e9o;Z)y-V)0Cqzf>{LLzP{7g_MA36m;;UOX?L2ZCfS?UsS1|9b+_9YFRmEev1ZyAyu!K7D_BD8*aTRtB69{ zK2dqZaz=JgzT}u09hfgD`MlL1i&p-GbXJzb&Jx&SICCJ0G(uz$1va8K>+2rm@Jr8c*c5raq*i& z(Gp+9@(i_sT($kvs(aO@64gDheR5#bc|>U3{4_wOm{a04W5w$&nu-*xVex85?4K#V zS4ZJH1n2q|B)Dp|g=i1s2qp0%v_o9fdCPw*P4?kKQ zt(r5?NKzO4ZoQe>#Lb|n$D|PEiMbAB{od8`{mRkToW%3X#BGd6Oin(=HPC_-RFmdA}!*nMX||4iBhVXLZ7Re?mB(Dy91 zz><~L;Wv0vA?4kxb4Z<>sNs-7tl3el&X{!g%xrjN?(cF~s8?Ba#Ts>?{VFZs$J0=l z3G?VVffW#Pi`Crsz~GG*F?-AG4fQxShHuNA81fm9EpWA0w|2~Rc+rAUPy=PE09CFHTn&ZBH%Z|#ns z{&vesD>L?Y?rF-hO~cQ>C4l-gOqNIhF6>IE|0%NB{@?o)>7|VvO>GQ?j2sN?&1?aS zWB$!=Hz>>7Yy(nRf^`_Tp=2r@IPdnbl}c+cREHOTwaLIMdD)(V{aDsErx(vtc=s@w z7n;&MI?AU8Za+Qs1gwwQVN|*3ys&`0=vsI=Xk0yqjl6;Kbp8FF7qGFg054THf2vq; zYyw$ZVP0DcQdQcOd7iD5VUDk~1qk3GEv4X|O?rS(I0mmR^c5`1w#=RlVN-EF=(Mc8 z$6{-(;D*SQr~_GBF5|-#C~=av9^oN|g{W2T)^2bzi=@)sI3BF+3R*G(|B1S-nCMNF zPP^MmDPezHu23;1E2_=kt~c+u-u`XG$dnT!^1VF?aHWyZ;IRgO%aAk%uu2LU^<`;Zr4YIDGo(kV= zV1s`4j7n_m>RS9({Z%7NuZVp=jLQ8K>F}Pa2>$EOIQ61}b<<~nh{vpM-whV(N_Np6 z2vn&e?l}sA-sKo20`A?ePVU+Hy*jklmlU|2qH9fz%rk8SlU*e9%OnyYoH1g)i?0Hi z473mGa*t{^Gz18lKJEt;gzIjISyvwCI)6zZMH1pQX|k)-0DM1bH9aq3W_KLvJFt(t z0-k+KBcX|=2P6_2W0*seFWJ5>3MAsS4C$P<|6W6oeQjcqK*0GRVA5tRN`(ALjF zNvv1mr4lfh7Iru_e1usz*B(!Cw+v-IfQBas(HH&%=%7$~6WR^W2+BWG+mS3*QDzEq zgRak{8^rBY4F!;1&j!+lx$zL=#>xORyhPZ2J6u)3{ut|)$v(Kin(C1Pe@;vxchtZ3 z$D~2a0ISat566*ciQ|c)C~m<6G(4VL`0_i7ntA|)!W0AOevl? z!EE179{w3~ zmsqadwB*-sdVrH2@pX~EnfjZLonqE`#KlW1_Iceaf+tJY`_mvqlTy0G9t)OraBuDU z3HpU{8YAcWxyHlRRKma=m!~-h9VsEcVQ`;Kk@#|6sl4G%#!{+DW@^UT z-t9`$n?-Mv?&X{yI=wZtTrT`=TSuit7B3pOp7LsSAT7QwCY>tw0Oz4NumE>R-rB2iL4oMc?3i7iil&< zs}-L0`MsNaDsnK4EMeE2`3dYpjwcKFG4)`@%u6Pr`x`L@$ICDx;09sQ^!gjwrR?JL z2zM#WfJ; z&pBck$eD%lkT6{<0w`2NsX!0<6}hnHM5xM0CM~M{9Y!HGK7*;WngH{~@IK67rwL+e z=1qb2e%A4rUvm3&P|3@}i|=R>nkTcV79hU4-4%AeGls$>%Cy!3elu8O^32zar;1^) zBH(e@HJj1JhUu|9I8j!SG*dF|qWSOqj5N9% zY&mOzLsp}vvfvHYU}y~VPw*ni*B9@CSi@~hpMoniF!tpHVGl9JBxj+z_!q1^ESgcj z5-<8O7;8Ltu@&&cx|q5ZBFSl{lY^)UB{V|6FS5|MGx)GDlB~5UPnVlgSMH?$&dSCc z(JR(DG{MZb_r;TnWxN`L5jc+?isr|9^h1*ZMw!rbp>TbWzL;T>`i-(!NuQGbSQst? zoUs~@mJiqe$dmGjjzppA0z7*|w$o*w0bgqygh~j(HGv0Rt0meOX$c4pTq0!>qEl?< zVz zNqt}+Dx#R#)4XYCFWjVAy<||Rc@<;%j4KL8deUxzqF&B3^=C!RxF?s4TVUy_m{thJ z1ebX99G^Dp(0m9Zz+bf_XN$8Qq>fw|5l(RCuScD3#li*-oP?2nr6kKN@Z6y;i{col z#WVSJqH%`%oqE(1*wea6<9A+>bJ?%<9mYlt+*or72q$$~7Vze}9v?!Z2j8Cw6Xdvb zwje}ylIp>vRM~PU=k;LJyUq5shOeCqyz+XPmlAX|_1G!4$*~(VvVw&)N;5VMV#qw@ zDc!T&yOBoxZ?IWg{l!EL8WjV@Y>woM`QA{gIwFl_5T9KYbEtC?CRm>;blXcAPGoFi zU&!eNqoo)Ns+{bYCh|*OKrS-G1i{69j^9^pn~0UgYU-nd$fWQFYFL46AoKK1h4>OO z=hG#F)hk3xGY`R0ZyrJ``Y2#{GQ| z1P^!gM*F}g6=fFf;H-t4O4aDn8wAw@sKEe#M3lAy1y@jgp7l7}L$~H94r-~hqoVl;6sbM)wup}#D zv(Gabc6X&|Am)E>?fo<`%E;_Nq(9ZeW>htdJiog=CjqOEa&g@>yC$HD)+X3AY~>Ri z@YWc_>R2$dWPuYf`t)K}QTON%*qJY16$)N3H`d(=%=rBmL$0(NdmTOkgGQ;>Zz(Im zs-$X1`Blr*Sso0Ico;6!MY?qt(&x|Mb_N@;^62k{sQ16ew_bJ;vEbt$?<`7}Y&$Cx zJQqCTqxTrtP`o;or#(!$zo=GJs4Q_WuB6k9nu$K#4^)916tdH6g*7aADVKQ2Nb|N+ zY?i8kVtjtQRD_2{ngP-@;jN(O+owWxjgj9X;X%T`ZdQ^_uO@2od^lAz>qb}l%65OM zQgG2#`MO0dg{N0>Co!)zNk>Pqj^)$sJ&3A5vGyx z)rCC#?@LeDS-khvZT$A3L#2T6cZ`~+nwO?4PS@@OrGUxz2{l>Cjq|26<3WmZe?29RiuiT-KJ@qf0t3tO9*SsVSkV7&6=TSZ2o0 zCsp*bvjTj-_Y8a{HZcyb`KGy;p8+M8LptX%iXK=--guwo^uQFo5xf zf2ouq)T5NdQ=1Y%1+G;?EF^*&OoeoV)yDmj!~|&|=9cs*6&=~TD4E2tHOw=&Nr}q5 z8TPuHN|3;b&?A265t!qb%%ECrhzH#btV;6A2zx0o#^VPdoP>~kpm!)AdEf<*B?_i~ zgy}c{(%`yj!1yRBo_M+%)TpMgw+NMh?*?9D3!#7ZX$JDm+R{luF$gg96qiN@V0g;q z;k#}9Ag##Yk<;nN2Scc58v4G12Z;T()RqLZhXFY^aX^PT0(2cY4v`>(!=U?_y5fhz zW=FPIfW#rjY?iV9)Wvk>Tp}|QOj%%cWCKj*5zzd8zkVQD25MXM?gLE*xqIlNkB zIiB0i_*SyEH;tEBU{qFkPgZ}Pg5bsd+}yZ?)q)QH%L#%M5CiG(yondN5z@HDp}VBJ z4*Np3odc|a-(0w5ste!xCO?U*x%H0K!ceVwn9g1pF{&v^+yUB12{li;is}4-ZF3jB z#J=WHay{b+k$;YScf|0^YT-2cxfQNAmbV@^OyvGfs%T|l94Bu<0KOO_Qo~@=-=X5+ zMPr;4W z@twEFi35T6P2{HlF%HCP0O4`8<<9>@**gX47PQ-%XAmo-TDw6lhV#fUWWXXHFNl4dX!i!WqAw-mb3OkStVcju|$Ib6>L~(r;`y(E5MirsGSL? zpe)~nvvF8){Kj#Si}1l{F*c*UlDL7?0P2|Jv`Zslv|=bQm}4*1?R&SB>0J9{yTt3R zj_1S#YUamJ1eiKm2Ui^;LvXTW{PyXRI+M=Tzaz;!GG@bT-Fxas)94zWg^Yga#3Xs)gzM(3Oq<8Q0uevsT*DEQ5sh2C1E`ur*P@|=Br;uSp? zr1+(MzmbyPkWj_N+!rC1NGbM(NBaJoJVX#e810A&=56+r&amO#pa_M>00Z9j_UPn>MN`#Lw!5%8GLfZD>NP z$Jm6;6lbx9FGYvSbyg5{0e#+3bONX-3<68O`fj+pS0=9jr*1{J$7BEr zp%MFl2sy~%nT&bt9CTgmD{qB_)TQ}2AAr_AD)AM!R)OcM*~}KTXz7z!EEDZTlsr43 z3NSNY2$q(*tMNeOhdImdB}Jb_CC*fx>xNWAoNG%WO6X=o%A4`9g6NXSp^+>vJuTN_ zP`Ep%+93uOQ5fFl(E}}?ha&#-tvW#wYjiKTR7hWRF^qSM&iA}x&NZmyeEu~=49m>DcR$x z`^vS1%HA~XB#PuLkaOwk423YSx`)-_SokiOat)w6>o;P50E&c;P5yt z;0y@W%D+C714@KB7kRM;*9-KGNOe30DUz5bKviUX(cYXll@*q(OKb

    ;oj$21%r;gvjIC8i}7DfQE?Ar&Z0GoVO zAV@lhy=3+@wDZO@yuXOQXA!&r=2F(ll3EP?rVz0#S&u!&e7Uq;R?pq)N67vekmBkw z9UXtarQYEER0=iXDb|K9ZI0~*p6Hq-&zO(UcbSv=>x0lMm^?T;-7#UVp$2fcqgj-( zZ^7tx(IpjWkN4>m#oz6`me;6d)RkuzrNz)OL}aDQ18Te(;&%3cODBXK6(#G6XeZN4 zoHD40Yx^z(aEbMAlaJoGyW~5GAGrgvANe!X0CisDX$TSUY>E3_-e0_TEBuHl+R-5z zHL)Cr?}Ii%<$OiA{CUBtHOx-Xz0!?V#sxj)?auRA=YLix>;gY1^?r+~R#5+cD-?=u zwnhRr*3L%uzwNXC`?IF%*AcPL{#!}y(O*mnQi`5m`lsbIpEl@Q#Js0pqV=n9XRvz4 zKSnG;;k^3sW(tNpjoI3^a?7nYInKDrz<@i!+j&_CirLMWYNoZhp?H4#nqOQBi5>Tcy0)iB;-UaN(f^{R_!O)Jlx1nh>&(P~cNWnPtTshs0c1 zZgXafs2@M4p_b~h=5i&!!bx5KVyPlYY@!%9N;3)E<5RX+J$2VouYg)zJnp8x)xBYi z(807*rh1T7Zi&q(C#NX2)#3cZq*_W$B=C!20iSWeD!%Zj`(0Zj>?FcYAw%%3hYV=Y;WryqwVr$a(8 z;#OcK+x9+Cht&x60;5mOjrPTuJP1k_Ll)bi-wFan0Ch$w5(<}v-|a_=MRF0b*VmcC zem+p|$@6FWwx$OG{U(U@6Oyz(@(OyVf)S?geu$w6*7?h7LIhb;K9ke={fh~7g;wFZ zD|w~E3X?5uli)z5%y%@H9MK&{B>7}EgqWN+M3o2;eg`7>iVdT}g^mS22KAmI0<|CS zzZGgL+!eeW^$YU_>_gC-qUYnpxM zNU?~A8>OBiY#8X{QXQG`kb5~%9sHdbIVsmYhnfLUJ3{=6%NA0+=OFDH%)lhYn2VNx zb;rRRy+?Hm{&26q1>d2|^e>r%V94A1R=N1wQOC<2?dF5y#=8EK?$gY7>UpImK%Gv7 zvt-J4?JD^qs_d?5K6>n~Z3niyS;-8%vQG!*@8~Apw@ZY@@x9e_x%Cs^>u#Z=!JYR4 ztulIKMztBQ727)7(!b!dbnM+6>y`)btoE#~lP(H7lD(;Q@-9p6fS&a*N`=`#FKLKZ^)vr*3Xc`qh=_h7P!XR&7+Lxpc zT_MrR7R&^7uMMeOXdeVWxBEyFJCEJ>3nNE6Mo!5EBhu$B*FFDp`8tihf*$x?zW&1f zUjN@lhyPY!{m=5HQd|A=)SvU(kkiq*n>m{?jKQhR(v8G}dZ?|=y z=czmGu<666E!SXCi4d_pz)Biw2l3zrKhWaO6Ia@-Xbd3-K`vKJQOc{i5I?~fRD*OX z<28{zc8Vi!aHwi%bq!S0rWS|7b##Jq)2nvV`t4-OJV}x<=&8b^K_-{0o|el~P)B@f zm$a5zA3*+gfF@7g0;a9;#~KtBWX=3h%ipbrE*`)d5bxKJiil%i^G4--NP$y z|I)G@$L__+#`!a7A#9_6!?m&*CD>kdhYfKrf+*c>?t48g+_~<0DHkx?u5$l_6H+Pb zE$?TRIS(4AEMWB0n`Ourh#m$)wJUwJ^>Xr~cZzjZG~msUOsyc*)w&5y(?0^I-efrK%KU zZ49MS5|1!xIQD@A=@XIu6Mtxju4(@pf0&vd=Z6$oLcoJ3h7OuN!4KGcvtLaoz#VGn zixtw`J1XE$lyN2FrxhhY3IalK8&Nm%ONR_>rsF_DwcWWRO!+u*{Sid`Z*4a6f{&OT!c_5v1Zu#P+HKcGE(k(A5OWp%3bPJ)XED+o zbT9GO$mM{FDk=OQ8QqtX8)Ku`uaS$>e(OI*F8eCg+qJflnY0`FdN2^?J%6W+g#&+2 z*91>}gX7R~wqGMx`jI-5#<8UzT%R)?N1V-X{GkW`tQRX+^e0A``KYrZFo%Q zH%;s4V5LJl&)6Yp){y0bFQQvxu^@}A^u}?kQtS;H?**EXBbS~Wo zaNA$M?rRXp>YG}JXMy+49l*kd7+!?e$O-Fb;B_zKPeU!j?H~Hm371SCEVFxD=`T0| zbG!EU+VkQY(;J%}-^Pxfjm&Nv{4ZCJ)7yBHs=9?uideRk9RdY7nC&@CGR!2A8N){= zVfqc-`Ci8tsDGQO_g!EYwJ;0%#x{&h32#2~8z>!UEqFMJ?HKRR3ynV848SOhJgD?W*%AH=TBFL5j_~>zA}=Z z10g;z*`}&BZ_?@<%|ss>2r4{btJ4ftLYh`Wnbty{JNIw*`w#yuh-r$OnE!V<1OWQa z-|z3Y|McHI>JIh>{{<=6(J`|&bJY1|{<#7Fx0=^~h7;4C7(V@K<8ppgRXG2L;RZHV zwt9}ggR2y$qBH2>yC0B=o&e(H6-iPco#2#;p4UB;2(W^8AumU9n{V~K{JK@a4ZQ0w zjbCQ5^*=Ifyt#IEU4kf*x(#RLp1uu`N9-Pl8ddnj%Z29vluDrFh5wdD21!?l>p>t| z+?AroiuNF|c;c0^VoICA@u%2Ie=vtL&M}xXy>9Bl?Og8gw6jWGG$<}N#-1pNWDRkF zFKI^=F#xIra-0ZLVY+}2WSMdHd+H0y=~7-*tIm2l1{f%?RG$&4kC2>*s~nGncMn+j4%02+ZAo!1@GZ(KoPcO02rniV}_Jf8gltw4GAx4ixu7KoZlne#*Q zN$$@hs^DtJ?WAe+h0$YkgAVLhXUG(ed8=h#WWBZMnB^c1_`E@2^pI>rfl9H%DA?3M zZ0`;Hm|331c6lPq$$5AiSKlKouDO<8402E%Cn=@NNZoVk(W2}u`SQl+ww8uA(SARXPuAZy5@|V5d z8V~(DfJxuEd$%FNzbTzL5yS<2dJO)zsri_ex5{dC!_1A_Wb4mv?yAx1f>wwy)@CU$ zkIh**^2-3zQDM=ikJOL z*{^>Uc=-Q^_r=)ETJJYQ`&Vvf>vpALWwFQ$|DCPFun4z3rX`Dz$lIq(@u%B&S%z?> z%g}z!xdF?Wtt)%q$Hkegjrc`gLkQW5TZ{DJeUgc3^Is}=Dt|d6EgFO;*S|DwhxpUo zU3~X~dUgC&f3;Ei6GRw#_PuzBZ%~#(e+*zusiML?2l=U_;ulahR2B{pQl@!+(le5A zjRA7^yg0QgOC_pVYz;D4f3pBhJ|KczavQPdQkDe^C^1wCf8mlTh zR$y&qag9=v5=2{4c2MJfN#Cg`#HJy&*--x?8uD7ICKk^h^&+&X5%Z@*E@>q-xwa`2 zdzEzZaQ>o_?RVe_yQn_VsTqc*17aX|@&O|DjXYhz zQK?*Uppk`T=M6#(LvcInQt~D$u~ZtoL$Y?{!DvMeDl!AeKDV+qlqxDYgaf1gd^(@) z@(^&6!U4V_>Su{&Nt14SgvCB7^!75mcsDtQFQo)Q?YeL(avKZz zJ?Y{$7`U?(jMr`6A15|k@+#6!KKLcYRDre)adVKwFFm0`nq}sP ztGCIzV}C4C7+&BWnkOq=#`HOcpyodQMZUQsbEd>}f<&C(sCOptDJ*#IS7 zmInUvi_o>@%_+-Q2J6n1=U)9I_j!dcs;j&-Pq6HQzjDSgM=wQASZaJ*KCTwRaq3>G_^ljObNC;UqvrW{cS&`Gut&UY_K?Bf3AD@Q@n3mouUdm@L>5Gcv&yDi&8`oI-scAhZLKST-Wa+ZE(dr zg??U$%0@&V!VicysAYkLEW--P!#~3kjo%}?Jip_m4%Fbat{Yws*yw)3XtJzZHaMb8 z+{jf4=asOF`ars84nRT+sNxKJd$;eYV0#8+B^&UgSD6pZDCLufF1I)}bR*6#OQ!*` z==>9Difms|Yict>EaOe6Bi|$^R}H6(^`{@`u^c(G3IuPV)%mtHaO+fR-Y=J~x2;PW zW*pE-)=YZti8VrHsaG!9yeSBA3z>u55b4rxW*(?z0&g00+rkqfkK_OpP`%L&3HS8Y zO+4d=^|B#!za2IY(Lds&!#$vfI$mPyB zl@2PGd3wXo zQA=x&jD*jQTw)F_O+ zK!FQTG>>^ah0tsdTY#+ypF9ZC&9QvT=^f~&?*>EjrzM&D!^C-SI*h1lV9O%y&mC3o zN)pto@1;*D=KJkVK!|8aEZOIkkKRe$jvQu=3Y-co2ABeD0+EmZ;6;bB*Ez$OoqGgS zo6gwC0|Sh3c6LxwJEm)3JrEfPjF+++^D18pygY=bC2$)VWxxudJFG{Kw(x|UPntc@ zi0^30big%70EtC(K#8D2pG|UQPq`-K#;|yqCwFEe#ee|Mi$I|PUyyCYPfrrW_RMf* zbIHjE-iN@W_dvFVD>%>8h+8H<=o?HqHlp4R0Jsr>})qPrNW-o$;@#{$&jX&OuL% z;8{s}vwZ~>F}xIo2F&c1XODxL38A@EN-|7!SS3IIu4m5b-RXOy=Ujc@Dgzpdm%XZZ+%(*g5!qhrk?6lq= z!F!7Uy$>V9VD2TPt0jp&t*X*7=N)(Zt9GHBT4C>8qYw`XXT-Bta6MaAfbLe&`TF&&QzKLsV-vc-#Z z@k9TZhd^>cu0+aEv}us^F^>9Ekyqt<`uA(3OC)HIjG-}d#q?rcpe#PDtNZ+8<^-(O zlzYRS8jQ-Cyr6R+}qjN-kQz#+TIzwF*Vi?s2`8}k44+VjQD5< z@p8Y|E-~C;qLl2vTMY;6*C>xQwvuB}89H7Z!ARP5$@Ik}UAw1n{mT6>OL( z0M1?I%a2=_7!1!Im1%N*`I!+SaWceFWG^>!kiWGoa_U;+dA+NzBTRzTzy%NS=q}}q z5s4U z0RTLG87Ust#aheeHz*(!@ci{f8-=g~x1chfPY%Aj(DoxOmC0$OA}~Q|qqZS8PjOeg zLn&R2qYkq6t0sNDVrnpEp#rFAXc`$@SzmV~O60)E{s%kGMZ#PQIfSGd^&U_gY(Lix z?oiWnZHp_sc?8Vx48}@lH&7?G{U;aPfP&oSZ|KFq_O}e4$0et)T(P6w_-(Xv)AAEZ zSoG1*>c)N8Nz~KTjG$eRhbz}owA0sl{wDL3B{JgRXQoJXiFZ=7pvgR zY6d=_@Md@$XGJ<7I)G$fJsM}74uH(zO7LcvsQ2bjjiv+UmbJuH6okcsM~3)TAZJR} z5?>N^YIeL5j`Z5o5#WZy1C!Y%fWE;D>oz)`nXjYZE)ijLm-fmAlpU&+@?_g_rv~Js z?JT$5WBGU|`)s-?v~V_!rkL=aU-4V*N1d|X)LJDiMEQbBw5;-LM{**Ziu(N$BvW>W zG)PjSc;kk0cugB7y|$;0F11aG)aWa71U>cwJMG6&uc!_5PcWzh6>NqKD$Wo-xKQP# zmnC*F0j?DBHtQ@ErfH3w_g42DZD%#^Mr-iU446|Tp+%X|Qhw7)!>gM=i&X=T*dAV! zm6er8O2I?I!Cb56xO@f&`;VRV_rc~T-}KZE1_543_!bPU(E;UR~JWbXGivKHh_Ko{avRoC9&`W zGk^7YW4Vm}axpk9+LQT5WiKy|Un0e-{ zgjH%}O;zoagIpG2ZfJoSLUO^%eI>oKfChHTwKQti-r9nt&U=uwF30BM`mpBT%w`_i zTDYzaTu%xW!+dF~d!i6>j_jCo(lZ;@>k;|D?2#od*df26C_aPM`UGvT}apPfJ z%{JgNR*aNBjA@m-C2svR_HI>p+YzdZ;83w!fD-Kh{*1rvtxYybn4 zu(8SzRa${1 z*>qX5QFawK45fzFZ$!Cr;c8(N}EqzBa|k6eo9esQ?UPBkh+M{88v?z^2iXiZDiCXeln-V**0U3 zFg31^P~?$v(LVYWbGN!O&Rta??2JBJixyYH7(xgh0XzL*EN+$9HMuV8eQKyUo3k3V z&~^;13|&ZjzdEb@&Ok1HUTHs(`&>-*Hjt#!r&u9eQR)@NVD}sPz$vX>zL6i})Quod zyG%u6^+e6boRxZ5Ic=Rr@ig_sg4;7*z9UwLw8?(dyCF)5^&%*AHsx2+(-lH7_LzXT zIN}?j*z@yqELi!eCrge`&hbJUd53+d!CQw*D_S2DN`okMeLGYE#|2VUlENEI4vFvD z#sr|W{R~QhA8>O^3BXR+JIyD=%`-yJmv*LfNqH)aUj=+&a{HZ#D#RF0?72ew_RZB* zSGU`Pv1^iM&i4kqx!1#e&bRlC;p_3~@fA4-Cr2Bmw$8xCbVbb%fj7j@OL;+$wzg)} z?!t=C+e6NerS>=1`}aOqTW9<0O3oGH&MD%Eh|NwnH|IA)8?JU&KyXZSOh`(Bp`$x% zXKUZS&sxXlTLxbI1)dCD!a+ofO~^>WLWDy<;cMQ&k%$W0s-O7c)=O4XDd$&d!|>1I zV~#xuCthxc2m7zEnX7FpCMM#L0ZGP-V*>&{Ig>;+tK_ko5Vk>*46ar@KR+@Rth^}I zumKwN>xvUIR26SMZx8kcv5k`tUJ3bRH3#Nc)|KjiwdW@v4+wIEh z%LQC&L6$WrJ6tunn=6oaT0}KVQhO|v$)8)LzbPWTB|7k^UXl;zVW|0k z@$EqMN#ls-lYX8{g(-M}4#x~Qbs#Fq&Ii4hj6%KOba^=G?E8M2_Skocw}2xuN%0mK z0gv5xSQNP_nG&l&QDTvS_HWL#6a9UFP5{aQmem0b(sFYnaQu7kQSiUMfsy_-GE5w= z!t8VQbbnIOXWR2+qZ|=db)bq|rM=s=gQziHm9P8B2%#L@yc)zEy^k~ z4g~h{Vs&*h>3!IJl70IKUIUDM$$ZeaweRsx6i*u}CDY%6lZcAJ3tCUnkO!E8j1d{H zm3TMmY?Tz@BY!SKKt`g>0N(=nn;MF+Mj)%+K`1*f4fW9M>CXXr*PQ_wN0#YjSKp6y zsn=>1-uFppev&9-T;9B-rJd@qOApmogWd3?-enpS`niEHpZ8(SDg+W#v@8PLR^7R5 z4yP*ds;XCli7WLRg@%q0DBzrt1_>rMaaEG$`IV!H%$-3iveQCBCiRck8kZ&&5RxUo z==-H7Qest63Jry$ILyFJLHZ8g-Ap{O@$9OYUK8fj~bho2g5?edRg>EbtVPO!lAZx= zp;f{mxl1=-LYCD?lRYKcvpvaRM#B_bMIkHn6GU68C$nSZUU`;JF#Eq0Mgoy5+TH5- zFF2($F*)f}9^>-`v6wI;2KK3{po=Uvmd1hR&vPq_Mjm8GWeCNFu=gGyjDFkgIEo^6 z=5xzFoj6nmebTPDF1^r`*xM_L1**+;D$NN516PT*fWWA$DrVH6w)B2Mk-`c5B$4AS z1(I*4O_lP5zcYi~gWk%<*f?a+<+GWwwvfYDV(jk!33uhU1b%4^09LPFiv4I93o|xU@B~S??VG;vf+~&>a|jn3ty7_ir_hM4<;Gas}>oJTR*P{+l_=B)!(U28GA z!@mzOYDWHmO)n=eg&ou~B2yKLDcN^+S`i`*GrrCum8j=Z61mD#9`{dVpiifVHS{i% z@uJr*87jy9{ZQkn&-`9)aPYq{@grrkNSV~1(u}*CP_|@4(;uhKQ>STyeCags6QVEYJ9^t%t5a)W%eXuP`eB@Y)mg;Mk!jd;twz}(6<1}PUf6HQ}`72P`-_tJMqj6!)r&JW<>m5>7~y9=yAu6baR9bvGc zRrBU+0`AJQDgn)&UC(T9lTmoCrsjKH4r=Nw@yw~mD3I6)Di^gvtN1~g#CyQ>@t^h* zE7LS$oJHAsbMiaFo?h}|Q)x>jDL%gm&hMQ;h+o9(KY=ilXfloRf|T=Z_E)7A)*hBZ z(_dWD>W~W{xT>WsgN8vwP3<-KC#f4Zh|?T7~+A^hyn_+7+E5#PGXEO+#1zfKB$EHB>L_BJ?DQ ze8%V_=p-^+P9xt!kx#z_ji9`0RfI|bsxb1B^3SoZ^s>YsSATA(mFJULV-th{{pnJ4 z>%38D-W4U_n#J*F>fc3s74NCWUFbWDL;5@&+@Rq52aY<%*B6bYYiHjJSP1T?1^Bfn;E5GsJ#5@ z&IPQUq4-07o(nc)KQyb4N?=8rB8dGU!5 zN&BnLrVq2?58W$@XkfR>VfqPS_~l$U@CSHHZ#Y>Yg(II=`XBq~R>SIYTw%0gTKB18 z5HBv}$(g*oc1?}uG7to&ZhN58Hq%n|rjB{WK!@6}L_qvYuH%<# zeI^O}VC`$5%*W-~Faaf3aZNJ8HW}|1)4Y{x>us>5H9*X(Srb@ZHCflUYl0s3rFqgH z^<2i^0+2{G#EQXk8~W!0cM;z*=~5kvLBq<`PdM^Bs){8@gNd$dhz9J8vFiYiT9x@B zov!*ROrMlVVb9mlEq_jcrr47u>&cWYftq3_A=ZEod=Wr&EsS}odsS4e1ccRpp`V#( z-=LJpGWZvvx-gAb{)fEM+QKuS>s+p2a5Y&aAZ5~FtuOFB1 zRprx{Km+}3wNgyTQf?4}9=>8rqhG2Yt$-FoXPfiAD@2FVn^G>K#+0aXO!g-{U2J8J z^}$o@?M+LiQKe8NK*ITPphr3UVwpuHoT6$_anSX|DtwllrJ0rkiCF5KPDU3Ln%eU> z#BuY(l6LXmD7RJkg9CS(YEsjVA9=V;TuJ7;wAn`>%9D2{uo_<)O`7Y45^);qv{%Q1 zZ_sq^=!x7BV5UK4G&HAtiSoz9c~}wTonP6YAQE?oL1xDaFud36ojejjw9l{sdFf(9 ziN>Xh{F?W@sNz|#!MKc0YkL+CyQXtm-Yomx8L8fX!|`5rDEXwgE}WOmh=c1k-|+nq z22Yk194OD$`5R@oPQq(%-` z+Vhfp&)8}nBH5`4-0sU{gbsUf+_a<8>Xn;VFySk@+tb!6 zArONF5qR5KWor#v(}fk}EN9Dk@XmB13bT!5@_YrYIH?ziiqQ%f@r~kh59ClgoPoc= zmY1lw7EcxC;v8WN63+}C%frq*lyj#svO7{8f;vp_#fI-=s^lnKTb4<~bp+cuQS{|X ztpvBUg9J+JRc0t%akG}`Q(4(+daUgt_7~v6O1<_O=~F&iWNQ9@_1vt1x9W_)C#YlM ztq>9mpT*6#ChS8wCKcIgho+#2Gse!$Mnlnv<9d-ZTqPDRfavO~Mmwr^05bQMmWGnj zRB-CXK_|nLnr#BCBwE1j)eP=y1qO~=P|s%7kG_n|X7}uk90=xxagzJ2E|5{_V!KkO zPPqhNng+AfBAp0|coT*+q;LtGvy)s~{FQ*X`0f5sK(s2!XXX2|%AqqAC@IZLDq=~? zGl&%C!QMmN`}QvC&ibfAO??B8c+(c%^VTm*VyllsSi8*IMO;sCE8Xgg2P`&Zd>7%4 z)ZyYA5#~n8=?&N>=%4QMle4CL}bq%B9M~G)04&lm1Y|H zL8=mA!W9?s1C8CDZaCML%rRlK26;My5pDuM&kK@}xgMt`~Oc!JKW4xqEV}LUjq2Zo6`GIX& z`Sot^zjuR}ySIdIFW;o5xz#bks>cXtI}8dcrj!6M?3Z`Uutx}@{!u^^>GT`l4q zR>lD52my3Sa40a4URY^24em4#8#^rJ$|8!>o7@`NY1Q1`Ma3|&a$Dq*zry-P!}E`# zwF(R&eV2Vge{I5QHungbjqtL|I@U=y#trI-cPa13`=Ir&F zvXDiqDMaUPlj5kCr*#Qwg?p!%%IR@za%rS9Pt`KWBip#?y0Xa_X5IvHAT zPMPk%m@qz^_H5Dyu>!3Z8){1p0X~=q`39sBF_B`%2lqoJ^HNN3ts`G}(#-|Gp9n%Vj-U;SiZLEn& ztU{I;G_O{%PaZ2j#Am9qjFisj)mWoOs0yt-Z(7M94YNiUu+3^~|B~EOG66KDt{?8d zNg)^5(D5g1ltxR3y2&$9xywdZ$mNTn66ej>TH#2u^QMH4$Q3|y8~d|92S!j`NW=5d zR{#NRc}#UL!4G=22uZ0;bTOebFyB|*c5B;~h{1Dr2FeYZcboGK=3$OM$JqwL+sZp7 zu)Pa*av7KK71ec9fwp0>cGW0y06)!cGJDGlW{x{qxLtL--cYsBZ<9j-xLL9NiAkk4 z$r9~=xWno(F+ED<>Ft+>`CpLj*0_+R01!2_h_egb2z~TE{8+#C^v4yJq?Ww@q>(CE zMs}|k$Qcg5FXV)Lthr8-L8VDM+F1=3~YV;i^I(uI}1crtI_4JSpy|ePr`YN)2Gig2hv? zfQ-cEH8lG@d?)i#)gxNyvuk;TgVGQ<$RX2RQ~eyLi>tt6^rLM9?* z7>vk#5U(Txj3v_1<0aE6kK#Vi8no%X_$5XaP$@*Z`5!;lIfmKCWq_#5l1Fy~N3Af{ zDv@KTLRHZ!44=$CW0%rEqG%XkpJB5cq+@_=T}Pr(?*%JQz!PY!7;x7HAjWY?M2Jq# z6&cf}1Ow?pddMz5!sYAyn1kK%RR$b~`#eqsdVl1*dT3z)ZB5YfFSw@jWMq@#GiSV~Ud-;|}*jriLIWlBri65v8=9AXr$its?pOegcqJ zL*acY6AM5@)74nH2DnXPJ8eMa<~P_J_S&A60)wcp%yr_V~$&y{1klA>EJ2_BlIp*ZWKfwB6X31PJp z26NIVJKlpOhLkW?G*QLniyU#pud*a&$$7V5KP8;ITq3G68qP zrY7Bn>6h3|-5f_Akcl&=M0~WsY8~VOa;k#iKDfy!cq5AIKJ0Wny$|AaEJiyHuNxms z_7)VebBZ=Ck0Si?k{&PyF#E5hAOs*|UHJuQN*iTprgBGC=Z{Xe)uk;;k;7@26iX zwWvd+Yc*^`9aYw0gn=yA&3H(IuNUb$_(&8anh-Zpx*L5il>U& zD3Hh?Tw-Ldav=ZJZHh?W6r|@6GMw+otxt>NT{_E*EOr@?zU>B4{weYAQMH-_!q>79$K(AvSqgR7~*DvzUn42R5u5!unnBreFI z3(4zc{%QRu;Uit?`i~Pc3zsZ~KDFehe1&yhy5fVh5^9wsbi^E*W44{YqSWK_`Sd*< z9;Q?swRloE6XWBr;lIFln|^BSjusA!p= zW{MW8%f5VP7Z-!3R$qVIoa`INCf0OnxS651r_=LeS4X?1f%H%RY+S7~wo;uQF^qjD z6zA97OXg{;Motr2ju4i~<KE3k#HHGZo?4~iIer%UfO#F1^L6?xSrsd;wGGjsy#BX)^vwjUrQ27f?~Px#?O z3Gu%atA;Zk%gIk24K8V}=C4(8JohCG!FDJzl+~MIa}PBpyut+>5njBjwF2SYag*|H zo>K4RGlpl=ht{mIosQ0rk8xY%^AY3v$xF{x7w{YULeh+M-a`V2$Hp@Y=SslwN9}>Oa?1U+PyE`D8F2jF8pT-7t$=mP?L*R zX&h0xGs!bwvboc*{0LPtqK_iHY;u6O^9t+LV4uB*JxFk|4a6HnZkgTlDP3oRAzFp` zhZ^BPaZTmcytzb~Adg!$rWWE2QVRv)-ThKF5B$t&=>b#y0>iHTMx0D3SvVEyt2D z@^R?i*(EhN#2L4BcY*`wr|2H(rmBAtlNV1Fde*HvnP0(pqp9mCSjM-4dn?MX2?Q^@t3=kz95H%P;~(RiS+Nj6-ERnd;XAn#+FP_;2hY4rT{Mv zlc6}~1^f`+KwQuVk#h1d$VZc`{zHmULa0aHUI;o}j*`JFAgAR|;2}F{oZd?QPBarZ zz92Ft6|PQt?pxEKRKz5GSq&~aW@D6V7}C>lpCY*28uMW$RaORGg|^;`5@tq1=_J~Z za9y78fb$_Z z@cPi62w#Ca%gFci@q;rW9Z^X%!Npz=X_QC&Sy zM%Shvx6>HO(f+jJ{&WuYDm!C7Z6Q%reSbrFCksDQGqZ|vEvfp@dnw`H&P#^4N17Wk zN~lyg_>V%2Txh;n>-hR|Vfh3_;><`GQk^t^J}3d_0!Y}^JMmR({`aeSkq$W7L*a`%Hu!`;}t8k045WSLiz z>P!jQu=amE(xr9HvTpY~93SxAX6WnER;e-8*^Dj*0|ZAfx|`_x3l9?+2&XqEH^C|e zdrM$e&Pd3DTlftmnW?UUZG$?$++M}`^x~${F+*wf2bZS_pB_bI_y1E^3 zbDogHRBhbCF4L7usPsfrE7b;yK^!&!ytTS5d?{Tf=*82`okdA&C{?B6{$@`D8(?sF z%Y1kaSPQEhfJiwDawDX`ChXK80|$d&Hqps9hj-_D+L4|PRscZ(9~;gzp5aiEYsUnS z|BJA9iqb5Kwlu@GZQHi(KWy8!ZDlw!m|@$tZQFKablmE`_g44ls(Lst=V6cYu-907 zpE=hzldOyM*QxtPVop67Zs+IC)GB3wKBYX;=8;M8M4J+jc3Wq1KpwfuD_-gPaa~P0 zgkP8M5H^$hHOnq~dV`C65TtjCbkgz`{j!NH;@EBW#SJ2O@<}dYb!CUUfX`I8T7t78V ztjwqe^DA($%1~NBqAkSNY@!_%kOPEKLDJ)MDfRUTpm8G40Ss9u*cx8XunZS<3@dKEjO)y%mi6vxA@8p=E~Bnq3D`9KDWlKtOgaJMGM?ON)Fq^M#QUCc-2;3lkop zA@}f>?&+&Cy=OFBa>bSc()SYDs$cxO)l&1Z75r(;exm(hE(dzFV}TRD%(*ksV{Hts()7k0l(QvC0<}1d zeG0)4UktgmSaQY6{UIS_hjn_t0V^z*R76iW&L!J&kI-324_BwEIX&~&oJI+EouopV zD}Xw}`{>sy{rweJqS&5a~6n-txLF2QY>Ju>f~Jh%Kcw?U1T24^DH z?>66w;rBzaPT0winZT7ELd> z>BKv_8l0Zq|Fu=ui!;w)zZqiHtBeO_Iym;(7PQ2LJo;d1((C8YXINIqBveT!%>O-{ zG1Bn2x$iUQ;bzeXp`Mi?UO+Wx?T9(1F!FlxKDL0})yu1E%cn6E4rSWuYq0qX8)^At z&$lfA-YGjYOH70u4|xs!#OHNnx}f{zf}6%mX>k@Zd3?Bu#E-s=Zk`G}-`xO5s76th zOE-U=!^gt%7BhJ^=U#^9CXi^VU`HYYnmnVSooRjQUf3Z`N4qK;6({G8xl<7J5t)q~ zJ1RmKom_7^(;zN!mCAzXt-R|8X>)YIVIAorDeP#mP(XFUQKkIKLEk8H1$;d%100@u zW%;5FJ(uRfndu{d`S)9Y@|W{Hq3Wz>I`#Hv(u5vA7MWqJ>B=oKHWT3216%6oOK1;K zjc5iC)G>8eanbd>%24T6r!>hAN=Z|7nMugX&Mz@|Ot^@{v+Z@tb2G_+xYdDz*B#4f zzYX+F_U37_-b-FfIBcEKzK);g53T=G2^CMO*TiXpg^lOhNKa*e!OzcUnV9Evg^J#d z-sL0$bCe0b#l{7Re_-u&19v*7T_P*bw^v696zBqKYTnx|IS6csh6_-f8%X@)iUWhmri5HI zo7zLO$3JAUVQhh(A)!JbiU673$4_U}J}3E^?1(Ag?r)!Ae;XjoSco%jl8QHT85KBG4`Lipl-q+aa-&0%z!E`#!L@>JHQceXGMXc)O1&0lBn>RuKgg= zo?2sI?plOJ1!GCV4u(IWA1l3!K(djg#$bB!&|>0YvJvY^^`{`IDFd>sjFV(RqB<8- zcab+xpMEGbOiA9F7^`5yr>>VC6p%;eacM#XsZWx$-yAAq{WgX8ik{Ua)MkANl1Yi$ z%}vD@R7oTr+o%c$1Zw&Kx;R;4#T|D%6v~==pnRG+F9^*@o4{TpoUw^b?c~N*y$pzz zj&D#j*|atiNAB`{f2L`%f}E%M9bG?XC2thL;?nk6mX@8a1rsM*-j{IQozB$A~t1_eh4Ni z2DeUnzpjU1L!JD>SHsy;_wD5O9<79pfd)U5P!O~s_PQU3S!@&~ADQ=syP%{=?Kcr{ zWsW29k2%8fUuc1xhAYRZ3__hemjEEC%649kULV!^`1mkV70+K39uO8@OzhpAWiOcE zQ(bsberjs51@K{FZbcm*N3BGGa_gti*W#AOpr9XX|M)Aqz;Qp~_pmfURW^X6in9z7MR zEpoN2_Wl&Unbpeyg*NWq@Adk165c8Dcduwl>XWDeU zT)+wCyZ+A1=#6Pr&5NO9oO06%$&>8HE4?@;)4;ORy~f58lxHxkwFLq!=w~^FV8$&j zp6C<=b@?0Y*=ucX|4z)Eudk$!wxrz`FmcCBRS-A2J-C;ahoX;YZl3qUd*Z#^?u+f= z9DDm}aK1sFt7>$+J+lq%AT^!YJKG5M*By8~4D6;rFB?6xgYaub?LQe)yYtMM4|%`# zGtV;xX6B!&+WV9D`Dpy4W2GLJShVTCfAbR8UkaIzCu_}ffdNSsVA5>429w@3&X*f$ zr+Ni3K)T;lE{IGS$0#4395jEG9!?b5cys}XlyOJqlHkWJil4_2&gR7q54XcH!7eWV zzyIQ|#jVe};&rk+U$IT$Dn>Uo7MUS9r2}=Dy;YvNFV2cN+gdJj`TejmJ2E_tI|867 zbj^3~o!v!2nAUzQG8gG|uv3>C3LPGk9J4mFS5KF&4zEOE6lm_i^F>-8A-p+_hF6#W ztanAQ>E@~P#?K@(NMpYXo)o!n@&$PO-hNk06dBJHbYycj2SFwka*`&;Rx^0b{|t`u;nGk0+99Hadq>Vskt|B0X+$MWb>BBIBr?JgZRT$==)z0WVDdS4Y9-i)!z90TY zCp-_H)A?P|C))7I^Ce}q7W73l)aK)t2Yb2w7U+&ZAOdN9N|RtQHZ+k;y>UX^$bG+P z9e4gG#i~SGL!q^ArWW0AN|v+q*0zQgB=$)muR@s$Pw`x=2CC_m>gm#gACX+qxcqyY zPF@Xe2>NjaXGea5oM-^Y0|lIMtnO{Dxk7Gz0^&*kr4v+ug|saTG0k!T=Z)fuFW}5p zi$Q)c34XZrn!SML5Km=2f0uGgex}06NqZS3SG9La#l)?)l0~BcmDneHBfAUdy5uH2 zmX+hcL8jvQ(l)->wsyyyz6GaVW_YeAZ|xvUQQG7rfRfiYr5#^CbcCZv>aWem?C(zI zrgc|Fjt4efUwLf~Torl0xbWU~x-bV>t3WDyL1&ognebiL51B2+LZyL)m-V5LH8y|0 zr4sM`gKegTU#Cx5F(|6WEaxmoNbjo5Ea^Mt6{;e0b?4A0Cx2mK4r|BJt}k&KTCO>c zX5cYvJ#OvPQ@i5vWwH=JX{4yBDy*Na72&y=|TRYO{QDtzn%P`u-0I(oxG>(~` z5ehVvW?g^DSBDp7S3Dfrdiueu70c>euy9VUZ!o}b3DPzZB_LmohKvU2h`Te``Y+jO znNDqbw7%^W>HC2yz0g9e!i>i%T67tW2-E^Q92gt0?a}n>FfQ294EainTz?cV)ENETYgEyQTaq%kG1J(qbK4JWe&yHzcTf%|@_vNz3O9HJ#9~>6@Ge;ZifzMK1I_G%GaZx{&%W z7j^@8!Oc#ABFy+|BYQfscW5fln&_UHzM2-g+mXBkOSc^GLM;`SoU!hj^sJa6TJE-b z#fHoxjlEr#yfvsa>T1R-)NP`ZAWosBT940bur1i1K!&&kFY>AII%$xxN;E-ZVl}5o z-q;pnUFnoNpsI0tn$Cc3_k|MZr(f5-j`}D2DV+@m#agGUqV)PWz&(klMmDj;CP(Gle)2z{fGj>C`8y zu+5axfLs4d(jh<1=6Q~g)cR<1qFRezz(1hfFosNa1P)hC|0(naP&q$-43JABvVn`n zaf(JC7~AM(nU~iHsuW|uK)i-pyz|q;k)!}D$^v6Nngk$5d!FAu^56<`@ygY0} zW>?#BYlq>8M=rlo=hwUdnI%+Wwys_nAv`ACCI0;}m54?zg8n4#W>b% zXVYwCI*~$mmy#$p=8VZwZ(2cQPCYiLj|8rD3zop9A5EVA?ut8TQ!l%gO!vI(8Ss-+5u$@qt?hq zzpUl4e(zPM!+n@pGN`!Erpba}3|XfJD$hzBiav;azzjbYJoJyE;1d~;WS?x&tY$F* zmXuh0BnnXn9;qyoj4mD0lc{EbREIG3)U+AY7`59{;7?Jej6#taCWU+&T!anjU&e*~ zap+*#3DHeoT%93NDgT)WV)OQbdIYQ&I@v$cD6e_!p@ir+Z3&SfEho`?{#YrmpOcfP zss8Ia5bY)zc3e0)QQYSzDvXa!iXNBGd?+Q-bKCmY71skFir7Y|g()Vaf4!+bmZ_zb zp*W@$$~DWeG?c7y0l7c_MBO~xqR`vC&ZjrTI~a(aiQSl^e15+ukj$tP<5A)I=Jkw# z`zP6Y0XUgZ2Gk62xQfk{u9W7cMXa;J3*-6=hO%HQRH9w2_>m%TG~xz76{UY!i~J!S z@Qr1K;E=#4q(Asfx>?tC*O^yKm&f*$qr~)B%tFJ?w12?c#hOFyiNZ%$-Nlo^6Gg3? zYe0ahmLM|c^sGiw`4NIe%FDJV*y#4@e$&QpFV0^;HWtlGgzyoyj;uJ|2V<6(3g$QR zSUb%n4cnI`h#%KWQ@pw#9X6y1D-QNmvM=QxoEcuXEh4*wb|!AC9d1teBA{DgnY}qz zEp$x`+cTSGoIZjWAAt}c?JL(}jNr}eSnxGZ553I{5`{qtg3?(o)^?Hnoz%k1E9 zAi!5vn}@sp8FXC)4`84PoXW?DhPXRytyRnw>qiBSk9N2@FzPn`&0Ss2o|CWp2D8p; zOWC$nJGFw(*#<>78t@4Fq|LvqgVEfD>=&AB-W)R}D-8%n-G2JGc8yl_34{!#*4>}n zF|+DEDRTjmL-3Ouw^#_=cZJY&(sz5T_=e-ZEsb(CO&z;Puy^pe^`XO5o-{Gu86lTH zWP-5jiV=B9p$eY(t^Gx?N<^8-`#WK=+J3Xo%+4lE#}U3)^!mMYiAE?8XhFlHVPbA0 z-(xGg&7n`ST!~r)`la+bv=wyw$GB2iA24`xKzbXzF;^t3Sapdu$nsf)NoSw@PE`k0I1d^Bdxd7{P_}(5Z(b>^(|vC7(jN0Awy0jHU*}d3Zj!H*BfHtkwUDSe5Pg& zlWC$x^h;TbWF_8GXQgWl{5biaWuWps5K$4gjVmYa4zyqdEGsJYeZzFS-aL#|qo&Gz znf$XJi=6RsqgyU{z&N^C(5=`2KC0AyjRKT7w*&Vf-ULv$iF;gpwOm%D;nylkf&zS< zZZ6wi`Tir4cbk;8NJR7vlC`oUJMnn%+Jg9n@LP;`MbspX+^I0MjF~3tH3J4d$%FAT z+?CKI!rYs&aGj=GC?XA{qXJZfnE1Y-?#j?q1U?{_b9|bSXsSe7JrpPe#qfqCc!rOM8UDPDqqUzH% z)080moT5KMUJ!21m{L~#C`^K3uT$8rs?5)xd=EwNF`yo;e3ld4JuMrKn0q4(HO_70 zMt9q@PQWO#iL36Z-WTdB`|PKjn44-23i+MYk1pW&{2yY4c2Ep;a_+N%7(8n2>h;ty zMSqf563Z-yN_8d)Vc2oBhre_k86n7w+9iNV2kh*%`hm7;5(Y zdnVH!WKmNYh2l$c7F!X^?&Xl!fjs({fm_|nJG}oUlJqE~3*3gS6mdsdK1m6{i75ru>EWek`C1Jpk*RvM5~SE-0Y-F?>{W=~ zgY(*4`_T-u3{OtVvMC+}&OyP8DGj&|6Df)x>5;T>BI?N340p^ZDKjVVs7?l=V@@T` zLCh+aDSrld>~fy%sO-B-6xDF(ebA@KTZ@eKzwoh&Cxx1H&RRb;^%NAShIgJWmuHi*n6V6xc8y zMDO{7YEuO0nR_d#AILJxE6?RO%tFuED6%DT$R^usEB$H=dY6XtV5P-ZIfG)$Gek&& z#zKL$poUTuAAkyQsPh`dBg=BXWKCpF90o{P0nI{$QykA5O|!Mmq0?QRHB_4${3 zoRZPR2SC@5mP++bG&IMmt>nzCxF=F>1qyXk(>BV1NoHuk6*U>df}QPgNsMLg?MA640*05N63C%&%T_fj9RTcn&RE`Ke@3=J%waWU`zA)*g2iJI`_wAymXCjA%{Ni;u8j;|yOg!v zo%gk&e@qk^XsjOwpYS~LO)C@cA7Gb#uYJ~fBUJ&L>DVzEMdx}_R+XHhGT1Ek0|R1G zZcZg(D@kKj;=YVF`E27*h`B|WJwiL)ngt@B-oP37<;v}v<*4c z&EHL8mx;=F*ZC^oR;s>cmF4tiSjyH;&Bk~I@Dm24vixW+@l3rd!^m4aW=>7CQqps% zN>Rn-!kQmmsIS>9ii!Bvs+$XNXgj~jx6fuymtCG_*J_#Mw5KEyrLpn^9%U=lUhSU6awQL7o^ANCIWPMhm^06I$1d0D`)H(a6kZQpy3eXD3M$nF&j z!-$bje}2aD=KMWE$JR;;b%=z$rKYEB^#E*>5mB68au}%mkXf&DF-ES<0m|S4Ir>L9@d*lJZ(WY)O%{$==b`Xeu{$0l*7&g=Wrpz%9I(t^>$LP3 zo9{a!)Qw69KyOQdyl4DsZ|3l}G$LG2Cw(6fO-Mh5abco{wXlhdi}e{}2%#CCXn=4i z1LsrIp+wcslaY~2q9At!P7e?a3Bm+j=aIvhjne3wAIEDkTqEaSBY-gh1}~RXWnrsu z{1w2eH2Cf;dx+)qV@KYv(4#Xjjjr)wc^^F#z8{xjqiHl2o(e4~rjYPY1Xn8xed=yS zhe$wbAH&OTYpX3UVPjNz&^<95ft!)>0Ysqsurtz8Bk!9!otF27tTw*1$kFZgSna{6 z+3&XChNA`3{V}@b-1eV*b7`MQ)(v%e-1EX+Tto-WuAZHAD;s|m_0@SqR_8ZMSAag8 z;YNZNtz=r<>`es|tShK1Qd?#ChCzrJqp+|_5tTc-{_P@+0LR0~livShAi=%Xm?xD1 z^3#VB*C}Ta^cPFVVAx;aT2Ya0Pi4i_=j7DbNb_XX08#ray|0(F1uXNpc9`AOAENCF zILE7tdGiO;@JTR^H3ljZz$rS7HUz3mN(8XSh>GOuHqs|@Xu@ed=iEw;Nk4pY{MFn~ zYnbx5qm2%|vB0P>$|>z8r+msSD0i>>25|Hu^1+^NiWWWHu@2b|q?}}sIVl?*v?L@F znbZgF?wNc|qz9*W@`W$Xi?WAo-n-1Tgh2W@8{pYggQj`Lk)oj+xpkSpAE+SB1dw6R zY{>T_hER4GaM8VKoWcxvHMu|YY!6qy zdDzo<8mOwT?hTd!kXL13;{SblB6aT+9g}@VIzgwww^GQ@IxlE7KMdO< z!z4!YnRaDyIK`10!L47CF$N{H*I1}j$Ti}hq~GHIY5qzs9=#?FMwnQ-w6thF)ST3x z5i_3GbvQQe@)q>DU==o(CoY<#DW{X6zAs=)2jk-^q!C%G4%yA9aa9P7Z)ZOOhoG0% zGPo+hzZA9)yhT2bzaTxa-&h3F(iWv)E*EVBp&LVM6UXcQoY{n~dMxXX_C+W7LxnCw zB|Fv$&Es*$k2EqO96oPHD>GGy5KN@Jo6}ki-IHv_t3HY8-cQow4T?i#Ge6wG==A9fw&qea}j-ly@OiD+1vXYV8v3A46| z$YufK#2H}b#4aI6gl=Hs%t|)*EhhhFQ>G>y=@OpzF5*wV{M86+-Q4uyd+PXHL&8a@ z$=%t5HMd1vm1`k@UWm^TnDz;sWCo0YsSIbX=~ZQ6*%DcjACjPR090?z-au^16c676 z8S$Y$4l=$&mLVrl#r!@ehJVd)x*y{9!s=cZa#`fc<_^iubBFo^qP%}rFVASd$8eu@ z#B+Lf{@wzdMy-^8EbLbJ%a05PG-V~SVmCz zupSr;SmxyVKwVkxEJzc@qDk|ChIUJkZQ+{l>aj4=9@@e)B+R?9wgLY79MVs7udZ(X zA)zdPpj?`gQzbq{DDhwX=hup*H<2Fioc~=C{{i%?|M(yS{{i}eAp-%?{%?2Qe_{Rp zbKfm#Y}x$myMPx1;`WGDx{Osch@f7XW&ybNzGeP6V$-)xp0p~tWX$#+f69Q3r*+e> zXHnw|g`SR&t%%Dt+v7@fCQ#-iZf=w4&g>LNWIDVPe%i@mHCYs9GX`NH_p3=IL1h-w zREjWaG$omNTiP^Y@jPd2Q1mk3q0l^33@d6F(8hVk9PNi--Yo(Xp==lz$V(aFd7%9& zRp{oG5nMrO7m9hoKR9o19#_&cJeW?%D2mloF3vq zSYKM~$LaVDrF&l6RGqg;2R`Se0(Co=F>$AOu&+rbw7*z;-DBCK{XczywVLjJ3u6SW z#{}k>1@cq749w##WXW@D`2?S*QUc07ee-3KEFk!Hv9MI`8Qy|*+n2) zYXj#&z4b;;3I9q%-UH;mjFfUt`VGDIN&tRohp-R^pvx^PymF?MzuNEkZf@`qh2N{) zNjr0Bo}NC{evHuB{lJQ}D^^HZo3}&=W{%0DIeTr_H)>orQTlON+9duaQa_=iC?&c; zeaFnxv<3}0j9TpeVOL!t1DS?_7o!MWfw<}|rL*37+0etp+faqN{HV(m80$kDGviuy ze0#YfW=<_`^Y9J{SQ;)31UtN5B)T#WS^?g~LjC(%5Bl;%!ZjPyJe6=eB{y2A9(b^B z))xft;wTrxzm+@w5`|R5f3!5_C`kESeEScZkj?OB*B5xi%yR=6Re>oU)Lu=x34z+YNPMjcARD%=1*l>ji=GB90DauD!8rI!9j zHHjSt)q(s})8|hdAl?7An*Ikp&&b)?$m>7LDGsn>q6T9soqIXG zv*pojzC!K&^0|Cppu2oh-hl_gj@ zWbOT(8OT40V<`xIoam~pJH=7^*1W00yEMPK^d$$%MNR@^XXF~FdU7gvZ_}vOhtCWZ zaazVGJsC{z)xhGTJ?TGq!T~#$$GR_&gv^6!ah;mD^gu~r+&t$o;{_P;Jzv}EY zQ9v$Pj=~M_B9a}6Xgptp4<1@0z+_+zZIt2- z<%6+RLWMvrTpkQo`^qk_xKAyNHNuC!F*)IAB8io%!QCsOeyB{0;%YEqB2 z>P9ffz{81j8M-@8!x_CMzGqO8*DL{|2NUt?Q60X@6Xg~#i^NZMxdm@3F1d%4y zkI)nld?f?6?O0_#6mYog#tk;L!Z$A|p)JkBKHop$hW>XKlW?!$S$AU{O5GD%l5SDM zsYg*xMifIVHS3K-?YAEOP|6eLe|M(97%~wp8tU-}2p}L4JRl&(|8p&w7}?qy8=2Vr zXE|MI**R=*ApN^G=xZgUGpaV^oq}m(Wg2mMsBeU}Zkd{e0HL32Mv-2mqKQ{}?75Lj z;vlD;mW4zU|BF~hMsO34_`BnPx=Ou**%ZuWpA@H1xPMk3_lajqq417!`Zc&-!;+A- zn}|5d*){NlY*={RvTqQ(5ZQT4YnW9+7ZY4r1Y1%I&vTK|9?P-cF{2qduY_%q%Lz70 zf|+$(gn%s8n@m2$^SV^0>bTR>wseFons3q=Pc zDGe18x`kxER)Ygo%`O)oF);hSiB^(jx9EFKeycjicoJDbovX^DYN56XmrMr7HwlWv z$#_M7?DJjs)c{sHK&|R82b6z>a4FCm6m#lcvk^mZHR%K=mIoY@%IV=1oH7V#ZsMGo z9U21X;S#->rZ}Bi`xux5lb@b782S6x(wR-<456-SU`n@?IG$MeUPQlr$CHJDVR7h zo08iW5KYbnjZ`L@iXRExk2Kq%(Cqe!9RK*ZvgJUy{Pg z#1rjuke>CfapYk`r@Fi3%xGulgh~ZmH(tlJXn-@RUeLX@?@BkFolCw1 z*(#-6SAp>7XeNqHU=kBUsWr9y#a|I_Rkc);_{!?eC1uXmVUFc_j^XE&#;^S66@NEn?L11FIclk&80H$GwCr zj?%p15wC3CW0pH$8Cd-9VR&eIo-@<>w__`Hgx@z5iX>Ed<6(vEI9bontzRn1v+^pNAlPJY{hz zI|Fy)vpVtLn85x4;HEvP@MPz?NnCO;n4R4@E&PwR=z#sWDp5YMlikyWu}E+t z+F+=VMX2MTc-~%@-w2nkn2m8x`AHzI$=Y9AcSAJ~zxa;GoR9VSTYQ@GMnQ)`yMzM% z=!cO(vSo*xmd^ViTLRW?@zUj0yCfH5;PF}UGU!&f5dfG~)`?!mW zVEk@Qsvaq*D?T*q!~qB8KAB4+$R9?j^k%e{biO$9P%+FCD?8|m^6(b$5Hc4#k_P-R zW!SfgQ{n~ch?E9xpFD%l{{6-2&7NOj!*Xm*sqD=i*Qkkefbne+UvKG_CCa|rag8E) zsB@LB{C_Sd&M}uaW@{%~6CL+$3i=_=F3+#&k^u$;Savi2%ts*CY?Z+E+)P0)#=g=y zx*f0#y3opA_M%!t&z99%Y|?@y0p1G8Q46uigP0iugu&5#6cEJ>r`RQ6{joxbzZ+PxmvdwU6rxwyz+;9#2pYF!Lr!Lue;*=Z+M24qvl+m>MaAx5#JcZ6O3gzS2p z{lb^+QzC28aQpOfum4m3ocW=dt#xflTfG5nwV=l{br`>*!Ncj3MiTlL;ZoD&j9jvJftz19f zsE!woR(Bqspu&e zOsCnTE$|#l2I_IE%Q?BF!{AE#r}qg|>DzqFmN*s{U9^y;sJ~pT{oo21Z6ugF8kd1) z4p45++KI-OQ3PDI;2E#$9bbl*dTLEvOlAG*GxF-B&I99q0|@eGO<0I{!nZ|1>r6*^E9!MU4Mctosz(2>LVWv7S{D|0gUEKu$^|j=Cp-T*JPb&sv z**4B@{A3m@Ej^DBrk?Wu-2I(|!yP^jkDuQg1+fwzSpT6fLCk&uM-X-j@?X8|IEui| zX5ex1FoClYVrH~}7xfXI?(2*i{`oLXLVu;rWfaPeTe`paya=+Iq6LuQ!TX1nlYNbz z#voG0gq`rPxr6rn;iZPrqw|N{GJBjA^5be3gOUjy#9~GYy3mxM?4omRa+yVv^CcY4 zscW!BybeKsT5>+>hf=6^* zCocE?iHJotq&p&pFMsTbM)HaT#*Ll~T1jy0NxXJ_d0CV|gdtj*14c*Uk2J#soA`p9 z&A`tBFS1Q+3z~x==^k8+SO%HeY7vcbSpsh~8fT}IkimH+WhdktL*hLW-vq$2yizj+ zV>vZNF|OvOh&FB@;&aAFT{wTPN34GYCfi5^567amgL-?(fEyr}1;-Q!b&U3fo&5uY zLdPM*6ad0^>~gMH1aZg-PHuxtI(P8+#D2iz2og^Us!GVl;l6t!wPL(4KM>-tx?pW{MoOcL*j=xoU!E3Dg$QHv) zX&-~YVz+!4EcZgBB7SGvqdQD0A7&DWf7ex7#hnD5PlJ!%pueMuvz7RbMB@`kL4j0dGwO=(rQkD9aSh!{fXi5~(M|@AXOqCMX%?3(Cs-^z4q!>^l zTpVY7DVa=@rs%l_F`h#>8O3ztf{n6Oh(Uu)mHS5kxYhJ%f}}qT-~Aq;*a}@oYzVC! zys}_s``TV%yC_>e-R}yRQ=tXG?tsGrS7{Qd3DBo-U+`BFBX+DW=08Hxgd{>}g5?W2ec*6f3;cfiXXsrd+ZF^&$1WezFGB0}?;D1e$(QRdp}Hq+pW{c|jso^i)D`F=AW zP~ce^h$@_agV~VE!Qk6#Nbq5^D~ajb!%3lc?Zgr_;mt@8f1g(qnn~u2sMV$Caw)RL z$8iG|&pU+NW%O#n#O~@wj>#Nz%&_XC(nS~s8+quPr}LlNVc*bfO}fzQUj6df(9>K7 z25V)*S(SJ`V~7{u=2%RPMSz;Hku_fvQ-aUECJhyOBkBG+eL>7_zD7igJ=BL8sRP?| zm9U7_6>N%RKJsJRlh(a;NZWqxiE_vVUdzgJFfcy!lrlMjfxXX*I z`mL7e{EgfcL+Um5{*x^e#r&YJvmFH!ke=Rg*!{o-q&>S8YDBFXu@Wq~@Tbn} zNBrLPscX}z9pS3!n|X2RzO(dP>|upqW@`WN;?CVCND4L&MBI=5!vQ1tj|u};+0CjXrSG3qKz_i;z;2jnt%xHvq zUklo$2l@{+$AjB~+OMM#5;;dCo1V$J)`dl$Ux8FyYS1F}V#KSNhp(CpR)@w%H-GlH z1;hSW9vJr2L)xcaN}juTc@p!FCL#i9^r8Au^yJpYEgTN1Z&8xIlR6^e?2en&_#L@I z?s`O0wAt7d=cOSB32HmEiv(;@F%;5YY`OW50S z8%LTJ(jBc$z7?>Xh(_#a1eqBSbsg$wjxS^N5d&A!{H~(`&JzFyV6IJ&nxBOm*`c7* zrXNZjxikce9azu|@!(|Mg$mkaMJZVOWU=<+Hg`d*O;E&LfG#HC_Yr3Ek|f?{)cndR zRzTF1+)!8AShpwfsa{q+=uejx9bTrG1~QV3cQYZi@NG-#((*8^*~KVMKTZ79ZJ$g_ zF)6A5=Y{u$qMmE8llrb%=EDpM@VLfA$QGzF1uRqCW!P?^HT?=|p4`6JHhKZv)N&&j z=qe-IFmc;5`rH<5r2d=(Sk^nMKBSWTRc2Y5-`BcQG|cnnbfJgv%R~+UH&5|2@om4Y zU)ZbVIf-BgW=BKb$XLi;M?CfdWO+=AChA`g9*EFATG51Ea3G&Z*Bf>KS6*HlM*k=8~t zrFWV)3mjp|rNoHsi>%`aZHuZ>o+n&?YOhe}L;PuVPHonb-st&GYre10Dbf5JyayTn zD#V`wEf%d9d7($nq^=hk$OkLN)2WH9YfTChtV7cZfFuarr8yq?QVb-HJUD^BT^`xS z$qUd=*#E6@_d?vcgrqPG<}6HKLOy-$xHFqMf`+uR0{4{A8WjXh(3`H3(7S?mc#3d%dKQ1MycOJf(BfIbZRXMG#z$KfxRu zh<*EHpyg7?t@jrno50<07!B{*@Jl5Kp~TUARLHGW`NxgM8X476VoU@( z+tpES94mwC2F6b!##XWkVS!&)FD9J85%NkOX&sEvZ>gGr4J?1M$~R1w>Gn-qIh_ zUMfYDoI2C$M3teh*>+LWhR@7SSIShue)OYFe^Zb|jhHMFR>)T#u~`+Utn)qQ8MUxV zzVd@iTrpYTbFqfO`U(wR$r(qW(97tGh=rL1Zmb8@I37b?pVr+8!HhTQe?XKh@vCT5 znx3`M#_Hy#OC@+E&da0(xO}>=cSw96^I?X-0}-FawN=^5Ay#fp3iKBC2X1(SsD8z{ z2b3H!e%VnPPm9b#iCIUb)lctBMv_V_3^O&Ch?GC^uvZo;zq$5FpD^i6LUy>)!6o;E z22&ZjUUM8g(b~14p1hp$wxqJqRGbp0LIMDx;S2x_@!5O<6~ z?>Al|3#K$~wXhGdegTW_FI!frJmiQ-6mw^JjjSenOMv&&=81!j>wLlJgXMF;7YGWV zjKRmUmXeJtTK?j#Lizi)wn;SI6=g&Uen>S?g&YvqH05eNEkTZfCgxSwXGJq60SE<~ zp@LWRLC_q^CEgATjS8K7;WK&e?#XfRol13g_p6H9tTmcL9MggbJdwQTPG!!|kmEQ= zuUgM?N*~z%VQz6!OR)cL`HG3Lp`9t`&Tdp3J`%iKxTxs6J+!tg4m;90rtj8B2l43` zM`5Qr8OB`@0-Q$V!)l}6Bu)_Rg8CB)^l#c8_y{HpeKb?pH{PDo1Ynx{+VNdn0lR4K zo?NMqm(8dB2!GMGr}C6qHT>=VVb1^-V`BHOQ2o-?==jN%9B)7Qp9FrY1h_IT8+ZMx z5-c~)w z1Wi%-aLs3`AD%CDrxc2%-A#(G3&;z(%AGS~$ta`gZbHlU^Ip`39VXFcSq4ed7~;qv zy~?P2ByKaBIULQGif8g!7T?-=lTL?M>%3~eh_IV1l6)v?aFt?(^TPLvY4T+jMP#~{ zuxkS*r@0m0$uH?1fIu)+e{>&A%V22)0H*sDea~O2OB_x@S2!_fw_>%Ih4)zC)75af z>bW6zq%k@payTia=j!sZT&S=EKCaOtg!_qQ%Ld@iaE6PVdk{cFjmc9wDuJ?h7y&QV zJ|jFh+3Bs7R9n!cGiQJ75KU$(lil%l?92Zf3BKDlxDKb+m0J!)`_QxfkL*rnU2UJN zepbTY`k{#X`ySU=(PYW|WcoxL@FSbhWchZ(JGABBIC`r~m>6T@U=+ORXKa%?qCqey3CML>W5@14-%#30g10 zux_gZ15O9p37p;(D}_ch0Wv*20t}pVLRT9{w?a%6XkQBKiD}G9G8)aV&FqZYHh2fx zDP(l&`Qq)I8QvBZQ&of|;o;vM&V{?U-XskLvoQP5y>=PX^FUO^kbHIJ7FheGaU#Jn z+goqGy`fd+wg?`zv)5k2j+*e3Hme0jL2->Z>f zA6vS7b@s$=lsrFkm#u&8DxIdE_bOFX(r>9szKmOLSf>{HX?$wI(RX_uEqbM|rzx5g zR4KCN@=mkdHnRhoVJ}nT{Tpe!rkI8sE?Y11oNz-#=u^8>qw%3RV>Fo(Mk!(@m&S*t zm%rYZn_EI(XvmmoGwI{>i60)jnZMKwY;wx|8Z!)iH9~B)g z^IF1K!Od>sq4Y0%D{rrXW5t}_GuKZP8+gd2B9MGC>_DxL599cS>nU#LwgHor zF1wFjZ02zEmGCNki3!y{2UhME{k363qM?6R%e#;_R}-_oDvUX5WORQ>y2RVE+oj>3 z94w?wwyk?h*IDvIE5}e<@3{N}L)inne4}T_ubt_9$l4<7n3{3++H?`|~g3gCi7jZc0aG&DEG# zl=oOVWc$VDpU=r>TQn)Nu1sLiioXtQuM)PJDD2x*_0&;JVOg4R#?y2WL(40z17~!o z-Whssjs6T-{~ggQ_6P^PTL104#hmfth1(re6u<4az7m=fAlEkQ-~&~cX_E0%KUPHt zZLW_=KfUdSZnt5<@sFKBqF*en=L3U# z`*r+X{O`^bGwAS}>MEbtusC$~xus)X)xBBnacJ*%_6ptbb=XK!^EZQXl2_A)5Jx{B`B z-leXIiTeXgqvppnCuDEjxXUp0L#PuuC}{nmD}D=yr)SD2KA12xFUi_&s9%Wt4&B@W zZR+&^$78zdb5*@(>z|2zOV->Ny;}Xw^6O-Ac{kS_b-A(ado+p6Gx;j5`-j2RxQ00Ci>2^)=x`}TG)V`{0dRe&F;LEFPO%`(<;y>qQ6rKr~+T1GjW8q4P^c6a{tyL56 zuYHu_7N6ZT!2QXt6Q7;;s~>WHvUBUuz@W24ce8mzv=7w37<1}LQh81763Ymqflfm; z+Wnlwwl&L@9KW_C-*iWX0lnzZwbJ8j=eHT0&ydOXl0S4^t3mcklDJ#MyyoR6Ea!YF zIPSOZ^ZAav6Rn-p^8xSOhaXm& z;9o%R?pD1!CvKP5S(%^{ovsj*OpJnm+qE<^4l1--NFlqofS=YZ@PJ z*=$4!iTR!sM>&4gap^p%(|T39kH(CVDsi$>_%GZo_ zpS@ppqO?=;y~2X`ttV~PS)W*5(QMbTN>B4`?dF$RWjo~56}!G|R~Jsvy>YNEDM#vq zLFH`c8&6)7Bu~sXS)Deq)i^@os?jx(PTx7VqNjW{zF{wXLE?_|^v=-KQvgaaipQ7j_T2x$-Pzf855(X0pYP^d+p~SG>IdowI<={-I zHLs3$RIgjR)m;2(bgpJi^N2d~@uFOP`uzniiw5qFuU$4l|82P2j|NJJf@tR3BOWbT zDbj6~AMfZk3>h*|BzU3Yfq>Y^6#KyVXOgM2Wj0TJBcghLyR^!bNeOk%R;T6$6F$6& z^%r{q`>SKUlWS}{%=YUz_+&YjJ6jIE{8B1)9(AGG26gegnsS>QXQz$~o1M+F$Cv8+ zSOOU%P@Q+tvP)JWE<6zC(D|b=R!EvEbCuIFeY+rJ9>s$Nf(! zWXPHe+hx|wd`>sN|FY`oQ=3P+{`P0or${N!J&-caWx18bNMdwX#pF(L8FT8S$nIxi z3$>579q$}D<$5x~GAFxHH!E(!XtVvhbG~Oit4Ly=u}haZlAoZcx>zK#_T~2X6^UcQ zq)V&X;sP{>HfNl!Cwn~*G2ihiOXrZ&&Ov2fdrP+}T2t*m#4~cOsJZ*joE#YbGju*wZYn7(jNW=$@I5$+y9?GaM9`|Rstp61e}v=hyuCFX&{7M_y{Jr;7l#F>tCTY#2!?jCZF|(qq{9{aaGF8@HwP)-C#7% z^?L(M6rZSBoRkfs>}ij0dHTlU;-t7sQ{U|?m_y9}b#YEbY-?S4?w;Z$j@keafKv%bOe8|NH8gm?C-3*yMHRD(lwW5sAB+Lr!)V#Q&N~jc?vfy6*1sGlYBk~(t@5LXOoy9PYLc;xhLqdluOEkZNlG+tb*lJg?$%_gRMxVwOy-SN z`gqm&+V}~(oZdwYj6Ru8(Mt8))GeB(Ue;X^m@xXa^GxT_QoHE?y>mPsKfPgihWOZ5 zO7~)JZ1I*gzf~jWb1vxNk0Q&1Qdi3UxrI9QyJl+Tv!8l{TgZfS*?hPK8tGqX?Pj;R z(Mqbc+ih!xNa5AO>ur9L>Jt-0-<8`xn&x3tZIS1i#}H1go6thm-5IktNH$J;j7{?v z-veR$&v{NbzO( zXc0b>lBQR$l{fuM(}Ty9aFa@VjmaCT4IWG%7{1ML)tVyG;)0Q@?MjW;zfxx?=u6QMWUo-Yur|HJg)Y4lOI?*9h zESLN*mxQ|H_Mb}FQuE_MLAIgs>+_3zKi$9Sb@h?MAwp4Q?jlFc#6^=2CQNh6q+eKd ztjTiw`{-*DS4JC|m|TvOnC)Qxb^4E$Bjcs)VLxKJczlu2lbAgzv5$wC#HbY#rwzU} zD`WJ}Q|qdyh+VflFl6BU08h0NE8T!MPu2Ge&>Eq@^DrspQA<@LM`lXm7mJS>*B#C`biENdcdaHrwYdpn#Ch_^eI zxZa%k^=lZdU1j^7#i>)3vyT}0C*HO3Tx9$XPVQ`6;h$33`IgmVbKNhclX3-wky1Ki; zuOF|FmcFsRX>wZI!q*i;?2e^9^H}+`{roDwkHOnxog}_De_6Y_Y=(@2_@X8$c}>w5 z6oV!D`FD-pDV+G_8gA_xQAS7bwThw}2I`ul{NANlTR%R%bW6sUFNOx% zomv}D#!@p6Z_R$42%JKO3v0r?6~K&is87nNj>yYe!U9{qHZ8?x_8 z)YJp6&R%Wa~;Qm?_+x4G{i4h;{X2hszmB%<%%gCO_a z>dEyprJ^6YooJ6!x;UD;m|mU02$$CjY^|Mhbkw3zLYLUN9q;y~_($CM_|tFO+0@pdXZ>zJS-W<1 z%QvwM)tw`4{I})8zWm6gHl%s39WH*RqBHhoR7GFTKQn5ZP^*&Bx^i`5WUjA-$4Bq+ zdPT=Fb&Eb_X6NV23a!(h@tT}iywD_fhR5-HMc&r_qgD&kQy3}$Z_nPz))-Io{I)+s*J6(-}Z-BXSFmA`cI z`{{F9B0cUnXjvVcX>xS*Hs7ioEz^UvX>r}q;xL!JqU3DC##{yEuH6!e_qQnQQrT-C zA+n?4x~r^Yw%o}1f&1o^cQN;u4L<>&8kD8}*IJu@K>tSEnD%3}c}K#M&RpK<5n>7JcyVek1a9^`7KojdhME$ zql6bq#ttg|H2Gvm+WMzT=Vbk69Q)Qu{^_#Hv5MjOth~GF?cmt8#!?dNM8^(2w_V|} z#g?gV^RKsCyc(8RxJ8^gD8%ve)Q-fopo>96M;d9Wo;<+!o^2%o8x z_%kQeC0dx(`){e0KlQ$I(Se~nsyW+o#ETm51y%}*hbJAr=YMdyiJaATNAt|f3Nd+C zq)Jkz&nr%B4Q6cm@hzm(I<3CRuhgR;uVKp1yvG4qjgk%}^A31;i3a&kOnA6#s_^Wq z`|2WBZrcBRllFw2G>uVI6D7%+Ze~Gu=%nOw5N!!c&qxGdtxeabOo?`D+gewp_x zF3YL2>g?s+iQo7BbFZ?Svdz@}V>h?Jc%;DC=n6hAIxsMHj_1f%rVV-WdGwPItt>c!|)Yn-^IN$lv8Ox6QwuMc~9h8-8Wfs4^!W635(o%LbGvV`fak)?Z(RN zmjRCp9*W7YcKwq6b+e94eZ=vpBA<8b9ijMK7m|wpU>2VrCsAGfdU)QXuM-Z|Pdw;vrbWwBDDuK&30QiD#;&oUWOM?Zda=DOIKDJzI3*UNyc?=XE{jB5+ny1S}sQve{1wVMGoC;7Yy=BeD`y7 z$*lxpO~L%ZjOw8SXAP}8=j3LveCO=2J=>;rHK?Zz(?48g`7UqngrvhuB^ES^FvX64 z7r%2~#7MuwCf8pkXzy*APM4rcFFC6O;o>ymm+xBwQa;%gj_k5k{JLd9Zii-F{Z!Q^ zd3RaW6KARl*FG5dq1wY!ealzN`m(J{ZJ(_POWEjNH2L20rwHF z=`fBC%v`0@zA{GGI(DA#QxadhsiJoWEKoEg+|i!@$+W|IAxh>u6M3DKotCp2{Zx)NJ0S|=g-x! zwzqO%1;BbSYVocRNW2r^@F`(*4?X-c85aPOr3KEGb1dxobYk7K4D)->4V-0xEfNks zp>6nrLcr1@JPfRewGCUL6FOomiN}Dm8QyV78_dO#HuzF}v^2fL14DcR$$k`en1VX# zNlV}lq=7_H7!`>=Fhp-kI9)S<9vZ?9Q$4EKe)=>5q2kA%F9G*7h{-qyyfibAH7$aMTh?U(l05iLd_Yy?T6NJstqAo)@v$RQM>FK(Oa zy4z{VvIN5GH3WimZ;P<*`GoaB@*^(C1zD!Q_w-}vSmz@4<|rli0r4l11IfOmNL-|@ z`5zTWC=dvY9qhw5V>b>GosONRrqPL1Itjmjvj(4Rdj=7C6X*o0RvXUu0p#4S0qMKc z7sEl@NhSmWNUQf@GWOoWfq@o^$G0PaA*1N*kLdCM>Gu(sgF zP}Mt2>H|h25ve`_xHt<$KNPJ2jA|GX)FXCKINQ&!_2!sX@D2f_W(%hsiix%o!1zC2+vT zKGR8C50*SZn|*K!uW<(RC!szb2l(PH`9aB`KtsV$P$f!wz!~_DO0vOWV$7Zw<#h+( zq5w{@r^oQiWGr~X4epQ6X9L8fczNm*JrRiV#sDam!tHsO46jDcfWO`1KR1hmTe4Hh9d8&VJr7AQWQN znV<&PWhp%mlQHf!Kkz;7mkk;dvem)LZBL=)y3lgX9$ffkGOFA7f$nv~IM~``y6LF!x!|VWDG&(oSMS4Qh=1gW=}n{s`e^#l5a*1w@KJbsYr#+g;V8T?RrWqi zMoKqFG#^T6D1@yV-qb+<0J!6Zj4XOESQQDqh+hEN1@DwzP;8_U#f8~tbS$>RI^F~8 z6M`+Ja8Q#SvlCYWVmQ2$5Y#);yinQkFnzjH?R@hR;M|56^#TdXT=4$J>^KedHVj;H z3TEFY5QU-*TsN5`AS_bo6iNt96P-FSF$RZQeFy@j41*U{%9Yc&;6;!%!r-H$Gzgb* z8krf;P!F*HAsAlN;j8L!1;qpVc}soTMgU6c19nH3BbZ+xmP6~SHFn~YvX8|=c0E*x zpn!~Tx*kWcpylBtY7~y}PE^kSd=XG?z^edn&%^9IrcjeS&a z(>cyy5SD}bu{#FL&Fl{j$#sSx!xNT#sBIKq=7Q&sIR+XY;FQQ4ulyi?aoUnyfb*|& z1pE`Bax#z1%vrxaLi<9r*n^=@^&BC?VTDAc`B11N9GzLbs@k6ns!WF$9c9%v&WK^$ z%q`WzHNy|!FmppA>AJUEAlY>ga|hp3CfLt~s6NY^y}zs8afFH>dDFtf!YEWaTYs_S zls&UR(OsB8cm)&6bWp>dhshWqA;QT|V}Zr*%t}azPY4m*3{At2N(?3SSx8_E^|3i6 z7Qm1`h(OTqfrnovBY%h>3}gytj{oAAi#wqQiq?P9#yvDlhLt2w4j))P(upvetssVk zllrbCu}<96?l3qG#_$s2WZfS0-<{YrmM4KP37oMH5t0`g*sfSyDb3uab+HB}4pW6c zb;TYICS#KvPl5naNSFp@#_S71tm(V1ygPLTn*IdPReNB6H$8VUPh|XT0tOo>R%v(z z!mv2VOrb&KDK(z36u8B2MIb3cgH9#kYfUZ1?#vSiA`M}!jSBqQG@kG@5;c$*61Wls z?nj}9;)*5twMTsus7xH(s-}U6<%1q?0(w*ki55U2(b*jxCK@O4NQocNb|bjQa0UWB zX*N&j-xAlrCPZL>)+GA|!N+{*X+=WvR%F2~&VkKGBC|pLKs& z@cWXl>3Ts4SSCSk6797M%r;DjV$#8H+lF8yKA#bxo(%^f=w`tqTV z#3p$5hh;i0%Z*bJCR>4JJWK{?Ut)>)-&_bYms~G_t}N`+l?jRcA;EVrM=EuMK_Vt4V-ym%&^~}hyLHT?DJISi-evshXqolTnyD5yxK1h0_JuxZh2MNA8IvDJj zHwwgc0G1sn$-Ux!klYs%D-H==v2kwrXG;>awNe-*^7-0(e=Rt_!^?W=;qup(>&;k>PyP?wB`!E@n+Wp|*Iw~4F<~~mcU-rtGeN?^~@W*Ad&~70t zRd9Q9WY6>xFX)LY^aNc!*IDw1Zcm~|P^iJK^uWHAWNa;=IAzw5ELc4=z&!PPC54Uq zMtlAk!r;vmY&`=`e;_&R_>4vAT^1v69VZjJK zzxQD>Zh8pD!g0|!&0qe}W(X!Og94+&aebL!6ycD6#_L4KhVoHA0o_ShZqj8#qUE*xrJw)|%b?Q8~l5Co0+D z_f8Dlfv;@aQCkHzat~ZPnvW2Cvn~wVoCbr?e zR~s&+$Q3~;`HVK?KD5+N0N3UNYfIQ%)B7+P&Nuo=$$#8&ePhwvMa-V#YWvB+T~I zK#pHk2!weEn{LDZ#l|&03G#fr=x#Zde z<>3{xOmLp<(+FI1fD2ttcy<(3s9iIdy;yH2Fz^Rcp&@Co( z3pD`!uKuv`oe3@hFN@UGKHRipB|s9SwA< zEc!v=Md~{eCqYiDnJ)6Mnk1ZqpiQw?iLvQw_&kK>w~}GW3@xYuYQheoFcOs>NJ2Ks zV1P|Cj5H2GNvIGRmPnPH>LCg=CC_wUCT*WRDk| zqiI=@%w_1g9u$GTejzLggVhJJt%~m~Tjbk2ti&Yp0Yw;N6(6$tuqi?MW<6tcvqa5ls zf^+mORbd>gg=5r}C_>0DpseuCgSBjoc4N1z;1u|-gtcglcH@YLz#RA%$XYT+DLxYg zrocBq)`BrgVH+qg1-|{UmWxq}IEug&_~yr2EJi8X=>k*WTOVtw7^OJAR$vNz<6|up zqZDRI0#o4I9&4Ex%0;~olaafL2L)!BoRYejS+f?2(bg|Mzz;N!L@q|bS|UbQZPLdD zqTrmo#b{Uy#3)T_F+VhXBy2Gn*7C4 zY}V2+y5~rxUI1(^$ykhqwJ?mXvmU+{fQ3T>7Gq#73mYLaIsnJ{xlT^M4TS&EU@Z!x zqv0aX4-=bjt0o8sZYdZeU@Zls!yr9P5CXhHFow@s2u4?`o%;OXbIfaF;H+g}bd2XX z@CD8_myI#77J*TpaTj)G@U}HaV6fp@0>)TaOTcI$V752EEIf187!hj$7*(fh!Tb<$ z%~N9ptmR)ccObly9|A79X$+pV_>1PIC&uvu|1Sy7VJjI%!dm)86Z;F-2}Z(4yue!c zMQ7L#oBBz{J#t_z`=W%VX@U{*8AGrZeeICxYSZ?95%NEOv6g&M>y_Ru7&W)Jg0kcW zEy1~XPmrtyUQ0v`k5mZG$JbQJTHZw~u4eZHqvSbhvKDvI>A$T`Fe)C?Cu?aJUH?vL z;6cPsVb?=zhdFCu7mX*^zvi2QLs=Jd4yQVEqArHbTGU0OBePciu{o4< zF#^_-E*ieHj1=eYbNb!@zy8(ZkQ1{T$;IX$~Ybs*e`t@p^J6Sl*Yt z*h#Wjzb_cdy1`KCL?b&%bzZP+aS$H-aO>3x-B9KAfej9*YTnZ08~$J9gGXVOC^r6? z2!Y@Zn>J7iX(L_~II#{M#hHiBnNy*dJ`eUzprPYDQ$8v1qZ>T3ZAL9wV_?hnM`iY# zRH`j_k@dtgc(~^`-48E^4h(_~$mps{%aKo9mIwxq;mcg92`R(jt`-~iy%}~Md@}rv zS@6)Q4;uF^hC%2AZAIe;)ewQud!rOQmgrNyrlg3)#%mYm~ z28%_*lU+Hyps|~3h-6=y)*Npdol5kfoB#P2u$`vw6l1Wr*vc0)hzbV>P{ZAOjDI33 zl^Dg1{N!{JsPD!0z;Iuz;k2DZ_`*$@J{R2qAeZ?4{x3EY^BQ#Y$& zyY59suxr4Q+k7)w1(HI1xv>Q77_EU_6dMREs6v0J;+th2+$G@%=Y_Zt(IzFh)&P+; zgk$KY<)Vjt6aAemxbggQ{psceo}~=-+o3A`-!r~>u-T3qiS77R*Ze`_No26lY!q>1_t-q4=`m{Nw*NR$P3QaTRr5 zfNu$*DC$DBhVhRN3g-<6QSqi5+lY*Yb)e8E;6xNk&@e&}0=Qz-7ndL}XYq$z=)oH3 z0a_p`8q14-+N;4@w$85RM4te-g$ZZQ{3SV3C=@!82%SLZfz$E=aM+QsURaQ5KGeW4 zP8zlF>*CK2&=+BFGpOH;Ruq`an!JJ%%wc3!2=5#24<<4bVpUVbFqJg~AaeYD`zr*l zkx}y?2npf9*64Gp=?5sWiyFHKTXh9s`{#2k$^L<4{NXAayyvbb4A*lQE>wi(vjyOC zi3)>>hYUxp=^Q#0_}{bPH1O0^SVN$?_tio`qTfgH@N_SD-hhp3#-K0Cy#mHIV1yER0Gkux9y1(|aa5^T zQ4+k8ET}vxgbF_%kYPG7aDk!35M1EG^M>vW06q#X0?n@|L*fWGqy9%7erKM-{rpJO zzmup7XbCPwwanQYa14vEWH9^U_#1^Mg{4mb^p#p!1n_xB%E3 z+deb)$k%V6(T7H{x7}$aU-(>79^219-;JFa-verbwtYK80BnwJuW{G)zXkYLA=!Zr zZR$Fn_(){Lk;1g`#_Ojbo)HlDp&JE`ZQ_@LBY2hM#Ih-%mTI8zXnx}1R-WKJnGcTW zwQ>v6<)G=mKsp+Tk=kYPgB}^h5jxfL)@*AK{4p3(fK zViRT2aOpqe|HebA;{6VhkvS`<|`$IB7gaxwj4GKi}1Dp8g`WGql8nRHv{5us8SG@p9nj(C} zHlBQcKivemM_=DS?3xeO%aHIQ0M(IAhs41hhCuoO^&!a}{Ii+;iDbC0&DY$eCI>;k zzJM&z#LzqVj0q3N*xN6&C_0Je0LL&n<^aUUdD>#R8tg?Ke8E@#c`V7{q4)vyo`?R{xI-Tf{6KhE*F3~eT2*{ zn&!&?DKHu5?8zPGhnr;3fVH3j=(@ddz+k>L;IGsy$#iOzBP9^d<>JY2ougFiTHule zD?+E?wPFHuxxu$OIVWu{SB;B;D=!IBK!xU177yW{${JjH2nDw<>9sn+SWyxi8Z^~8 zQc3^PSr5V-as+1(!!GhC91gK5fFfWR6u>*%OVAKs6ia5JC? z_28*?K;;fE>TaAz^n=Qt!Xdk1F+6AaHyC{tFjUkXd=Y>LoAey1pv)Xn8}7BrO9Fre zP7$&#My#ay?+sVK8NaE>U;}s@aQkdCfsm}=6x@c!xIN&N1h^Y(n7vJ5czPZtqiI6F znK;hhN_jEOaIYUB6b1;L5A2lrXInsryB%T<8N3 zmM2(`X^+t0m&tgn!9NowC`_AqO$uQ@C&z^YlN9I#>UD-{@z3V!V#Q4rUY!^A$N*L7 z?z6~#IE^RCpNzd7Rv*AR#ldu9&Q#{wKp*m9Eo_cR?Xo^k3YL^Kd%BpT!ga0*4Gsc3 zy$rjC&|wOlB{<>VoH0i>+rX1rQJ{LQli0hbXD$fazX0`5DwiYS^7w8wBFHlo@=<8_ zBy9yDMDn>D*IaX=RvS6+YNB7P<;K zHnzK2q_gqCpu|A?{Z$s^OQ-BgqKt?ym;)9a91~58i>l17m9^CZ#j61&hwk=1OvVYW zklwHmg6no6X7KmINN~h>vb$SJ6jTTMup^rFRy&N}^FOWaRUl5l7B3kE!T|goyxi2@ z7Gvrina2(Iw=NN})D|A{+KozPi_}zae}v_M8SKMDGon}Xc_RD#z0DX8yb|=ZuIgc< z?+5~cMr6-M4=Cgb{#RS^urbX@)9CQ|4h?^!$}a2EH+f=*K=j86xTv@yp%P@B0PYqo zIDM<+2^>lc!^5ppziXyeKKrRR;AVoIp}11DJaOSmz<9WrAR6nI+e8EIHSjX1Zj?Oa zh8qeuH^VZEN+SAlP-ph++8CMM2SE`-adwp<62TJQRpatXce_tRN005BxN;(l`ePtJWC)4YUMyNps6w%viTqrtB z+~zmHAbbR=Zyi4cr{9Cp?dwL2t=T(g|t+YWrs}2LLD%<5~Ity`cu6f4H?6k&rhB_P2YYX}CdOppFKEuT0k4OLT*Y7@r!}iARO?A>d z*ny6M=%Ibi!$(pAX?_&-;Kn>S7nxO2frv^E0EPcOUBJ3*!DO&CCg6Y51pJqA|7B5e zaau_^F?o6uQwLK!6H_~5PkkFpJ8Ne;7Y~;sp#L9g)c;OxXYWGmW$8fcY-ncs|Dr0~ z7E=WW0sycB1pq+&?^FuX|MTamDqD8h0tmim)TiJv$*Wc*d01pslEdJc~jF@fvk*a;~d^(_$|$t+hyz%V6dk1FVX48*Wa>U>BqT@ z=btdWS~OTt4VRs6Xj->a@nitp6yDKd)HYdo3Y1WQ^XOSU2nUGp5Op4eBZTj#Ff&>; z!&PQ>&F?&1Wdp8y3%l=8O42;?H{6W(m^2H{vap$WYJ*J`LmlkI!G9(SIEeV%1*81x zlX)yf*&;TW^atJ&YA7h`wfWjQF45x!el2F)SFN-I+XBjEZ8Lo?CF^dr``ShwbKngV zxO+WU^SEscVyjlulr)kvQXG9i6jP)G2(JXaa46PRCD)bbOBIF*4(nB7+Hg!#kR4IY z5r)Tw1dS_&`r0~P>Z+|EXpqjQlHc@k8HBGd7?f>-ys@tq9(Wkn)UMRylzZ}8#@ZH5 z)(EBwvtaz}dhOf#*e24!gy+3wlM+#JD;#SHrxKW*&J%nV_)AUG`G5 zYKkIo-jnj7uQ1Xh)=%Lb_N2bS3(e9kxyo%sK{(0CZh@0AI}Wv!uzhLu``w2GWJq`l zYq=ynN?@+#14g^xmG3XgCt|uHW8(kSu1!XNpK*ab0KX2jY7gaZUcR{f?9Lb`5AV?* zJ{M8S7TU(92N%24ACyEN!2eH9wWzNxnFj#?ki-W7_>Z{$Z%$Q~S5*=gRsIiy=4MX0 z9daOqy?Ua{xN|M>HkUw3k|#rxlkiH%HB1$Vr1_9!?t6YD#tg7&I9alB%3XYlN>)X~ zz`HMPEE%C);~9g05g`pr(*6&>7Qa`+gvON9;!0XkO=w3>yz>Oh+gch*gqcieT?3 zzB=f5OlUZi#)cQu8iG=wrwPyzl3^tw7E5(Cz&7}p;OQKtkT*EPM$wMbhR!9X?LVmp zmV_+$#}Zd2WFjw-0=ELrE7qh;B`Zq7s`C3kbM1)sfiw_4>VY-v0j*a9yu<^%cmsSA z2YfRJd_&KH+1#4;BZB9M^b3OI2=_CBa?JH>jq*nZ-Oi?}k1wm#phohS#Q?dd zrPEPygli}d^t`c_lo))hF4xH`(aA&tz8gqkI3(y9*7h?El77LU*K#P+*tpm$n)zwu9VpnphVoy3KNiPr~JgQ2!oxWBQx6W$g zAnmf9#`+ACXR=1H7FrJVY91ryI-ct1I6Ug&3@ifc;U4a>?#8{KJH^jFW6KMlnMd7U`r7!* z*_8=9^rKMN9JjYa<~^&3lMyRtsIpW3w-z$qiXpN{$OtX~ouD zLb1A>5H)u+Y&pl3y${ZfWa_$IKAEIYjdj9V%y`kA@ltUm^Y*W;Z`2}2zB3E|k?1Lf zkNLLEFMQZ8e+X?G+OwNEJKvF3nzNwqg?H6^_+MFj=i9{XOawC^V5g;eC8Qw|kRME- zBlv!@q3z|GJhV+=@Zl-IkuIkqfzv6WY<77XbQECwdiLPi+j^V$m-^?m_Q<^G4FAfy zEetJJuTCLoZuS}i)plS!dgCe56BYXOm+-NdYd-e1yf(_x?hjEJw;H#)(uAG}@qNFA-0w(Z@b}ad&z4IbPxDv-b208^v!NFTN}RI37T} zbZN2oa3=$;V`(sLgWP*7X4PM`>u|L)6M9wGAq_2Qc&BX+bOl@F#&kG(#5Mg3k3z5Z zo|tdT6=)NZ*Enj2zs*6G$_1ne!-DH$X0)zmT^(j`{Qsv0?AkMO1^=h{{=p^y+JD!8 zF7^)kHl}W-HviFq)D+4(ES-R-!(>Zto z|6joVS9YM%@?XEmfB8QU{eSm2HvXBOdJdB$C&&bZ!U$XkjEL3@5P<>~ZUB=QmIo#Y z7em1Yj4WB4&O!MuNtr%G>e(JX{yK!x6$a*o-kI)BTE_WaescK^l1ireetVnlo=$r6 z=n0xcf-7Ll-u_Bi%u(S^I2a)q`fuYpugw-Dp*u5q{%5P?|62pN|1~aSds_!Xmw&UU zRhfy)W&i8pmt9Gx81W)c`m4Yq~OfoyRfy z%(nOA+1qyyrA+NLnOA)NF-944d>Uy{7nG_JTL4fkhgK5fu80YhtCli^M6$lGz(|nj zLuB>AuVBNHvxFB)caZyJiDFt{v|@hSHh|x|+T;JjCVSbevf7exrYfF0!UMjd7hB2* zs2;+3CPt0r4nmk~$u;O>B%)|QbzP@1@8c3|tjt<>L98`KdM2fQIu@RP2RW+=@^w^D z8w_qCLM;tw3TFDZ@t9%j8XBkT#PZ6r^a=Cn0s^!e^&`mU=6h5)b|HNs5Zym*u#mW# z=MP>tZA$>GA-g9`$enA{*ceP z)p;EEXQKT4$M;FC1G2LE8@c6BXN^hH;-|7|CScFOUD&OwRMtF~4 zv-a*k?1>3(E9TBb@Bp8m!g#lJo-zy9*-UR)`0!dCf;g-^v^(6!Gkl`pSL$=oS^4I=lW7~Z*+jfWMGQy2Iv>5sx4r0)xd6o zU?+Eq_ENOA>u!HAWNvv=teHq-Q=5i#D9B7Xq!peFfS{{nd`l=~WgZ)OjSB2KscwiFf<3iG9doz}5JBNh~7I z%;vhH0U|~G)_wVTQ?VpwdY)<=#{9SNW~@thUbEA{#*oRLW036~$J(^6r-~Z7RO&JIUc?R>mFi0d!v(en64TF6P+s= z)OYV?{wD39PY{%fGMq}siNu=Kz^mpx@4bYk*o$QJIXi^fT{j<9)|aev8wE^R#H zM6v$0ATYuNMZy)HXioXGruV&q50spI$mkOxmT(bY;cCQI8{%uTa#6L{ZCzPu*!$Jd zchjEx!2U1Y@gImy(jO`-`$P9ljK!NTNK5XPnYcRW15+Hn>T|m zL#T?04js~mhc}bYIq7_VpTMiQNsC}TNOPRw3<;Kj;~)w02b8rW2oo4fwzTBHS!pJ< z>=l$9jg=FGjCoOr?1FSsdx*lTFi~?_K+Td*C$ zYE~pz+ZF<#devh9X#=$9q(ZSQ#MIiR`xY^tvFs1qO4>FWiEJiIiVE@)3aw+E=+wd8@&nXzN2VnB7$bsa^^iiEff;&W~wG z$UJ5D2F6}{u6jPW6^Fpo(DzAeOtrQu*&gy?o^B&kQinqyR!N-D{@zt%`LUzCV+$du zGKJQ00V-+2hE1^ylulNP2eRc|FmN|nSl_!s2v>GIidwR6L4*~Yqr_6sil_*y&ZNfT z*4Hgo9Jqr^cKS!Q@W8TS$9eCYEiEEhsZ&sv?EWU3Ha7;T9f&)dL+MOKN?1Ycq+a_! zc3`&fGD#+DD1Y-xjIvTmq^l*a2nSSBAKS#OXT=7u1w$!VDIw3a)u6{NE5t8*8qtl5KhruHA60 zFe*^eS977O>mUWf?MmEpO*GgRH)dj6)+6sv?E_qf&EW^Cn zu)MJCwOJt(7c8#OvX5r$YH=rQ6KXqi*Yu{lxZnTzK8D)42dAMnJ|Bf;NWRsSuENCR z4p=RcDweGUGlnmFlfOR>b=t9WHsL=&_DM*W$(EECD;FnarNti)v$y8 zvj_f#sF41bP4M64+W1;S+a6mKB~O>U*9IF0s)UDt#n)0dg~Ku$B#LUetPm&?ZC!;5 zW8SQryN!KgN7HRoHfU0r`W~V$M1K_9Kfrl*_o}3_A}cD|iZeO)emaM9&wVy?&hD%8 z_{*@~@1*%~C7@5c-KGrwfY87Opmqc(TNEwYmaRc^+aMUM*`Z~i2o#OJX~1Y#-#{^W zgqF=h&_8j44ST>Mnb-6R(s&P!i^d|g_}XgO@Bs&!>#A}D>`ZgNA$OOPf$tj@>nsnd zEb-Wk2nPO`LZ;+2Xb%<{F_eEI<}-0V%Ur3PYT*E(EKUXN+p!{cdeCHA6sezv66u_y z*s1Vnd!mGaX`kxgRb-T3>tv0+&X!}gEF&v&lI=Zhd|E_|iSKaBhz1$aid-bdi;cw$ zb1oyK7peq3)IdgL)5TB$rZuWe%*!yRYWj&Lrw1bTGxXv0lCCDQ)e|}oPiCKHAP!?< zvXN7$Eg`}Cjvx>r+M*E?2h<}08iELcAX@^($cpf1y>-eCyQ?WU1aO?=kiosYJpfM! z+}!SQ#~jBnJv`!>LrEABRY0DVz)S^{fj_|r--|eNVi^2xYGm`^?r*5?hJL7QoGdrQ z42YO7qNwn}t^PGP%l7~X&1slW6Kj^(mjMwn0VG=F@HyvEQQO!GM8>IA2f1(Z?QM7_ zf2@ew*2?!_jL$+S58N2?5b`^O8VqSJP6C8DLS~bOkLX%g67HR$epwiNyhR$CBWLT~ zD_uwx4~iq!gz9!R$ME$w!}}l6`z}^yCvQ};^T;eQK=~9vWG3E71{s*xw1o-=SE_-q z(KQwyfieKsY4Xern7dQN5Sp?Gl%J4ll%nkrs*BH9A{OmCckrP{Bpx0!oA^e3dWOqJ zS=?yew)L5CgbaOu;2Aj6*(djZ(I{bFaU(o*NciBx^|wtKQ8FWpEt1ng0Lp5^`dm1v zlC#Eh{5?qim;EE3zU2PYdsX`G{Bj;L7$74Yj|$zBpv%g{NF>Lo~2n_mWnqr+U zh=P|XDLx#72R;uR%jh(jWdK7=JAi~;W%7=}Z~Lc%L9@Re5toB)sNB^pgS}AEHD|zo zk=`cZgkVP`Nf#D)d=;*kP}RlcdI@4pbFzw$AlZzIy75oVE%K=Lp>FMp^Nayn`U$z< zRH@=wd|lbXvK|>*BrYHkIxu8!JZ%)rjvK=oO}I`_L?VfziEeF>JUTbK!w_B`Zkj4`3T78x-#P+)86nlj;ogs1 zm9t}qsw-`dOsN6mAhi-#2|Sgp|Jaq@Ah-{^!(RQ&8$A1Oo4a}g=TtOavu9TdwadcR zj$HdrgAWahTH+VV@NMz-cypf7N>G0-w=0a_i@Om5#$mQgwH@ClH4WOYlMt^?$PGV2 zTRn@3JnULg0BuSOvi>{nJa1RTa?QtVr^(JOZ7QJ##PY(j3^PK72Q5Ro-`SwY488tc zvjt{D!OworKT`~WK%79Jn`qYb)@yY1qhyH)T%zw;B z9rM}hMi>BqUm*a1f15}D)Ah76bTia9wzsh{HFmMIw{!lF6{&M+kIf19d($VB?swPW z8e7tZ4=30xz;?M?nl6sm1x@p3M5XP-WAi@79#7Io^!Jsxa!lee>WE~EMgdsVCbEPg z>MJtE>?5Yf=O;<_NE1Q#rup@Vo&GnHHrK9wi!;{TlR|@NQ5C0j0Hk;B2qdDII?<%R zfA@hJu5U;|rkNmao#oh)N-1sRYKKePAo}8}VipjaJ}8B@)Zq=Sz9Bo@Cdr&8=3Qob z-DJkhX!el-+l4E)SmX{yOK?{kuTj1J<7(w**RH&I&KZMr!@Tc-L_0!(X6=gIkCF(l zgeBAyi9Yjo?vX}5rc{xK$) zRPNmh6z!i&@Mcd}RC#`hw{&M7UKxKek-&FJp?JZidQ`K@?~f;QbP{Fmg|iIvLo3)W zu?=t(RL(P%q>eUb&d}!%S=M=KJrgeN#P<%P^?LDHsUat_Xr4~=m=09KntUzZft6OH zie%0>tv=-m=T?qN{MWce8I4?zu(w_Qacg4(P`f1N5*0pJ#VY>k98%{`-4I5MOMQ2We=(k*^VvWWbLXArX=ZiOhg+ z5k@!7ACyQdG~f(iNx*bqbI2lsM_+o>gYG$|yn+*;hAgHQepq0{i;JW3h6w{>yP=p6 zVEpv;_&22n;MEa)UE#Z!SYtLwgHb~U^rdH%BC@=p76KO=<|Cd3LMUwFBPv97hCI@1 zC#nrmPsZh|Ld6SvSw=(z0Ypk|gyK9?AqLV=_7}zr`ztO%@Bu`A!$WG=9F^A^_Er!bY8HA-hEmr9JtkQTjOMfzA&${w<3Q z^5hp90?ng|tsqPS1v(du@Va^m8m0C~<#u1dQba}|#3*uL1o^iR?X&wRPyrA(sz>cx z*Zku3#-t$d4T4z+y^$_{6Uow$T{drR)9VH*ytAPs;ft!=_MbI0q^Js1S}@By{sT@L zX2iA%S(!+UQT3vt``UFD0JwF7o?QXsC~lS#I!yQ?7vN;e#oq|m6{ihat2aL)HHo8! z$~Gi?+lPjIz1Pc|cHbA*TWxm$%he=vJSGo)fRghm)p~V7!0EBx{yzji88L!!^VHU> z-yf)AZ)y>@LCUavV}i$s*beKCk#04S(iV16lIKl9h<-bS7z3Eu#tX0E<+r2$Z^Z`~ zui^#Id_GsMkkM-w08I-hB*GmtNAO+v8`dw#$b<%niZ6epz#Kk}5nP-&@UR#S5@Sg$ zw|wm#_M0_nIRw1hWuP;G(T_>Htv9dt7f&v1w?z8bf$w`CX?0pP~T1E zR&?edqQzxj#B|zO1u2%=Om8h3I$X6~gRxB7odC2)Uv8pf+ljZs`qkPLaa&Z&UgyWz z%ngE^tW}F}GgNJ=4Dv!ovhf1U_ya90b+iMjm7g2DcJ_2q`PxG9YSvf;|ZcmK*m+S}=4dTR%{B_!!bA~YGDuv|_`zC>g^^Q1}*2qGY?!l8Je)?(x-JGDs%n8t&s zDQXLRf(Es-q?7f=CeWF9;`^~KQI{V#wfuQmxBwe;6yNb?hGTQ*?>5eOX5>7waJvJg z_5g7p_yxF+irbEEn%2N60NMaZs$KQZ+X6r`NWSSd^5vB#u@&TsK^FH#RTTkHgA* zKdJLf@RWL~@)O_|cKyV-~3cE`ChHaIa#DPBBN(Bb?SHG#x~FC1KH5P)Ez& zO4f5sZkdGU36h0C@g3RXJ`tyoW_3AGEXBYV4MZZ>132B>as z9UETT+;kyM;l#}2g`4CdWodvKLDq=<2yO^-TIdCLuJ5~Xz!Td(0cLywW21K*YLwiC zC;&I6q_F3XxE$L3k;nJG;tEhKb8(!!i*svTeI|{JI~iHu+LJf^W>X3R>aLhjc4Epf z09Xxbbi}jdBUFC+_>%zA*_4($S&f`ivjFvX-RBPgO@ZT^4chn=CWW2D+0shWa&pwl zJI=K9vLliX7$UqR2IkEt{?wB0Wv<-#&mnL@?WMjp^3HQHP_H|vO+BVwUq;Ra;*v;b z%7}q&r6p^lnzo#+74F;_<&N}q7MXxVdm-&No5I#;7U zfc)`V*mjSE-}Xqojx*Mdo%D4qr1g?_w$x4tS9

    U@A>UUXm)V+{W_>;Fj|vv*k8` zk?|bcE(X43fQ!gJF;QHP-r5$lBbuz*bmwWeHq?{DJfGuJ)nqrve3k`_SRSp8q}V}V z*?YrhqnhE&MhzWg)slLgg4$bGS_-_n*24-Eb6%GmNNS2y%a&SHeJ2)!o{zo(jeWW7 z_!~xWsH6EW))oiR1!=fai|>{?OiLv84GzwL~3~Fg0gnrDM1#w zJdSKkK*zQKokzJhcNaf57mi+bfJ39heb;YQ$*3dCAgx6+#kC%i(h^6jGB(6v>SoXx zE#e)t_UWNdn){2-WN^*>NjFm0O4MhO^tPF#n?E7y&<(1-TI(ThLV;^6s<)66UK z8x2!VA$z+z_c6QBT;3(XA3bO#>Z3(7xdfNz(kIm-D5Yb_^V59YuXM;X4P=S3_OM;y3R)YK`9QB^TZfrqQb>Y}IA}*yz;9MB6w*T$u1k_UmbT2&M#?;cRq2KZ=7JpV?DOafy0(0hrK6LuAfKudoJET5O9w)fu-712_({Q{+KkL( zO{OE)PT`H@zo8l$&D8ZV+}qiyc-P~o*fLiTw$0WRpctRD*V6_s)Q*T!8*eD}d6OIo zCnUOfg}S(=vW4rsG4OTK$mUNM=6$md(n$mn(`({3(i%B>@zcF}u!e#o;O8T6i=>o3 zRIPmpu3eBEA)^86&_xpZERlzxIMx9(7A4MWw6P1Z`5=_TSZ14(=7-#W-s0Vd_@LjN zn2u)R-8(9R+^qeOO4gW&Y~QzFBoUi-G9CnLY{63Pdu-ULddiwdG9sF`Vm)~9^l;+l z=-aSXztTZ%jeqg~%gy;uG)bvN}$9xZR|)U7U(?P@elE!>!yAI&c(BvH&bjFOC{8C6Og$!7nQ*+!vB z_$#2;2Db?)u+WR4^mQ5BsEUjD$f%5q{3KN1 zCLBQ(xoQ3$6*1BM6&E>)sK7@}{nt!Lh3qt(P$3(I>aD?XWE49aeW*)_2XXIqlF=Z2 z?)qRvOMI31Ht1*={}5`}%Ydezf%HPytFR9pegsV(!aIu|_BQyhy`>=@>WqZnljw$* z^h$EYo92d>_Ufq0o$AI{io-m_8Hya9rSRam@_6Fn%|wjjPlNnjE~clS+WOMOV|l1A zljl!NgFNY6KI)65s~ii(9G~d0t#5MHqSBDv3eJvukl7IQwknsN^_y4g!H(R#i}W$DKrJI@bj`kXcW4v!0?N|zC<1?4a5kLomhI}XDu`sP{uQQkw0KPWR8@~cnShQSyT?L6AAktM^L zX!0&yIBZ!?NJhC^e~vy`es;r-?nUllR1eYW1!YcZx&4-}r&a%j)rOMlfbh8z$cM|w z+(Oq&*@*EZo<+~hz%;0s$!XhBo->-jBnDkrWY?U9mrRHkcAMxU-p5L)I1^^p_0DbR&mr+CCi))?ty+^IbqTI@74K&T27v zr@bv15A|$RMd~GIk%fky&(og|DUXy#4kXpp_7{h}Bu$V@M=INU9QeCmv31^C`=KlU z&~eyIaCnLsW}CK6ep0es3ms8OuT}Xc_x0kf!&?& z;e!Z-1}9ien$HE`%LriBoDqIl_&DCGcrzw#CC<_qtDek=LLwcBT^S3R3-?gDOOaI; z4e$sf>tNV6L8ckjR$l<@G|m`LKzSUs#i%fYHja^{rmxnl9gYoO4?J0SR1h!@5iV>W z^uPdP77i{dXd4$Ng~+{}2`CrVL(5v1%X`3doz|bcu)$Jdc=WLq3-csmIB?enWbth& z_a22!wosezV$)0cfvi!x-H3NhyHTpJ43&Q$m^pj7LDTYdeZL_nH3dgCYk7F2#)!ul zXvReUBJ%88Ldp|}>G>!$wly^sOkc)Q5e*^(Sr0~de6~NWd5J%jYp_PfB0$(Mi=MQl zsz^W$kSh}e!Yj-{Tg;L<377#C(fbQy@XSJ{5sb+e}5k#ww$c-eN%89N8Tv;m@#lE z1}HrPWtN(yl*u-D{%(F^4Q}vSaBw8Lj8?*;Q}2#4sY;O0JteU^&m?ONKVH3b<35+H zlRx-qjiBVh-bElo@^9dykO&!>i2`IT^wZXn?Nt2MapfzCj>n(DGw>Paz?pmpm3H#roMc!=K zHRKOCAK&%;zp!|~d~5OaZ~ zJv2G^Hp1NW^!=EFrg4;_cJm2N!C|S`Dk8(=PBCIcW8-)y!3z|#%9z42;Nam!0P-gc zLEIb|;pm{MdmlCDPem4*kgt)mcmuvf(Ca6>j@^TxU5wa^2^;Qqxz;DeN6xiLe@TBz zpdt0H-MZ6;g}L+;<;9+lRpiy4k4v3LKLT=@r`j;9tY#X&)COk{8p_AS+#3qAeWI5& z7kGi)kL`@}4AF`quPMHH|Io7YktM<9-rMj8!Vss_Mcxi=F_?iv5}u2`s?zSc!3QB_ znWuBYdA}kX^jp;Pmbsnk&Nm|hc#fw%(k*o`^_Mc_S=o+oRmGb9seLC*Y&g}e7@ajh zNaI1H)aQvSH}o-JaA;Jt3u-gK6JPgDm>5zSQ1vX|`-~EgaZ|wv$gw5-Z ze+8gq!@@^}yq)hun7ASvBV$uF2;xG=8@4Ng8BNuUr5C4Do_seczfaM*tR=p~5Eezb zz;UDbQKI>#kDw^(;UyTm)eVgfN(l7tm*(z6XvATD(2;5G?l7vOq1)-^>uVb4E4UxR z;7P<1SySH~ofkxagSkg0Vlyt@G_+MRZZ8?aKSl-N2DI0^;qnt;&Zui&WoSre)kB?R z@yD5oCSt-Fof6`V4}Qg_U%Jj89DnuW!vI~N1Vax}m4(t`P_Qf}1>1c|~!;b`I~pl@v})>a)(dM>+N z-I?#X1R1Y=i$fiM78kx{{i2;;9E$HqKYRIDCIXar@p8&2)+1Y%chV@pdv&jFo0eJvRI(Vct(V+FsXl_%4;Xo#^a*x41p+Ys74W>XBdd@}lNr@2mZ zJI+N>_1D4H*6#hhqu2P&@Wo}g+z)%hRYkIwgK0=x9qP&+RX$4TxD-+%I%T${G8-|Q ze86XPJKXRD#K0jNk%=jn``H)|XkSjgp&m~K)dxTc-V%x2TQXLd(!dB^4=ys@_m9@< zXx(AvG)9y{<|8t!grPS^mWaL<#aRq8GRJB;gBl3b7r-?P`zO_g!}DIu<#S8@=zAFr zJmMkzBTm$_SK{1}d-lO_f6TFLq)H;EH6oEy03O1JvyMKTol&HtbkE_0hiu7${V!C{ zrlfV8NHe7@X(HUZ;1+7cuqQ4`TnvsH_gtLBN1Kov{p}+LG;1<;f+GyAioAUca5QTl zE;Yw7E)Z5RhE|pZH&4P7NTGhDf}KnRdr}!I z9j^e5-swO&w|z>1({u+CV)kyY;m%D=fCsA#wHkkp$T_^utup{%?rn&qH4Gf*8T)0@ z#$Oqd?On!?k+&#Lens{IQHo1jDW`r^8Oajk!$+Bie;j1AloZ_nyOhK~-hz=Il`6NT zm*Tx1{i?uetHnSLv;=0fT;uvtV{P)G(>TD6OfyNvtMpgk*&#`^Z>7?v7ey47^uyh= zYn^Q}Mnj3M-zcmz^#j_6CNpPVCd*gC#x!&f2(rMr!#q8a7IikaPvH+GzA4kxE-L64 z!_Yk|o}$ee*HnuL2?rF+a>P2$wSs}u(H{U%G$g1jrZQ}hn?GwWy%WH~y@(&s=@W_s z+Slt`%@&`3OMB3KXij>2!ihv;20g%M%u^PV|K2uB3HL%)N>|YK2>WNv3LH4+pzMMBE^x@#!$DhUIfxX7x9`$!Kth#Ap0&Ra@U!a2$RqP7xeM$R1C zLa`2Romci`=w?$9>4zCEPP!0?Cinx6qIq@BCPd{$6mG~7+ zZ9n@(u@**>k0>)K!jmLqORCpw?N7A2V9U}LGL5!kOdj?u=CglHjQL4YxF{6*hqrNY zDEIo2ZSSRrAdh$%?_4vxJ$?R?-fu{voqcO~p8oiyVWTQm=Lu4RBV?{ywD#-E=11c(4(N+gXJ`A|1A8K-9u3Z%{r$M?Xz*9!LTH~1& zQteF0sb^{~hWsPo=Aph}@D_J4_&mfmx$t?YFPeNFaD~z45UI89EqS2X7hE&q5<2%g zxtY}8tzq&~3qt8NI>A&z8bWBz_qw+_e;y9B&z`*LcuOZid$_E^v$?5O2b9@OK+i7i z=_WLUIJ3q@arLMTP9(K9NW(+&VqHOMRUPK9Iv=Sj1Rp0mq*T>L4I3F%;C2*t2w>;t z8f~^xTivZ)&~3|9j>QHi0+U-=jjZu@?si<)nyq*+iR<`LWj#2AT);aR3@+b zh66*3`7=<4-G@KvK<{<$@dQgSN?x@QAVZk=0#B0#B)IHMbfPUc*j;+$+WqBmQ1B+B zM1D?N&Y|E%A%|s{a)%~{JRZSW=P~CMM?!G7x}5aUU&;E{j50a5Ps7%YJ|nhZPRJL$Idk=mdoV0hR|+ioeyiT zHK+eLNTYiXEyTfe8SDaCbm9Dt;oZ#6iG3clnRN8mKRxX`HB{5Kk2-RRJ0)%+VUm06 zwBTO2IhV{l18t&C;SwN6_AzpFZI#lj>#ImA#(>K z(!R7pKe1=!&Nq=nB)8P!Ex}oaHyO9=RaY6+Ys3@o(7GRl1o| zw3c-&uUN5J%IC~0Uq3&wtC2ub;jc-~!?!-KWvtq&KFN{hv{CW0HXugboEzw~+M|Rh zN}UL$!^wr}@zuD7==FLloSUkzP7lP6L;F*rew)Je6jTz{u$8x4j4~7|D-Sg_eZ+-* zWkn5>gd1#z$3K2C7s0pK3%8UJ&pHRn3Rarun1`VAP7sSFJLWN7eHRHl6E=Ew5YH-0 zP!pwJD*7WB;=f#ahxiUQTM{p251Bet7+ct1qvav;%4#{a(emUPOEX^#e1@kLwBZr8 zm@FHw+!R_CSiY#{6S@F8Cb+gM0$Ap>(Ih_=`!n+fOWqd4s74^gU9L@1?`(+O#4@YJ zCpRQ@oC%<0ul$G=Q@MO5z5L9{@CF&gm4P4+4BYMZAbk4NQ$7*kz$?sd}aKF~#@px#zv z6H&q4XM+!Y%@aXnmKo92w&9275r57rC)tWG)RFrxX1aOOSG}*(UgDP`(EO~tO3Ab2iNHK6mb9Y+&tX9^s{ zXB^eBL4URA7OejZ-_~9c@7vUPG3K#VYw*iht<8b#ZkfM9k^6H6ZxJgPI^~9`5rH}* zGKdDA!^0=YWSM*WG^p)1KQ{PN#sHIkno^6FokzTkHcvq%Oh3w=#h)3IVTkMjQJ*bx z5-j%`+Zq^I`0Cd0m7x9#5@T2+Q?KWK^x;TbB&`WhL&UGMYT;q9-gQIxxlKxDE*jjx zx~i*Aq?ub*hHzBsiY&z*1OB9x%%h~w4U_^7%kS-Z-E}et?`l)Nx)nRxo(}887YI|{ z;vGSFZ-AO{mR$+sJSi2sXvie@T15>{y9}lXiLA66&<86 zVTL}e`WB4=NBM^+%YEUC zBnwzL3oiWd z8T6=q;p^WL!P+U*cNYr1?=iqXz=je&SB5Mgez2?O(4d{ei<5V9*~QesF3^wB`VFzD z;hkOBmt>s0icZRTsQN`MT-- zJ{z4(SkBE+?lp)C-&CJ;R)R&gN@MAPnRlm79wReHI=g)x<3>akb1-bLX9}l}}9SnBukB{9l z^{@2)_kAE3{NisrZ?@riX0JTXRNkt#{(!qAvMr8*LzJxl6m8Ck!4h}fi14VE!8#p; z6IAIt1uw;W1e)!rIUoi!v^TesGsW=B|IQzitHM%VEoNFltBTb?l-x=MhZ^gQBC$o=wZ(ld3=$i~o|2=jSQWmMvWJ9VS3(J>ET~ z*u$&115iMHEe+NqH9O4BMLk{eUm-uKS5rS&m^~O8i!tWr%!O&oYs=acUO%&sjf?1| z_OjpaH;V7r0v8t=X1woj{{PxkiLJ^Obbb;d&ZGnYkgEEhmz)2`eY^j0cFf$+#nj!< z^WV9#9bRkqEit#fXX*y0Il|1+1~)d}3l&S#NVyb^c&VnPzi#)#B1MHblFKuwe1`6+q%80Z)CoWjh!7{+VUwK z5m`)AQQ|6>MUi+-XY^F|z6eR>p(Tn~KMDNBb6+*qDVBJ-r3qcN2n)5;M~aG(~WCgWZ8$7-XpLqZp81qAC{`@mL%IU(H5GrqAGGzQn!!g zznWa&CB?-}9tfhzg33!wRFh;?!zDR$qKu;3+=0ua?T{AR2B3iSk5_uf)qwjRDv>RS zOArNk(UMPRhELB^{|^9TK%Bpq|GcxapDvrbvf7Vc1K2x}i}@-#PH#n#W;G7l4)r}r zX3J(-i6n`#`Jw~_mPH1GERsCKq0H-Hl-|^3zHCIoC3bdXNoe*{)-aq&nddOp(6?Fv znu=eSLY2+S+glg|^|P$G+F~^=C#piztQLS^r3%&Ml@n2G@E7Gm6vOD^ zcF5IHx!FHSJYt4onq>%#T3^Js@sK(iFPkin-)40)K-8_CX!_D0yINMpUkyz2 zYDDO-B#LB`XTTC+*cIM!;=ZEN$R&a3pcru5_a~#DVERr^56_O1v%}L-^hbQ0oSY&J zBvth`7b(tIhfo@f&LQJ*0WJ%GdLtSkioYr}ECNxBEV zH%)#?s|2{@UEfgob(NuzEUAE`5RhY!ccK#f6W9Sz1@OxnxWxn|R_qW2mahTn4gJE{PyxMz)h;%cve>PwCP)Ha$xGm=>mkpw6AIhUX^u>zvExZtk6HES%r&B z{{~*3Qx6jY18oe3&V=p8@t^oVL~aks{ZG3vxw_#%%cjm`SyeFbdlg9Kv=(m74Jd<* z192i|*{5N&Ti>OXAQb%82?~rA&jauRf@lOnHj5Nk5aM-H_wl9UyIsRi3dDnEFMi@(Au2fMpNW+wUVpp-&3H~Tk-$8nE^+V%6b@m?>Mb<@ zd~5t2w=rg24we?{b5bX886x7b&s0ZhMJW+wbq%XYXrMF2QFJl`U5tx@W~8t(tQl$* zsF8`qotUFaq{>FiT2ZlL#oY6KvjU)T{C=_e<_CNW&7Bwd3Oayjr_~B40t>)9-SlIM zDl4dipi)gi%fb$q#U4B;rfD_h^@Az{OfOUi<@FBr2gC`RpDfS3iGL{(g>cD9togO zZ^6F=r|&dpX6mk7<{&|ZHD@#uguz%>7)R8!XpjY>_B~cejN^fgn?;cOIKbm;kp1JK z>5wFNMSY;N=Q-9wephI`nmd>hIXS8%!mMfD>gYbhfmWnjf$}RXsx6lD8{qyWlZT=I zT2v2eLKN@JX4D@=eHsJ#0C=d!x4F$1(VH~OEeZs)1h_bYBAyW2z&&muL0b%?0=t3< zeNt1GD^s7|V_oRBNRaQCAcDw}q6McK9%1bcf>Y!8<7 zO;z5DVsDWx#IA#;DhfISu~pd#R=DG-xLMxzkvpQHdaLP?Y)92mdyJKqAcYGIBa8zX zQ0mnX%+G1LY!1+#q90j7-}ST{}aufR6GY zSQDobq)<bAIY@|Oxk#^ZMbCgbiTknum@RW!_R$#Z?UJ!Mn4uVxvPw-( z$qb$t2!r+1c+(!U0pi8Q1Xx~KG-*~Sjz&$`H4Xk;5x*(M!5xK(sAT^g#$gu4o0HMo zV_&Q0hu@7zsa~s78{Fdr=m6)%)~*ePpQQjh;J4C-)pArIz}P)cPDiJ&!TOZ#Fc(e) zF)J=N`0(;$H$&?cr5sLJS~$0iQS{rZzx32Xb2?>uJ?t1EN!bFZ#cG%7yiz-Nm`)r0llCE{*;&L)JWUE%8~=r?K;$NF3uJj6yaVd zEzvZC3)EhrAh(G};`G^%`UGj%y%@dKqJ9FOg#l|;@uDpA!Or%~;Oeiy44gjeYAqJ= zr99&Q6pMIP#;`q8z&D^V-<2L$;Afp8HU(VC@|Xxk+gtTFF1T@oBnlw( zQswc!OWAZIg#<~9=qp83xaF99TL3qYaLJ|U^j0fY;+B!_D|L-umsusIp#PJlH&x2d zl+ki)O;THveZlHT8Wfv^`9s(cQIfPv0osHeSwB#o9=9n``(@bg4_;QRF)_(U=lxNG4x zimvMlEQUe!N2gcos*t)3xEOR5r!biQ;Q-jt^(A_pPtLBB^VdI&j;@oFV1$U@;Gb2oUa;TwBpSO<^weAd`I?Bsh+5udK!~pUcraoNKX!6r++NEk# zGYY-ZiE+Vy=)ssErnprIRGmWmu+9PKs$53*Mfu2H7Wh>u<>dV-dc|{r3QR3M;?xss zCym;l0J6v`7zf<_0xc^8VgVJ285X+rOXy%#uwi@-C|8dln1+&hr8y85?j1J3#G<~^ zqWI+kxLh_N)X7&bJ}vTWk~R5CTG^#kxa1(*tccwqO3Ep5q{lp_;{Czc_$RED-jl;? z=BTIFmN!|r!q6YRX2Ezh;`(k0cH^U(_xS4jch||~@lTh7wbdlXPa{3vA~{~p=PTM# z>@&-=Hsca#A#0L^Hso3xajH~0Au%I&?QU`a1p|BVs-t8nhOaS{A7> z2~Y~`)gT6VNm#H_km3`)1HLyE(XG}$)FQj#~%IHxX^Yq=6%^m1b55g!>6izWq zCqNN5%SO6;X~RzXb?WY6vu0^1T}IF!9Iv9uQmNjtpB5R)O**-+W6Sbst)gKhrTKxk zXFMct(gAG_51EnEsy;v_?y57s{c}K@UWU|_8igOJjmbt?6O%u1D&*#>84~gWqv9xP z;K`<0GY~sI?DkH@2B;RW*tBpgW+_YqjxP;D+-k@SO3R~duO_vUBnewPu-p@07${wM zMQg*8U|LREc=+SLuQv_ug;-dQb!9%(Mg@-`v;I z|ET?x`h~5!K>-{z53n9qgsp$Lh5SBSJG$zTTY&;+pcMU8&U)5%gsb7W>|J{5Tc~@R zdfH|^6U}2;-AmI|?TJaHLf4LaMmV;FutA0*ClvB9q>HU(Y8ytvbZB?AWd-}IipHm1 zPY%%wRaBgZDcF#mp(6lE~L8A?CnCl8ujuW|;zl35dRrtms z3|m8Ex5`+nUv*o`NsL*tkLw0_&h6OlM=e_Y5$e5Zm*{H-!jue18APB$3EQpGIJU&Q zqY*}piqiq!g*c>2yEUyseNa7TX_h-Zv-YQTkEH7qhE>3Fo0`ot-6vUsJ7L6bs5{oo zKhN^0*cL%aYOj<|U$8^WVWzr3cwj% zUY=i?dDWp_T)?bY<`!+XEuAOGWsS+hZV{W`TBJj3mG@_)L5Itg&4oxCbyx1F{C_8x|dnsmsq3*2cl951Tx4vLx(&mti;9tTQ{V0Rz7QNix^Mn_HWw_X2}-`@QO>H;i6xYhcam zTRg#VSVXiR9hDQ3zqlcZanjI3IXDt8+szvF{3y04>aoG(ZqLX7GoeuEL#uA@JKP6U zE}%Rh->AyRL_>5!eC0G*8#3Ok%gqnNW#TPnl}5eW+))d7%l!hIP&Vup;C{1cDSqAn z1hg|6QbRwqwV8*y7GxfOEu*`|TM4VrGP;eWZ3Fupqua8yZBT2CZfK_;bd>qw?|yWL za*-(<%v>m&&W!7v9B0o&ssI{BxQ`@yeE!o}^4;O}=%+(>M>q!gh}+q{Wi#9Rr(TF@ zb&&CT@=_(%bEg;4F&$~dc+@|bLbW_INz4iu^-9Nq>zQ)>MxB0wzlCo=MSrw>16F*6 zL%{!gTmpX+bKMP3K5{#^za}D?+E^!|)e2DI(Q9d4$Amdy9`=GWp~WBe_g;N;tGm58 zbO+7tg^URscuV1X@}kzP5vLr|NT#J=Lw5?z(}06n%IgF7B0;dv7tP9pB|CE?ph;)% zg}4(1ACNxat|}eZhCR-}7R30rUau{CIz5BrI zwrr%!8`4rtdW4cxerh(pF*Q@Jhw8MUk?m zpp-ra&wVg48eL{H$BZcel@-F}pCPokGf@;ptQ`;l_k{+=UOE2m7BQhl2d2TW^`lRZ z2F{FHO>#Kt9jrQL3vMeq!qi~Sk9^Cp%vI%WqH0_H21Tg>q0#}v6!7-@0MA;vG2pAc zj{~cYUYlc+_`!3h?WgEffu}?S@Uo?)l9;TO1nWbYVPuGOK$O)oB0`nWqr(zs&}KL6 z+mSs*T-Fao8-T=C7l|ink;q@zexOZ%xR~-(byIa1LZ;0WTcfY`%+!PKDr#o7uWb~_h?Bor#&;lN=8m#8~i2JJ4(DdEjoK8?Z(!ApZx9VS1*W*KHt>OI83ez{_MFIErYE+1P zv_{;tadi0h?b_q)Ar|KqanRm$@WymjYf*tEx_%>K+jHmv$Q_&Lmf+D`+8Qq$PQAP_ zb%L6}{ouUDpc6LfL?47SIQwVbjofEGxL)pCXS-NOV*e~gfnvZJZ51f7js!s43JL>? zW3Sh`v9}Sw0Y|j1CIiu@7=Z=N>C>eW<+3mdZJ}p#;a5*ET%C zyh4t;?S*sENfvToxb)Lf-}*k01{9%wR`a`|!{%J!$wvU0zqTfEW?*zES|K)Szkh*d zFIciohTL#9^!I{IpedMWbpt#TjJA3rinPi~J3j*ho51v<4P6sMot8`7K*ZyIk5$%a z2O`C?%40yWYbZOT`T@KB5wpw~Lyv7Bo*-{7+f;5KwvNg;1HcoTnbS#Os&UZr=#J+p zNI2^c^0^FP>m$BCTMZxGd&&{t4ozeG!+ZGo!UKHPkoU>mX{W)#Pg`+(l5(?=F9wEI z&Swxu+L(4%uM^P!t7azTaB{f``=Ac|lNA+tk`c!=w^<1{s8-$a_P^ZMN zOMu8VH;*YJ7+c+NItPf{wa+xz&0&cn6o5{>x(OVGu&XuY71^t9wF)%5cD)zMSz|L2 z39M)IM3i^hrC@V^OQSr(Z&nVa!)UA@kE5!kSkenyudMNi#6)_VYI&w%$Kb@yq1XYd zUDV_zg2^c(ldJ)i&5Q)9&dOpmy?5*cA3ETqugR)uZ+w0$wIHF(CO&<`}5=E`sa&LC``^jJj>Fb z^P^o|A6}mvB^NN5C;y?>HquU~`zEXl*d>U^Y79B-yGE<1fAse7>WUBzs9F}L%to9U z`#WSkg!&g@7LFube+GSfsb{dD<>$)0fQaxnWATK&o3BVFO+*v27YEyv+cx%KQM=o_ zu{vBpL66VCce+{|O_qn}{Os!b^4-z(`Q;XC@)pxAh! zkIy3?fekWvG>h5_Dg%=x4k8uENqIkycREV(m(MV{akEr0g5Cx`bZr8!C<;aql41#r$1k^<3~&PcV|DIVR*@N1Z@D<0blY$+fAN_dv-6&`} z*jzw%t&8i}{jw#Fy=Shauxgv6aoEx^LRiXimfUat9VX%$24w$Z7?bCec2cm(o)4@o z>-VScGx6B+TO)tuS7Qv`Dj!vpyAw9(r%OjhqdALxT-mTgioTJ|Qh9t$G8oTf>CfbD zHjun@b`10Z;waLK%3_xF=56Z`+hRpdGB19_LlWj@Pt)pE1Cr%9D;kf}Ozo|%5MLv1 zW%;Lx7!8hFRR=CGx*VMyjYwz29EQ4n?0G8vc$KS?Psq~=a7h-HrPwcZPA!}$ocDn~ zDDHtGSDTLDi+yjO0zRSKkXZrte#Uzx9vovzvq$hGdZa3t_uj;SBcWm{ixQ95%w*aO z57LZ+Ic}GET5$}Ml;|Nppf%(N*7!X^f;@rgJuioMZ8Ad`0cafwwVsRnYKLi?^-2Xo zA@~1{`$|*7Js)!+wX_X6Lge`q>7S=vVvi{~8&F*vOfpJ^0#IxXSt1jZ0~Q}w8TjUD zR_Mz#FtT|8Rt|=#HIIT~+DsZ7i(RMI-r!eJ3$)5tfdaygoORPXtE3i%#a(o)nFfwd z^f63hwMq!JvCIQTs_=I0O4xRK-IyR=FhCELE+j}5*NwYPi-KKk$AYWJAl9_=nF3vaByP3A7uo>)IXIo9L|t(V4C=pVn!CQZ?r8j zEw7=e{$Nb>f~nHLB;&*=3`Li;z*)6olR7DCDllaC%Gpd47e#**8R`o=%u=U7evM2U z0&i!_TIavmUJsCx-^wS)4u5CmFNmyhANSLW63rfTUN+%YB&L<9L?#Kx#mg4tAs!BD z37vn?-#a$bJ2%qZgB}>gdonXXIhX>}ou{%G(n(jti6howdO_)+ZR*>11On zsZL%wQk5`DAbLqUBOgbQZt}CbK?0C-E1Vn-CJ&l|LF6jCVHl9Vgq@fs>vW#44cOA2 zhvWih`~n1bf8a;6v}giFy#LJcmiq=wXw$D{&=e3w zy{v@cS){X3R32Z^G*9tBML}0{Pz7yvpyC4@$&%I9wMzJ*)VjRKPxl)c3jR(Mm~o2cz4Rl<`N0+0+>rp~wH~Df~8&5>qf1ThP zD>?8}H-FlZw_$b|4lgu3GQ7)ax%Vs_y#-J44RWyKmUfCKJ5U6L{tjj7g$S06vX>)=YV`*lde^)KdLKTpt#Rtk*S1BPJr}fw<*81N($raK z(^&p32CJn5s)_%+WQ3InYQmez$hV67N~y+_MMz8%vQh)fqJvj$oT#<^|1y1i-6rT7 zjeJXO+|KlYDO|%Yu+9`>?aWZ5E(fw1RYE2i%Vu^L%|&|h=F!YQWy{{tvAL%qptFWB z-@->?AG3{!JrV7!qXFN{^-f;PgP1{l0t;oErEi^qaISoC=j1fKml;@`3lgWoy2yj_ zxY;*%;0M(g%L7dMgp+?kCQXoM;q4GvDc()z1Nowq^Ie)Od@jk#@G!&>@7qZ%E7(hk z%^G&`+@MkQfNniTWG?1qMY(SCY@Sg&mCR;bmYB7CrTkJx@Hv^rgF6BiMLE}hT+6av z$I*6q&c)4 zjH5-KhSG5!^Bd%K2s6T~x})@lQZq~22&3JKHi_u_k6=w%!vLkVzDp_%2VdFX<>I$| z#AL|^y^*qG`w`}l+LOPu*L0TUji}@eDzwkiq3!lSJNfEOS@tMTjnP<6SMYF%->ko{ zQx4vuMSDY0-taWwgw?~mk6vgVYVui=1xqRK9ZOkz-Cb?JmQsh1;s=U9%8A2xZ)ibo zJT&sk8_O|{5&3*arrll_tbjp5udHrhbu(j_-9pu-ut#ef@xdBqDhwUZa9dtw^g=Ku zDT=Y0(}sDKYDwyg%Xb+Dk^<}8&1`j z4`9;@(^wOA#h;c^FUdY|&abnIq0Ke5I?Vz6e2Gaw8~^^4fU)y@6T$HSPn(DJfpPzT z#(rbq!vo*gU5JeQimAO715B@S&ZgM;u5c+Qv`{}UD%d-Qg$I!&U-8qT0_%cis=jV< zi-ShGEF=7HB)+VZc!A5UCwVQ?g+Bd;YXaGE%Qb$UaMLvh0+1Slu1s&YHgJ=OZJjBn zrSwU`;y5R=w5{tdG+uNQPVFVHd#5t=c}c%3v8)63nr6-NuzAIY zv28c7<$_p<%DUl6o>t!+enjq=Rxt8Yf{HrQR<92p!e$kdE^}v?QP~GgoPty2x=_&< zxaPfEEH+rUPs#X!6KW+#3FJrgMN5!maQ{N^#hGP?lVu5$)pu*Mo-v5^^mtCLC&hYq z=7-*xSK7p7IxlRBbrN>Hi121PB?&2}r7m-n`;#i8pc)KoM*5`489OyMU2bCss9=aU z?qHTvL7S&)6Gxalo#gze$Xp|K4XiPD4kmHt0uy~TNtc+p5cQF%NORtoj@{1Tg?1R4 zPIu(B6JhIYsqIv@oAX1vlkBGbG?K}r?6iH^a;mNC(W+(9AH*$x9rNnsxsrc7;NVw- z-++DV9K`2u6Rhn3OD;$6u12`_+s@rrYa6)9K=+Bl95ZBRZJvV)1GHxQAN+X7z9o3E05o?>A)YD3e(nAXVY@2#yB3LDX@ zvLh2{Xi?gQWE#$_n3esW8k$Is4b2*7%xnR636m$b|M$ThpA7tpmz*I1nC#zP{$<0# z-gd%Qr`=m#acFS&2LpQr&xjZvgY`;*NN_eZ8_P-YZHM{uV-71uhX)lXWm^0+eq^@8~07Q zcp9Up%YAaknH;tBewlE-Qz)Pkgz2X8m8&GE06JU8mze7B>voyYlvW{~Gc&rnkE@ zxh9=}Esol7);KNK{mKH=oclzmzbtIW^K{u_ILXcX&kZ$~hs9!R~XNTdmoY4(u`8xY#+!pQySy^v741AZg>)oxc8xSAqB>CQB8q z9=ds~_324xG@XF+BEGIry|4(|O$DWHuQh~cO%4;DPjGrNEnh1p27e-L$XnO2;vXnn z+FuS8-Tc2#oSiTGowKuC>&~Tr@o{RqFFkzwjmUZJ0?PP;zz8cBr&e-)HotAicW9q!Y;s^0^AK*j}7K%d6PJd*at*rWzyE{ZR)5b8$67{(hh zuBCmvhbHs}-QUk*v2II%`b(M7oyUdvBs00cGz%Sjs3l7)ob_-@*Ml*MNIIlZXl*>% zF`Bw~I-R<;bR~VXg%7FO$@GEl%1OpK+td*~6$@#hwv|hOdh6~E>a>$BJzS1YK$5GG z`{P0Ms{Iy+8_5Qml6h{_Ysn*8xzjS>H`=DV7j`Vpy586)GO1ChYUi_t-sGPNTC0>A zt`nS&B=1q-Mo$bv($=|!q`H>&^9KOa{AboU*wA75wO7m*FwLFP8J%!@5xq|9Y(jg3 zbbB^h@cctmNx5ub;iz-mU|YLcqnJ0y{QjM{P1sSq{mxS8UB|$gpTGGbx%%@6aWAK2mk@Dn?X$N_TN!E003~Y000#L003}#G-@wuVRm6(XJK?@cVT%haCu|R zeF<#RGB0+8g6eVc6J&oxGFu)vO zPY+4#fZsDc$JFW={~l3| zifWFl)p7gWKDD}!@0C?!LNzDV>ZEG!SF8K&6H{t+ik}!&jRUHAP^}(R%|mMSkbUB? zT0P89jH$*ws(D1M9#PF{wK^?NR8-@rY93Ro$K-rmHIAzl{Mx4)_p0UzwR(c?OsHs7 zZQZBV>BgjbTd6xry}5e7itfj+)d%?YgDTvw?l@}8RnY_Z;pnSVI9Yv2g$LB?!z!dv zKB7Vz<)bR3;eAAf_o&s6s^~El9#P@63XiJrnA#}uTQe#=jt*B(s_3Z3H-OA zRP-?w-lyIkQLCp^ct1aIT7?hrk69Hy$UmM?q02v>RN+JXCcX|H}lg^!!7AG24VRUwV;1r?sQmp-XN8s2AA_=LUm zDHT3x-aTWlo>k#f=IYb->ZeusjJf)8d-XXLe!^UR)?R&Hg`ZT>LB9GadzD6W);#-Z zd-XFadQnAlDtwMF6=)6V+sD+_OKN*mb?zz3Z@PGVYaXveGq@2yRE!o>_&l$AO9Aqh zXI>YDtH#x`F#@a3gSEug6I@Xt)=H z+Gf2Kx$AMe=?3GMcawIj_vY;Q`0GJj57rt{$4xea#BJ6$HWRnj2s)e6L9!XSufBfK z?Ic00`wUucw-rWlr`C?6@fq%QDQMPP8}59&bt7seb?Pc?yRCNO#!;=^Y(}jRW4~)7 zowoEz!&t)$5#4Omx1+}HNgCsP90f@fy431IRCAv`V+d^6Sl=cAPBD zICy-74qw%fvFuaD?zBQ_LSOE>UII^pA^(loEOzmBHBl3-u4{O z2ss>>AIpc1h(BqhuR?=>z1sU2*G`lf-%Ee;Mc9VKz z_5uOINen&nCEP&nPxiHXd-m~@Y4`jvuhsa{43?bfg@FXW)~FNqC-=}bB95=0U%W~y ze(~bm(t^Jah5ksdTJ zT%Pk*k_kq`R_KG862kLSL{9uTO1g2&zY#RL5#!WSyA{op4I-HtW<*LSFWHaR^B*$` z!k}Bn`$4O7Gm5$AY$wCPeh@FdCS!UM&7mhXRw;hM8Fi+dBMwuvViq@k5u+x=V8L;V z$s=IfVT37Y6maB(=6I{5wnog4viUKJnYqJ{hWyS^;?e1Q#CQ4HfQBk+YaG&O=AB!Q zqVI)k_c6B0lly>VbTj@6ek@=Ainca=PjruGuh+V9Oe=gkCKefV zq9?eO&SnrtvN&B{(h0-D*4uG2NIZKU>=3I96HuylhhYrE2oPkD{RQ$c5)C9Vu z$7uOb&S|Cy{-kk71L!DsIbyHw!wD%bgvz9>(D9B_xJ~FVsrJfhYroo>!be~f#34or zP(drY57Lrc2)ZpI%7I&bSthY6f+o=_5jJg%iPv;% zB+$*e?)h~R6y;G8+YD_^xedM5g18v%Y(`DUZsu9H+c9`yxJwXYRnMk4MV;gK<=tmq z#a-&|Qme5`AAn&8@vd86=LhJX?fPZ_d75O^MuWN!z=~S!)@i!X3WGQljD(0w{Dod8 z$$FDccswKn+hzhctgo*}z&y7~kUuL>UxlE}43j=T5~-Vqm+RKH!a{QHBbnu;pJ`8d zI^JQnduHlJ(B4TpXEC!u9Ah!o-RM>l2QFQwuVVtW1MW)y=XPdt-K8tF+3q$-D2Os^ z#xubqsM;5*E_WS~{C$d5gQ^)sb@pU-Y+*g}Nx~({Kwv}@TwNhSsd_5?AdS%m`beQ@ zH1HNEXv>Z(*`y`Wi({Tdn;l-aj=NpQL>thJNy3jZP}}M@*P z0B|vm+p+fuu0Mvuqqxabo8$#F*=TQ&3Z&JeweE(8ukuFBV?=wsGe+)!Is<8DhP5Z= z88oZRDm0REzjh`x#TR&B=^z6I-h;LgT1Os$kLHHM6d(!-%9yLmXz@O@d#LP86c0Ja zibtFS&bU*i<6&poIaVU+1pKg+WeY!uQ8FJmDv|>W<9kLZCz?)V9(P!p5Ew^P+?V zjvJB4^@N{*S3<McV|xQpzY>GDf5Yy_XKHwtkQh2cwo@xL-VTduaw>m;go3onO4LkeA?(+siL6vVfnv%W7*9 z#E{VIp?3_30#VJwYYDW=rAz+B#fvX3dZt;%fO^M|7!9As!;Fg<@oWZ{qlv^(TXIM#q?Rpa9t?HJ5ccxOA0 zSoOMlIbIMDU1aIB7MVpO*=(~NV)27ko>0GM@(-I<)rBBJO;{L|1>X!ryhm3ZKpBHt zfT2jwvux6#b{j5WDM-4p1w}nJ>Y6Ztn~9_A-Fh5_BqOpm5JIwSlFSjxu<~!!f~>Un zK9NS6$-XLA?t(VD*Iw2mN-n52IN`+z0;Yf@pqtID40KHBGaU>Ew8k}BNy9&j)=Es! zr&4ILL_85_+G6vgSRe1$-%OH^S|f=A1lcaCct=xM`muXI|7QJ^fZCFIym0mUjw;b` zw#QZ6P>$o!`5P^5e#R)4*kU>!jWYxG#trJnQIp6Y0h+dui&3meXy}I#jQcEJ24NVJ zsPBCWufZt0xU}NE>0iEderd(O^wO(~^DF-O1%8*fEiE+Zf=Lt#XU0MhTfRguGg%h3w`V!XuEi;7O~T9o%n3&eXMGLKQ3{zW-m+0nT6&Jt&tR zxPTYpu#Uq94x2b^=ceh8?LY;d{ubH*`j(waaiUnEzienXRE)h#7NJKBLVqrc(1guV z5eOZXV50_Fmw>w(tw#-7j~TSCz!HuhRm1pfOo!N;*+?9S;nhmeBnq1* za)40fN9bEv&FFi^&SLc}x3|_#Hc4@~5v5uK2f(ZcHR8i--9+2K(X`v??uh^TuG5xT zrBmADOy7g~Y9;kR>oa1!1~*;{YTKPz`YO2j*$*_LX*T2k^U5e(zA^;NQmp6}`UG>Wz2xaPv+WzIP}CubCG~NqUpn5~ zcE5k-U0TgFHkv}(J(4Q$b3n!sAROe zT?YOh`OZp_nQ`>m49;3t$#xX&WZVfxA?~nEz1_{v=x&W5$_7R0x9t5TA-`8}_>`gl z_+sy>(rRBiAKt5Y46DyJC}tSxs~%!N_>tjIX)uD{S4QfZsyOB^^Vggi;gT}ZC!J%k zWQQ5I47RPMaj)}ye(b{=AO_BciKBF!m?S`7s$!TmKkvg6P~#-6vittAW>GDg@L`ZpZRaa znId`iFiEoq2F&aeN(?j^(~KzATFokP9D}3h&*7O9xq^c^4rVy2?R*f(XqOLZ#pz^t z@z7xDI=woLZ-x+}3RBJ@XMBJ;>J>p zkwl*6ueW^Q|0)=wTVPkNQR%E-Ps=Cf8$^&`GHvN&Ho(cyG;vp+C0(`QLVv!IC%xRj zS|n|Ry47Zes(}SZ#@eTS<&FA!B*akLyG$s+23y8Re!x2k`|jJ>mc$cp9I|+NHbXv{ z1au4uFcV-_&8r&NJ_ipI(+aQsA;E`tlU{wqVs;sN>Nx$K1VEUGBnuF0Ie;jtT^Fq1 z0{w04oGWS^c!J=e$nbED;h~`76+;-zE7v~{4njYej9MG+?>i}s^nk{zuU|9^n6;I% zyktbRwH%O$P1FrZ|Ep;sks}dkxnmN0&g&qHEt`k^+xW?GJym3izIXoa&kl%7pAUN`%U@@`G_>?WzgrCTOug{LBxA; z!(aeo#iHk3@-o82gXRW*J)?$p%(F5y98(Vu-s0Rv@0QZ)Yeo;l^Eabln`VirD*fO4UiIzM_}3g0v$G$%{)u{sjvK#Uh7~+{XZ8K%BqQ*`(;~x$Fw~2EMSWcF&}MkgBvPeYS}a`u^3{ z!&Vx5Lpdb7!FUnB2_jGHeEm&c>Z=4UG#>&h8KoD7f{f$1XW_MzHYI1XOU%{0KW%t6 zuZnjK2VZF!B}GRuDDcceI3Rhcebcv-&w|U4Bw>u@`>iBgD5@gO^I?*Or<}5Lu+Z;- zhgfx2OWTV%OPgbZMrj7OSQNP9WL)(6Dai$(Sr98rg;!+TT3L`!3U2(zwlnlmbNcfE z?F^}LA=}sou`^_h3)!an>FSdI^4!Ye)w!Oi63%~VA#z0^*81j$=gnjZ_s1<KCrzpXuNgz5*EMe-=aVK++2PH@*H#X2hzJdEMe63 zkmHM@=1#K9X%g%>&`iD~8ZQz*q(P{^UH3gawKOwj7`F#3lps;&Ie!L+AHiY8QUXM& zXajs)-x0#_v}H?35DfYdCyjiYMhbj4 z4&P%1nN(fwT5igpV%6m#YRb@rKW~)%a)CAFqm{{tiOM1GvdU=5Hn07p%4^9`{}cxS z6-*FNQ3nBen&lyJ)Qfi5xR*%4DW~_$)MumFQYskeQ1THDi5E3nu8e9KEfbE(5!`7e4<~wZ_KqHR?7gK-$m4#MMo4c) zK}a{**ZG=CeR3ozBt$baiV#Aby&uQHwlq9N=vT8YI{ACTIV1uw7`8P|{<4zv;sPIw z>eg2=%L~_jlGEoE<~3nE!$ka7%z43_-|D2?apBr4I%i>PM5i#s=lHa&7e}W=lsQR% z6p5J37K@pa%h%5-`tPc~D3bT}!>acj4@e@ZQJcriT-Q5>&)BfzOKI*1Og=*JdBVM~ zC>?uT3tGNjCZq4*H~Zwo%Wds{LA=qCFju0yKA)j?B(w8LQJyj2A}cQ?HS`NZ4BCQc z7QOz4WAe91_AawO7I|C9DTmZERya`568X3WftRuT2xJmGA*k7|sQ5V(@yyw=2$e12 zqITP|B=flfz=dF?#n~qobVAmZHW)h>t{r0l8Zm3n`6-_-sOwjBa!bNUN21eFf{~*L zn7&=s5)rb=%Q`uiB%r-feSME5ARPCz?|4kzItI{KxOPC@uBg3=PV059e}f=oOk2d3 zL?PxWKVa5e=si+=yuXYCXaAkINDDSGKX>6mPImQv0{4#@WEC6gy%vFi&tjSlqa>(p zirCDk{uglgDI9(hhrfs?Cp09PoR5?%dSK<=Um6xY(4ljX&V2trrC{=?bF4t1LjPt; z`<>AOax4xJr`%rgc($ zF%yp(Gw~RQ&PhDon6V&NQVTLm(Vta4%rle7CnA^_JQ0NsN zXSUys`U7epQ!ALPdS+X7m4;Gj{}3wy>FJncCkVeo&RN!0>=*>6xnjx%L54Y(_OT0eK`(-_`ai?whk& zo!hu{OaN1ApO?_2ydci4Tm1#!Wd55W{6pn9S$wL}AK?Z8!L7 zdi7|!IQ@~>pzeZ9Ik5Q<#=zPz9>q8YrsWX4& z(T9dMZBj6d27zK@zJGtgSb_n6Bny?D(c%=ThBf00u!8WH)Q~gT-{eeo#X{=cvetzZ z6A{Kk4D2kiWb#KvZAG4^_mZ8^d~S6nr<2uI7HwEAfo9vQapc8};ULb>(W@V`Q(d~3 zrHT6p*(aP#rkzE09YYL>lc4Yb=6pHfDG6ECh)?C@&1D$gO1kFDaz;n%#}WD16Lu%O1yKj0mXEf~&Z`~tmtwxII^6bV%&RM{!?QTKD~z-ig?~^YD`PGOJM^MpBHAa$UZ^IACn}_{pC}t;;U)7vjwT)KYxKEDe8(A^BgCvuhwor9+R)~?} ztU+lu*Q_-svuTO2yY053NSm>depY9Z#}@lZWzH*Nus)S*PTQw4Z)QsR)Daaz{l(o{ z(vCR^V?32PcXQbG^?(OD-fHTUZ+zoSL8FzRn79z8(fKTwh^>8W(C=F zE~J%eEjB!&2I>)xv4i=Ae%lsjW@E|x_3JGuHo&poc}rHtn}}a{bhwtPD?ptxLmfNn z`%29G5A}fDyWj#Glx<3wV>cI2EGHT?H<{v~Kl!j-_1r6(JwzN{Z6t%-zc z*tUKic0X-wEXjEE)8O}0cd1g-x(}hgq>^qOOm7^uh0=JD%IP9;(xlig@axeo9Be2e z5w;`o-Eb{AU5ygD&kRS1L7es$&*^fUc@Lbc?{N_qU5}-=Sfhu$>RvGe))J=A%U9;- z7nhe=*t)p5^2((Jf91`~iz4xHhA#Au<(0XW^Yi{?>g)U)EF5x#eg`TWpUz*HTV9qo z*;dUWi>*r0BQOrzX#f(uS}?|tl2mwkm~Cd`?8Aoea-g6iNAT4U6J9peN~IR#h7x1x zAzimrfDcx#Ndtu@q=c)z5W1O4FUBsK8pB5P^-$Puh2_ICEMW$3`NP?&WX)9x)@c6H z((;OTWq#$7mtV!dGhh|(HH*Y+oSHVpNCnfB_to5d4^d8pi_{g`$Vk0zYE z+nktL4%y}ec)M6IHmA|DQyHmDGdzPMT$bXg{R!ER^s>XEj>7{ZEc7tKkED!HH<)9E zD5WBdKwSK~^lXMH;<|O0fJsH2-h9cs@ig1vpJ=XT*qKQ(w(RWL!R)NaD#MtHA6GWh z1d6Hoov{{-7yuW@XQBk4Goi9vsCGf5P1K7e)gh%fe^(jH2!K-G9tzw6px;h)12DV^ zQ`#mMIr_^o1F*jnZ#n@67#1=C0uo_R6uxttqe;OD_rM)Xh7<8mQy7_T@LQv4ZDe?I zg!O$dv%Z2U7A0J)NIb6uGa{h{_}y386kh^uf6yBAmlFtNlcg1s~1T{L6Fmug$%@ z==~$y{f6ac`YtmqrXT#0@UU~ArG8fCycOYGd5xQa49+3QP9Fjn`=NrV=juS+J}A75 zeoxccJ%y!#lr?pC|NL+%D`suah;*Q=6QeudRkZd#gJG&J?2188bd8>SD6lWry+ z#lU!m#&+)J-G1|(C|(5=PgA*~YJCi7sIOmv2)|d-8I(u03%sP=+pS!!hEW6a;yDCe z<05(t3)6-|5&|uHi%av1BAG#76{{Ma7eh-1z?n*pX)4LBaV@-YeMLV%G}`&KRCqA7 zbHX`1(jU=Z}Z2hN#vkb=!N8*DZcDW{B z0kiXwF7(a&CxGO4Cvy99REmk5_iJXggoeRilaea$pW(pL z^sie2$uudAzWK4?lb~P7EKA{0T9V`63DWb=>D8aIq^DdiOieP4)@XiN%PIXptP9ZwyorvF)P@v=17GnBgxf)xVWdA{qYC4255c9g$VIJVYoPHFC{ z2l8$_b`WRfS1)QGJXe}A+D{m*bEp|M@yLxyEQegu47JPR+;rroRKnq+TLG%#kd|dN zny|e>UCo@VGQ6NZZldjmzD% z6J5H6?yh$`wnRzI6s5JcGq3|GOq+1xf=x8D(!?IC5#W2DfG6aRsqDWuqT*NBh~^+N zu2s>SMEIkr-O4W*Qpij(8Q1Bol9HB8XbpzsXTtFWn|u{;Yl^G;|0)b&I^zGJ4h4d4 zA_F_bR}QJI!%{Rr?+qmqV6i_$p*MWsuAu|@d>2Iao?lM`2&&jgq&$dLPZvtUC z^c#I<3i!L-ol^v++|#bRS?3UA8y&OWTMZSs9c-u1@K7?J+2Atv>+AIz)}ntVcbNVn zZSU5uFR6)~mY7QL98hh`8|6d0`vo_OG3KwTbs-aGA0u;AtmPknA-B3X7 z(PX~HVl`j%Nl$%c9J{L$I>(WVBYIWYcd%+%rYa_-pCq;*i7(` zm3Da$Ph-VsUN5-pS0HhMc(WH$IDlLvU#lXq@R!IATRk7aby)_2oP-EwAZB=i<8MNc zgASmM9U+mOw5S4U>IM2m|HL3$MmV68OIl@?f=^D9CUdE)Ec*UiE7_+-N`7o;QYacO z7p-5I_mqkeV%p1G(mN-G5;UiZnj1YxeadEi z;NZMV3<-j~!xys%-hAU`OYgBo$T*ii5VbnpI7-oxg%%F6ll{P&0Wv&VylD!*U{ytM zenPek8_MkA#WVs1t)*fD-pr7qm!lSTl!2|7S&f!nWxqMIvk8}5U;z3_naFR(lH%^ z0n==(Rw(n$ks18Swv;PwVME8vGraD zH3eA|^7aHF6Bh+>6DE1~BgN^-(fUstabV8#dUH$fPcI8T z4MwFwzU_1i1Y1ZiU12^=XI$zGRkGKHyd8q1lF-yVTr@GX6U3DZqs{?m(z(YO2WRff zuEx~t9?U1QXN%80YSN;0E{@{8n{`h5chO;avr818Q-G5UNcYq-)zt?EmBh-6NI6K+ zcV_dCyimgKPr)Z5*zd865T;{Wr`((Nql7b7W`8IxwG8ndm_$F+ts1*{GNiqwSYd_hcR_ zFZ4rYuiiC7T9nqe#4;2|5@({Nv34uU&Jgone1rE*9BiJRz};`!`RfjU>NqM#kFFPz zFW4C^j+5YhBo(}EJm#ZHg5%>9na6$9(9g2a;OfLc9qq|1fX*Ruec z?jERjkgfOTjf&;1nr=KYKpaHv*$<#uXA?(Vu2aD9&H-8IvQ60=e{l zos-W6kwoY*p|P4XXA7zIl}IwyUacZXQctFUz$v6UZPaG<3o|Fj(=}pIL#zdBP-1St zLL>ueM$LB26>%H&W}W+qb%o=08>Py3jfumymoG0q&;*?Uv)m>(EBe`<4~L$So#2`E zIEqfXSvyNlFDbAwUHIu7hz(KP1{6sOp>H+aeRfAy%Y#s+KT(;e`4zi$wy7X^lq78y ztqY>}mb8p4K_MfVpb!?kZpvbn`6>u<%2Xh0ck2MnZ3z4wNoAodBX}s zxp8W5{7v4uOzK;-O1v#=x`*&D(ljj8J39@?i-EftHFjKVPwxh6T(ep$WJtYh_2Y@% z!;7d*1{UaPE3b3uHwnDTHm}KI#^%cE^FZs~PJmreT4!BUbvmujtQB&--bkWYZ_ZK!?8RNWcnZHV|9>fvz%ACF*Ems+t`0D2Zekm3QOD1( z^PV)P#0Jc2GZ*{Mspr;;o!3@6n#b~>k6F7VDz(i}%O^}fgY6Gs-Lv7}(XB3s!|arzpL{Ly$I5x^D- z^$yy+PN~s?wI8g3j~N~gIW6fX8z80_u{t<4a?%+kA!};nqxgSeqHqj3^My21oLy-g z9BRnz8B1>W3OaKZL~aB`PUtQSh1l%C$!sglEdCFQ?|oZoac(lFgLSIMw=`gfimZ-c zzB+UJg=*gKmGb*Rj+9~SV$JOTeeGnOD51EqQXOs>i)uS4Y(t@G^j0##`u`D!|Ei1?g0=16v8oyJd8Fpj?j0(#SDGl8N)vrMH+j0$C-y;u9@1mp;eCbj#fNPbE{SkqJvg(kjRq;bO--k#NZA!r ziLIx&b{us(0~9A4p0D>5idlHdnP4*pf}JSN?60KjaQDQs+7#b6mGQgnany-6QS9A) zpCBDcQs$%@uEvy%nVo|qwnPdYTgBNKQ0N$&;u;3lZsaBxVkXIe9A=9reWz19$8BqPMm+cM&6j=Tv*`2kW)kWclk@9O@S|PD&F)FgtiHQoAhj`5* zTM8T=s0s^6*kezS9FNpH*5sP;FTK{^>hL_$zf?5oV0jE={ze_?RVS+`|g7fhsGxObOPbLY_zc(VWuj?)#H{gGgG*+Iu(;crIXTOze{2ho@+(^2=Q4bEaClqk)CA!-zTO0r z#q21-8F64fUA}a|N9E?l6?Ww~k);tAqHqt1QBBeLu{SNdo%E(iET`CovH^*{ zSf2n7^Nua{dFyN1n%WsL4RCo*6wQzzaRq*$6&Mbi15&Y0T7m z$UK?!#dYTRu=OuXq+mL|G?6IjnXS|P5tT2u@k z(C>O{z3qL=Pz>o&>UOReqP`(Sft7DOB^Nw)_dP&I8?p0(@loBXGV!Y_?sXevuJ6o$mKp!`EJ5FUVsw?Sx}Jc#)+34 zApQQZbAs;E?}DudKW@m&Z>dZT;I!cPmwzU+t<~=7GfH)CJqH64wk-T zMlzMz8{+HCw%R-I59N8kmPXEY=Jtu0y2|xk$pfNvu$xR{H_r-9kWyBh#2?O%=|eFR zN_dzHB$+JD95g84l^uf{O^X}fvM~DjJfNPi(E0~fFp|xpqagN1&=m5kKa@?jFrIG# zWPwYnJjtLb8|dM0sc?YL4syvxT%CbMtXC!bKRA4kQrU{NKn8Aph{_X$$-iN^GfMLn zv7^gGb(8C0I{DfRx#^BENhr-Jg`9wttuNcTkV=Zk&FF<7lUuu;^#$9B{o&|bp3Nl-|qYXi(rLR z{vu$Iis+YHY+)s-73aUErSZKq2T$pMuL&)45%~*8jQoZ|u?}K^@$&!7jgN`+0krfy zJ%uqIgb*iVOkXF_>H(s*hnU(v?&zxPPF8~2J!3vGrT!BvwoO>EO)eeDgoQ|pZ6~|^ zBQSf34gZ$T&1cDP5bkE9*Uk3FQZF2GgJ(*W8-E3|39Vl1*6G5T+ahMzYU5p)c&@az zTiFc|Ae6w!krhO&7d{&!5OlHvirp}drJZ@bafRMm!R|sjou|@FpjRx^qgxSF(u%A< zs&okxF2O1$rkrPk^HDjeoa3l_YLh1q)8XYg@1?ny7wNqVsDd(}m6Y)FCMm)Z@-j`P zCIH8LL2RPQ_3I%=gp1S1*OM?jndYNF+b0=qCy5aK8>bU2_yHP7qWKSYA24`F01$_)b3kco5XClArYFcR$GjN7qOHhrYNJMj!U)B zy%F`p3T}^(b7uEU8B)51(9NsYe}VvhlmY%nc;kXGV7gYs=;j+zvq5LX3OIu*?RK#u zD{O}Xq`_-owk^>>8QqF%T@yxW%s8AY&=ab#3{uZwgB<(8I|Y)~yK==_@-HqfFC)SK zH4BMc!k&Q&J~}qabj0E+g#^V*X1B9pK+aT7Ip`Q@KICX95vhlNN^?W3qOzYlSt!}0 z|1xjEi)8=%=>JC%AXu=0@Rq;9-Yj*##fc>`L}uBG3X&<8tJyB9cmw1;WH(n@)aW)t zg)Q#w3~mQo67Cs+GrPDFU71cd?k_Gah!2rNic&{U?JE$BD%UM_getn3-79@rfhGCbBWv4P)d7!f2`!{NksD)asL#%{s zcFuDp4Ynr0*<+v%=#qQrzhg!NjIwsh7G8g}fr>fyEODYqI%jTv)(5*KhX zOFH!o&rf=ZHJKSYPrsdm8E_L1HKG@<`&Ej;%XV|-Y+jJDQ0rebwL z=g#=WO!YabYRZh2aSU1;V>o6&*Gjp?wLFJxY6SLfm29!O2~%v&@J#-ad1Znuzx)ha z7;OQsJB~>jDs6Fnl4lFP<*G~?pVLaXRky)>=R)3zM*vu339nI~7a$w2W*-m2F#nc) z$wuf)f=sq5H(^D)R+q9(COyc6M49*NnL0_l1dt2!PCYK%CKUTV(7O{(ww{3zhOjBz z*N3%8vfq6-XG z79Oq>r}F(TmTYB)@c~1BARYQtX6Vm<=SEJnMMnPJxsmrpzCV zc6di?Ub9AfV*{Jsz-%C5^lsR$aoC>h;>T{o?wSsx@t8j|b3m$=>~_O)02*OQo|!kN z`0X>FQENAMf<$)E<~R16J9htpbXx+xZZ=F~tl%p;Q{0;{BY!1mg$;U<7mPCzNlx=O zoutKt7!5RY*U>qb2$ z5;MX}e^XD5?@N-f?+*{}WItg^9)1DS+SvsDInyYg%H_EZa49T<1QY-O00;o+t(!q`GgFRU4gdf#EdT%z0001Rc{FM-aB^>SZ)0z4E^v9Z z8f$OcIP$xG1uy$S*2ZfjZLquWZ5D7GB~90{eYVpFnn0#y+E$~7LCHzIMgRNFkQ60K za@#HLf})AYd2wbqFUhuTYqnuz%abhelaNHoau+hPWUC-%DapiM5CvKA1dlXciY1o% zbQ{ELvP;>LEcVuTfA(4v=~zHBND2I-Bx9elUltVMG?Q#YGu&mXRp19K&d4g^WX)p6 zY52=_$G1t!tcYcsWJ!D)hQO*#7Q2i*F}^1tnMQmLvJD9-Uo#T#q6OnAX)TyfK`;v> z-4n`LNd^aGy9i(Jj@_DgaIpmq9#82WhSTYV`2eDKGO*0{c6~r0iF%8Lr z#r`Iu{1H`=ZjxuVRA5;J{CWoZ1OjbiZOL#= zp~6D|O2~09<)hl%?$17U{3P0<1tdV22+skAQfuv8NLbuwjB-x*FWUDOQ9rl{kZrI< z3O%kcizo#M$XBL)3e{set-Gv z*>AAtPW!#F>-AstCa`sMe}80IKM^<1&{vVFZ*Hwi_o{a{n0fHGLf{{5 zV|CcOb#Wa+1Kow0z?|*3EVVA)&)lgudp{QYfCd7(*X~3w1-3O94X^d{z_u<%qk-ES z>ZJuR8{J(DT)nbH_WhgN-as#1;u$74B}%Dn!D-xyJM0&j+_rj?N$%4wZ4K`R1=rZN#=ZVuz3YO*78E|{O^ZW=c5mI;tI?%iiGa$;yLE3b zT(qa1*w&lTO|kz3YMQx|-fUFpX$_p-0~KMzcZK{bq3p2=Sl|c3K+Lw>$z&v`e}WU< z_51F0icfoh2ZSkH;>_P=X};&Fa{x>ZeN4({(}3c123T=kb@Z$M3_b9uiEw0;$?K19 zZ+q~2EEiGCV`AUOB~9co25G&ELidv>g3F&;{T|pLCz(5+kJPSi+`(mSYlT~I4ZuV0 z`l!_^K!5<#S9i0^(c7VS-J7{@!AGi4DMAVNsP2Tg1Ku`J_pGvi#P8!H@PgaXq(=29 z!o64SU|gfRL8{VSt4I`);Of1(ao^T9KjNnQuJ4W;h1udZsI!M*i=?}SpX*ifx`z8O zy}PK1vo6G$9so&8AcGsYcTGTH0W@%jjUcRa@`(%PXoM0Lhl5PmDr9~ZK(;EM2JImZ zb2t2Jh<0tFvCAzdAeARJ0Y-uua2o_^X7CXnF&#~WN)G$X#C_vVrfw4u*FaipjY?C# zH|+Nsc~K2>i;=kjBGM2yhd_c7qq|`v$V%z`Dq!K#P=&mH%7GgFM?y@SAyP&7mPH9S zT~l42OLx$OxI{z+(wohC{Z|L|1LR+^aBC13w@;AzYH_RAvg9V^koQQCT9_j%Zugy~zdUlKyCbQStyFq&aJJHyhsxKZgr5 z6Djhu&bnRF3Td=hQu3)y_FXbeV&=4Uoh5(D6-0sXM8h2uNaC+f2t_$>IpiG60KQ{( zjgGENnuuWi*bt_SapOC4-kW)%p_GJ2@lrLyoF9zkU2U!ny_t1 zedtU9h9C@qeuzWogpJ?=x<7*6pF%gZfgUpqMT;iZW9Wc{F7tgRJ1Xc~62(|T#tHO5 zyr84-x?A8AkzAy%=-FhGfgPNrpg?&Bb#fO-X~GgPKZW?s8FW)I_INz=qz#J$cep%g z%f7$2+i(}*$1bmOxyoVj-q3LY5=)TSc9eSSR#M_M=r?RJvBODT}0_kus2}XvL^o%pVFh8d}EvTyIT!*0YOqas|d_Vo@VNn6SkP zP9%Kn{d2}ztHFwZPiQ(_L*20HW3VZ8bKQ#18zRggCK>w-M5<$G`Hs(tZykos*+bLv zbs{2~n|w7c|0XW4;(ou(mGeszlrKPzrkvEc52^A@5H$dI!7%~2cB;F2r;sAQPL@8L ziJ|)U5{77)c2MlZ>O$eVURZHHiclUy%k=RG9oD9t9yMKUOUbNzgpFV9XIH zz*&z-G>lbYEC)5M4|>GXS~%j0F3>LZaZyW?cOG(qg4C0<8*j^#Py3dLD-1$b5$-PW zF%px{=M58aUqi`prgDKXQfQN)!#Zu8_H8I`zNSx98ZDm?$~l}EQbP2f2N(_`7H8?4 zL=4B3W{}g&R2W-)WH5!1+qkWZ`|@U~5|ro(U$%V5F>+ZMp_I?$t|z+O7LC(q^rSV-*fZUG zeu%ICAf@1IN<6wqYHjg)?qGh4i6jGzeHJ`H(h={tFgu_DbK-g8GqPb*K5Nd5$tq8B zvLiE6oTRA>nlpM-dj`uq!LGoUXS2wM6WYQ5^1}QUz&Kd}XYvEoB%{YH_w=9HI@&Un5IB6myO+ zz}LnR-8yA)wJS(T(GyUKrK&+#`*{=in=((39cokD(+m{fpC#Vu&*;{3_0ZX{Ficv` z+<~)*l8uI)vK+{`E`lsg|G5Z)+jk0(S2Qv-P>PF|OYEavDIxh$SYvl`GCx z1HvD>7ZY)GlKB3PNsCp+A9q(Y=p9zx?tevge`~ve)JQAItm$3T|qu4|*hq53zSn^sC^?fkd?X z+KFWr`T>8|J+N`r^o*f=$3VUlEL9b`lLzuI{kpa`dT3!E+Lj~w!$Xgj=n8W%DGO(7 zh>eVU$u5H`#>=w?Z>a@5R9hz#UY<%yOpzufZ~>NoKZAi61KK)F7SM@T!P3ecw_w>N zJ=q=#V*v%KM^iF1t8kZpGf>3(a!ceGaaOZM;;UUOUu24}doudS zrllrEevk!Jq=fapxD^JE5W2}6o4L8Ls-`qAQuAErWycK9N_$m@6MHo;EGZGM+hsh>qX_G@tT{sOS+UZ7hX3vSKF0%Ggkss{HXG> zM{HHrI&`~b@v`I;;68GWRz!L|f+{>lFUhokQ63CWamcfPR!2sEHH$++om{jnkTzHh zS@xXpN6AU^{5Gdp;+3<_ckwHVgzsk-t6`j|z9lF1ouaCCvHw*zzz;kN=-fl3_3<;s zY{Qf1m?UvgVdJ-tkRW5SfZ*X{{mYhumb^TM)Y3yWgWU3S(YJ#KdvaeKYpw&8c#R_z z6=8L~X8Pk&5YK-k8PD5Izz#zEfUz!ND-9yp6A?l5Y*WM)l}F1dMKbXRh9J+wNZ@&` zl!YrO%hPq6nBtfEafInW*2fWbs|nPEgwp^Jz{3CLj)?{k=0b%`zU=wY65qc!f8%pp z@3hSqfB0ka>3F(^QSbBNSM|fKvaY~_N>wGTm=_(V_w^%oyhF)L_7LY4DjjGgm-$e* zRaZmxWrUX(_TSxm%Z|IKQVLqch61olpaj$tg_1ej21m#GFHlPZ1QY-O00;m9uA4!) zq|N(V6aWBfGynh*0001Rc{FM-aB^>SZ)0z4E^v8cwOY+@9LI96S$>Ht(W3Q5eS0kX zJ(D&?eOQ)dT9Un7t|-dnEAb8_oj)V6qn6H9&2+Tv^ z9R~9-ct^lIg1oScqaa4WJO%>UPkJFegEr0CS4_dK<(^Fy8@j z3e0H`r@=fA;tZHGAkKn$0mN%yUId|oc?rbpU|t4^e^W4FzRS6{F?R*TIWXS?iG*+9 z`&F))#@zcL-URa+h@XM^0f?W2ISb+~Fh2w_3FdVWQ()cz@iv$@LA(RzM@^VAsmQcJ+~7_qZgIv1u>jJ8 zHd^{^DtQL*!1O^Zg7mrcPF5O#_#`b|%1S%bQGK$CE3bQ@08ngfNWBvVanK4}-3vt5 zmAa5^$Cr^FbDiTkvGWjLZC`Maj9PZb@z?ckB!%AGF0O1}xsx%ODI!lN((%vMWBE8f z-?WJ?>m=T=W2%+yw$pN?AM5QP)YqjiL)$&y>4u#kl0{F(8-dVUw(DY7(|WTT>knnP zt)paF!$(fMp}TgtE_J`_HDws-lTF#OQLrLPx~BRkStfgd?ShA6hswm zq~jN{p%>{vTaTQtq#91glQVicUZRWA2HaoUZpDh5t-C>7q(&Z&g&xHr_7|~c@^Nd! z_SdoB$z-^E+mq9J%eAp)#vOGC2^&1S9fIviv>S-W$%K5T6$oj$PE3;*#Rsv_ZbpII zjiu#yogj=A^&w267*)1IVKuK_!Lh@JyNZRR;z^B~j9@Y^TMnu{xt|IKF>wQbJuYH~ zCRB|D6DE?Vs_j_XVQ6n(<|=wc%6Js>PL#gw&)%4n)p4TEHtO!bn{Pt-IKWxYy`^>Z z45-NY_STw!y`jFFgPTQPHP9Jl+?T|p6EwXGa^=PYyqaKlZxatwAho>GJ>S3$L zrh0VLV_iL@dUVx8s7Fw2(Sv@}dzqX+O)EP8I?sAl%{`S>179lRVaYk&o@4{0B)ZJw z<+$QPQ#esPn2t?MVu+PdM z=_iryH$O%?GNrZ~*gwsH{r1O*^ePbMb0i`yUp_(?l%U1S7VnQU@u*|^4W#1DoE_%~$ZAIwG zH!wbOT@Ej}+(+-NS)%(<4&e1(e#Xc#eCiY;a5?3zP02x>|R zm{Xx%9Ofw!*KvKedqSo% zOs4Wrc~eQ}px;u`WsomVrur5J*7C9KbzF=DMbO1KuxAd5*(7t27*~Xon3>;7x zT>If+%GZ-3hKcY3)ji?(g3v*|5Y%xzq@oQPk8a#Pe@zV!L861RL{W1~8Z-(9g`#Q= zam4uIYM;bre@E@9H_#4hgW5qa?^j7xU4C z8q}BZ3MGY1DTWSO2u&5G6AdosBtCy}lpw#Wwjw>hOG_@=$BG}&#&MQg>AVi5YQL9P zTdB4VJ?-W{KSTgR$`|PU3xtN#c^Ag|5<(f&k3qbRG16rqvuqYh@GgV!OKHqogcqOQ z32gQ+85nAbhK0O_vzXaFohXSeY?BqCICzj0zUNQCAnLbOgz!DkG%e<0UxWKkw8V6~ zzXRI9(WaMA6CQ0y_D2jQRIH8&R(Q*$uY|$l?P=Yak+>Vj9rT7xm3QoiHUmet>sEVP z-uR(Jy3i$GyvbKR(bU>TjS0MN&^8U>TCIW!qA*hmj&?$e zY`LG}T1v5xDSplQQP|Qbk0aMlT}-L>6Ls{L(AW5HDXQpi@UJPw7*qT^#INk2_#42% zN$~1h3cN9t%vJvx9aSXNXSH3`+bRFU%>QF8J_7O4JjFeTkEINLUyCOQ5;l+LC?3DZ zOX>t{of(DENCOC?4wR=8xe2VE#)Bcm%^`Z%m4%jXHFU?!vKScYYnRq&Wwcc1bn+GG zYu&e;OK+`x&-JHT?ur_AiknD?^qxm2X2>w!5lVchLW%0+8N$)=bZu2vmzJ!?gO!T$ zCL~7+%^+|&5}m8nmMW#HswQv88&s~Euap;;OG(RPY3ZHP-IBFjx^FF38(e_T%0k6Z zWJ776OSS5PDmo5qw#cuxI=56&h5J(vKKGzesjD(t=LT^!hSMhXhOwCPVVwvCc+0c$ zpmDoawN{MU{Rdpn=+o)O#(ARS4GFu@%B_k~sg_m0Tx6}w*iz$fIYPOhiz(kcP*|OG zl#(&GXw22|8KH2SBKDW!c}b#tAOA)Ct`OJJVf4GB7?p z`IsyTk#Tb<3FC3uKEowEMAW~;@cpj{{cyXNC!pFv>Xiuog9%14Yf*X>w7vbR&1=);b^ zR{!2O60}oDP`imfKu|n0|BXc7!AAAG2N#mI#+3;t75XM;*?nlEG~NLuaX0kU_GYK< z`KC#c)pF^yUa?4rNR_GU4u&qzqw{90Re?y#m-*4Z1=Vmyulepa?(k9Ch;AK{;3AnA zjx0jimLW!aGQEjV<1n4^avi#P@a)c1Ufh^2J!l~-N$GH(1~G@RJll^poYqFZPm~=) zF{;ZtNcw$SwY&5-U3sh}FEbm`b%V*NwJFqv0~p(#$;&|B$u>>anW$t{xp^~I-OUwZoW<)<@h<2lp4`h!|wh=|}f zTm1Unr^W8n4{2JRqyR6C^0fMUbs2Ijaz#$CcJ^Ey_tO_;Y zjZixXJzJqMRjLYlyIWhFXSC;o1Ey62&gkE7-rOKFA$E;4eVidz%x7pX598GuwD?&3 zD2{tX4jIPBwL@A_o6yF!0o+p`FZ9x*xTij#4eqh4p51}z2#(RYWD9+Q&SnN6oF*j4 z-DR7$)PrfRx25jI?rI8Xzg1r?mn-$Up{F5)3e@@R`jeM27)Aw~-5}*@WAfML1+Bt$H)!H+X(w5<;|!3iEfsy0 zR0K#b#KBIWGsTl+jV_l~*vfhUm5+?QV~}iJvn|}VZQHhO+q-SMd$+rH+qP}nwr$(i zzWtmp&VAo=&UfS9ida>DYE{e;wQ^?8ks~vUir;$)FOpMRI(#DPvh7%nyrii_=R<>T zgs*TpGF<{X`nHp0bVZ2KN%yX2-R51I5$qmWpS9q8CYJ?WnZC=lz%wj33ZATw@Lku#4}9^J z*94Qa#=Br8aWYrybWf;(IPTr8T-o826zyh*J#4T>ypr8c8h8ewIq|)+SaQ||m8jEp z(s0J6SdGuy5A_px#d=7b#ce*qm1iL4XCOWaw&xnsH&0y3E$ylqxf>&!wq+e>@ZM7- zb8t_F<46#Yyxqkk3WEX0_|JXmg6Sa*EI(P*0fP%aDnZ>8OrLEG~>w0$&qfCu*L=f zVUDM$aQJNuvf*=Uo})7BNMQ1r4pW!`?Kp^Y`a7}suQCX_5XW|i?=mBy2G2Jd-J&yw zBLTrV=<>T8$vGC{l?fAl0G7goQI? zj#isMwp2|3iF4lY_V(Yx*6&1lR5B3U9zWyT3+|EW0l15AyPpF$%e0DPT93Hh@zM>Gcg=8^v&49Ml4sh5(Ord$$g%T> z$h`!dKrCTYodA~KPWhe_79wYHcAq7gAWW>yg|FP_E>Bq_oVu?*+q?}rR(M@ec4+)d z!B|2nGX-1yzn<&i~sL-9_w_y_7t#rgQO1fuu%H@kMM6HC$mQ zx-5Yg>W~AkckgG#65oNG(n}QW>LPNW)$vsDX$NNal!lmrj zR$GUR$N_h_`_^xXFa3f;V5w8Q@m$(C!4Hi<+XGMO24jq(!`Rrx3U#dW;2BEvizq!+ ztOe|w;^rnLDqkfn2|4&MO^-4ab?{`K#EGQgt99=f{U%dd_2d)#3s3O)stY18$|5mZ zPQs&&S-@Zf@1cl7)`3?RUA(dH2`Mf$;lxCT9hsuS)+dXl{yk=_5ZJLdlV#sJO zTH2>QG+#S+tGarY9er;dS=JFI?T_z&K>oQ_1c!K8NY|bYcXgKt__d$__?%MXviDHpgA6%mfF6D zyUySBmP)*qq$7^T-J0~vRov8b&cdu!?0*WzI^oUrcr9jMuC02TSg1dx>A956Q6$;jlyP7i1?YIW>NRyB2dC4e}^6jB6*%xSQeo9jJs{hIlaSciAU ztZ`OqQ(!uQL?5jq(O&b8zJl~=6b0SbnddLEBJVsiFvVu5RQup5&B&XHhy})bCF&HoR2j`vB+0#B?W^i(G@mk~0)!DkdfA36R zYXL8)LGW#h_QOE_kdXlJ|H(wUnm9UH*xCMwVj^PbrD>_ACubsJ)N550*k_s5Pk;da zE1CK6NJQ%Qb8GR>^}mHR|N03dqwmRyrx3{>p$Z@rhW{#HXhfTz2o$g|O02Ol#IgZU z7$YzwFs~CoY-~A_Ede6FezKNIW?Bw9ehuJ!G(kA5zlf5Bsa&{^l9Z`Kn4YbnqmM&y zb)J8Au&t?{q-C^^rDwa3V3A}p-y%#`z&H|IE?vgvXX1RO3U@5+5JAJgt@aXvObr;^gc^udi=mYvHV~|AQO&-|qT{u+Q5PxsTMfIe7TF52Y&%%nQ9W*%`lx^R@Ws{1qUT zK=T#9xF2u$_4+#5IT`=z{{JB}{olw?hX{xP|MfrogN%O>TZaAz+B^sV0N#IHqoIeh ziN2GwqlK-R{y$iM5C*GE*lsW&bY4(vd>FKa28)g6G3-iXv5O)$f?B@BqYa8_tuaw3 zk*Y^ONxnsYYox>%jb0N6z)dV&nwzyAYE0_Qh#1c6*c}^odGdQ*FbkT>5*AKUmD5O3 z-siIR;vKUJBa!-6EbxUl5=a;dB_uGQW%Q4Q_y&;~8w@t{@MT!Zu& zgIJE}L9YXSw1}G4do_L=2?v+C_f9epF=`nJZRXQYoT<*7*u-QC(F{zSSx9HSMdaOV zO4WoQT*7kS{mIRfy&7Szo0~p-PaL0Xh&c%|x;uNYkbj~ z7o45Vbk{%c~xCaUinlZlKPe+EUCh@1KM^`uRkU+W*Ko(W;sREBYglG!y`L%rMO{m*D z`@bHwf6O5=)zw%57ytkV8~_0S|N5vIHK|P5VzVRQU016m6LHEZ7BZe6$43B>sNo#W z3L}ENS@`2c8fKw{(t%VJEp9pb)*r4T8subxkLJoLRK zJVqeZeQVApWPmA{1&Ooig0omxC3Dd@8QyNWnb)E-uxI9y@1rD_Qy!%*XI~p^nJM#B zFZD!2a7AZ9djUevrSJ65iywZ5a3qE=G0AvU!TwIE(SHvS}Svn396 zmRusTi;Kd7V28m6CY}k2CIl4}1QB0=$=HpuB`Nd}DYrwaF*%Cqhq4>{Cl*J?08eKT z8|?5iG>~I*w2x+(kT54YnblU5k-8CE0Y&VB>laZ7N9ErM7@NlRO4CVNSTz4$hm165 zpG?!`D|AcEwU!rnyh~qSe(EeXr!Kvji>U(d>sVlJelfX16D1Oh=ZmNw^$vB`I8&oz zb0nFe6Gq<)qv2KxiA+a8kKVBp3X2EAha`kmlIve5s%rJDm@lg}41TojVim`nrr%cg zSPn2o_<;VQ?-1WWAvrIzj3%5X=Q;}4)4&2;3~v}a9uQ`&6M!F>K-<^s zu?TVYh(Cc8?WIlN-!(C3xC%$~CHL4O>E&3<#42EVP~54+TH#|=Ol5~*iPu|lbzQL# z%)3klQZ6ZRAEjF#qWH7(@=>n!m-C|-`cE#%_ja9wK>G^)Bn#G>q^5QU`t8Q?TJZwu z>9L;=6^n>2Q8E-qAsf=4@-TxMI0qq&;@?9QU({L;0 zE;hogx!~K$X~go7O^|K{6IVXbS1PW$11)hDcKxPjGp}{bg<8Y2x^wyZ(~;metR$@B5-p*#2jgCUq4B%)eKvnGdGDZ<#lW z(D-eWE2&B*9=*NChunALY2EniS=jJGuBYQ;>(Awy^>HOC12AJ8C#O+(XGVfOA`Q+7 zFZE=RiZl|V37w#T>(2y5enlqYM6zH?6a}eyYw9FI(QHRdVAL|ep`dJJGz&^7;Kq6T zOs$6i?kzkcfea`o@JlJdd4T;YWyt205o~@*C$f3|T&%Y@QZ77SBgjm)e9p5iM?jf5 z*9=!2a}1a%Yo9 zKwC=~tKfMQqk_dHf_Tp@z2~|htvJy?jm^Gpej4N_%LR;DSi~Aw@@c(0t{wq&IYSQO z2x7c;O&QP$3fky25r$%}$yKU07+;#K$H}-2rF$OR6rHzm2VUo;e04jQ5mBc&(64bu z)G5rpu92+aKA+xznoV~(f@uEh5q_B_e!P@U1M@hGsSwn|qnA4hv)2L^UOQaBWaSlh z*#d`Jpsb4XPEz~oZUwJ@5=YS|!Cuu+z3v&mxyz~pxYB=7Y5Gks$>ON=%b~f}*CxXf z7J&C%KEH(}Hv9Bv`!t}z*!+RF)B?4y_sX7B{%X79y}7|d5OS|_C+WzhaeDex`!Pae!!#P8Rk1?M+`J`(H*-uH&faUi zzER`6iPVqH)FxIEOZkL~q!8-_@i$_gs?~qMZq#fymsNR%1ZWxxR)j2Q1?;M~l*V%B zWkVMoXG0m{@}n+|Z>$ex%!Fgr{_W+8kTJ2S&CN5=cWJoP59IKAk>JYIe+6(81G(h2 z9{A;nh+{UQc`D|1N@lb`&F^5{q|YDL$zCRccPnfBB@Cg4_h@O%o}c`===K{EpT*#2 z*Bfxe#C-!0Sq?wd4NuD}fUVz*zgh2E!FyWql?rUv3dd`P<1*WHu;}?<_(@ZKvsn%oMMJFK*2@aozXnkVE-{n<`q=d~7H`V1<(^plGOU zpl&D^RBzaGEQ-|%e|0}k!1xQI#Rz9zfEo}V%yW}F9mGz?TbI&{UG-c&Kwq5m1Mfb9UwK(6C@rrLLHUcunCUbM!==*D%G|ru8GdF%b>~dv<8Gm zbUcl{{;Q(ksHUeTDU*rwL2$v>HK75h1K?xEzpeZEZB|rv`n;p&8Y(^%vO+G#<{*1I}RWL^Eo2!mn z?0=2#Ip`cKb8Nb9_L^su`XN(`2BYmvqowGGG4hLd(cMB1hZX0NjdGwy>dyD8#h)xL zf+YxeIzeZu;wTM8#j^lPAu!f(YVeM{q7_S>?`2X^67Qk~+Dw=gW2&DH!)Hd1 z*hPh@g!_*h?NT|F;gFJweYLppP>1V{&ujHJI$EB|_ObmsLr$VY*bw@Ot%? zZiMQ;mkBO;tTCP)O}p#LFvN)9MD^B?Am|(s7(r+rw8IZ(d~RRV3IxBeHhY5|S?`Q? zn!gl9F{B}l7mUyCO+GN5wkR!29DYqZ0f(k#I;NUt6{9$RM4HH#7)D_e&PRQ}^^R6E zk*tVf&rk;g(*`DGql4SD#JIN093hPq2WvlvIBR(2p93d8_~3ls4ba!-*WO{%n%q8s z@4^eaej%OXQyf-d{Ahw2EZ)zUE>;~Yp2sYY6@HIgp_LO+v|OBg$2ciJOl4)V{Uzwe zJW^E_qz#|CW3We^Aia#S)O|=)4FRPf8@wGEmMz7}Z9bB`Vft(O8G|>fLb&dWoB+Bq zP7}9cCMt7@{sF$xCFi_(gLGjCTEvW3iDF~pdfDMTBw zb2gz0VyDs`1JXZR$w7*PfgU%TDy+in7IO$?psRKuNI3?6-BiVJu(R?n5r?HORJ~f^ zd^bSbypQ5&coHpWhH{ea!m3j_Buz`!v050g> z!BYn&WXVUV^F=UKmiejfX4C9Xt)$VGRC!r^V~GWrU`jn0n%%G$FnCQO)%q$>nn=l; zv@a=u{9+1#y#`m&uFd8Eq_h$#_Y+RpEN~vejzkWX%GZ_yJl1tEfA9lv+s8U z*nDyrMjQ6VDD=9+=QbKdP~PkT2uBicO4lXb{lR>aGJEDIy$?234%c0e$>=C}($4{7 zVb)&*a4TI0nt6TcG{!a=HlF}(T%Jl*5ZdODjH772Ic5>cRD|zuFl~^35(C$$_6j(9 zpN_*0IJ3ocw{{#w7?$?b+qAZUy~t=Er~PtKX243}w(%Kepa&tPMS+5|)AEG!){JcG z^oPOg18L$c>)Omb5Si_@W2qC2Td`>p4=5Ay8rFC4ze5-FByTg0ZFy@{N|#Whe4hs% zY7qC!p7Iq>`;^b3OLYwpMKBFd0!(&)m98>Z3MfHguGC_}vM8J7tv&O=n`Lu@XJP$J z5O28=if4)xE!z;cxMWDK)f_Oz4N(H`87OHDblUT&DLhTro0axdq+jp~XByYzD|l2J z_)##c40Ilfn(GxEl4kxUXNxSSfoED~045vAYc*>4u?K zJU>!up?Zo{-88>vY5TZI)6yvIni}aM%&r|1;=v=0A{$f&_y*oC-aOf7Y)_PKyOnp> zi!Vy(_mO0qs2P|nZ!O>&Zo7S?8Y>t5PCpOLS@fQ;$uNaI6ZrsIa0b%;9$vm2%tG)r z5_W)rs(0s+JfY8O#y+5WY$7)0h0{!kmMnu4%u}o1abbZ(CJf3TT#o>Cx5kAwcIc2} zJWjjDVB zU(_oz!Li4hT1h-YgH(WQ>52%U!u3{0PNeuK)CWQ(Mol&Wl z0Z2(Akku)WLY?C2km;?OI zacc)VpwS;L5v-hd`zII@oJ!DIV6}=maM!M8Nta>zDy^(d_D`xl8ss-F5|?K5cUoF{ zfO5OuH-iKc9!)CL{%B@Pbh4SGP(*l92=o2@y^t94J;U8t%h*EH39_$H_fFKvV9(R& z!GiIrY{}uZcJpIbS&X?|`d-P*K;B6dQ-09nL|1LyDVEA#jhiZ*OVgW6Z&H9P#5fRE zdX9mrCx>#^HuY*f*t8%ahh?_h>%B*^6*u*Fo%e% zUTK@8%dQ9PstxLId{M2R)2<`bRm1fs$eAeM#WdFgmPu0hmqsLG#T;4w-P92zB-hzr z5*@r0guY}`2O5>`p7SaGle6Q8vU|Cy`h&~27ZYjZBla;x>O6JxZ%Fghv){<=ua5dm z%ws1SUxY546ET!T@KlMv@r_jjuB1S=?W;_Od=HmhI6~$v?RG$=le%oP)l~v z@%I{@bvD+2#1oj}G>hsEJ@PVA!e}Ci>2GW*Il8z*2~X(%I_dorstOGl@I!$D0PI5l z4~Fl58jn$v+J^0h7(x~)=>-y}E1gRMkVA!t6jA+0S+ zImuA`1P9n9`3wC!_V3e^N8oq0$1W#R$svbz_K|>;fwi@*G)^d8T->xJ(FGE-4hX$QZVH+0Hi(8M~y7OZ+23KU`rvt>7H) z^IZC>OEYaKWD+!x9R+sj{lp6Y^(T04S!BS2&AU8eZ)!vR4|~l{>-NpgO44su`fgml z+2G$MND0@l_-AEf&N|OVg@%WB2I_UHX!6H#kO`~lLgYodzlydgPb5ubF-qP#Yl%TG z;NUlgybZ@4CoJUdZbf>Qcj~Y^JZC872QG)M$oCSaa@Yp0rcQR~0n8u%3Ip6y!i1TY zgY|k!yf%vd31=3?F^QmUrix_d%LQ^?A@g88H0yTrYPQ`zWs#R@`(B02@W>uSwmI-% z3#hw#P*CG;b6Pbl36qNl=nvh-9QWe6RB=GExfF4*SW)IpOR=loYVH1>YT{@hTVq%_ z1DoEeEp9vl^M-O!JncdVfAXHm9uZ4^z+J8mE~3=S*oS%KNL&n$mnlsT7iaFXs15(6 z1}^e}(?A}3LyRn7IYHO=Gpf;uv=d+*AavE$saBdlq<5ka^Yi59}$tpF->#(Y{ zJ6?wWBp7;eZw`3SGYNEHQN7i>6&PWo!a@x)R6g zGUm-Jf~FbFQnL&H3Dt2jR$Dp2h%D|P_$#9+1LxCj6Uh-ikaC{m;1y`obz^9+h6;Pi zt-$~@xpff9z1H(yy@(p`bRQA_<%q~Ox;+|O(t_3DuM>-{8$BG#RRz~Qu)MM0S1-Es z7_ffoZ}>HiN1gI@B{Pi+{DpUh!*S2M&}vv_x}E%k@hh5Z`!m;f?k3$6{$r7lP!YPB(S!b*f%@$+{NFp?zgiUw4FXExoet$|AAxAVIB}A>mG9TtI ziV9jL68t`>a#wy$PSYLnM>dT=a6IcAf~BN~OqwnS{JXrE?xvUZ@~)G7t7AG9U=X4A zxWO3UuN$F`zi8+#z+9pOv_s@yeVn1}xr?4~Hd}&a`NmhfWfXY<0pBqKG25q9g38r~ znJ1`iPLTTneMlPB1-hD=+9h`cS)0*KDU{z(2CyFsr(g+`__&LFd5<=ct&J;(G`tB{ zcxohCWnK&#P_Tdxw3dXW`4UC^x;?aA0=k3xg!q>Z;1_9;tfdr&DHrQ6iY|Vb1rHU6 ziXUcY7vPKY6=%TYoj@M|kSV}buRn!!>Ee04%PGl;X2}69q`+w^339-kPt*OqzFrkj zeova<>(~fyo~q7a%Y<|0qUqGLr?+`CXjpKJtGdQv%gE|KqmchN^Rpp*U+ed^;?w-9HsGFu zYGh^@ad@b2gtTm#ngs)*oohytT%@3iR(R~Wk%(g>rJj}sM-eT7uOq>`iH6VVIH0Uj zt6($+aoQ)wDiG|S)x&<`+L9}{BcFZ^s8=_~XYM8-40Ck$J0TesTsQ9<#4JE`+|nFo z7SlloRTRP$*TivOB)7$|@3&8FM$9f|ndESQ4ijT!9v8wRiS#6q4RpUQ)vh}3bhj=Y zVacHKH)PZI+u)mAxJP~yig+L(l8XHmBU9&A1&WL|xJlZgXLQAILbBvT`W8VwQlJ>n zmfdg9Lq2?p+TC}JqoBR7Gm+WbF|ecx|hwebEU@rkviqjf`20&aAA~0wR!E~(#3!<7;Ci>67?E5B4 zak|Z-_ciIQ@*Mq1Xa!}KGMBQM$|g)a2@KC9Fcv%g745Ot-@2~`kkSE4Wgi)Uyeqg% zzFz-`Q`ee}2)wIF2N;oTz=%Y4H;;gXLDl9a_LL`gUX2mwV!46}iBa>N-q{zME_&0d?>e3& z`sXCRd*PB*tH&c%E5xi1k>_ar%>_RBB8*BwLTagaKw<4%aSH&XP}O;`9g7_2W_jSlJd4zeIe7} z+trV=NDkBt>?EqT06c7j-Xn~BzaBYn=|x*XGfYh>ighm5oP~9MnjE&XC>25VQxg3Q ztuv#kpR!3)U~8kMwIm94q>RQ*=3C>~pF$`hDyWP-(JBY(S??N095!&MyNgc?cXW)Y zl*4x6wqFZZbtKmHzqj;W>87!9%99F5k)nQ+=+-e=C70>S7yKMeL$dOVW1uUwq?CPJ z*8}89zN4&LlQFbcfh&uff;CsYY>5sUO?gE&$=eB#NpJ)+R%KVW;vP}D#fY>?mvrzk z)`~|3hKpoPA}+1LS6yTFKo|R{(8#eIe1EWwK~~Yxo$}YJ{7~*huf^pN!<)5 z#nzNnP*>wnCA;A0>JPew4?BKfwxKU{QpEeX7jwo^n3q4|md<`mcU4^m6fHRn2};g( zWLW=e-%9!C_g@GkaftH9!&G4!vG=MjIPAH24hsnhLly-*Win~f=5M{uC@JFsQ%d4st2)?30m1%*>g1-1uRm$%_(M={Oso)d))>{Q3s~)9W^6d*$h# zzi>6nl{@#E2}mZH^Q-D-iLfKT$L!5j2a2j_CwvMZB=cr{YT+PqZBJ~z*>>KLdTg7d z5zQ*Ct9sO}1Zv!yjI?q5RB+;f`j>dNqJgeuVY#55#lmShh{Q`Jat zq~|$_oHEfE9bMVYe2%tg0Dp5-B7I`UyCw@_5MljkgQ7qbqKtyzdV5`d!(X~$G{idP zB?7x9X@70q4bnLL;yEI5Jl5lF@oLH*1s(?O67bE{3nl?)$p|qno%cdCS6#QpNtRXa z5?_pf#bw4xrCHsEQ$??`jP)=irOEZ#g%u7spm}?O`h9a!^+-lp@u5*Cs(MiFm9{j3 z`0*=|)`Z%e#v4l(B7$*ZWe0Up7S;?FOzL7sT%Q-J4D&X2N;F>$o?O4}lWXwVr@tt< z+5IbcScau3k@aV)B694U>iD*hx2I&w5_#Y4xOyQh#JN&e-nY|D+7B(UXD9EO- zN*7&lz^u_#M$(CeKFO5fwmHat;IByHJNzymd{$lde!Sa@20UVL z?!|<}hsm3Y+O#etF8ij|XvTiD)Zts!8n@sY{n3U4{JEuufY1~gw;BSdD49;x)$d0h z93uWia{Br^72WMG+%-2ilus#j{rHRMD^>_!(t9W8s;B|JQ~+cGsM?@1s5qeMR4S_G z#kuJ~!D7n0g?VZImSUe-VC2wCG`_^Za&YOy3b_X<-FI%t_dEHMI45)%g4yDl%ZHV?{xtc+ii0KImSttb__QkoJuuB%^(*iDF;?TE%wr*&X#G7frPKn$P z2GI^?fRhf_iM^S-(~1lc2DpJXIu7w-cz~VSyl}^fI9bu+Ie<=Jm56v*QR4D`oZ!n8 zEOPgQ-aLw6PJ(nw;o|cC9xBmz5#sWJ9-U@!X8U2x2HB@+9GLQ%pba!DoTxE;(8m^> zo@98x;U*h+BY3doGl46Bj-6@?0{*%MSG#AEp`c3;VJA72LS4-mDr?-)5;soPpCqSf z`@9vQPe`NF>(Zc9V_n^5pY&*To4)S-G)+m*x-VD!$z{Vqv6U%{4w2E!OI7PW$Vj_g z&fde4Ly+6gu0rvY;A*vRS$p_sD4w3xNb+Kw8 z3t?J)4bg5H#Emg<+Hg16?ETHp!sXrY1ZTTEl;p;Jk$n;hJz}xHMcwkWF`=)yypnRW z2{U3(!O%?0+qwcLdy|Ka4ZdX@9C?8GP9-257D=o!hH-A;UYJV;6p%7Zy}Z82W{+*d zRPoTH)*-`vqhp~{>l$S`Q5OKcB5KyQD4KV zYu*y1nRyk0XqkA&0`P>d!5fbpHh_tcM}R>l7SO>J+PT>28zP09y-{R{n7KL0BFPN@ z2jYo76)uE;8j!+|3P)ZZqcAV~%rN$B#4p<6r#AmuIwd)sk$^k|pe?xLr6npImGGY5 zZ}fcazY~wOg7a%3MGfH zJje==+v`&os!;U(8qC+Q(3bZ5X{84!45s6rb*5>R_Fmur$hskYW0S3Q%j5nuZi0b# z&loXE-LN%Lk+texjlz7Gf)c4{j7|!`CD@!`UvS~7jWCBat2NZK-qkxloL+ULnVsmd;)=U zrjjWgxr7$>07{5va})UR~0qV!7~5 z8W*NsOX(Kvid#NU@5HP1ylSh;(Z)42GRV!ggO;k;>*Fhky}EuR)w?$gE^@qe=&;=v z9vi=9>|eRV3Y-s8vSBnl$GJPh)7R{r3dGxPz^m%HP6>b2C|rq2LMOX}K)~JVcdNvH zeJ0-1FvO3()an^~KJA0|vyEO+E10p-#wUtXm5)KrLx}KT z$<9$&ghSnw+4qRd{c`gc8aUq~x-5V#`Wcq`Q>hs|l~>l7I`42El$w4(xmXONA8Yc^ z`8P^JH5Jk!tCy^8Rn%DwRqAh(P9+}*+=3L_P{^NFrBViVm#5m5? zBoN53s~4f)3F=fx>KI`W(a50josOVCm1(Fsf<1qw9HCq=Q_2D#Kd469Vac>~zBm%a zB<-+Ez>NzNu>}s5kIPCN>4`vW+IM6p96I}pPq-E5LeL7}+rrwJ1OhJYIvIP6>$+<@*Q`F_B>GhkA_*5D~TFdbu}R z#w8YNNF>&5wV8pNe9mZdFJ`a?m}e$~K2>QiIY1M0zrq@|T@i%+m$dsi3R658pc&!Emn=*g0gg5x?{LpcZ1fNqTWh5X7- z9-$7aCcCM^68A?;P;W!Ig0Gj84u*R~8hz)ybLoe~mgTwF4gblX;@H!&KXQ9UKtQAC zpR2W+@yxvLXQn7mh)lIt;+{LYgWc$+ zJ(P)!;FL@vjZDmo%*&Gi+)D_mJ@&E&DXrciIYnzJ5x#C?*)1@I{^7T>T=u3!B$gC2 zYs&XR_!Ghb&&xnQGtBZ!qnw%($%^Kh?*XwyKgu1iqK+A-k1GH#Xh+J=t<9E8dT zH?I#2Z5JMcv0||%mRa2>J+MkdrqdoU?Zla8souYNv&sJ*ZZ)1P0-R{!U8t_&NZxDK z9$<-@-)jB5OpI2$FrWTpSU!TQZkazY-tDmFQ$d23Xf0j_QT4|w%zAjMTjKYXWIk!5X- zX~+$B^%?7pMz_@~uBL&RZ$7P%iUkVQ(vW`AYlHN?vFdl2SJv6+1q{YH{V5tWO%6|c zriBSo5lfh@*7G0W7LQuUYw9>(vCB}H5!7uC*M|y-qV7`}BTPm#+A>!c4(tnt5lO|g z0>KacSG!rqiLZMCU2=qXJXd?1t1urfkw$eM&pdH%$V3@PaH2jnxa)dsVBWF2lzylUvVg2L558T0;5grQbgFqe4|1ckrx@6yNa zmFa>$(ZI6hU^n^t&cQX8?Alw*5OqG}b2`T{@+CH={2!a|#~?m@+dbkF|F8u@p4{IY z0q!y48T&+L>v)ZO-j4|KV!DeoWYyLa2;Fj@^5si9NZ}6%23cdQGlQ2;0{}-yVnL1fAqYC>O9l-$lr(g-nJrn&ZzyAC3xnL|dO zjwc;FX1viPyiU!G^PI_Av&JNJC)Zbqb2uQ{R$D6)jWlw^BRuPdO598D1@%SFMZ8UV z*mB*P%}8sLbpf!gHc%cMz?@Q6HYyumqs0gmk%M%fK&5z>o0zZ$2~M8p>_Ta*CI-Zb zId(s+hkg^>)wgu3uG#NoiRjdD8u-O&LKo~E=HvLt%+AHm$gZ58Ix&e$`McT~&Nn8p zSUkxW_+Lq+fBImqAuxj7{zwHc_5X)<_dn=QvtQ08ZvV=`Ug4}cZiwAy=`$wd0g#~x z`6X7BN@-V&s?gL_BUzAZ7ej*v2qUsV*aOZYJ&3)T*hp^eYT{fpJQAB`y0yUyA!sC{Gpe-m!8Up$9D(}%!5n3nd!YPq>Bw*% zySFsfGr4zaYVSRV$iQgYnhr6aDZ~QqlACTwMEpzu7Ly5z>gR8=5N6^V4VdD8AWGKN zqFpAG9~$D&k#g}(KJ|<)O^wJ<+o`=yD952Gz6b?mwM95qG)Gu{vn4gkJkp?Z!j5HZ zg002uiPaO|%de?V^MLmLJgqf?VYjz7^Qrh}OJ{Z{2n^Rzn%L=B!EKx#9i<(wdW1HB zALwY%r4-#CO4(pI2>Ar0hAgKPSJKeWlX&Mj0q$@0k|iuS`wckz@zc^&C=Q}g&&oxU zSm2(7k6~l=-z{IOZeAZ>Yd!p!?Or4PMFB6TtArbOOK)Zupc$>%Lp+uJ{a`4v^Hzi0 zE7w0wLUWDy@xB!6cu;=_2JaHtMp=rOk;Dwn$(l~OG3R%oVHckX2V|#28faHk`7a5s z<^KFksFQw8Kt;uCD2Df&jh$P2(-`YV2{KaaI9$>9C9SC$Jz5IQ)Z6;x!HAnfeOp<3 zKMCb+N1L2GHEj9j;?Prr9U+D4Fc5%WP4!dq0QR}>-sX(E5stG%eJU4fNSopt zaLwdtp3C=0hUaKV$Vdj`k;{cNCqWCXdWFLR0zU)1qogXzu4ke9DzDB(a+l#6;v0@m z4c{MxIjugo*z=$=0-VO1G0X&dF;FPJ@0JdMrxh^7DG*sO5el=4f0~QJ9Oh5uns%jy zP$y0>3YmgqpqeuT-sfs&M*bVL4qfPAv4rsM(>z)+rxW_j`Y1jPzZPtdxz~cxcthVK z)>0tMi_14)ibWQDzuMwQ9AJi<2ODbk%Y18|*aY`giW?QfpT&v`mZ1d)QL;`G0j(`M z)b{ax4PEmcApUnOsH>3jBjMo&F%y&;%TMPxgnr3;TERf3(4I{I!7j>M4H_CdDi(rk zl0@qm^E$kqKhH&M3_-*wGMIy{^bCQnAx$R0X-Ve-iKKH@tP91ncR$K2W zgxPsL!9f$nmm+N+=qExOgFpEBS0b_Y@vxMB`NcEk%iQDx8w(3p13H{JlBm;KCQJOu z6e^QcmSN$>)DC>5R+7r@t|S=3^O{?g_8-Cf3_MLYNnx}PD;vHb)m`GAtl$>e8^2F2 z_k-hUf8>3wvie|kH0A>ERbc4heg+AwJL&z4e8~7LG%d`4=>-VnxtrB1JgT3&Qp&4* zUf!5*WLGMiMGq=G5~QiTUdRRj+KASkNT;gnNmY`gF-ammPT2?~HPHJ9HMAbAh#Kq3 zo+h8F!o1lnw-Gy+@yI5ym6U(mL&SLt464I`2O-5^Qc$3a!;*liB~&j%rqrg2Ae57# z+7IA(KNSEd6O_=l+y-sXzpz;sxpzA9%S<)>rE)VUgO_;Z0sJ}m>YZ?kTXc9JoU@&$ zvR`B8#HgLv+9jE!i$1J8<*^z=1DP&umnw}bq~8mX&dZ_I+KvD~Yb;lF0KWs8KeDcu z3p+n67$b8DWVzH|)L(hj2bKgK82oNEpqXSskWx*uEvqfP1{!}+F z^oZ4U0Ht19%n8EzmOdc&G`Pa=J`Y&MA;@S?S`6y^5;AxWm^A-(I1N|8DIXn`^2|ee zv=RuZtC%S$u5eRG@QK#OmbmV_MAG_m2HXvUg8n!CAX5iPG(QFWVB!5fhKENfc!(`s zAAA0BJ!TBTbZKHWm0Za@ze#E}{K(p#+2oxaJ~7!i@3h?KQ5O}pV0Q{flqa^_;Q<*_ z<_EPK6xmZSlDAL^ePXA=Bv;MKSIgvQIL(8LT=2oCDv!V-KKW`1>~T>4={!FmcAp70 zAHBZJb!cibvchkHYh8ixm)Zco`(L7UJ{qsN#@3naN4?qdsV0(%xnPi|zG`v=@B_xc zR9WD$i`jAB37aFCJ(H-QLzKeHNZqwbP)fE7o!|AK^ooH;lib&zo#aa!a?>LsGr9~P zKI!>&)|$I%;loTPqZZAACT%AEp~5K#yKM4R9(wd3_!c-pgA+)NEC5sihY41thJ0Yp zB^b0Ws3AgKvKS#OI-WxwD&T4HuS(qQV-~l#`t5X&BP2C_^7?4jnyOB;H)_~oy{FG5 zOV%OOhDhmPY@rW1;~W@XKY{f^opjNnt_Y}7&^1Bb=t<<_r;gQ7;)A||<=$20$KR8z zFRQmZ#zkZq;UXJf1K9b^4`^N}+YyR=IfSka5QP(MR2AJ6bV4#783u#`ZBz3}bIYg* zVHB8h5W?)_y-R0DXAyUU@u6{k!2#&z5uy2EpO$)1IgFa_a_Bl%;;rI1cz~2|VjHp^ z_xh;0!=dCUa^iPj`xorpaLMlKMmNw~ZV|-Iq-GT(bR`|kX&?16IgGOCb0v9M!jU*u zW>w!#kWA0vm507i3@6i@+X^}1m`WnIppa~Dq)F4*O&A!tUnDS=06f6&vq zY0ogST&{$JK^mKl;qLB@#~appq>?~|b|arx*yD062LC{i0<*^L6Pg{oo74{TLZj_v zv$_?t1aa%}o0|T*ka|2GRw?quDJ#J~ZJi!mubNM`B80HI;n%%z4lia6!s@#|t5{mrzCety>a2{Y%947}XVpkFH5lnW+`M zzMARAEEl%xZ@KTR*BZZh^uTZ;lVKJqku`tE@V3>sC9pZB&2#08K%7tFEYt6&1Sjfg zL7rc3Lnv-RP)#Xc>fH4lp6pB4)S2;CLO*l2EWW-{JcQ;y_JF&UeLts- z$1@sFofr=(RBlq#3;GBjp>Aj#Tq)MZk7xE5h&~e)fe7Hx$3VXVv2#wfMsRn~;8GUM z#6B-$b{0x5d7`JmPj-D~5pQVGr6LsSA$?3sK!#Ui`TZsS?1G*sVJjkR#`Z;V+4BP^ z&+$9+1A&o8uVC*+!GD2d>yqnzE^xgc_iMM=Um}l!Hl-$WWDNWkHp!b zUF%pI_=Y%^^2ra9(@5Geq?R?e$S$Kx0NigY->kz-^aeA`A`FqW_NrgY!!hjAHpEJ1 zHNM;I6>2UGaSr~L$6bonnABK(FyDe_28ta!zuuTB3a0DVNYSWYf%QMlpe<|vV>3e8wzn+S3O|6W+*GA2K zA2de!lBH;7{4ed_{D6wBsE2QmNi=jM7=k%yNcnqfHF|jKRib6&5QrojlduF~U-Z_ZEG!s^sq~_ECc>bIa0_-dUEBGMDW> zEfKFcL)U?nlV*)$TYumvY(~fL_|9T5ZElRVY^oLLB1xPL>cw&3iiIf_2gCIT<8MZ^ z`rv%4_d*|nRq|nbd^o#-u;_lfz8wLO>HIF-B0ws#l2YW+=nmmJ!X4Z(?hibKbm6!A z$M=EO@jydc6;Ck^=80q=V`?8Vnf zy(W^N&d$xjzsC5gaR*ja-8|tGx~KXoI|N$La^#skr8e6u4*nrPgvO>;IyeLVBmGBb z=TUgI_C%Be$!#8%_ueSvsY{``KV&ct{l*U?VS%VHs|F%dFoM@^U`m{w2X(AyirwZQ zD}MU|0)XhmKLjq^@~GQqlsTHS({PFE5nZWL*kf#ksGDr3yFL|0{sgRxy1a+WQ$cWL z#X8wKnKyBByRaL7iuqb0=~nXkB3WE1R8UAmbl{n-j>XKZ4NB&hgt*3|9*%OFJFM0U zE@PZD9v23_Y5}!O@P8n313hj_2F*66qg$FT{rv&vD*^JgwHJagwDlgEfzLN`Am^q!< z`}~|7Z?c>vvq^HjvwJxD=g7g;i-T!#^&Z+eWj8`Swo~ga3LClqGD@SE!rA{IX!Yr3 ziV5O9@ca^*z9IVwc8vJ*yto9JR@*XrBY*0mE>ve5Am>L)HGI_cfz`Y;4&6`X&^iobnJ^b!qkEv>z2JVC5BM z3{BDQni!e(5i7W!;K8nuX(+jCfliQ(i67W95k-7}LTQXW{N93M3Pkm#?Vem;o#olZf6ABUH`i&@S=Wo~TVgO>-)~qYuB84FlmT>G(YZGx3P|OY+ z%fbpQdTN%1ZdId$!zP3s&WDm@$Gm_f8xk}*9UHZFBozd+Ww!@mC2@47W3P$n3Hd1HU|aWkFghs z&3C>37K{GFaZHka@csOI=3gK1zkB8WAF;^jir2~>d(6Wt3>*-_`%G4oUcxsxl{T`N zMA~Do5y6cl4hRmxJ{}}3a(iBOQ)csV|I_^ax>$B0LvG_lJhNGn zYet4U$&_nxNp{mB$vKf_G2<~o?m(tYc5M6R!}aAEAUstm7stdfy{c<-$neIi&z1gp zrZ<}TE(DsC>^thAk35e><8T7Qh|S1j2Wy&5V_hFL=~z}ZqQN^F)1Gy7(m_?S-aByi zusYlLjl`FDV3M~w(eN$#pLCNq)<wn2(+>WxVX&6Jr2>Cioe%f1h9_0}8} zFb#kHJD7$q!iQpYcYq38h2(wmWD5Nm9^2EO*PZ-y|A-e zXiAvIFt7ZSsGl16dym$P1(4{Ec z@z?Z@KX}pfPB=JtdJiQ0XnGGO{AflGC_GtuPYNgaIuO1Ywzh`JWyU9=DLkVYvKC)7 z?oj0+q3Ldki?tfAczTcU#aabdER$E-N`saQ&-7*XN`try&){W`=Gb=Sp@Qjdk;_&U z7oO3xkLFmG@>o^|Z=uUp9T%S2GhfXi;PM?v6aEs{b5$&P25+Scjy!JCG@j8cfG)0@ zS{$R7P}f)fKaSZ;Va+jvWrsjcrD@-A9X5@x28=BSBrcdZK*8B)oc*mOoMYJ&_if7e z*qkcQ4rDn>cD`qQTSM#1eVLO9!$rNy75s@A^o#FAC0DANXy|MX!~0SEU__oa<(3?w z^`{)qhOJxh*O3G7B2C+Oxq+IcN*BLj|1K~0p)fb)_WO;0nT&Jk;ac~C45Oi@@VH$_m)?b zur*djr0hH=ss}k-vjM-l`aLcaZ0`N)1AB%IUwnSlU2a`h+_|ye;ucp7uZ%d-{?buC z{=UoPVI$$DIUkc;{<${yMVynD(kutY6tX=Nzls%*=E1chrIM0=(Mo-jfPoxAl5Cj|tb@dXjmk7> zU%mp@%we-AvogAwG}`IeysUxAMVX@{1O?}*SHKUgo44ZBBZ@5!K0mK(XMewN;^f}^ zo7%(So%`Led=CKzO%njr{iTaqQu_cy?>T+YaQK>pfXiCJ>40LGfDHiTqgn?1f&GP@ zQZU&uwD$j45xG?Z!0cr$N*H+S#|n5WLT7c680_1Daa7cN1i`v_{qEflJ@b&r#wY#> zj3|Xk_k>P1;V0@~+GFd6MEx#A(N+YvUw_nU@$vG?DTYr;4stF=q-ri%#LCX}8&#>h zN(ygOc?0Lz%tP}r;Rro>0~$F2pk{>nj|JQD#FgznuN7=EN?d7bg6KXHvqE#7rR^SrG3nXB`PN|&#q<1 z<%@=}N&zwHYRzPj-?g8NUj3zT%cfr|Z_m3*k1`8VE^{c3*hDx0ob(qWnR0*`ZQw}0 zR!2V|*W0K}IQ_P&1=B_<(LK4{D|=WcduanPsXT1T0UK1uD37ER<3R#xR8T94=!o4K z+4y+SDgU6#_;MOTs^2Hz*6Lnkb)pe)Vype+U0Oom*l;MVIONxLa$Cx!hzukKqfmju z>$M$_Eb91l8Tc9?4X6Nn`)7rKIOS}XS~Xh;5Wh&iT=~irQ_kylsWue~^nySJ3j;S| zAk+VZB1*!I*uPAl!#%P%fhEb5EhDzX=v93S@wTeO5;Hgp$s!cR0{$P-l_=%%9snpd z&agsvfpXJ$vAvk-{+Ma6?ze|Y*O%dq!W42g@x)W5c zQ_P2BZji6`W($M?aZM!U=g6CXb<-%p54B`bnfPG2VEU2g7ucYIsHVX1mobRLbr|;T zfg_L#d>Wc0P~k+X*F2~c;Qp-=Htyjv2_S}xk^Kj^Ok#*62(5p0@*T7G0E89Cp|M2y zqg1&W!daBa_|Fpv)skW`_TN5rc4}b-21gAOyUd@mqM8s}G|Csn4m(DRqG^OA&y*{3~$D{GeZZ#$fKZ%BS zsB}FkhA(j;$Be9N{k1EGr0uaW#5!aS;f{4^LJv*Ej=7JDx|_c-GQun)RC^R;F2nG$ z`)bwpjjVG|=A~!LsY`U0C+ z1Xiv6>_nlrB;b{F!qCUN2>wrC_Xc>vNNkm}1ps{la3fE68rxagt25hR{wK~m+d1)V zgbmusCkXp#TRtbhB^LXE_Bd=ksWJ^%b?EqH#9H1X3^ER3jqrfbArlNjG+;w`yv zt&yeLV7J0S=b!Qmhsv#ug`>=_}tCy`x9C z#GHs)`b9`gQy@>{4igGdCkdEQs}2w(E`#0d3#!}dC1O~N$;SHBz3`O~ORCr3a5gh+ zC2RA82oEGPH6kXTeV+jSh;zA-6zM4~vW1X=YAR)-uy|jj7bH`LOFot)!-vEPs~PFk zx%RQ|JXs#NM8{utOF3ZYIlIAIG?1Jr(D4PMNf&HL1@uBx`@?L(zNBaBN(aRzY>#SO%{K+G1=^%BV(Nd)+W3w5Uao*f*E6YtxE;6AZ>90SSnW%6@$k&F4n2NgwNi= zgzgIGARF6imSxyFDe5s}^X0kgV1dyUk~Pc}eE1_=Wp;b=CRwY#PvX{?EZSbPioQjnd@ZgRI4=I!r`@qT3_5rKG~xU5QTv zh8uhJYOm_!dG*En4w0Q09nIuo3KeifD9(hFWFegqn0md zjha9cJ2PZl0`=_Hj>MLug6yv!XV|x-8y9akSdMGv5YcWT2$GyL>Hfz$mTLok8ZP9D zEiti_Q}vJ-8OtsM5meurU^AKq>h(3EoA6k6TUyluO67u^VJ=e*1~2lgy5a(mCVD^D0=0uQ z6|^H6TJTCu6u?D4qqukvVU?ns5RIX>Q%@X-F8jJE6^F!oH9`o;nM&hdC$fvEp8;h8 zd;g0jz8zau*L%M%$Z>M#Y?>_1=rj@&AxD_4j^RNkto%` z`;9+*SadHtn7b-wCKL-Ei;B6Wj62v0Dw-n069oshmoBZ&p8MasiUzQC~)hMTY2-v?yU>Mz~Fobcqy#eY)ZVTyUjzv-AUj-Ela^E*} z8L~$PxdGY?=nNN!*p9#ms}X1n>nsGbtO$CcK&-q2(PAJ>Y6)n0PG`C#cbICi{RgDE z$I_C7hD*@~*Ft93G=DC3LPo`^s~cr?JVyv!u=o&|X#p_|Bs^O6R``2Fi2r-T|ENq5 z(+aUql57WT;?Dp)+3M>6VbLZ$5$0oXM{Nb8qLQ#*f4FQ+mKYn=Bqy4w)Z+!gUt`t+ zK%s`*q%BJWdk||rCO}4Um~mPV4eAT5q>Ycss{?7w+^kxv*)5xLd+_-+25%V#ln=sj zgVKmU%PDeDpPO3yYWT-g%i}ep#mjKkRuJb_;0gLPxcLOwAS(xh-{?jJKF-kz{&vU) z5R=wdiMtYp407JYRe$C17iLdM8lIWq@)53zQLfI653dhKMZHkm|Ka2dVU)wHOy-zc zHwT%Xt7UV5&cvSQfuDh%|D=eT=U?9W*|`x8sO8SlRh5tsvk zvZ}Pr$qSb;cVfNy7+FxngCGzWnR(*mxbne(DHIW#UL%T4aaB`@d1=amxIqXsKGx$F zt;R6m(@2Vz>8El`tjHq=OYZK640h1nA ztIB>TN`}kuR@h{$NgmW+w1Xh6o;4ndr3FT!bq5ttt1567nPY@$&#kGSUf&I26Xpk| zE+Ib|ZTS!`NbHwTX~Svx7$ zEfi&DV$S^qEOdXM*olr1Tcp8CuBq(@so0ZAeq4ml(_l*ZVm_oKA z%;;>je9G|m!M0L0)o)tOlowb247sCWjL00bzccb5NiY2w@NjDTYWe~~7Lt>3zi5R`eP#%WC-{NYX?lj% zOyVUj$*g=inHV1)ypz*NZt~M72=?MyKC>j!&la8BYEj(Ief;TQ?N^FMI$k;Nz3=3M-Iue^ulngWKSyZk+*5@^;!lT@Ml6tZ@YtA zD-+eR{xd$HeG+v+r_lyALFLDNtIiJ??2V3eAMX&?&@?5U;3k#$q9k)7xiOKx$dC$L zRgwzGOAPwTSfGfSsVIeG5iS737{VZcE1Kt<60sd@UZYq5`#f*Ny^m*uODyCTh(hYS ztLsNfCP=FW0-FDAji9!RJT;0?Ex>O{o;p|m`=r>^>B$(AilfuhC2+yPWaCWd=P8f# z5-;CwS>Eolfbk=AVR-#CUIii54|07o5O0b4cGzwT83>6#0+Z5FTeN(Zv{*AN&BVu5BpRIjRI*4 zoQ3LB)xVix6XBLZuRT^zhA|i*;rxS8E*fDzXl(6)k}`qHU!eu9r0pJox9kOP04#m* zGvfJgzf+y%2bFIU-yBjX8~~=6`TW}%4O4+O>ddxm$THTYVX*@#IEV$~(tqiOJm_oF zt#aTyr-isf{(7OaD#r~}?4;XFJ-Z0>-qCKP1#MsPs(2hjah6;?b#@&M*=_T!&FgMc zHe-3pwPFKq%Xdu{Cf67xGD5s8OFrMFQL_#usD!s&Mm`G2UwJgeQk$p$PDMWP6g~dd zzywL%x0wqPw#4@ix*n7WR)Ujvj9-_I4Xb%G*x z=~f7+u~y7LEi0CM)Y1l;kO@Q_&_vyX13#7Q7A3GjX zVfSV}uo-+t%)KuobA8OTC&|$@?0`r@Sk$f5Q67jIxtA%R~1kT+7|?YyF;jC0MU)@a90bOcGoPmy^X&D z@Fyl}i0WCgCPE~ji14m}D3>q7w;Ic?)sU9+GTN1#B+m>98OOn@P^lsie8iH?;OwXK z&=3+}MJwMdELtSD_2qKB!V#``DUJne)*Q-c@-PX7CxhRV$Sna7QB*c!ojU|6A%0S9 zJTBg0l+Ao3F>M%(%*ac2m|XVQR#6_(e|Hffd4a0kln0)|Har0ic-Iamas><}!m|iF z=C7XsT8Pp^=5q-uM^Z|Hgna*Sf45kEF-6?6fryxWOkB~@m=(xi|LlQWLrWj>J)I|y zPFV?mHV3|HIgd!C#8-u`xl5`~Q}|n;CKDUv=qdpWH5NoH7p4-$r+LbdH z1OtM*5AfpdA4IeEL~zubclkax&An91Z`!)~MBte${0&;ld1RTu(sA4xDVMo{bdG9s zN`q)p3cDBRXw4YD!4+Wn9*Ag&OWYp|g^B472|8jV;F=xu8TcdKtClGV~wqtQIL^{cg zUXC!kS%==?M`SXd-6H90AyJ8lScch)A0IOuT$o@~1vfF7L{wTveTc2F`+abeymji=;;T^bsCPcvGnh4p){AqE0ix$}wf#?1Js?TS%Kb9bIsCD2(KcjP|9$PC%4$bx!mxEXA}e@zSg#YL zEk80Sj0r6PJN~|Ss%*okX6@eA!idA0qniG^)a01)m+P5g{XI3FHmMq->U?QSNu;1# z90eB4wSNit(I?0Sq*A+ey-h~Hc*}A`hQ15YXwxO04^ybNVyQ2QL?76hwbYFd3d`pi zpEg$LsjbT+iCdO^mP<2{*w=-xk|KF-e)Jjw)}C@h1IyuSW&j{GOZAMDA%VhYi7{C- z?Vi~NIu208cmq)Aha$=_(ZyfML~X_vQNw!fLt8rez$Y*tE3yD{!h*Dw|8hGx3J4;WB{rAA5bmwuQmNnI@ zAcgK`$Y_BC6h=1rGR}acwL$`G%aOe|OFd-rS|)i8<>0x{?F}*>{1@`3v5Dg%U%B^Ax}1qR(QHl6mS0 zXLu^xD#*@8rKGJ%x%dSIMJCj07SGKnl&bqY$<|)KzZ(3~0%f*F8A1}-4n|Fp_qKasNfpO_L`KQ#ZWZ__ zTIx1J6L&;AMXP)7F?I43~!Q)pMB|?TjW7QAErob9jg~Vp`NP*OCh&< ze`0_N2~PM>Z{^V%JWtaN{Z6J+--W=VRf;(9F~C_P3q$6aqqz=z#e!ub9XA+4ZiBF%%*) zZhmiyvIm*Y)N{e18WQ2Alvm5zvxKw^Z)9PT{T@^sVujuX(zSd&qRk=^Pe)->M6y*q z)-8tcMH*3EtbZbp6iBp@M*Lm7l1cz%3R@4ZcvRpvS?V9Y{2xh0qXrwbYTLM|K|CG5 zbitMrc5~E+#gq$p+)8sT+!2_G%(jP#xo}AzTKD?`xD}z=$f3`~goe}!@$bS>%?4+j zqKXiqk~7bcL5SoK_4a*~o;16HbM_0l0suakQxD1(Yt7IY=>Z2nc>JAOt^GBZkt&0I zm#hN>JymH{%c3ir+E+zgXMdk&B7&%J$H(YSUHVjg034?V~WGuUuD}cGg)O9vU*pl)7@R zjd`LCk1)`Q(L|7bK2nCmKp_rVHMYWLWJV=ka;u7}hzUU=3!CMDKp-0dpZh5_{dc%w z(?jpHs-DQB$DzQ9?ZJT?h3Gb{wV*(0fV%-xTcRaU2&RB$N5=pG6HaOBV`x?hDg15A z0o~CJ*@?y@c{LdwkvoR(fV%|@&fH%-9Wz2ZBBE-FF~r<_dqX*}mo{1?A)prLzg_EB z(A^J(74=EiR&N0{Uz&d#QL{XC7CM{SrEZI1k-PhC#ca6SjdPO-`U6IzJ{A2Lm<(zm zqf|@~zwxv+)5i2-YqAd4JHUk6MO51(!(;G!RI5yiNFg9)+d^{pM%^k9LE$Onxr@;9 z{kp8rpwBx0*<`UUuLyK?-Jbs;kC}8y1nxRY~z35K${$89T zCr_H7gg**$;M$P^gr2`GOoCEe9%n+)$6e^^%KS~HTJ8;UP5OCqr zj{x2S7y|};yjD2iEfYkeD~95Uh2R^&vq_ph%de>DbXMgGAe16{U?IF;R}(&>vp81B zG(+FrE%JYk(qV8=GF9uAI~Jb5M)7gMI)maRP?IGrOM1zhoAjM;ohvZwVp0CbN$UL1 zc40$~9NSh*+U&drb9y>shMd!I(x4F@+9hO>;N{Yc3E%;jTD>5CW`B zdB(!F(Y|oW$S9z$cQjT_SjL=)WKbPdOXyGaDUcT%6#ieMLeB!wpUB8iF0=RZ8|#bV z(A-x7v}zunWtb$WtG>w|m&ZRM=zJ+*SI$g2U-YG1jI4FIE5E6VEMp)3V1Ew9GD(r# z8Y&WoyB3Dd)+j zAKoG)g3@txk6dwD!nUe)bnDdqlkYv-j&nLY?jigDr}kH{>vK>jgL0?26UtSjM^-vu8lw==R&(s)EWMz{s7FRq9 zWMVT`I2Q|;%#_V$IUqQ1O=(nct9G?)_`7t`PB$(b6bwT;EvZV!N4BD(DH|D&ojZ9U z@e3LOg+@qkVG<~PRE6l zj+)EYFh^8S8uGKX1YX(FMfrGBPdSX*LgK2alv3k#u&ED#gQ&wchLo8+HVg0p>g8h- z`pR%qg(1q>9~f0jXTbg`~#mtgl4q2LOaep4T0)2ffi=GNp&GZ8Z+z#hqpVbDLjr zt6wl@(&ozUy4VPVziyR7si!`|3xjjfFD138xzif+y$IC%mT;2yD5liJN|(8}-NACA z)mb{7J3ea7<9swG!$4g~0*0}0^AN3RD|~ukNk`O;2TTE$_S#edBIN|9vu;GERv>ML z)48o|q^Hhb`jWFd9jinSw``Pt7keliQq4``#GzhP@TH;7H0-JSwyOT$vtzgUotlB^ zN%->4i*D}f_4r^st~v@;)06OlX0>c<pb%s8X7{U zf*8RYMKD;g$=Tmu({yL;l!=NdEhCHb?CZ{*etpj~{2hVP-M>buFv)s{nG@MtFIKv% zrMRQqzBC+a;`g^yYM@x=v7(LeQiOZXIex{ltuqt&dVW6dM8bc(ZUnv}`+zk`wg=U3BcL{q~JM;x_w$8<4TEK!Y=Y&>=Vs+H7Z$kh<*5+Ds*slwE2&TiJ*Lx=Ho{)2 zxlGhZv)N3I=Yow&)B4v`6 zK328s>*X}p^UGW%w6RA4vVYDaDbgGYg-fvNV#*Xlr12&ss9Y{)XHk$nns^;%qsNv% zlYPERVx-8ojTAiSxSG^Mk5xwgZr*F3H~+l*k!^{BwY$yXjS%7G!>3S9PP4s~OSbSl zY6?CJ-p+S40DekQ&;6;WFyvpIoP0v-)#{TSvu6du^T_v2A_;2WV@S~db-6Vkm zgkhpAZ>mwU6^l0vFW2tB5vRvX8*N>!_zT@G^vQuiAJDDqr@{PVQNcFmxLF0zrtrMy zr9onOOAmuH!>b3watv>Ykwp%OuaPoehH$hE-70Q%em?_?{f)PLh z!U3Wak!*Ys*(G#O?7s%{5fYj+#-=j?mwoYH!scUpgXP%M5X6j8JjCJ934&6F){qU7 z(rg+AYy%MX9-LJ9d8>K}h;_4~RnQfpv3Tl|NTPP#(plRLEOo0^WYN5EB&OukoeGSa z82Cv|Pf)w0`XT6&Q-pUrAIs5e2gkNphd-K zt@ij$Tj{*QQ1#^~d#Fui7X3&oGzk8r@hgKIIObsUpaZP|X=!RpEZ^TnlMibZiqz&G}Mqo_dY`Sz0w@mLAH_nyJuFK_^ zlu9x)$sP$x7}eQ0qK3!`q$}1l`24K#$=DAVBebxzzBSm+TSewvp-=i0FEtHb0MVF| zb$)R?I~~p;#>f*;SKVtjkS1r{ESjrm(z>)(Q^6L+n)N0Qf3&O@7H}IZ3LV^^05>5= zoeZik10?g(j;DsXJMD#~W|`iTO=qtEV#DMw%L8^(_eE32)}NX@|I%?*HEel(A?LC3 z?;LAqv#~`umAU{HNKN&(%NOvcQarY-UAtSY**qX;OuH_|juJDkH`CcaD?WG~5a@P7 zqWsjLTlJHrO~lvdYi;-6F4{q%JOOY>zyP2m%mGjmCIBc1LjdK)R)F$}617c)PN`Jd z(;~XPBx%oke)e^df{9(3O9tcdCuqXA!L2p`*O8{~-`h?}w~&P|xXV=pIqh!tF&Et5 zxDR7^f+nC@QK757M}ph-SO%pCi8TH3<>f&9M|Or~97toRj~zs+vKOf0#G)MX)g?=z zGO>344oyC-<%H&37}#YE1m_M6&#tSm)@BYh)N)k(i6P8|*4B+K5IUuwUTwbby2wNpforc*6djSxUEGD4{aQp6I?P&+e$mS16vsB^+&X^)8P%0JBis= zqS*7>T0C5lnbid(UJmmsyAoQmaT$se_;H?vT|HAP*oAyQKc)PvMwUvG_R#r`$yhWo z$&@#vRyrM*0)f|IV=CHibWt(jyiQ1uwMp25JVj78-hjjC)6kbhlAo4dH)x?T|Drj0 z%n0SIHfL=1(|qJg0HzgqmS|@4z%D*DiIIwWbwQ;IdUHXgk+sOkOzlZhwVcC@C3VGR zazr^gGS|pVZAwZlvEcO3bd{RIi&$Z^g3WjDmWAsr^?M+-sjv<;cKM1ijdL6@&pPfj zQy?&Nnj5N=;(9{Lf0kMa_UzVcLC8(O7?^}+$8O)0NqbgxI;3y5}4<`4Yl)&OBZ z1k7g{n#=K+wj1e9K9#EERAWj=*CbhyODN8@<_O*F3O{NAz59Xed@&xU`C&f$bMmWa z{}Dfu`}=L$0~v*?rX~|x(<%{jacMf+K6{;Z4k-%$fhNmX5iQmb?h{vo1j+mn zL~3iyJd-I_WNYM=dT8PC^Y8^`V4FQVF+cvNCtQb+4S@fmo7h^aCr?e}2Y8Xey_^Y7 zEZ_>`;1m+^f;HmPSB(g~zTMaa`L&KSLs=|I7WMQc(@=4pd?bchr~!x@zGr6sW8UVn zis|Fq@@kaNRW<3ddhd6}eoQVP@%yzm7Z*Z!F)+u_#Ll`LU{c}LxgO5{;#_sdJIr(u zr1)?D?&mGBr*jFzS=HfAv9*nLp~>$hrOfp(q~cg`U`6!h8p;C^Vbv(VC%Wa&3~ z>K3)en>M}=!>QiN#jp6|S@m`>Ly})vI_tmSJUm#&oYA*_ggfA5rIl2X)#`+9t!Qy> zyOkr|(510FfVq^DfB3}CcR`Ci)aCWAz-QU|s0L_3x4u3xEz2kR=dFWrMzh*O;`$M* zLn&+f+>FSf%)55-`q4%R4HMJcSkfoG7`Mid=!Pv@M>*M!bKXo;thOkl!WHbYY;SVu zg%ZKP5*;y40N#2KbSZJi(I}6V(5TfaT*zvoua@ppk zJDk5KqSS?$p72df#D`aLv0V3;@5mC!ugq28(EVMImI3;SnA+((QvWaZEj}0Ndhx~+ zcmn-;nhK3jT(kZVpm%4mo_#3|1pnTQs!W|uoYHn*zQ>ZxVTyE`TYQQ_;yOh(b0L#hS#e0PZgW%^(h4DkcLBqZedug$kJd80E@OD~du7 z9w8)1II2=bQbSfPOiPt2D%_?^Wt?Ew>9U#DR@SK4nP5iH6)r?;&he!bHZs!U?Ot`` zc96RRvwIT6#r%ctpQS1%tzb0~$FnlWpU}IZyB&F)LeY~QdLT?RmTlNdrP_z@Fwub2 zA+8``PyZHX8rX|Rdug`cLWUlA!0ALs9?edv(ODNZ{=tHFf4aoR{aPby-za?yv}8mL zEMZijoV(I`$@d}xspVy3#3{(i!e-IS{}Y9fL&ui~N(hC178jS6M!^F=zkE7EqfYmC zOsNSr$OK|xU}2oT&a8C zBEO{(%w^jr!bLq-?wlHQ3)*VXD^YtubFsVEs6|ajOAkV^MKsE=Y^ZG7nG!;2wZI{{l&yr`1<_~R> zjL&0x{OBx#iTvuc9jEka_}{IGQMyc-%HNJ_{!s{RVe}rS`V-JjVGB6bE-afG>+a63 zngEc%Vb8)6Gj zEZ`V*S^i=~M2irH`u=+k{Kq1nF(jeP{wsf%{B~jgKXsh{DS!WuIbhVIX5+Lef}ksz zddMM}#AachFW`P2ZE(LLFBB#zEu?qm94aFZCBr*pj&hMWo?^Yq)dG8XTkNvTCVXkq^JbnQD*Wlh%%WQ$ z-iC^wFmvru;3{DT%R0sxInybgrJF~Ue3kaKE1Dg(k0?Dl+S)oDy{gr=KpA|O4?X&8 za37yy!xWrVVu0m(k9lVGeB_qSg8M^>Dqn;$tSYb%3-;OO`=a92tBX+3D@HumkL^^c zMO@;Hy?ujQn%7u%KRlc)QF?)^yO^-=p{u;suk9;w_(zneg*}0{bA{5O>jybZhP0n+ z0{xpqmBQ8(aG6brd4mvL&1lMoHq@2135HgWDVPFX+owj+-sf1mX0@d#g7Xn7oendy z%^qEx79d;~Zdi&7uIlms!`M58XA*7EqOomtY}>YNcI-d4(XnmYw$pLPwrx8(+2`Tz zqlf!a&sA&U8@0xqV?22cgGVQ83*mXCv7Ri%ZS5HfMDE$U2t+A&CxBH0-i%(~*2X6f z%L>)HDi9Dg>aQ!*%jGo;6LZ1%2wG{njrl@NSqHC}{-53o@ z9+0qDYNJSz3n@s@G%YNf-Se0BNRK2-^gcmc+XJl%?geRQ7*>d!c7qDCcf4ca> zH77jz9xBNn7|8Z|z+sxeDHZ+a(W~s^QQdXPh`{x?#)Lc{qjSK_Y>Ky;&0uoyi}2g1 zxoQhV(@DdV&phOb{1&E@Zu!j{)p(`Uycw0FOx=q8E_`&&WK8EW8l+_}KoUdxgZUt! zK&gDSHW|i zLd#4YjWLD?GM*YmF^AO1(_@UrLLr4bAADvK->{3Q8VJfkY8U3CbVAAn`;?4J>ul)@ ztGVX`ttvt#p$3XvfS+Loby|;T9W;q|-m?jndKk>!edpj=YE*mNuv>b93~zbQLPMXP zm5?7R*z|GT;Jc@)uw?91m^gRR+9-b|@6IXB9W;JZ{(1#Xo!XOnRE>TK#aG556`BgA zM2X1!HUPglQfPORmnUj&(KqJv55`U&C?9E8G2$j4(9XK|6&&GrI2l68+4?+G;V%Ep zUN!?_y*pr4#b{62E#9e6i~Q|-)X9hNmXilG;`%QgQlyG|i)>s0;yJx`97qL)y&XV* zUO<$UVmz1_kJ6@J1#jtz7&&#YQOzvr#Jf}#L5Hn~3vteoBC3oe7jcYkSEk#d7tcdn z3xY#vJ%Q}AlZ38%c0``b##lLbB(TVO_9p-jKuDU7LiPSC~bGvq1^vt=-ui) z!RO`ILy+hN*{*odiLADTw)*fFS&39F$Pf49vA;Ju@vK*n)PNNfn?uWi@XSau`hoK* zK909s8n@m_p9tU1gErA@Ua+payujC8gK_8JxI1!}56&WbOKn_$MAo>SqCK=o&c(o$ zO$$b9MdzMB_#jAmPJg=A-~h+cH#g=(GK`C(U$4qtnNGUu;rCa|LY+wA*0@AQS}FnK zlG3F(LoxaXQsr0%D?=*%-n}gZpniySb)^;EIL zJw2QA*RN-aC+c>^-PF)`_o?Xde=%eK58sN}uT8}K13dK*{@=jU$@IT;tA?%J8Y9wI zj)CDUA}LBF&cdY#S&@?D_OQ6xT)B#x76by6qrMrB^XD@U1rxkwNR#CnYm;|B|MrBF z+ca0vCO`)iShhcll<9R8?6l`R?yBC+M(bLuL_M67%v`n%-99X5ERK%IEWc^Zq8&*} zlv2-x8dZ=Ax{wt)!o-17=19<^OpuYDCs@vOfCap1rVeI;a-UjOj2V<_(ET(;X8)*U zMf*_QY~ER;U;wHrHBCBzwPyY}vANR>S3EUs9o36nzE;UqDU3Q}P5J6Xg>H@b)zML| z_C5Fgp@cx!HOvGy&oQ~&;|@(Hg)J039wt(B7dKKlCX|q!(B^o7_@ztG%{7TP;tfJB8j zhc=ZEDM8-?g(ydfzVZI7Ijf^4GDf@(7=W9Y@8{}vs~tOeoJyPHKjo^uYd#)Jp+N!tm@M`i#8dRXN@pd0 z2s=D(GEqyj*FgLo_aakp4c`1LVpVkgh2CUxsiz<}NB8YL$&(pGAA6UDK$W`bNl|%r z5*6%#k5b;f>HM4qCHV*1aEV(S^vjau>vVf4_Lhcxemk|npK?`DWH|#7f#Mc>q`W%5 zN9bjm!UlkNk6*s6)awe&+Li2k214o54Ix)McAO7a{A{$-UB(e((#-)H@5p>R z>DXVE?N;`i;^#{5 z5IuI13Ho675bdY1*Im|>*BUI?VP+-9jaPUxaSQ&&COTl@-hNvNOKHL6z;DZ=wcW`j#((Ukr>=SZO+(_DtO%FVD zP-a9I19h;5h#`Dg*#QO)`=~q3T4|uQpbhYeiFJ_-nyjx_J6kts?u~3>t;$&t9uQsl z8ytk6(Yc&5Fv!5Gj;($mJ`=rt{n`q}l|`*c60qByrvqolk+79C8XC0cA(@rkoS#;^ zC7Bg`*rPYUcT;#$;?K=f6ynctIm;58R1ac}6u1LwP0k?FGX|yHC?bP&rA81FIn{iG z6sC9Tl7~ssO*1{sFhb$owm%Uj-FNU$#KwB6y_VUtqMHepv#Ogfc`8Cx(mQFgZ5TB4 zKL{(Bm;QdcjIWq_qW>n2h@uK&W(D*a2MB>bbOtEc9I?vVPh>fEvr6BAcacLblY_p_ zz$Xft1+8mq$|Up=w2#LY^7B>EHV|)1HNi%x%+s|yl}1_^UlVm0do;7*CiZnRti%$qcBXdZ`!m{jEG zxev2&NM#XM-4~JW_obnx>{w|}sCD5>oOG=XvsD}RMeP$jb~L3ml@rn4Khc@}4G z2?xHEIU+@t+0Rr_d{WHxQSxQhX`F160EcqU)AG>*;Pl|>5)@gE-X$PvnFj5ecH<~2 z>+GWsn(kv1jC(}@x#TobY^#WBW8$<2YAggbB{Lv|BlMx_I)-tHs^pW`&0zryvOjA-8kV+&Dva-BZ`jEX zNj60Pnm%Y3zEHjqt-cXptC`MmoN!F7Qn>F;Cm`ug7i=$Np*w0#!_|O?Ga!EJuZFBy zMgk8`Dg z^pEA>uK)dcg~ipoMf`8y{}064TXT^1`GHtNaQ_!MZ|h>?Z0T&_XliKUZ0h7pZ|||F z@#DM7iR81aQFj$Y+K29_<2?H(z~vX4<=KKz6`B?raKEH+C>0qD>EVK3uZU8EDTiE! zScEP33dR_h6BkQHywCQtI80hV)M%h=K51v|`;Whi+sOTwS#o>)8GO6h_=dl~_vtHC z8dW-~w0UwlnxeG3zO<>N+KsB2AUH=}+KkUYO*2|D@EIv@=sy}m_i_^$?Rz79w$|zl zmez-i;+h&<;81ps6F^9tI)6Ebw!@&)%@P7OXh*XSJtCk`^@xmxPQBg;A$wk@<+pQ; z%|^W`jZ>hP1fY_{^6?Db!>UzX=)=igSRo`gRL^EZ@NTyqo&)!`IOR?W3T~@*ED3Jzzu!NIwhIRmbSUY zAYo!c#*veVfCAcsgG(R+H9>YX^m(62D#qB+l^=U(p3399MxXqaRp``RdqROhypWT> z3O(6mri4MT(1|hV{pXGCRdc}s#Xynegk*17Fd>U^gqak%@@}B1+rw(Oh{brQGM0Hl z|K5u7dy(r>MMQBOch44T3fM=2Fs`Z>Dl6>6ng`NVgHiR{JfWX>BwR=%Fv(BOq5+s5 zdU)}IqO5vWpdLx;(n5wPw6iMoU?X6`xDhtD;@Ut-{qheIZIeS=g}R&~3WWoPanNq} zvZA~yqwldHQL+Mb`e487UOr{xzmil5KCa7sjp(Db0n7_xFk=l&z3Q@bpxcrppKM4y zC5jpEE03{7|MZFABVlqldw9JBBNkv`RjBvQYtQj#)Wd!@cR{@Rv)+JAjWY-$IwOp*--;=*Jzh`CVpt=dT&j!0pOE87P5nm2$9B};z1aJMIxsx8pk$-%~DYcDwCXW-OBlNMJM{ zY>F|Pc*#~Z5VU-FySHL?t`2!bm04D}$lXnDy=CQrSI?Vm)^Y+ku_b?&3JG{x8Gi|GRFDH#K!sa5>QUwGHZ`ncP5FF%#IU69WU1 zN={92pvK}1VG_0GiX0etA~n)np7t)RP}7JMP7yB=PjtQoU1D4@d9cV4Ib|cw5w)dg zPT!{}P9@X*LZtny?-g*6zD*ArJ1T_Q{sPhZcQnD#!s9{IK)Aqfz%8)!F>-*)V{G6p zIGypq%HyATyb(iA0kwgbVa_!WFGM1h$|y1@aU)IIVxP#)QW13FX3GFcU2(`MC%15ga4Ph* zOe5}UOmmy{YJOR}*&re!MoOyHGlq?Vy?H!GX5`9s!A?FD=H@7J3!xEX;S&GPB5hop zKI2|KHW8x!{cpVcS?sVf>p3+q)o}&N1+Sc_s}8 za{PP%`m4p-ag@yw7<1z$yO9xtArS&%lcBm%fqz6i0f2v9TmCbq(9_@D+%#~aeAu3; zkE@i(Rn|mM<2SBFf;r!-WVje}AP84MQa&&hIPSK3Q{27-Scsul3nN^DGmARKB-Db? zXf~hE!3J60{@;#}XjAA1BW``S}3#ahI-D}vJlpRU;x$A1UriPhuA(~j6 z6?a*~!v)sl3`zoOWpA4(;=teDZYrzZoY3Ful z^KgCL?T8{6cw*h$%jw`HhhLo;?kv+0tMazn?MSvXeThg6*9S zf4sY8My0!oju4UU=+PMlW*v^_9V}-{JNu`y1T>w6s#QGDi+Mj$SyNTk-iui_Kmpk$ zjkWZO=Ej=~v5ykw2D#NN7e_n#rBi@sea7USyBVAp!hUq+ej@r`Zk9fK0}s!(rC;R-?28)wj;E@BqvdBV&%JeC*s#A7o?at+5hyZ}aWm)6@A;~SZ0$Pt_( zclDQnSo`kv8=vzIknD&UVXG02>fDRXJ~&lLxtI66j?fj&^gIpC5V_IP7l-E1*||^p z%U#i=j`e|#TP?6+cjXX)93qiZC|Q$R#ei_UoKnX4kwj8*2*nD{UFFzPAGT3t9qMw7y#Pwe_wc!<&^vgCAPHkoXhWwt z6(F2@I6InPpc}Rc)cJxeu`LxU1+N3^BR^n+eap(85ndvriV4Jy#Xd=D)T0?{S^>>e zZPu)76G@Lfri!Ghob+qiA*8Zwqm_?seXdXYm*&<>zYKmbT3MwtfxL6tRhFsaTs0fx zEzOfMF;3+gQl$k`a=SdWSZVN_t!Bb%5sT%(HI_1!173ab2CgIybD9#I*)C=3Dby>l-e1eWF)n)S^)Orn+fyZeTx1!AOFK`?awO*I6@vfyHW0$kJSNv3p8i-ax zJwJ+{b@_JR)iD>@`X~IMbc6_VPi4Hw2Yti{rwWT>(xF#CMEHDVsRQ?tn>@qwlC5-^ zo2_Cw+UPLwQmGx4;ea_T-%U8Kc#)}mAIS2P5qL`1*PxEf*Z9Er=}~l*wXm${ zmn2i^FXkZ|hu>ClzRBsGrkDSVu;+get(339rX(;BP#6dh5Xt|}R@-~9S{eL*CfKI# zhPL)Lra!ngIZi=lP!MV8`ke}bmvjheO_d4_L>(~@vipI&!6qSmle81pjHCMX3Gk)r zr8pCyyY=t=@DeuApAQKp6dVh_Glg(fG>%q7X820<7$Kx34%!jYM)8sMCL|#$u%)+r zPZ|&0@A)e6M5tmZE{LL*PaegX)-JfYjt1tmXf}V0`roF}M4v(c8QM{?7|Q;fk%~AW zNf~k3ym_GmKc0AsqDaO7pf>rh>vpj}p>>VoDb4XU?Bm5+fU2MvoX~FbS9jV%iehAn( zhNyXDeppj!y)4g?-$6M_Pc!WUSs@>}|NXW24_zqaRZHvh69kl>YbDbEe}OP=R^hQj z6++7C<`Hmo_0L1XfpOD~Uy%w16NWk@Ld#x`y9uNasSait#FMQL2d_Ju zVENmjYdnXT0tNjX!Y&GYMJS*=DzQK zfb4t^Ue=I1Rii`13_R?M1Z_Q%wJjA!pW;nf9Idx4 zI~1p(Q_cvh`7P|P`lT9JhpY*>{gn_vBDz%qeG4>?X_`(W?Q6{W6(b~lzrDnG0iVo( z-enadV=Th93pL!L4)0Y4T;;Hq+|K9%jZ-*t`a#hcNlydoj*NNG8=ebK$B3tTRy{R@ zNoUK9uc%&RXk+^|qrrqN-5Uv>X6nr1DUNp)#YrICw$Fsdzcg{F>Y8U-{Ffl^ zX0>%`;$Fnodz18Kw|wnmA~jpd=N&ee_KaK8v;1sB9FN>*I+<_yd@M7(n6EtjK%@d= zm3*Ub=5=S2C9XmDdOU)iTO{vl^N%4{=9zI1Dk7Kk%PHt6t)Hl|Kl-eh*ifoCpTB36G2HK4>bbXVB*A|J*7lRMcrq7I>7Kwqu8F`v7(YqtWMSxN%J839{m%y@ zJqNv&p<`5slfCKn8XXSLuk{vNuOW5~oAvA%Th}I|pUHPyHyhmStnKR0OQ&9+srJ&V zD$g%mwfkzWGwi})_tFRuQZx`OEIcgDO!x?_6+f??{=7c4V%EQD2T4U*{3HE+s$Z}$ zkV|ifsy6axTQ3A@n*u?v(;o_D1!)OcO-UUPjRg&f9w7<30QE73qB2I~xI!QRh;YEbxHu+-pFKo_$Eg9&iNWiI_xZv_w^dUJohj34F?r5?o|HvXhk5x!dmN{ z-75hI0_4q{yi_<^Q)orqCX1`$Yk%#Ur_#WfLvQs1uETx6#_A>Btd+#UEFBuIfK)Z$ z+(~Ywpu>(N0dnc%7~SyAs<&ov2`P1p#7?|)CCcoZWB{3I+tC-);UfwbC|ECjMJ$No3L9NO=bxukHD zY(}`+$gM`A9mOhY2wCRSt4$MTA#KN?l||I}OtqCbBqSGDo(uLW@~-m7;!Tf)V#W|{ zLld)RoTrB^DJ^G-&0BMGB@noO9-J2?0xq(|JMq5YpXCtJrdreyUrtznr{~n=Ffv(m z+8YW(Ed7O5z}ZR%h3_ooxhr4Cx0>cKUV+>;M7i}5+G1W9e%(_B}lX??|Et?#*gbbGQ7oZo{fUj)AN44!nZsoe}b z!-n~};$CG0x*CsCXL$ATf$Bp!S(2wEZc$U3CU~&bhxZMA6qtck3he^KX`_PVGu3Wk zcsEcUs@%=}VsH-PQ$16q|3Z6b0Pbfp^mP=K%-Ru--%a&!h_jrhf_i;UbFe3rA(pdSewzj()mm4|*<-JMuRLIVnZ6D?c3#3#tJEU&T)D**yU1=c*v*w|W6w?nmu1g-nWjJ=Z zhKr$6llm`9spkz9yg%Rk@cdQQKyk98#J_5lPH_jL&sqie;yVKd6#xRYA<_o!bkCzD zn2=C*CT2m&i!CVsID_};J%ZAqA^(8yqkiSVNWgU=4S&abt)!y-P~PeuBo}ZDT~No* z$nM5Gm&Bg?Scq@8Yk@ICC*{O;0%&6WP0uS#NI3V%Kp0KPzEzx5ZL`r1(o`=#qL*M) z?odaZ8W^OkQ~BOf>q*CrY3D(^gii^W0a&=Co3R#;vJ&0Vr-x{Yu=hujZ(%qLp)q9c z4^28sr{IQ^I1t~6kuTM4_Uw<4@m^_`Fh@LwW|i`dwj8UxdBHgziNAHD7l|^LD)%>C zhJ;~Av!?ih>}9(pdh6&^_*znqf;?=#d`v5>5V!+LPVar1y6QoRMOYa0&gpvVnNFFK zw9mA;4r6k}(v4Jwd^j^|N}AMG?aNZO{*rbHO6~+jh*)8W@7iBpkk}d88HE>5W<-x< z3^P|>#+ivvqAaZxCQVf@yO%k8f0!NyLm+0WIQ8*w$iF5?#L-*p*-jICzPfE}8OClp zfw~v!c^_N6xuu^DRcwH{M5{4iR&5b@Gq5(V@{l|b;B9mcH|~;7<@G7`aI`d0_)g-& zy-hR>*#-_J8+7uw4`MI*y32cUMbhYV03--z&H|K%Tc@}AAuX&?E|N}3pZ1aJ78k5z zdpssa%2sF4IRJUfEFSMSrWStUZmZUN0HhCIZ2PMD?!$5xws7y{wnQINRXKR zMP&d7LiI3k-}Y5Ry2i>PA!ErQT;R^bzK&A#jw1XQB^V2K_z+ru`&8#VsVuRmUJb_O z7^F||w|dtd=ZwzA9ipDPYBp`^+=mJ-Prp3OS7_OqOp1^yu9|Kl&9WI9FVWP<_qJ>}4GSv_?mRY%r9*IAF6$xp*zLwcMT0eOOgZ zD@rTpVNX{Y#?c5j-Q6CS=_;R*96}^>w0o2_@^C{6Q>KHT3UTKhx^Nlkh_pCMTv}7$b2x6& zROlr-WkO^T=X_zFfw-&<`&bB#tG zynpt6B&#q-`SE9doiR91qD?683(GMLg8E&XJ}X&?X}8O#0}VC6!8 zmq7EjK~B7YQ`>v{ik>-D)WP27g1D zUqpX3!wHLxf$y(_2n<|-$XXBc!bUd+j%;ts5Fh@4hK?Q7CUD!WB-hu-rSVd&G-%A< z@^Mmg{U}N>`(cNjUDBe z<8Y0kR>J?mXJMN?GLtCzeGy|vzDw`E9Vd1JMJpe%I0kipoiLl9aCIhy>Z+n`MUOWx zo)TOyCUo|m8dFJD{y?YZhnJBN+w;apEjutvRS+f(aH`oYheavHB9IzXdZvpP41LMU zQwFI8=>bCf0qiw<;N&ShbzOQS23jfbCbkBCbRQa;o};wrSDrnK)D#`#cI0GOk{Bnk z9oPb;q?TlK@AdLS7nh){$EXp98x^v=e1)XhvJ|x~O-ilId-FXb(ByGy22N0=XRP;< zOvHaTuZA(Mh7q12qgzS1V&UC1d;B}jLo zW!NzI_?xj!D+SKK7tZmg(6?e+O{KzHoL)s^JVL?fO!$GN5vHj!0v|4oTZxiLaMWDQ zI79gF^{Po-Ed%l@z&H#N7_^$Q7n)S6_t~k=gDI4bS7R@N@wFNrG>+GvCC?|ygTJU3 zJ2&fptT;C$RhR!TJYN90N4+yBFnU-49jb+C6r{X%c(k?o4u-foa_6vg>*M^=jL}b)h*F=v&Y6}}# zkafJc_|NhYXsK7UmO50eFCQy=OxjA`gpV))V7?1jNeyG!M;}XyCftPHjRM23IvK8^`73-3T()R}J>*+-rbz0U(|&a)^|YZ(N-TZ5 zZa0E=N{$8G!w3gNT8@df9@{+yQ6;GJMtJ`U%l^2_xcT@rS3f){>a2{y4ehaj@sx)0 zFRAMNsan+e3Pt#olwp4XSc)1pUbKvrt4s!(>;Wwp!#>9?lL5bJX#Ju8=8Se2UjaT9bk9fX79XKQ09jUT47M zCuxWt)97)+2W{t|m|pKh;?-{iHcBvCSn0!PKCF*eU-M#=aK-9PDvAGM1b`%}aP^5I zV=M^<-2Tzf-NF^t-iHfp*9%)BzUBBaix{ra__iUcz58s#JaMwt=+-3=f^u(MsyrmL z?1p+X&<-s63W7qkXJoX6Vq^^TlhA}#@$zC(apHmY!p$q>AzSYqs0vyO;U&ebP!w4d zw_S9s-eQaO<5G8dkeQVsq01`KJbk2k-ZLUkBv^wqCfJ)GSiv+UvFa4&99*mq3L2}T z^}nh8gAbFtHmKZPo5FW6+MwQ8gd$?>UQDM(FVa-+ zS;0u!P9IHKzL%>u1Gf_4m_=q{7dB-tswxkzz(}9`MoTWkD=92E04H8v)ce`i!TT&) ze{>?T2gA_=RX4dR_Z91osrf5llginkSLnTauG>@!6lbt~P?Bwh@3bRLuIVK>dYSWk zETyxEXOo>63k#AS3v25;z1q(~ts>~8nko3vaXXDtj1 z#|RkOuiC}>HS6E#im5b+slmKaxAFR1;NeU|O{qjdyQEZw3z0)-vp)7s(#f1mE( z33(1X=(Nr}A*op^I#UZv&#o)x>!`Ix!lNFfHmFIZ^2dRZE!+#op9BXC{bAQ3Th!}9 zNhicPova6~XQPzAI3B?=x{9*@T8Kj5E!(jF#in5;wuS`yf_@y=R&%Rwwv35=XCouH#H19O zBh!=$bRsZQI8>d}K&A@`;`jH$(C+V$?o#(dnj_jLNjopVlmXe9pTcj|L0FBZx5o%4 zR0*r~j$Sxji+{B(s*5d|JQy+BS3Fm2%P$`^xMak+xwRagg?up5@6rZ~MXZdc$f}(( z$1a>J8r?N+UyO0uv(;`rhAYqYfHej^duIwZZr{*AAD4xF()o%e-_VdTGMaQdWKY69 zB1Rq>fFi4-izyBp7WHuJ;ruNhoLgoRR-_gkh{>Ky6Apxb?W?w-cc>mT-{DleUH&rS zoYJ^FGFv(1w#Mx&5OO|=ilHJ^WY#ggR2BSak^;%;S}2O>aNZZLLS?3X@Oodkf~JA8 zcB^P-@b=S`@$|nh_Hg=yL|gC`sO&Rt3``mz>b{WxrN(fT5v?b)lo=7V`DIPPb@B%> zGqH<$Nyj{sME1)vL)SU~@i;?I-PN^W2Egfmz%0+62*I|N{pPZHCsd2VfgG+@+ZYxR zy4&}qPrlxV&e!PRGGQ19tf(cAb!Z?vV$5)zsVwS%TZ~zvqbeQH7gO4>Q9;UI=ftdi zVYzPP0|Plo#wpU-IX3m%LqZ{TB-ziU@Q}TfIu?8`%$9-Flz)G1doiEPI}TXUB)7S( z6U$p*GT9s1uhNt)Ipsx{FC*cI){t(?YqmW5?^+B;g{iXrS+n1E5y-LTV7LR0-`Nm( z*732Mi3Zga^TEZ}B-d8<$4JLi5MP8^=QnQpzsTWdzTsf?@$x7BgAhdduhOD<$hg(mQE-`ye#H-r;2$!9FMJH$2A`D`rkKO4~f6IAswb(kPEs zzrf9^Y2Ir~81XE|#$_Zp{n=td?w76IXd3p5Pr;-yVOfxYgSzn>4kl7m;-sUSW05!Mdt%eaGKC{Atdc1FG`y-1N_Xsqeh zHVn{3yw=4hh->Q7?@6gg-oo4S6t~0UxK_5B`xER$FFz1X&En?+mYh$PRvli zS8CZPR`?ijTwA;;mC<-5Pak0~nW&RObf>pVni;3kCI(IG>VdqBR9RrC!oMWtC*gmm z%yhJoa?>n1qaG(L=S8`|wLn^{yOk|dGkO}>>|Pk@wIv^jV)|FzuX~JN;fx_=raI;} z6FDdM)7wz1i#e-OP7d{5-KWz`Y^;0n`fo*ut0PS?-qdTG<7AF%6V75M^Ta@QU^0i(D1 zJu?5{QVdsw}<>LW51T5}M0OKN=VsbccxZG>SDov zGTjii(!*;p7$)8cmHs>yxK9X$ABE^D5fPw6r)9{AG6i6x$)Edt|9)r#W=%R{0jSzb zY{=xCwNuxIa3pB~`RkPoBczASqKdaV$I-=#CZF#63nj?g0anVfY(kMRnfiC$2Cn@( z>3TD=pO?K`ZsRc7&zSAdi}Yw&MTMvs#&1Jvd9S1N4eokMy5kV{eEx8=sg9G$GAMC| z#QFvMrAgcQJF&$V#Yi-6#U1rRq?DVuq2P-?S&& z0Yp2ajfP{uHZ=);WRd+@hDqWwR z$9Pi6?3sz6&+C=?U*ST4N!+S)gtk=owtK8*i4Nfs*MX7u1`L7JtCK5zuTNJV@?dkz zOTg$QV{jVEGQ=3`*n`XGsv_1S8+CJ9cTEUu&tL`MJ~wDg0R4@ne^^pKF``Y2==(>O zBgNLy4s43L>3y&OdqMzPtn7SL0Um#ua?Zbw-LXU_Z+oS4^KaN2utRm_JTEIVEt%qR zb=HFJ>yh;Q6Z~z)7B8WI;neTP#pKc%kfX@NjuH ztPjH7$vcVX0@ONp)6_#442tTDDOFcM%TN~vXZRG4Rsa-py&5B8-OKIQ<3AS`4RX``| zbGGZuC!00dGS_%_0HP5G1a$Po?V{o^;HPm@c8-QcL(YJ=ms9Z z=jAVPinEu4Tj(P08ctBlJ<(%$qP)(K^gFGLjEs7L zUaOVT$erhO0*Q_rHR{vUN1)AbehN8|0V%aebs`Mkd7{>pOkqAM&7ZU5k=+0OSm-m? z-Z`1#_|6e?CM@yVXtRhTTBI=Vp?|GJxO~IgxjtwS?Za(!LUD?`wEwQYVyD%+m7NOw zLqpkPq)0#Bt);Fzf(IJ5VI2s5{DzZ63U*$cyRF!y9l*mE_v)mP@6iLAy9#3|nKNY! z#g?zvQLarWZ+c+lKRs1KoWI5So@_`00nvUIs`Cutk!yLz7sO#F0EK1Xs5qqcQ9>zj zHOGdQ%>eEZwAUgUib}#p^{Gb#B49=nyVPJ4y^*rxyE~vbOsV(zqd;LYcC-*pAnpb>Kx$`&?R)Ic|3(a3-DBH|jgc_ar?N0EJn=d%S zQwfI%0apb94HtFd2MaDA6Z75C`JurWHt)GOJTv?e9MHECG$=Myo?5e)Lb~W!r^8$* zJqTG*Bh>6ZI#5@reWeu_DFAt)`&6UA)at3onR~0J|4paA?OQsv`-o9=+n)30h3>vN zbo+SgYXi4rwe>t0kO~k$Co^3BBPvxOj;F9jafZ=N&l(7CXP{}^3I^%BJ{ZP5&#+3- zj(ejW3qB}J2XbcsfBf70Z=+#7MZZGVW~I2+&Myxk>bi?tghe#&U&_}Q=>-gu9o6DQ zZIjV_5U$_&{inD1@9_?}(Fsm6c9>WkUC#r|_0eX3^ z3C%Uoj0}SnVNLT48IBq-GD6vVnU~;F@XN3}V`~ctAXtaiDePXB%QHL$UU1#K$l97V zrG>6pPOHOlh2wZ@3(hubm+u@cig{mVC;u`-HuRWS=;MEu8_0oZ&9-#0WFC=aKQ7&u ztCKe*b(y>vl9}X3jk74si;FZvH|(>Uz^oZHY%JAETL03X+MVc2_A-=!pkr^!sfg6L2d=5*TMnU(;D6h+t<6>0{<<>)h012 zfE^fDRrY{#$PKa4f`fJ+>%}oukt689@j^bAv0(TH^{sEsi<Um?;E#32p%ssp1r|xykP3BVN_^2=(aM6GM)(#b8$|%ffc$k=+Ux0Yi!5?sUqESS9 z?n75GeUcwhQD~ehUy@67&t!3@Q`C6r_wKP(u-u51RYtt{a@NAs+kiErf4R22QIskh z>*3+)5ViyaUMV}7%>E5xwVC5YV2kYk?6N~i>8v*IwSpB9U4QXuyu@d4HOKl|Fql)2 zBq`QK8YYAe$SV|l$MhwBmWIs`e+`q18lN7Dgs4%Fw|9kkNBHYNbo`_Fb7DJ9F4}f zT^lit;5on8AIxwOC2w4W&81h@Vs5ES_}jO^gDjw(jV>#>VC8|H{N?M;Ye@iWbNVcD zSNul^3Ud)@PT9bF>({xxsk0A7=~=Dd9wc@w##*%|`9AjQPK2es+Y?f`>+|w4(WufoFA{U8tD&$( z=2{xzZ5!1d`po#zyaZI;2|x?UC$Pqyo~WfCZ=E)O+=V!WnRqmj8c=b%(ge2#fXyH!Gs}bWx>m5TM3oHkHoePv|p?`3SpuP$2F03!Xp;Cz zT!b|D#T(%!hj5&%A?9hzjJ2qRU{)!qC`E=s5qduO^`D-C@<2jX0S*Rc2?Yj5^KVa) z1X`MU*t^S^x&ysTebj-jo@2HYRBNT`(%dvY4++X<@b?7GNUDYF98 zf5{tQO>fc}+q1&{h0FdP_~%>%Y2dI$AtI^L9xlRW1!d6Y8pON^UDpOr*FL|0sZjgD zn$U+9aOs7owWSnPc1Re>ux)p4UOPl|y%OGuHU8M82PLj4yO1JADtI~ZM}@8e7kk_> zX2-Q`Dh%Z2eI2Frq62?yQep%M0FI)B+h#K`;Ug&=T1T||a8RoqXLf`tL#>Uwi3NcG84q8U{*E4sobc27;~z|^6v(dNY-`V3c_*<@ltT5 zA`of|3O~N+xZFiB$`1MJD0`w z2L=l=MHSpN^XDjSLG)M`h`1mso@`nsfsL&@V4(SwC#+l1>>V)*!ZL0!*U)T+T(y}+ zt$~rf0Jp>R1~cl{HNYY&YUgs$mfodLNl=OEvmQfPEh+|Ndy4%3vi<-3^)zrO3=91a z)32~#U|9cVTgu$p>AzQs0h;>i8(OHJ@houRwgiyhLbjdQl)@n82=w3_oI^TY9Qv6` zAE+nTu~(BRCKT^ygBE{yf=oc}Z;)?6Z_B6d&!f1zyhI6J9-cFS@0V*r^TuDFpESYH z%^KaY2{No<$~uB34V7jPeAoqdu8Kqa+?f6#GbkVE0+jv2XjRPB*YBa^Eb;@Z8nx;~6T-DQkS5QSlcHR((n zcUn;Wr)xq*$!_!@(QcD9%Z4r5oU3@F+7v2XXQZCqvM%hpuHi`SCcVkV*NZDv`g zKDmNM+8>}cx(RXbZKSff*-M17K*zA) z&ZlBiZlQ?=eS}^t+){si%PKiLh&9a#_c>_ zIf3`xrA513-OSE^i(GuV)}G+{7YPD`xH8Mp54Z9j^G`J%lSB1v?Lv zNe&JGiPICNvdFj)r;)1qkg#OVmqkt5%8}Uuha{|PaoJb%?LfC)kfi1a(;2eu<>pTt zhHlzq6`XmU`r{d1JgebFBuG*j^;0tfC_9Ww?E{I1fIlSYI4^$>c}wh= z1X?EBfCf(WS|9O*2VR9B$?b26Iyd9^=ZOCOgSAeyx0Qvr=PDszP4!VQ2^y zxcA1#X6S#%rd1e|MoONPQy$1e2S9mxu7JS z7rxemHOPhVnpqM5=-orN*o%8NZxG~TMlU|G%41asU||b@-|#BCj5^sY*>lJ7m(ov0 zXd>6#U8Lgn)bnN@-pnG>t#_3UeXN0lQUF(kjQcoqRb<3sCGD)U)`Qg}bo7vnk`C`= zpHF%}Q93`L7$&=Ut`vu|Ws~5ic-smc^B2;}Yj^;bz<_&DJb9W*5j&5KBQdmViEo&* zv2^tZ?0@P;Vz=$DzvFs!Dmi#yQ-&#O zoiIQpWJ~UrR>*z^w)xygaqipoGz${{2t>-fvNrC1zve8e8WYXdzA^w#JJuME_CWPS zYzt;ry$zsx!#F~%$n=fxDE0<(@dsh+&3qvPS%_dZO|(%|jU}Xhl<7*MRVK$@X)?GB zDNQy!OxI`|ZBx~u#QiQ1ufv$VM$L~6<}|rjhod}li8Gw-)B$wG^Gva-c*^`G7u1^A zpVfZ+cbRE~t>0jwq5USx#Kg>_X034i6W}q2?PZJG_>V;M$74Dc97Z%SJ-ly9*L~m} zE89!W!LL~rGLR;U3&!9_6cL7#noBx8v98bB;{^D(2Mik>m^Ble4k39$|D@Eie&=Et z?n>t09w~}Pan>IB@gh4UL`1TWfmLwUo-jlCk{h4~^5svrFtTlRPYgZtcnWE-F>Q2= z@>qo4X&vaTUe}nQp_CXYsL#uB&C9VX%9%u0$ILe#9W&A$u#T{aCcoB^k$H_-9H~k= zOK~7hsfOlP$+4whFzurG-d5-caE`hh3r%hzOQqh=46@@ee*X$d>a}j?8F&|Z+9@rE&nFjrcLIzlU_gZkVVphW#xy*HfXZbZdaf)7I zs^BEo7f|6~SwN|U)nDz_e`qj(Nhk;V;z{QXBk$2dsN%mw{~{2194R!og_chxZRhw3 zFMp3fPYfeCgE2e`ncIgL?pwcdL!M1W@&mg$0zb8G42R;%kE-}VMBJkJRfe`Ivi2wY zLSrnh%h?IVpp1S-14Utt*)hTc*2VZ|@K#hMP@|2h7{oBXKtU<3t&+3a6x%sW+sFQt zyp8)8vH^A#(S+ znabCXAf~S~oF$A4UD6-Il-ivX$aMj*maP*41?di7z;}HUb z;^z4<(9ci=Kk}Ck#Q&5)F@nypA{a0*H>Cfw1hW6TcKNGMxP%o5o8FT%2E|Q^HqU~u zZU|?Awji^GyCI$;|D6gPeQ-V7wEcIYsU?R1BuMkztGM;CN?I|ed7Zw7X<2e>Xtq*0 z$hx5HOZ*Eh=!UJ=%M%BVF#$+&mFG9zalCoodmQ-je9u4ve$q#?QI8@rHy8B|t#sCo z0*coIlnGo{;hZ+VFc+K~@0+`PI;CxNw}D2AnYj}I zE%|jk1fl42Y)3vEmj)${3)ruGyIOQMv`uOK z8y=xkz(dEyiL_cf8R3R8Pr^k*D31>-IN`38*;_LVGzEO02Sb+rbuRyFhV9-*PHsW^ z*v}8Qdl|*Bl@3c*XL~dEREd73>8Lx)6X3-!g#OYhptwNPizL2tS%o_kV51~&ETJ&A z{xsjSyO{cLc;H;3+QIbVsj_9sI6o`BUzxK4E%*^k}|_dR}VRjL-o1;8Oq z$B)b3EF^crrktgCm0(>7O8K06zwwOpTrsRS+sWt;bi7RWY})~ve#2V5^h)5Nl$^oD z?cD=AZ5Unxe7E%$htq8$X%6IBUsv)e;t{T+AVFS z)7+t^e%tZZ6<)oZB-L&}1={`bYX7-uq%n0})lS5)^7|^2am?R z4CDx_{U2p-^*>CIF)vLK2hy0(pJsm?J~aP0!KB}Ov$H__iTpuy-8kzGnc=+l+s}LC z_xmWEzsb+wBHQ#1iOR*FAt+cya%sXut!ZOZ1nr>{7Unv7^Z6Mm4$&WgoDeLmKO`?@ zAqh8ZRW1vlF6xxRC|#QdYl0QYfx`3jR@Z?D@m|aC#T>nDDvfq}rFPaHTe&@C>{S9Y z$hlRq?hB0;!oS+ey1L%x7G{!_9;?F0Y5{Kaft5YuSCs|neVy`TC|FH)3XveD(zR55 z4$#bF$-BHz+gs|f)fC@RlSqq?pXQ!pzFeJ9Mdfa5eo?ZmUXI4vQYtrV#^d1_Rh@!m zPlSt;oaV~MPRNAS&t$EWcHKD_J1lXnI^XbQWh1d793eWr&Az(y^K-!KHA!A1= zJ-7UQ+WD0N^HMcI?@MWc6f48~z?*)P$Hb$~gya-1-0aH8F)O7wLM<~lF5%+F@xu~+ z>#kPn?(vEWGBj_hxnV-BnZ3@FZBvoM@K?Me%)YF&zR=?WekFHrhum)tc)V6`-r3{U zu=dsVEyG(COjmj3I*%=Z5JxGVK&mbG?gX=z4eD8&gp(>wnRdm;ap_F?!;md)iKYMAu`G}yss z`B}=<8oGQ-?pZabjsI&L6kLaO^SG=7zN6FaTk3}A@Sc7ukmZpo%v5(x$&OOFU&p=Y1SUWH)r@nnTZAK&;;xQ_!Ee?ThjPFGQyF8 zcG!fQ#}`^A;sFkC!kJz8H=!6eZ~SArj$j%WG-bA7-EOJ1jZ~)FccDPzz9zJTf$&G# z-mtKCNRJ0kif^|J(>fUKDR+9(7b9M`LbV6vUw(5qNnL8x6%iRz&n}sqLn?!4AHm68 zwAZ2^^fb%x4~QQse;h-=L&W}YEFxNn3|(S=Lxv~h;tkWL(U)rOCmnuG8T{qKa?4UI zBAfmzt+Zayu{%l9U31j>^o{4qy>eH_N^%bOmy<03YxXm{h^N>4(Ks*ZP%*kV{=R&n z(eS#KJbzaoZ;&PAIO$-aBYq9uX&pYN1os=*Wt=2%?ilqocB}|7p>T&ZDKjD3WvPI zZ+c!-DZ5ZahHK_v=GqU7ezAc4r>zmPyPixyf`N&_{V$EczijQlOij~Ioj?=wYfEs| zqDj;{JX+HlYn{^?oqU~Mw#7!T*e1k2)~qIP$`x5*fcdJ}{8uvnx~F_r5}rtKLD}Dx z+f7AzBZVK67ah;tfyW&=L0|9hBw)~jk>*G|P;>~c>e2{Q@NE{7Lq7Wb+U#bu6Nxu1 zq&u?Zqbe%t&N9>q$)sM;Ymw#{`fM7&@^ZPCbZ{j3HO>x0dz<(Hq6qJrl4W9hM-4lq z>Pc3;X28f2pslt=kQfG^1rOb}LXSl=fP>6;2-U4)-%+o1f>dBLaPX2FokGQ0t!$aT zqQlD3`m`;QKb?aucOZ?E2BfZ?WPO z0i#}TW^9zO8?p<8qrRrPzn7t3mHM8^e<#X?s1YR6=?oWzVlm3EYTZCyv6Xb5AwE>D zGVhF8HlYV=kTWL;$&$0Iy9FgPS$_0yi>8}!+p2IjsNxI~% zOYxA^v6@yl+v2n^?vxj{6Yd9WRECi{x&^A^en*iHN2YcaVomaQCbwrzB7fZ!q)PG; zNlIK@__mC#i&(YDR&r@R*jn*?OEH>OiX-b@IAYED+kx_y12lILqU`Y}1xn+$jPS>} z%}sPLi4byNf@mPc%gzoQ$;ik{31rZ(g{U?2*g4-=?ezG>601EtYML@5lTHG&2(Vz0 z)9V`++e-`xfBA{`YMMnxhu>WM=9$0CFjJ|_Ut*7ZBbLS<6zvr)0Mi8vB@uZN%F_q_ zLacYQwNSh%pNIv4QEs35=;Q(KICdmokpJ2Ky7=`Ny?g@$!>0fPL;v^o|5xq%uhp;7 zLG(6Qa`{YX?(%BsOs}R%XKQ6MIY_g}(nhi~ug-9QSKv-(66w>eS)L!SGHmE-Y|vMS zkXtB`%Sq)lk++sW2bj;Bg%S^QQqPL+%{Id$`}_c1d!>NnvSrA=a{ZodpWntK{Qo|^ zUkO=FPX--LPG)6w9UBE5ae&oo?vqUpXiG%yYC;+#7Gl<=C*65K?!b|mDE87NIN`l` z!jZ+?5kc-C?{a-8!tD&~ARJyF+&`d~k*r>>dLb_i)(XL<)PD zi+Om}5`VC3$noiE(v6MNVE!tE2ydc|Pzfc|1c|dVWmMRd`y58ciR8q!HHF(cQ$y3SD4#%T;SHa%jbda*uAKPphL=!W zn>doj-V<%9Jz*o`(Nc(y+|sXND#3+Uxu(Jtp=VRBm1IPRr9D?hDl6)U0vPN@>@Bq{bK)r?w>=%ROX{eLuCept zCmb$UQ%KVa@vbN3%O39GeFs4#>hNRbj9Tnf7}#-U5GxGoZ9G)vR1e(7MoM!G#9Ke6 zKjqiPU%tPu`*uA!Z7sJH`r$+!+ZD3bQRIyu6;*7Ca`K5yNojbkaS>e1t7 z)h;@dOKk&lMiLA}*0ZBXWL@c!lv*6!BCt``!W#PGEy$UjJ+OmhII{?2hrj7rG-@CX zp-9pscjSj`he&>~!ebW;(a4Zy5fi6}F)`Lh&3Z>JPx&eeCXPNG7iAgv%ZgnVF0M<` zC+UyG5zL5;MuS5e%lOhV_m)%!V3>#9PClKOurtFdj}(oCSW}a^<(y2!Q8vcA5)^m5 z?4F=G5kEl`$O>A9(6RN3!$iq zyUxhPUn{wWB~aTj#=OF8cx{$uKRxta%W|saZ-(DmQ0-RrSOmZq;8|q?$1izF3j@3$ zow6EHMb_TJk=h;q2x0Nhqi*l@!~MZ%mn(Pfy5xSOOKGgUBb|M*u0K zj#HGw=pBP(G)t~EBKZqGnUO~?P)#GIGlG8%m=i42q&vE2<8WGIU?(p;#)jRLkng#! z=J-Se*pmq4Ni3c5b#-(8$cg_wjRG+Cl97R1uMb(5l4kQpdyVc3S6=I)^5^b?tUbdc z*w~=@tGyQv2L|ELsil)tbbK4Cfo6nW(Lls;P@z`Nne?Vpc@6+e)eb<|Z;Iiy8yST; zPt87hB$bzMMU38fsHYftLWXJ}hnWSkmCxHw5XGIhr22_!S3`ClMy6 z*3Eec~Ewfq)~HkHhqPL;^yTCSXCm2tKyQ9C0)6#a==Sn0AW2}sy9Q9Lx-DyOJF z30X2LGdR0P(Rw2@bRnW8wlxAEen~lS-J%Ek_OOM7dKO)~kn_!w@Ii={-9Jn^Ekjn))Ty`GtC0&e^w? zox$R>OKRd_nasoP`^t(U+gS}Zsq7Ma!@X#Ldxvix?(2R4aoPy5vYxc zpJVHv$6BZn8{788_(hY%2C|NB-egK+=$|tFth{H>>IdRf+X2j>bEu1TrF+Q~k(!tg zw+xW$mg)~i7>*rvx^oVzb~Nsm4PD7SNT)2aqY_F{hFJ13tPuU&{RQ*!lb0X*w>0Kz znsahnF)uOp^E-VkT(7ir7FPqv8cdk;yu45AOCA!HGS>2rvME!v)8mY__Y zXwz9_Jv`f6)~PgL6$H8b^Xcx0O9Jd3X`t6)#DHaiUK>8}QLld9ArXtWtt)8QI(g2y z9@IzdJ(*=reoamD_AjEDX*Ec@ zQB>Zjy(6b}&Tg<$9Uz$dkVv=tFDR=on4ru-`VE7nGF!L}|+|WO?dBIq<{oPb5Q~6HFkAo#iFhcoa@7OmThaK^5TjFGgW} z!%EWpr>LJ-%3~5!Y6JKG+GqJ@W4z?FgY!ndE<&kjT{chMlK0_bY_zi38wa@W>2XSu zvIne@hj(4D(^3t}a40640O)J$F08EgSsUAY-8j9bB&}9mL7~3brt(}0z_QU8<%jM| zo(MbA^lkOKblDGxtswVplHLS6w*`VRIHZRa{kMDWyuldlAqbu@{y+pzh_!C81kZ>s zlB=qFFq+yP5K)+3X{rpl1IQrYW%VvFY4U~{IQKUoLm}`7VLZA>F0i|op6Grsf6`_n zqn)^!YbgV84t&7{6>q>qJ5W#&{uqF=Yp5zerzp2y{Q0^VcUVMCg*B-Nt)83z1s=JA zgm&}?c1Z!%4S{5Pa-=Jn;Pip=-*ePjthorvEjz{tAk#r6Z;NYI&(s^5EL>k6Ror0> z8rebds6D5u_5c)M1#@~!sby6!o6QiUcuqRfCcjSfq?Vq}d0|(fnmNFw-J(3Sh)%BN zj665`fTo27f3+T#PcqCJJOk{Gq~a9psF96Abw+oOuwZ5EXZz#1m=yhB@%|r*@bEvENumog!W17qjbL@@iDl z>`|VH{})>Jt*ra346{joK0lBAl+;`~{p42j!WF&j{rm+mvk&+I@egH4A~wr2%B?Ge zk8{{XrgapnnX%yAW>8d~@O^ClhPRz6*q*+tx43n)K;ZI;<7~F%S})|ioZ~y4o8e^T z+JdfCw?L9?(}3kgaEHp3C0`^>bdW@7Yjr1UC@cb5ZFM(kV@!%@2PI>uybB#X zPS%Xy$$DUaH@l~%WKGfovAZ6A3t+#syOS-twV6!fkq3vJaC5nQrJRiHUZhRnyMD*_ z0(y!-&@VjwjjYlaB$4li(I3|8o_MZH_+=yZ$#H+ik$K-WarKXp*ex9HkuAQh_w!Fn zdH3EZI_xg2TEp~nsl_)O zo=D?9b>GJ{qOPPrZ#>0TW^n*x6u}^c<@kO{W1ZLfuLF5 zuurBeVDZ>3!b7a>|;Dtv+bE`qstRUyIC0b)t1glKkB&WZzLw%4~8O5 zt@Ry8#H3vmCPlpf)O6xet89V(Vw*T1s;WQ!d_;nAt1r8U>fn&>L=Vnyb2uOrNSF3V z%kgzRT>o>8fzR$PB8kTH&HK#6lV5;h^3!m1+&i5i;MVyCH1i@%=NpNNzqH72z;3@x zBPj2_@H;!!4-Zm>iX8PgIl?=c;I$HHN50E9{1H+Gaa0uxUyX=ofb_Gya!$^lVzbXH zbhybSGA~?r;~V&()%q33<~(rYTX4llxQoA!OSxwIwrr60_48b5^4rfb;P59&1}pVb z{(?upP#hV`=^@OO@C^oJzZgDP2@gt_5fds0>IAeu792E>$zi3>S>PJS;_q%tt+8nz z8cp6Y`|KfjS8esNwhA;ejGK(%Uc|vdVI{1T>Kv!c+zdJ2JLI%j=eXDdhc7z!Hsdom zqaWmm{h-9hL*V*xsHTS?X5LrSDe&S&sHP86dK!;mt@9vl7u3`#UV4{J`f~94H=@?t z#v@|tV@sN-9N+~}$mhDrtWCR#J9co-_dbPhAC=Sk@E9DY&W&Bq)}9v0xX*pKd*Qio z7=s4^#gU-&I513+XMi(Kt#|^?ImVpkLA2Wj_Vz@u7_Aao-q63!MSt3+cSkQWXSzu@ z<|cKC?y6{|NU829^VgR5+16;NCKrT zS?P|T!*8PK@PMv)n^(YH$(M?guXMSc)t0is=-(jhxxBwp7<*qFa#`ER>RNfaDo?D1 zFl9PsYhrVLD&^PUsWmRNpJ#BwI8Bd8Rg4C%ni}ir_T%D+KZ;>m{b^g%*Jx&gCH36yw00Hsh00H6tk2v{% z7T%_4KpUtmqwz9ij1!Ra(?V-&K%xT1NewZC!6X5QvLNKN;EJCaU!c{4;+(lz3 z-&OnYoc2F19KgQ+2_QH+6#Y`*(_3(0CSbX53{?-^SM5IOk1O7YW9z2>=Oa9r^-d`Jy@A!Q1oPvV9&B*(lv3;uUH3am;w^uw4<-ZOo{OUs_bCK_roBAT7lPwB)Q?4 zAQ%XU>e{A z*nl<$d}bgm1ksJnl4oekt})KqDjNl6XpQi(bVC*?NNNb+Qa63Wp=rCKvf*eujv~0$ zPG+XU?dVmL;^KOfmA&G2E0UVhXa-F7EaaT5=`(C&=WrHjNXtCu*vg=N1D}Y#r$m4` z27=Ujwd+$Y5i+CDWweFDMq7@TtMuVjyD7^(`LuS6j zfs;rM!^)(CLIP&TElw~?M{znrgmshNYOo{&yn`I);+jzd(m|#~jX9mpih$c^dQUn| z!+37v2Idra3L7o=Sd&X;HAFH$oQXLOKYlW%MVO;~mB}Eb#8PzOI^L?HS86KC%#zU& zAUDEn2Q?(oTR(4UR=Qei3+zs4qT^k&_*V%lznZW^y{1Ij4PMje(!4JdlWflA1#yP% zHL`mXS#ZE2RQs}U9*Tvf%cLAWWBg2_t+;Zdh|sDrB<#8-1Cu4=Nrt{y4e8V(P>C*H zC&@m}WKoej9as#_j-rVDVQf)^=v^5PYW;g~6IoIO^yDGGI=3OHQy_%3(ioII5t}MJ zRkTPfZb(ubjAL>XzIJ9u!bOo(%-SfLek9V7Yh$kVkXlFf`zkC*z)?wo;6`d_9A@#X z*@V_qHyUHOcx;1hauUPRD3G2!Qu0YaCN++V3t%(T`0Dk!mKl#)Ej5%gO;IkEt3%tSQy5ipfSk)`UC6uDj}iAqd-Exy$*ZcOM{ageD8^?+ z7VC$DqK}=6e2H&u+Xt}Y@Xax!HEYfSTq`kUu5oBT)<_Z@^|%^!(5r=~be5<6jnTo` z!H#taTinW~p*L=^G0vMCJ}wkQiRrXJu5#HT#iLSwzK#L|iJ*))zPvHGe(Q|MuV&WA z5fyW)gTd1uN=uYg>MfT$`{*K5>LDkJ3f zIYCO;;yKiqb{MFPjgEBXYU|d`xF>goh?UFj$SB);E2ynoiO9iKPX_4mT-vDb?(1Z) zXll+$erC|EIucF23xpxXAVq6k%D#@fOnx{0i8m#4&(Tg?-kOs>TGvR{GWiXUZ+all z-5Ec3(qcSE-pEl;)P2Z>+dOT{#!JI@Nz{@$h#iih1a<-+v_)!NwK-FmMscBJVLv?m z>nuz%f+tozzJ;XjJ`mFgTootNqoGpvQr@@Mbu^S%Q2R<9PD{(wmUVUSN#}Zbk$Jg> z+sGt&F;)dh0cIr~sxto2Wm{e)@3(eUYi`Uk0IY0?k<2dusk5-e8->+Ivs0)qIASmA zH9Q1EON|AX-^-}}bAmM__mcEb5W)NQl(hQbL8|!xK5@G|FTsNxjyTH%FTZ@g-(0Op zwvIHuf}mr2A$6+O+xKnxwIUT%6GqR93FW38U+C?^1PUybrzb?<~Td zHJ;dh5N6aRBP93yU!3$BoQ$}v^q{PWdvVMW@%bQa2VTBX0S%*&gTM8Qz=maTfw<3% z${kju&L;=%__ZaX95n$iiCS5>HH0m-haNmbSbptnI(3F+p$L}k0l&(LAgdE6F<%GE z@hr?)6z|6eU8PSzp=ffKgD&vo1S6gxaHe7XyN6wh1O0#i*OuL_3h*x$*CJ+e0O6kzLI*X5=iIp z>#zuIdbUxNF6}c*B;Q8Xzwb|>Z)ONSh>gi!6-_?`~x18ueqBy2nGkU(V}Y73_{ zJR`DQCs+>Nmm6mbE5|SV;{ZFVHFlkc)x=>pr2Vv!4*cGM;)<;i6YGwch~5d+s|(?G zCKZVR#L5ZWxrHI4SHr23grU`Jo$B2naMhcE4pq?WeXfDH%-?AK*X7aY)=j zuZsvB&K zvIbB(mMi<{%>p9C;9)=Oi|18y@F$f<_T7yVP>E0J#|whcWUmwtC%&EVeq;HgP3e!A zrSe4cGmp?)&b4(e_~EzpHCDrpyJoKvkNAVpN_;gsAX{Qy1_rz;(`2_|h&vdbrH?mv z<0hG(qAGPPcZX|>{zcP-U7KP6C*KxDJ7Rb(e^M4C+rvJwqdvZawd9KMcSRCrQRT zexr)j@6(ohNHqfPkqbRtP{Q3cz;M=q-!>zsYGA(vBlA~+hj>WoNwGHaf&RT5e!J#{TkLiVljRkb|~ z*Ji2)Vx|_ZyJ?}^P$H(j|Lb4i|E>f`xI`uUL<0i4Wj{2wvSe=P=R(Sr6-SzhKp zHBIEm-~o}8AS4w)3}z$(BMxk|90N%p9U}cJh!i#s3jnYfqBFJ9x7gLVR?umoRl5bK zT@$lGC!=56UF+58>S$HjF0b5Zm8kXncHB-+PMQR6?5^HD-kffI@}F)w&-fl1+xxy_ zA}w2a>g66BGSkl{AE@{~5NDS>Q{&{1*E=vXikwU00J&5-9ZGMm zwGZ=PxfD58R43AkB*P$7=u(PRkCuFyHuH3e2QF0Tm)%3+%I|>l?21Pv0@XX_bK0~O z7>2s3!=^VIPMMrE_4oE%92dK#j&X56J-TM=VvCtWXO9|GI?t^MTRyF!%i>M?ZD-e; z_yjq@mE+^*K-VA{`>DgncI`-6uDm?)$#;+(4)1p%2Kxk2jB$lNaT?F6v>?74LuHRA zFVCzgn^z}^)a&>eIfk&3o#W&1B}Il;$$#2+Lq4d-?WoWM;G9(9^r=R1sA*6eewMn_ z!`>#_-X*RXsAX1*_UhV z&o`7-=jP_u*2h)W8c7@+^T~0;ZK}>?hJ+GqC(}ok#Yyf(2ou*3p(td@NkmtAYKx*G zvYNtMdnQzlqTStVG*?SYO)uC=B`YdETy1S*o^`c5GqrJ6Ytk%SVqCYixp27IS>0?b?F<$`d3l*w@Xe_kX0Vx1*rq8jsvWae7cpYU(tUitBd+Hq zx-O!dxLm1>e6l1+y?%KzZdDC~&u1 zr=yu<8(!qeh4!(rg6gfbmT=RL)@6hc@fN{W7g7~?0>C?R3TzVuh_QkM%u8h%2*g&f z;BHt^Ce;E5aJIAv;4_f23)Z-teF<@Qu%T|6Lzm~1?nPYGeO*>-1&SeY>c*xt&)fKL z!$;H9HLC;JC1Yfx@Jwg77{jN+u|VtIi#$5OS1m{A*^TL5oJ6ZfYZisS=>@EE;9#X- zhPm4)PGopnj`g8JgfX+oP2KLdV-!A)zs+?{h z?#km{1ga0Ug^-%)I;^X(E@5ooj7jw=l_tHt4bg9a=T2!6M1>+!;vGt)l`dOv{!*tw zUkY8}LD@qM2@TAfT}s*@2nwLEK#33@>^1znSV*v{|H-7In~lkr(>(!)0MO_X3X|HX>PS)d%!C&O zf^JJUX8bj?$g8b3EM}-OuwGFmh8%=*9(+#9p?V(7R+7p(mdt4I>C*V2i~*z{gPtpT zPaK!p$25RnIHU_ct4BBjj!$r}&7Gp9FE}_}iM=!v6nPPE)Qmpc8z#Ta9L2dZ5WA1& zg;9e*uf=%2{337=ezCAj3WX#ymrTUwm>1b*Uu?GNT-q5?(93)c#C6(%9u*(a@V5(4 zKNU&ahlT!c=h%n_rD5mH9Vh2tj!T-L`=ah23kc^lUIM1GCKm!FI}VkMw@g8TGH<{S zmp|wOK?cR;muyh)u$0WMAbeZB+xQpwTbsh^E#?=cf12fm@VJ*o__wl+siNE1SHf~lAnft{)GL!$Wke0Zsq;w_I~+0 z!>#xIeHiI9v(S26gEa*IWH0+0#}jN&-eDM-szO>LF=p(@Z~;E@Q7_w@C+DxoC#hwa zzaYF)E;~hTEAg=s+)s|pqj!R`=Ep$#xz;Nr*+ethjG_J%Pt{gKKZ7966Zl7AKSLDQ zFVypW-Wys_ZmBuogM@Lx8I0CH)zv@cz|m`9uU8=MiNI?zU!76fKPdtwJ->G@f5xBf z&Hl^XBa(GA+%W!*e+XGaBACXyU9ftRilW$?pDEbiD1??oQ!)GgAK|1P+Z)@A6qS3i zz6qD8XL$+Qsk-{0x2ibFvUbXE-0{X6(68)?SaWaQ7m zh5+|GFS_aeS+_A5w=_sNyD@L%emJSf=Jk6DR^ZAw{!$Y2R0c+tA=p-DD*6}`N&{_t znlmnrcGM)zs15}ai?yyY3W%b!_F`wUCc^2An%F%0nyC~!BZu--wi4%vv1IxuOYxO) z+18qsDrF!PJyK`>0v&F*+R-ty5s>+9QuA@NOjfktgqc^U+?0!xv~AS~-P%vXGCNlb zPQn>isR=J4A|Z-|{(*6KGY2sI4z7oF_p~|CrALKL#aNCx7pG@zN?}1lw1z&0+CSY7 z{tL3yecz15^>BkIb0hSt@CrYhNo=~N25=N7di8Or9PV+xY6ow^9Wyen(*-HLq#Pk-KS-@K*591%>abwJE^aJll>HyCh+C($(KoA-#7l!Uz|c{}L89oy zY$cj$im~sXg!OwzPdn1XQs?Qdo@N3$AB-jI5u+Ol0d5zQuyE5_s55McCBE|zBH=hs zNr@hHAbnORiiIJMI`&oj_uPF zSuf;;6=MbC#>3A$Sg~RB`ibmBg|yhYjRfo$#FBbM1VEM$#|vTM=VH{QN}UmQVxQ>5 z3l{Jn1f|+JKjL%0Et{BghWn-%OkY*xIv)aB;DpFnU%$B14j z>1q5$@d!z7;f)aEkNF=!{+K25({@EX8}OAR#!9B<`4^zeb_%|pGL_i?m9)SMUQA9Y z{f3GQD(&f`_bQmyp3D?Bk&hsl6~=kra)SK2LRBNW662@(4eB`jg1UeN#;?w42Rh7& zI6%iwRi1!IJXc@;`j0#%c|78BFaxT>x{z%2_FRM zgHBg&Hs}JJEEATtJl4X_O~kp04%;%K!n`6?2PDoaV%@wd71JlyDq{Syl1%Kn#6B`( zZR;k?08Bu$zwTeFroS}w*JdlZw*X1@L&11pO`R;=%N^phocie{Yiw*43qzwCxA~0w zO!jRyr+yB#bjy(ne#@b|4caXzB*O{(*DCc3C7L-6+G%k`yTgoOB`V!!%tH{XUxi=t z*LKMVqAIV=Io=1Ie_5vaS$&!iE5;uYVmwwPN+~gIAo)VATp8*1WVJig%2Q^m6Q_#< z%jS$SWrjQd`9~YFb$AI=ilc8zU-k3*za)pCvC#;BnECeB zhJnd!|8vOZCRJ%0!ckLD$Yi*qIb#cXqH8dqb1q-`vRjrPmk;2AH^W?85`HD@&xV?Y zf2|`gzwX#HYS>LKrjuQ)qp5CGxNAI$m8_M=dZpv~+bhx7a>~_|7gi!g+6$mZ+zqOS5H*%iY0tMd4g>Z8O?ENoROW&CFS3- zg)iKZ8@H5Jz0O!Kn`wetg}FA%i4sR@OMWHgHYMdwic56#qJr{V-zLG8elV4z=g^;A z`r%0HEvQb}O0K~t)Pjtr;HURnj>P|^YwJ?z9%Q>ff8EwyC3#U*)rmQygy|Q04{M zw+%fIdlhc5?d7_Mqhrtq(9Xoy#Li{N)I7${DTxIhii1+Eku-PLU{7ck2nvWw^HNnd zN%0bLdmK12+3?c^pXyppo}zZdbmM1k0%c^2!y>U`IC7XYMPSeAO=sch|HJh1!@nN^ z#aB*dZ?Zt9%io>9O#xf|^S08LuU}90CyK{@fi&mG0fhZb@#K}*i|Ze0>+6FfwmqHsL7)k9YjV4*Gp%5>akUcqbv;y=Z+n(DVsmtpmeV4dm)tanpS5={5X; z7NMU?j(KZP`&dYOzu?whBUVVe1=s<$j$)QoexPKb>5raMM99u|fYzCPw(ms6%vz z8q~go1+#*>wSfH>m_s~OlFVSzCVpkWl-r|E?o(6-(*|*C|8n#qpE#s|T#W*w6WSsX zYVI*f0lU5WxiC1OEg(a1i~j{KM|- zjR|f1A)#h{-Jxzl27CoPI1qPWg9#u9y#L3~m9t zg3w%!wZ0mBQ2fgb@bKcg_r;3PFY-4fr1v9?d3*d1;~rioNZ0Qk*@}CF7qsLF=_I>+ zq2(9PQ)7x?5EA-jmq>C?^#C{^DQ0p~`hk%>61PAm5Vm?`qsW#Qqf1;_^0L;4EBaho zg-o!=H~j~_po&BlZrHZA8kb`pLT@6g8a*UID@k2{$nY2z`=r?R*b@* z5ulN@TXF>jX;}W*z)9E7txD#h#z!SzmMExon>K_eXeV6dzq1!!Y#)lvIhzB~%9&$T z9MC9A_tdVmd`+oN%-U9!e66FZjjb1XC;EkYv$QAyn^(Z>mumJ6hHZ$!FuNWA)F!^E;?{Jnq0p z`am@GUVrYO4!dv~QvU0l@VJmy0>fw;UDIPeNGitpFS9EN`d}U6Z;;Cm#mob>$qTmK zP_G47uUyj`T=9}Uu`iFD>X-f53%>->Zr~;N{#NRFd^}kUwYFMyGvif^(>P0k$c+?x zp92iwl$u|<8RrxRuo)|kAKvGf-WmxU^VEc*?8m;wb<6mS%X6RDGNK{|neY^efn997 z*WqiUoftDK{6S0=U161WZK3yI!G~RTtXN>CP{AIHZv8&iO`(vaCQzeAq!e?PjjCKeVgzw)1~{6PPpSquJhnXece(m=I7hu+AYCpuEWRQ*r|~D z&mz86ZND>~vtpCB_ih81evoR2K`sQ*EtzPWM~b`ao)V$$AoQ9`f@_3ikb&npb^#s8 zh0e2Ov0Ok~WO(D8Ra0B$cmoT*9^|1utVTXq8IL7rg41QJ2L^wEd=O*ocIw6Yf*n_3 zB`>zjvspX4ketaZ`~#v~xS7_^ACdeo`Q8ZpLOZ%=wR*|UMQY86IBrV-xm66Su5}@& z7R!6fvPwx!f#?S33~y!NH1C9-sDG#Q1D7SY%gnUvj)`A8G{NaQx@XxpxUv39uBLoKI@ijiB%{^z4J^ zqi+HCee1qIM$_tXtEAJ0M5qHlz@m;M85U_rLR&tw6?mi}!DqvB&XD7&Euz%&bW!IT zL@VDm^lxilJg7@+Ys!k1pw+6(M)~uCoh>(6OODGw^n$Er%I(Twi@Fu@HtVi)db)j| zsdhYMl!A-3-f%elvL4EtLm&LO51Y|#%DjJfGQr&>oMAqqiZ>`EjU z6}R$*@lq3hun8%yxf&38PDJNi7)>q-&H2;`61Z7oG&qjYG~5g5eSn zxl_wcQJ{+@wzFK8$Sbcb4qBg9HU2kP(12VGSTF`(W`{fe%j)YW#b56!5E&2|@R16& zNTnpp7K0U92fUFEo-qrGWpb_VGL=8~_ikj2wo?%aHWqY@2!{tx@#3=!bBejSR-?Z# zx~R2A8ps6U-|g$1Udtttt=>&5;$(C1RYnX?>+=Uk;1dmvk67l zE~dBWBmDn8wf?zLu8{E`39SPCm+@he5~emb_WwsT+|Tx_d=L;2BoKCP5OQu1^kNVz z(M#FcK4K6^aI$&TLyaQe*+bnf-+PILQ{67sVi4^0%}69c9T26>6e8u#q%7jQ6iw|z z*i}e6{qH5B;}v!poE6akA7IgUv=pL>VKOhml4!sdcnZ1ZM=n01Iuzm4CxvMVd0IUa zBO@~-(_bJISQ1bcP*_mVcr#NVDIn7|!28cqYsfZ9TOLIaGj!LoZ5zKeBsFNo`2Me{Vx`6~n6Z(JqQpEll*nzVX>iS? z9g8kNR^~~_p646Ro-F~9^&axAD0JK|vT9SXW|y;R zt)oQ^pP1QZ+JZSkJ?&PZsN=#zq<_~*$(?JXe6A7I7PLYR6EG5YS_a{Y?Pd)Zy5uhu z>VlzkoM^AyqQaq=x{v}G^tH$gRJ1S3v1!rOqIMT2*RdKtd`_F}K93WqGLfim=We7& zr)5*>q~6|sskHE9sT2j(jK;(i6C14#$fCrOAdNN{?@% zRUKoshipQGCi8fAV{x$>*s{*hCX-o|IuO@g2>!5zZ_X8dO} zh_6Imyu#_kg&p}jNh^mFta^aDO{HriSZ(c^>DNr7k?z|nd z;4X3Z`S^c;WWJ`2e{TNgeul&TmzFjuHCZ)FQ}_SZ&8n-6V~Xm>P6SOlDLxohqm(b9 z4V7$RYny@~+KhmP*SCRQqH{ssI)K6>xD zyB%z*89Z+C+HF3|Is2IV*xlv-`#Gu)WLJbKigA)T7ZG-l7;C^639;EkNFRknD>`rC z&1W;@1i-2mX^6A|#N8#-uLRrc(h|U4Hq`T>uTUWQVE2axwnl1>9mg#BRcN$QKWa(# z3wWt>&n2o(PEkA?3D%`fiuFs$t;&kUP+3f*cLbWLPKE|{z!pbuGI`d-OPf_Q8;;6( zlWYD_YiMp1mJqC}av7{Q$_`wq65!y zR$i}&d0HBn5;HTk@+4vFZEo)_%+Js!stHV4eo$3|ipv6D%v5o3z9ZhmoiI z(p-bet=`yStgb7^C;KI1tVFBbx^MMg4~rPCp=_23cg++PQC_ue)l2TGyP;+)6rhdy z(A2os!$8LX`|>Ko(q6|)VtdB47xh109(6}3V&&R(w2GB?v{~eGqa6=p2#<<5QW1#E zFwjlqnP^_5^_g<~pHo(|mno%=lUU3L_d@+>wICiD z{?x@iJrn+M=Cb>epPAkIg8a8WveNp1>1IfWoK>8suV_O5K92*=pO8L%w+EoCh7fe=WPXtz&JqVi z!|5Z0Bp)_103u+CDs?B+R0F(x-e{z-!_2`3-}Zg*%u{GQJS;vT={dM4D(&U`0rB%`nbAt)W*yuRxUmbR2F0-QWLs-?Ua)a8g5m+MsO+gDINX zTJ_uuUX~QS?=nT;8qn&bmo}f+h2A06Nj!q&Z}w^j4ON{$yoSl+93pPvg$Ri3aLTyt zlGe=HoSWUD%%qSjbw`K__GGu52@|`c%!$weKWYr9sWaU)oGS7?ac!8#vaudX(tP8s z)VrnjQ)z;I+CvYfjIq!J5Q_E>r{>J*#kY97w!!~Id){_!yQ8Sgak{dbEd10RgkYQn z89PM{TOpa|zpKne^qrp3%6{0m?qWnF*4ABfT=sb@l7L7`@A+RX%{$6Lk_qE-d(GRiJqr#=MLt{aOk zR0kizsI!ra}=vqE7)k9m41Q$ySet!-@wBH1p;D$00Kh)f2vT<)WzN2 z$y(LL(&qoDL)T<&8HU~nnN(hFZGZp3itP}C!g zH|1^!^E-$A5{w_pQTN3Kw|p6jtnJ(S-KNif=ez#BexM$K-LYU|+VLalp^!{N_l%*A zCWRiwhfKL=9+3@{%xm6T^!)bF@SCX;z| zo``F?le|VNqU8eSIX(d%M2t1xaF2wf2O;i)YAeKiP(DB>B5QP<<3%|*JpE#_IqgPU=Y+{ zIL$Jv&w>4PGs(8zI)jA)w5hDLuurKL7os(o9x=rM9F=AVPS&i|*cyYK0l4gEbLiY45IKdF#UIx=n->{1DMU&9tuFoS z;AI$}v6gd$Wl!Bs;bqm`GjPT*^ z#D1|W1Qws6W0sU*R%t0iH8krbms)Azh4Z~(3PPcKAr-yLnc-Q zc3{QSsYFlQz5w3M#iBjivZ-rPVcCU52M^3@@~2%G(>N`CyT+j` z>Gdu~&dRth$#;v4{S$ACuJkoB$qRRjrUy9WGka_H6Lp8-3>cCC@c>ulEk4OhL-CEA z0{qRapOwNUJFl{PnOr`mUxWjiFx;)yvPk9>Hv}-Z&n}2A@2}5A2NJ~Iu4P~2%i3m@q(4j7w%6Be=0XZ%r~NN( z&Q?OW`vgy(KDN2|Oi0F!&!60z!ycJbtBYh|*nNz+Pp{yepCCpf1rs79^_>+kSay&g*B31w zg6r;|!#n{1ZlC+dg8rNMaomh+>o*2(b%ooD^BbpdfSdDI&9r@FJ~7I`k+D!MDc3 z7pI?2P$2kWa`M3W{TlRr-}#zr|J$m9;-W{PM-!A zb~3hUO8KtX9V}U^A}t^EGK#W9J6-99{+xNI8R&7m z^o(GKi8+-&%(SHCJ<)Bl`U9!1EjQH@vyiiR9pLok0W!J`I~8*p;u=u^21FZ4+mRZf z<=P4;nSbnE701c`7CNwGa4eT#izk4rr@aj{87uKXmJA|HF%47AIg&OK^e|;!ap4UU zX51*La~IU)%J*_&w#5E9LwyIOPhvpPJR>i%+lSBNZO2?7Vfv`((LetkymB zV0&2~c^>?ztT0-$4wsm@A-=p+2{3WP0ItiPJ`El>k@7@OYd_`|cur=!peWh*59579 zWgf%S8xIfSY=FGA7tjpsWwNRR6PkSKo6-p7^k@CD@l`vKsHpw6i`7pUIcXEoP3glAoLr{iFn|Sk?h0+$cA*r1YDJJrB(be9- zqljsUFVY6J20L{ipz7Jw3= zeI#Bo}UvVh+b5tuGAAUxLMy1H$<#d~%T)PJs%`>8-FSMGeI?n64Qi<78JzFk-`B@h6#57K61018CU@6GvLr zYRq)9<4F*45ERYV^eOz)uN6N0{t%+-Bk&W-U@2IenRMd~IE^CBf2gx25bn%H_xQ^1 zgsJ(b;>y1`_cNSW+=H1jp&H0jeiS^u5}~F?R?4NNBaXO0WDol5=PWJ6AohKf4M_$H ziE1*5)SWw>zylqZNR_2g8Bz$2M-gSrkXwK+p{IM!6XRmDKT~Ao+)!?6zYMENnSWhD%DKF(gV`4TLruu@oy`=K?gS z)2rsh2*$SRmm}F(_?g&}89ivMxw%qg^n zO>11FkP;H)VPfZSa&lIMA`G)j8>O47TijH`tG~V2g->eUP#rsw&bD*#w&Bcl^zy&q zjFUW|*k%?;R;}6?F8>(y^}0s67*s40R$6ATsf`Ps8r9^vPXGGNcGdLiMv_(0d+9Sr zV>VhxC@3^5dq#{){GsZa2J~7oyJe9ycwjVUoL7q4q*&R)JXtHEgM5c2E4EEovov6o zV0Jg;|41J?vHMDD2r@#FBoMhQ-AHJl5GjEBRos7pJJ?TdJTe&Q-U;XX@8D%r!j16i zAuq>{8J2y`hR->cvq$x%x%=%PRB?*oO?^oZLA+A{yde~8U58oR4Pz@TCJv8V%y3Lx z1cd$>ty7H?kO#CsEiPu-?_KP*kg~q)}L#R`K-AgI>BXuWHrvJ=9bVu zpv+3rKfs^r(mycI5;n~eg8Tgm?Z$t=$9PkiC2X1X2<+Yv2>^Sbq<`Sgy=j`w0sFLq z^CLWcKVzlX0(k?4+k$v&z-Whm2*!9Lo<-6)OVU`$(cs@w88pdqFK3d8l z|7!oq9owR>@H7Y0sh$;>ox9V=nd6_@ZDiYxvuQGAki#p7~dB!fgynUAPlW0S1wk3tB&hOx&~0A^XIwf+6Fy4i@hK zmTSibqLZ7hMawT43#yvXkko^G>IJ7%1{0F~B)aG%&;oMkE8#W^5dbaKC-SE=9R-5_ zG3;0A3?0a;utdDY88^5XxCdf)#kBHXuQts+N7Ce^%3U|m>|TFADMy<|dZP@rnCy+2 z`;<~mPOu2p`%lWk&O|Vc8xv2k7Nw)w!pa#gmki!(r_&+4=ioQj3by1M&Lb)A(eD_p zN>{@If5(H);8bDC`~av+UaprDt`|%-(VKbeH%_ls*)VoQ%mRn;B*$49Y6(P=10>63 zvWfYK`35`@53u`4-9zs123TWH z{P~pmkoF=Dy!uytU}xB}M;sj=@#uMj9j5T;F?VQ>Ekj?bGf7vlbIeSdJoT+QagHrx zunfozY{3B zPt7v@41h*X%3tqn6yJRe=pOKXL&@aMM2-8$ z#uSHI=xw1ISz6vx8P;^e0#10)Vg%5PdbnDDC+v0N;2$imLO^|{UHkF_g(!IW2l}_> zB;zXTfj`)lOO3NS0$MLKr@T-H6)&)@UF3WCk2OjSQe&E@=LboK39C*SIzqO{A(&!l z;W0b|6wI8_ZDvFqa3;`V#NjrviCsT^C8rk5g>1#RiMD0iHf4)8CDk_-X*~(6_CBG* z!5d?c<{y2-`Fzp5Ud``aX7f)B-dx_`B9_E=lCS!`w2hLYKqge-{x14bJ)~$G8Hl`a z_d0ENg89an>6%;Fth}YRtkT#v@1dBRv#)8+3FO&8mSJr~1E{9e#mMAtLvVA$PMNCpB{Ch*Gnr;) zc*2_ViQxMtd3yniUlB9*Hcqj26h`Wv#Ji&m4t)x>k^1yT;)_mBeVCS@)W&WW499H_ zJGJ?yafb#XCiZHWQiBs7=|0)3r}gxh4T}g(!Vs`ZP907qTi*=^KjQW%ct5L@KOdWN zjO1G@QZ5OJOs-%*)$&@7-fy(}Af+*#!a5bN>|j3!#tjpxqy=T$P`50rJ874-^tfg5 zv9g);T4YxnKodI_9!ua8Pz1qv`DJ8Zm{(6*k7)Ev&s&EVov=R(7VkRcD7&R-iHONw zDOD3pfn>DWdT?1~a5izBwPNp~vbz$7raI7T$hVleo6_Sas%q1Q`Lr*M;_=C?O>V*( zI6;TAfComVeDQc^NViU%kDVdpL+%1)%d+RC%v|!Ant06qULJuQ1Q@O02-u%g0{SCm zLen|Lox`j#n6oF@R)g{F@ud{o(j z=9P`w*(gjUHiHr{e{jsBKPmaOleZ=e>T?03z+tvEl`W5(^dnCwz21EFeD!Ca8D*sV z`W25@Fz^_QvYHCxPr66akSBV}6Rf?-ogw;9jOcrQo#~xau*C}_Aj)NF<97s#|A{L1 zC*~(7a=Fp+ci#6<-O>YX!^tz&m3M@#qYO|X!Gu@7dDAs(apLbr3RL-^k}6fIB)2$c zILUXw8qr9DaCZmRVNHW@vqkX7l1coHMKInpSh}e${Cz%5+lZxL>S7N?upuK%x-mY+ zDmjK0?-9n4%;rv7tAOZY5ebDI<=n9J?Ez{47hIT%cgCM^OV%!4u=&lEAN< z-MJMe4Nf$*LAiTPw?{)mb&G8QLjzD-BVrK2;^axHbafUBB5R{;<5C9OF&jwZl_Ilj z{A=b~Qs7s#*)qO852S$&^v?T;>G|v4b#`A5t^8RbHecNkSTLt)796mzl5NQ8AfM`z zk8!yabzFzM-#KVujov#evXV90oY5h1X>iQ$-txPci~&ERbASnBLgN*nxY(+#aEPay zafHtwym7=X{^2r#d^7*IAfaY;xhZ1{s%jy#8Vcxmxic*^ZM{nFMD)1ePv2ec9R!_k zI2NAZM2^DpmKCueQ*GA?h|4h65x3x0%1I23v&HM^9LPU+^`eo zsgORn5HzxvC0&zAOuN>spGgEq=A!Kv8+Y;wp`mqtNn))8|3wDVwsRXqa*T-_Qi*1J zrKsmAc=XG>Gg`KSRzgNI8{K++lQ9rZbq%8bP-9i91<;kXh+ftDc@t^ewO@Fh{s#7+ zZZhS(u=79v0u>qjADuG(4XTW>yYc@3c0!?F>0iK-hSGpCQiXdshAxGppt2)#mPQmZ zQ_#GN^7o@ki^ zd)0`Q1oUIXI(A(^)%-Hmn*fBynnPYTBQ%2I)`OJy@8^Vwqnky|u7_1$2v3 zwoo$kaTrA&*<@I83Fc~qzHRUcJgWo}C5peG$JDYha#}(ZzveB8cat*b|IE6FrDc$* zm6qTjJCkZk#jl~(_Z>AFkG6|N69_H2KLf+%`(sPbxPAd7nwjMgj9GLgvNusXP}%cY zq^st_9|I(k=rDXm*BYMwTWMf@ZzWRRzd-$WFoE!&p#JY*!vDZ^>fgAA&LoQ@{}QC4 z*+*&@QDPR!hY3Nj1|uF6GYja$r4o@{C%G4}*J~%48~ovT0*8H)0>lbo<7V~nKJH+> zTA6vL{Kbm*vNzD2giqa}@}xGXcF9Vo+rQ}9K_e&2`tZI_4o=Gq0SB~S(`%{d+(O&F zS&1uD;q^DZJz~>u_zR!?Vw@~YjfE6sTvmCOKYy_p;x%GpR~|AhkyV?y>*^jAxwS|K zv~1%YZB*e5mMbH<>^2rK;9n76DF&^F-}7u6%JD3pdXzX{=3wvoovc}gZ>-(9SNn~P zg{r##rFNZBJeR7q4<`a*@| zFsF^nj@_S=!GzG@O&eqYLv2!3FLwd&Ly4*cR|>N%p?H+a8%?y9bEIDEyac*%)Hl*> z9E=oA1KBy3EemcmQPtZo;t<&-F@){$Ctm;C=n?AAW?-!gH~m?C@9CwwhWl*9xrc_R)fz8BzI6Pq}(}Bt-zvn;E)gdlP&V~d5sfPW> zWCJBHS4#)`e~yJp)>uvmeP}p+Eh*$A-i(GJ>=C&Zh{rtBL4L&A{MjkX+< zqVWW24nm)v&~i1A?qkU|Ia$bji;lK1(~Eaf77>+`T_34>nZJ+Ema|@Hhj`wtbT@z- zWOLy`uBRW!yiVBNZ$;Ts&^BkT9a>CvZZkvV#*UOcan{mTN*6Txhyot{@`_RU1gO}5jik2@tU#S zeW~Y1<9s&-+OyOrG_uobV&>^mCt}DRUpOT%s#XrpH%V_L)DoYt?FV)*E+*NZC6e|J zFh`{rYsO-uWiv3y{9>fP>x#ZfthQWXEeINy3cPKedKjOJbz0$&Gfa8CT7p=hRv?fX?( zw=~u@(GnU`40e5`sZo_8H4BJ`roEP|a||bw|$t zl~H0+b_hQ;r^PphsT~Aj>$E9JPe0BQi2;57WU`sTYmCbbb1pJId$sBuz)&*flDd)b zyrJfbFgwf3Z8g_c(oCB2o)Wyycl3VUJyYLVy@5&lsI*}O*G2Y0w7)CFn8W0~_2^6* z)uunG?n^kUC+g%{H@hm_1I8w6`)Ed_4={eWtOQ6PJq5>ALiduM7HW%VGPj{E^_Q3{ z11Nr-v*A@>efUGtcLOmDhLoeYwjlA&ctvW6H{=a5*o`$0Z{V#piQcP)6$ZG|#Pzx0 z$UFIC*s)8oEAhuo@YB!N7UKhH!98YA?Dd3IF{bw5bv~>r6-|RXl7VdNJosK64j(uf zqs!-`+*Acid)<^VK8&W{-!om#7peGaM}Aheo4D87#HA@{c2s>%qKY71$ zIenXU$-^aZcuxg;1saiA4fYHE2>5xTADjlpj5vb+IQ736dOnrJ_;H7ZxMzVkT*D=z zH3EG_Wp4x$u;mSQ#6fC3Y+%E?d_!Sj?ON3O|WU%{DOB_nS+*^1`!J?~CXseCp^8 z-8dnckHXh^=Led$EkZ|&e%J`$l@|O)N!T+$`Nin}dn-s*1cz^r*4y6yhPcq&Js|vZ z<`pGKITYVRHH4+C9G?$vV+EfFN;%3Q(Oiq1hg1@NxS+O&eVf0%R?Ggmifdqop#Gt>WV(qlAr9dR|WzgAMnbqz+!Z|suTriN0;*JS<(p=K7x zWT9_zMd0!xN>jPgJ6yV%jcw_7)Fpz+GYT;Z{n^e%pWk2-GHzY80!}me9RtigcLRSA zU;%#+{Oe=y{?j6EdH%fej?5+2^~rbV`}6zttKr_^`|rgL@*Mwze{?37)ggzm*1mCi z*odhsw}P^>m)4$fav1H$rFC@n?(2lp{2A@XS%s5#LiNxYwKtE#%xpvF6FWio^Z=)) zXKdJs*94r?y0E6n14+Ou1!F)|n8V*&R6!ThhkA+M`E^FtiyRydbhvFn%9&I0t@vCiZg$_TDQCtN{Af&j=Yb{b)bDO3dL?V@> zz(U5Dq_}K0%QkCe&}g%ooR!+)(&7x!TP(8M4IG{9rZIuq)n}TTZ|gAqisf3S3BwYf z^|dnYqneJ!<=}9e%%~>fHf@)6ucPBkE|<4y46t=F`}VDNrc4NrPBItT;xPwIsmj7F z6!TljVTzf_coi_2tzrbEMlpBbm=3cJEahookesbJb7P+HzitKp(iLST%Og z@tV_ptdlc#FEe|yd+Wt^6s_!24hVbzDNe`SZ$BtN862H?6q?dp+5LLbkL0SI%Su8* z67KEa#EO!5i7kiH7&(JebH`MJ9_8yWeXb?>*93K~a%s~OGzmCIF`c{mqIVJV30a(k zwX2>j>IA(+VZAyrx^6tOsytibyl(5V9v4YP25GE?4zBGAHMh6r3N>?H_zt02R6XTI zZsW=|3N?SL*&c^a5wf~PcVg!L@c6}TN6tOFy@77cTK#G%@;p7r%L&tO)GcW_!;d*< zK)iTh=qgzt@Mjto1(H()iHk-=4~aK~^N^{MHgV`Ue1aaSqjvYq(R<0BBDJs?Fjbno z`jCY8JJI2WlK*`m7tjukCbs7j2zBZ>Ot~)#@&LSQ4k(w%EfTsPQ5m|(o%!)!=n%zZ zC~WUGUvH^3?@FmPr+AQ4bw$|ZV`mR`n0pSf&kgC9MGe48q{$5MW>~^4)%D$$#VTq{ za>+A4k8zWq_;Z&WW&U>2mAYFfFwmK2a zZ%?MWg_m5A;uY+kL73~Wd?U5&J)+N%~@DVoww0E!WRn?`%Pj(Qqy-QnZ_ry zN$aXvx0iw*P!0dcJBC<{&J&=dO@f~{Rz87J+c)3|YNemm&~PSXxl8H<-gJKFqBE!L zjsMIoREOnGm)8R(xvYiIQ!vc5$t^D=X|F888RUn>-(e4C{3@1KyQ0fi$b&k`J^j1t z(JgWytX(Cs7@2If+JpaykAWkpq_22D?YqMCEEPjJPsfKlbR8*i{OD}fS?6nqyo zdrQ7}hJvDqN%$(sBHcU>K8$9#lIIBO30%-`^Lu=4Nx`E=7S{5#F`3+tmk zkAWcmX9~1wgmrC!ufo)et8CV;=m4pagosz%#uDoVO4lD>JNZ?B_XOb#Yr-YXhbLL3 zH#Z#?-xIlhOhPGcN=@tfKTWTL10jhP~VT+ z6~(=p9J_VHMBH~9AD%kA{Tjod6V32<;(C46mx|bc3+ZdWwDb<*Pn4}!xEv~m@dw`7 z&uz-xl~Hct+&xr5Mi^&-zOcp4YBCy(?_3B&rJ#%8T|t^Y%riJxM-Su=O7a)uP893D z-$-D)g0y|wSe+P~$PnM+Fw@k1^uo70NxO}xn>Y~P`Y;4aK^wul0aSg|L@zxuw*=%|*dR8KGPk*?o%GPL)P30Go<7K%l@Nj=KnTJ<`|Dmy zoE8PqT*f91>seI5inbcdE#+6>WMB!k;Xrbq9m=8g4^2sE%~T9eu=-oV)p)H>K*u{m`$=`B3mK@kr$Czk=oNx8 z8{w%Y`cXF!y&s~jP~@5@J^1K)zlm0mTM@$?y%tUB&FbCFs z|F>kPByOSTM=%hO3y6PA1yXf&wz9YQN8+H0z9O11I{!l4*44ZWwT9R&9I|X^J~U)r zP*H9?2u{??QV6_R=#ZOkO4YDOK&av{CV}r9rrf_%{G+0o&)4V?Xwu}5x4B(zxLsy0 zS8{T?L9qHeW6kyEm9Aw+*pdGTikJpc9Y%lhw_%i&U zw{o~lm3zPkLNrF2+NU))^3@eg+qeg-CY90Smq-%brX_~b4VW_=w{fPIKK8qDykf(6 zzDW_8;4+?vfL(=&726ay*>f#M0`#^0+o>wPa%t>mZ2|gV-gPQ2KHIqE>UK4;NZ+trd?J67&yK6ZB zq6hwJSzGe(P;5zT?uZxOK@Q@rIPVzw z#{`>31EGA?XcX66mmcn$L{PgkF(Q;rQ7RM#Dgz}noPqa%Q>d?8n*poLyR&JI(?PBZ zbrascO$#vs|B&vqZPA7K-c(zvGz7|lu~}U@B(i}4?VFR^vR>@SJUM3|8oc#-g~AMnz$X zLz#*VJ7w*_SEMk7`m_eB%N@Y{mS0{^`zx7wfVK5;GCk)R>RaI(%W&Va&wNJ^!gUR< zXN2)IEMDUv4zi&pGY5}P@fJ_v9O22?XlSdX_5cR}r(p>~{1Pe&A3PZM2zMw%@s*C- zzjydgp5*ztr`bV)fIJ|8fRO(GJTY~!vop5;>B1;%;^OLTZ0aidUmO3$nk-ElO*BpP zElx7*XmA=4bV7EWD&k*dkxh`0ze$EjbZT{~n(_OQp+tcS6hwY2I(AFlIXdtDt1CY? z)KM;U^gs2!g#{mx9*|tCte>-_O+*4nX-p?GX8}l#tbp^9rc^-QX{b5aZ3QyA&h%8A+pzjh8OSMjIM z6ypVkKq#KL;Rogoro_UR)}U0##a8BiE)bF@g#6^Uj1Iv zN{rHJxJz@hDGvN;I?A}-{WTis$vJzHbVEG{17m(DNH~~myorQP@8L^&FNkXbtT|J6 zS(}Nf0T#ZA+DL!hDEzge816MdrjlcV<$H{$O2Y!iMbnqVwz;s%nBVDWs)bu?b&^m8 z){0;vlta+?!CbpOo|o*Sj2RpqG$sK-8NrfN>>tSo{dRs^L|E4unsJkfDg8>9zfWg} z^y%6?zjlLc?kP@OWeklLsZZ-wtW-(buM{pRV3)=x$ZBiFyxplFYig~a@w)LR$vr9` zc)&i**CWq(;mi8Yu_e8;<4G__)SY1UVHGO4f>+cwL!0zA<@K&oisWS+ zX$X2LXjFzSJcH}i?kKiZ?l`vhP%_$B02PiUK_2RzVbAKmzmU``hjAb+)H6j>BjX$m zhDNv;Yb}pAudGP-GIufr>_`sRMYIt(!qj|&6xDqx8hm3J(Evq2y1#wP_GvGtgBUi| zAq^{DpfDkppU;@LuvUJEdM{&VU{UxHx4e##)Efd>)W$@1_Ia;MB5&?Q`*0d?N~sSy z2B_m=O?Oz=8d-mk!NrXf`FLbE)hmH!>TWK0N=1_O61br$N{Yx99l}Ys=-)A9UnTb& z^yglB>cjkA-a2XPkB0NWv_FalsuPqF&L%(~Y< zQC(ZMh)rXQQKXV$6}%ao;aI2G50i4OqxZ+xZWNO?)u@n1a^!LhTe}{fIVujtx#V+w zHI3m%OoOU6o!wZWT|31wmWguPBYD)Ads>hAaK{$2k29=?4B^UHzK5iTLy3@GSyvKD z0yz!@Pk#e&vp(Pozj+eoHPRVG!)1YoexeZ;fCH17t(>3drFnlq2N9S|6|BjDCvE!-Le-B9ZAm!7EmV}U(0Enu)Y7Phm_ zhP%lzA`fjfR+dc7j3TU#j#Ai?KxywWZhv;R7>0iLno)I-JgQRN*mVYRgH3Jg+yEdT zrcILZrT{Pa)(tgz%b2f_fQ8b|ATNT;9>;?jVAQk;SuT-p=d}p8w{iqO#3S;0?F#71 zxCugj@}eWsy4|KcNL6cbw1=85Tf|PZU?Q9KtUt!a3w>J{ z|JTmnBP!PCRIFp3WVqm2Z)9tcOQYM7rOm{wM&F4&>eYPlv9F5yRqW7TMjE_g zcYB8YGvQdD)))rGsz#}-SJKB+C1zDW>UmS%(ja>^#N?ehZbjOm+CKv67Y+i_Q+9wU zgR+7*sD%2$H6PH&DQ7}oK~wFN`lG^%?V`$M&QM-!zXdH~H?QB)6HIX;hB`v)%sO!k z{u$Y5ABrbrn68mxnPX=)8IpEL?9}S*>_1#--=vZqB|h=IGL@qW+F7P|qQbN(WeYEC zCQ9wG`8c(vONTGS1w(=)m(5T4dbNLK%@zFjdXTc;c4+U=ARxu4|0vk~zqggM z^S_(f|KqBP(ToZ}*93e~tZiv?pg{*hil9k_iqw;dq10{}*+Ro1OcmIQkc4vBawM7= zH{iNkpv~*(>MjaYR>=rcH!BgAKF-n5TFSq1>hxFtu-%QF;bc`uYGPb$g zq`S=UpYWdedfcxE7O{~Ar0gW-`4yK9x1yqAgPPw{JJw}Hi1A@;L#a{B!)6r?+x zQ5t@x{G}eukS0;;R2!1E1961Hgx-rZ-5N!kCCl=RWv7Xz)ILsP;~By2!QG?f`4a2g z>Lymn@=9u`bj{*tTVO^qe!A;)cu`EC7ER4UWs{3$VL++F%@a39QmnJdAIHWwy+$$8 zlHBFx)AP&l*7HI-LfuEPw9TEeg^7Not~hzzR#)_g#5eJRYFGn_ShR z<4>@$nlU$3#b1t0Qy*;bo}8603o3?!TlraR=Hu_GHgis@%+Jx!$JP#7P4>DXE!a?; z5CqWNm`0X}D+WMFB$#^hr2foDR|6Dv(WbMO9*vRTcHYKV9FXR(aX#3uv?d>oy<2rNY@L8-=eiXq&#dlH-}Ty z?f_4JB5TUSS-MB-S_SOG4`{lJg_h#Z?mC6ve*<5k4G^ll?)z-#c08|oQ}dn;s}s>^Rq!M4LEJ=vGQ_}Kr6jjM{LGYPi0Y9#=6~ObD;swI~JFcJ&w-& zJ$2ux?Ma%tC)b2{pu+(Gurqh7@2fcU%zoLA#P)$KIDd=dU%5y5QU)ZmpNs#!R|;>+ za@kK0n=-@i<|B~KvSIeP{qWn>oJHS3LSyITu*{$yw|eE4^+OM6X!%kk_Lue}Bh)UL z4Pj|xHvHJ(wmovk`UT%$<<_TBcGRKtC0OiH5+R^Jhy4RGVCk0gBWsI`9mR-*?E_qs z?E_77RMDlEU>bW(Ytxa&BHgEsMoVj*%hge$p<%`9$5W-%cXQcPM^uQq%b(~64%&06 zA?HJwg3|rrdiV?<63R_x%cyh|3SWKPH`UxF4a;-o! zRtbviJ@wkPss-66w)XkdHDCl^#Ej8wo{h2-v)#Qvk#GuS;aLzuR`t`T^ohx0WS4Pw zD`Rc%erm;nsaXw)ebZ|VQu#GIt~YK zTjCcS)jO&`1Hjd$y)d0O6{L~OU~_0iQgcI6aw~t`TX{0{cjj@;H#?)beI~`&Q}q(I|AJhA?u?AQMqbfT{A;F0vw9BW zZ2%k6b)@SjY&iQD-rA;9*wAIx0~>C_bB>NCxBR(Pk^B1?P^Z+pUyS1bx@4R~XX6V= z2UEd*&XyuDlE_zFLJYy>e*Yb^1--Aam(|Is*T6DA|^Be z2W|gIW$ZAkn#>h7uCzc-2j??$Km^IHTdN1rgP1(XzV z*Ims-Fa`BWy%n5uS7G@Z`Qn|(1=Vy39xaFK4tviQ-QxgS;Xe2PUeIUyo>c07(H!#T z=Sik-TKba5^rUJ!H*Vl`12~sBtuC|)xdM?ee@(=@$jeu-y4af#AV4Ec=)LtX@+Vay zBLkc@pUo4=ER7k(qi%2ebplVyxGmb4k?I!Pau~n4uy-L*7rQ|RQPr}xRmONdyfeJO zOi=pI-vk9ds^Wo7Jb_K?W|TR(VcH62o9dE6?yR-Qpz_KTc9zL$Eu}i+hqQBudnf{3 zz8upC#i(y$AH-2@464P^!bmT;mc92}ykf3=Tb3 z?|3g?y5>OJmZ7}BdHpvb+E#B05F6a?>5Q8rOnPLk?0{9UyNo{hnQSdJe5YrC`;4Lv zw<5I`b-nFA4!_MSJBd6^cIvrykMmt^ZCvG{5vs5Pi z3%Z3Py5fm+xfQK`S77?&$M=8hdm#l&>d*=f0%G{%ANeo;M>vTZ+uHtzZt_zz_CS?H z?@9!bU0xzXCbg`xYCvvAkw(-}PDLbt@dHC^w z`KYS6b7>Eca~hVF-t~7oi}SgQGvNFEm=T0?PPy)vE^N2T9lL!6)$TNWaV^0Jr}MUw z6Xvs;4&bP^EI))E(hPuOKU7D5^6vbi(SBX(`76wM7NCY@(!6lyn7WtmZ^_4bpIv*| zIap~xS@Z7Wfpsh^uD`>$M3ZddX49innS_lR7;G26SM4~K4L_5Txh;gnKgVPnhxXaN z)G!D&;7YM!6rPlck=M|^bf8|dad>T7VC5oxoCe%t(v$rPGZMqo0td=^e_LKLIq+;% z&)Fc{lMgh@_911J-ePV(K%}R#mXN|CPmDQuUvYSNRZPcV=%1l?pXdZRVjrg87M9N? zb2|DV0T;;_>R+dx2`td=Su)Xp8+-Z7e22xWR?MaP42YSxN!aTT?VXgT-7ci*vsHma z7E?$4ExP9_t6={o+q}(N+>yL3-|2dXRIosxiQ5bo5 z-#w_Hj#x*zk>dUYzx{}AkA%dFbmnZ+#~AESfm8W|BIRF#@#9@! z6XeY7gAX=h1|%7>g9ak6ThHxsreG*IEtZ{8UP%#dxP++rkYBAqMiANc_yK5wuVXeYol`4KBD6Z`O2 zK$}7({O_bYx%WG|_f4cT37#P(VN~|WJcSpA$i<82yAXF=|LB<@u70ee^YWSqJ^cYn z^cp?-10JW5Qv>Yv7~~87pH=|!>y6Pp)%A`bz-F!@@r7CCyglBlQfTivYyDd$a@m8t zukcGo&0gw1|Cat8L*FpzvAO#f*?)q9fZ+a5(*Kk2W^sKA{X)pV@4$6+HHbwID$GAv z_SQa1t=pd(Xl zWky1bnPeV15G{D31|n!}F$|XmkN4cTvz+_pSTDk(^uKhpC_BGJED=(HC)3AkB6$U~ z`Ee`m8|~tQXK*Mr<`u(w-&w0jK2}S%+rEFU!-1?{gBDCPl(r zhKheO==eddO|X+FJPj{Apg@m(!ZoRP(gx#l>Tdr;{S0{_Aai@Zl}p&I|?2*pg0 z3lEmM6|P)Ys{#@gH&cbH!GWTOih-M4O&}UW2NR=FvJ&~Nbnjf%oUSxiNN9Fw1PFbL z^gM<&Zh24yf+V(UU2OB7WUp+WY>Iw;oU{mn@Wm05aD_-iYLd1B%b<&YTMej=0-f}) zIY~y1Df$;G_3R~>!gOH`Vugl&VznH7Md}$O7uSg2@@&UX;9Gr|L*N_(4mWB+_@l+k zRKty6fI()E9*`p94N9~0fx)U1>*XoVwgqVkECp&(@@*cN5bXkh0ing1Je!H@2P^)W7;4cB-XDI_?5{wPeE#)63VIby@!HV!z zC8TG#X#cjeS4?5LolQ>K3~^7f9D2xO)uu*A_Rf{vdw)CAI8;|!7KO@;BN%ggZ8Xkj zM+X8vyR$uPbz&&pDUPM#S=rC&7|v6=;}@DB!8VC|%&<9*SXT6#eqq~*gVpK5vCL0m zx-IM32xes%-q(88kj)lrLt2P4u7$Le1b$N&n5arOxo*_Sti77?JHWYmK^)?QBv>n! zNd)$ORiRg>Ua%4DU>LOq=47-)Jz!$@&_F&|DYfo8UW? zwf-`Obg1zBal>-xPFE1C?2218F_Dz-Nq?xMH{|1KIP_zFCCoE-mlQyF5E!ZGC^uB{ zBG-39)7{H7S-9PTR~D?JB_(sJy`U41?T`Ce zzQ&%7Ds1~z*J)P-K@ZIGmMu;_oRzL%&eK_c_V=H{_sw$V=)7BaLc zbNQL+^ttOgtMuk{&ZxrZ?)q4LcMcfA^jj~mZnBU@XYEu9zbWJEOKlvWU4udHED7^i z|K;Dm%qIh*RqHSC?Ji00CUa$UV;NS>-NjW6rv$$ywp%ijVs@g>5Gvmq)^%X~)F~MQ zBgWvreTF`)$=MI1i-m}?EX(I@LD~}5$S3p?RVN|aQq8bh=oZ%3)5b!2LKOpV;lJ|; z_Z5K{Vn;OsanCHl%i(oZV1&5p!4q6B@*viwJ8FeDT?2~niHk4qi? z3HC^k72;}(xv@4^kO3~?i1W1rGSp2-@A$&dhfhw^(6+d&auPmtDQ6p8SPIiXt=h;@ zB`OT>Xnj$cdrgRdFzNxFiyS`5_wv&Ouf}#SJU72s%IFnfYFe~1HLo}N(OyC>tM5}^7&A%C2Q zcfd5SfaAfbp+rFv92gbfP)b-CS0Y}PP)58%9|ZSI>ckg)P&iaKAabTX)BVkGVSZ*c zr))ALBhaquLwE>g5ZKEm>&E!Xa=B}=&%M8f@^l%y`jZlXO_sY$4oM0t0Z^eukIw3C z-9SfUYrD*@{`hZ^S;s7b2-s8_u_?_Pts4!tnK99I3f$a z?hYT1nXZ%UF50ixo46j3&0(QI*lXw)k-^j#88HHn6Q*DvW8bNfcarWtZ~@xzRT1T0 zQGpaOzJ5~y9}*+oL9pwPyQH`sXVqgTcQG}kqhg9(USPbvYe)H*W0NmcPf|l{)tUD7 z!rPYZ+E&Lzt)90A-4f2m(zt(JdxL5z-)0TEy9_)EQqhE~_X-D$SP3I$Ot;yDUkCL? ztPJ50{QLG`ejX8g_@!agtnpzzO=kiZ&eQVZUSELr$BS&%H$s- zZ3C4i58lo_c4o3Q=x#TU8v@~_Ww zQxt+0-d35_B+s7M#ZgA#hUih#cx{$FtOu=Rab@eV+{DvNB?{KY;aSAnTud>f3}D7$ z^QvBROlG0s(B@5Ur8FF(i>gJ}P!E|Ow-H<^zuJH4J73|%mX*e_F<4#N-EnxSz^ph0 z?+tgjVa&$g@@IfOk?{?fevlXw_Mpn=*;Rz)=jxKclV#uY;L(&iGyJ+?Ajun}xm*#&L zdkpU|ST|6mBUklTB=Vm#&A*`dmtZ+tC@u34 z5c{4sh5mbd$QYTJX!j^3V4n88eo)tPTy+Al!)eaLY9BG1m*wt}bU_Sgo=-H_%}1t8 z)v4tp5~VrY80$8nS72v`kutU^US6=hD)RQXA?2PTJ$gs3zP-ou+08WI4J=W7tA`y6 z4jd;=k7BfHSBJrBj){QHL);1%&ouRC&JcKRQ#5CRQL1dj^CR=cE?T<>Fc7{nSPYkTz7oHk8Cqukd$LB=*O%f16bOh7;{VI7FJWb7`}04{L9*uS zzl0Z77q~P9Ja~P0TC+wYgEab?NsFbrQ0_X23|(X7J~9+MP)^ERtubordFnc9OSj`j zLcvEN;EMYb?rp}k?M5CI6~2l8$9CtD?#<`1PvXw^+p8uB=YBr$k*jh)9LS8fWMtF{ zr5^syOq^W|l$LvSk`o16!10KUA}%wL5XDD)|Bb3B3vZRjUqKvPiDjF2W62emXyPt2 zfT!^8Bq6m`vggl7nm+zJw8d*O2cE9 z$;5-l`}-2vCt^`eQZ3s~b#}3%gtFbI(x{+ zv&@ZBK;f5WN);HNu%L4zORd0?jf~v$dMd>qndwffF1=Wa86@De_GDo(aw3&~{}8-u zkyPyBtSE10q%*6at;KGKq-UPGYr5uTluWjos&-R)awA?V^q-$PQ&3Jtb2IL z|NJf2f7~5j&v9ejo0g|;C}B;d)s{p*o|&|s{PW&@Omes}OOA%H6^#BO8WYovc_WieR@{%u z624vF9e#t)YEl|!k=jFeHppr=R{t=asH7cZiM>jnfA$KNd%p);ua)cI2@& zP`$_1xXe>|M3$>$-weNIR~;X>RymNR?7%%F?N{uIV=S&Jne51gr>G>7x7%j00c2Ga z^{F$ew6@q3L;0rl#CuHRnJ1dVnJ4avYiICZn(d`Kg6RuSLe5jH;qXW#SKshBS6kt* z$hgo7MQGU^tBwAI!Ax@jz&NI>F9n@nETgM$M2)L&jDss)?-oz@kVpUQ^I{DB{3od3 z`HOU6V>w>(^j&2Z^S*nX2B13;AqwrMvUHQ5i|#W9TPt+15}J+tnM}86vS2G1 zgRhiOM3r8t68S*W_jv<`8oa&1W>~FviZfapMrV~;W43x$dN#X^jJ z4SMLXqYkYizC+Wq?nO~TebcZ;A03bP#l*O>GGt08DcZyJRjtWtj!nCpk=%EtD$cWY z3)3Q1%K#qpFVs#Eydwo#ijHUuS>A%6nB?&4RS{f>_9IYxc#Zfj zf^D5MYr31(c*hlN8D2KwA4_x9)#$TL7d6UZnG!&yKRcJB3WceVY!e#l$bVt0fo z8Xq1n_ja^|YaU>0bsT?d>ExQ;m!=iaqAa@l1-=vww^|{__s8W{vJ=!nRUUEuPbr*3 zcgTA92^bsm0%`WG;_X*ZyANYtx0N;`0w_`#9*&EpHRj*rC%lv>%W^X|+%#fGN=+ZG z4!Egs6F`G@nBQu|=jFF3U&I)+N2~H7DSE>OIg1HKABP03`G(Vvs{h5-H%4i;WZPzC zrES~z(zb2ewr$&XrER;?cBO6Gey8v2`(F3y9%GNOcl_M3=882VV#W-W!niv^Q;h*_ zH^-wP+6!(__2`rDY*Ig+vDNa$2+;pg-gW+A$Tq?bQ`4 zrG~xGrICcF46^=Kqz!nP+f#m9qpwt(Qfz_6Z#N}U>bPVjHt2I5@>!40aMBB#4kdZX zV^#Jnkpat5@StmzNO}YwJzyj={z}togTxxp5G|TV^`eT0NGD0kFC+n$lmS-gh$rx6 z9>AaIvwN1H$^yGC_v++jT04OL9iXV)(ugAfR=fx_PSfqOJO+8jZ_3n1oQYTFAiki& z4;khl@E7WH>jXA31?fdf$7EVZXYLt=pY1(rXztj%JmM`~4-RPuQ(~VviEK!L&53Ni z2R7Y1mlT%c9!}-d4a@4oTZCiNCx5e9HdQp)%=@bl&i+?5mbi(vy}W^$$-i%Lr~RWA zqKx$Zl+a~_p#@l0scKb&K+TJE7Fj~0v^G$pQ0eF+fgxcqIxr#*6}#4MKKpE6^yIKQ zt~q(WIeiCyG3oT!kwqlKto=NudG7Yw?#^@Dmbn|*?)?DU0sAO3guX&#FEB)Tu4sdQ z7LyMF@5e~`Y=gp$piiV)G88NxjGrp{ogAsTfh;QdWU(qOb%}{eW@?g@@Rk}0h=p2d zm>FZjLG{2%THrt(aS$BhmK{H(3|MI3BPNBfV2=s|!Ba(GtS+YdMy-s$ALjsOz*cx4yZWCS|R;xp;Dhfm5vty@l9ZPnj;t;eo0!ni_xt)>#6nuDOsf zi|+Zb?yn1b2(}P6t17aEJ9pKbj_UVMk~~JSJ{!wjVc5)VGBtag6t3~^lf`Oba9Eio4=)zvH7vy)FD_&mT6lZV(1cyB2@Q0a zSl!FzP+PZ?o1;Bl3r%We5*a})YxgeTF2{9#IFPQ*btyDcwC{d*pC9G4*oydF2pmT_ zTLL2ne8TvMv)VMAdSAAB?HMGYl$WPgH!Q?Cu};4ZE_=a#lDQw6>1<1HvKout%2+{= zO|?g6+R4XYLxKzXnEPeGh9f6#65@r0j;r;QYn4#M-N!`T&>=*CDd|)^@G=a7XO@U8 z%rH1_u&8SI)_bbS%V6DJ_%reFS4wNr@K)1gb|@)FgU*?yxky%EOxh$_068;vgcGow zg*(=a`AVb}hNk3zEc2WQ?93hxXlm&3O;_l^$)DK(Sx~Dq;Oog!88(|U9(fcS+Fb1I zs$I~ovR%^bg&Rg)M0b?|D|eBBbMu88uq_mKseyHOvA&ApP?j$0#jSyNV0`lz1bmAZ zj64YMie08`^cU>`^cTeewU^A&_l@wTn;7Qr>fdfyJtOd_`kY7{BTGrL90DmMc72<~ zOeNM8&Ok~x{NF^USl@#O=e;L#3{jwV8&WZ$>2_N+S~V>UA!S-vcF!K4Q)v}Z2CsD{ z414b^u}3aR)6BzEOw}3ighghJp$D@qACxI+bnp%UIbrN1Zt7zIN0G773iz6MeFezo zUBB=AO3QlU~y~HKrQc=Snuj}4vQqR2YHrT zhs}D!1SKIE&3b2>746SrWqqs&xB>U+y3t-U7-k0yno_tg0I=*%d^Ew=JS#tgbw7F3=*ic(C$S_m0s zkiAAQ{QgMf9^#lL>S$R}DWN&*(pB<}LTRPesvW4f7*7RMnU3IwDUik`KabGYzIknR zNddVn=Tf$6822M_Dd3!9j@w6jrq+GuMfBoVXwxTo#VNeKd<3{AG{i{IoW4#G(4)+s zmL^?(8E$*L3LJ}D{s}pVugEoG^tu9DlOL4@KUhJlXn)nn#GbE1na6J_cWzl58UNOu zesXXIdSnleIU!8qeOrmMg^~9heYB4|7MT;`SUW{ZD9o~F>P{)tvZt<=@N%dDGjIOc z>VPsx%QjxoTc@DU_AS%|u~xMw;Z}DDo{C;SFXxkDwZF``p}R_7H4Iwyifi?Lz>F7v zISRZq6jTiz4u)38iR;>Wg7ur~i42537h!ZA;ZezsbmJ`i4H~*1EI&^*d zsVd4AEF*&FP^Hf$YU?xsxvk(LpED`REFJyM+~Ze11P)Z^FOz=B$*_rZC1#C*l#K5n4O)ixn#32V zwlz{(?!DbIb1N9BJ;XG3VI;M}vxdQI;xVLK9`&&$W0J<`KZem_?WRs@xw3dhtW$HB z`y?zgTpSXcOn-QX@HtAyKB}aAay^=J{DP2xS-8IfJ@Nb`Huv%KVdHu~GMp2>r8EBV zkUV>Sh>d^1r@UTkjr(2r-Kx1?HlRd*K>vGvOOa*+a{>nf>i=t^NAw@{?Vld!e^+qn z;i#d0fw{WrGjaoy@Jm`?ts4`@%b=DE#sF`Y2)pLT14vu;>bQQ#Z|1nTq9LtYT3rrU z`Y2Z~nKfH_FNtP>V#hpIKl0uB)xB+UH4Y%9u58C`XL!wYKWATiA8$0KwQl+DbNajO z>tTV_mFxwE0vxgK>aZ2U1b_0<_QZw~X~U9MRj$nb+AWaRo4L|3}5m3(M}ih{5BzptsPF6xQ2~%#Q;ts-9|s#AY>BX|cZiau%b_@Dy!oY-CRA zUZkuX` zn2M&rv9Y14=hHr|TD39uvqa0;f+*2`1Fxdr6 zT#K{=i^e!j#)L1u%m|W`9;TxjP5=2LPVuGo^r%@RL%iaVk0zlAPJ7uet6HPrL^em^qN6F;8L>w2@n_12 z3|&p52dK5~VvPkF4iC987aCs%8#E9O5!7vk+4y>LC(!lXfSWZ>TgF2E)}sEOqIu`@ zN(cGGEHHtK`2tz^99 z`wL7eU9z?U3-(hLqVz^7*|n}2q{I21MtWI@4NhJlRlS)hD)FIbmk*%YvX}f={M`{w zFo(daC1>t%M9x1B8dYDyBXYM)z*W;-_BT$r_(A!5=$yV0YAPR+BUWt%d&``@LCY@M zA{j?-8JxITEAzKn+H<#|2+KE-vD>@PoL3^+BTWcWf>r)(b@#)Av1wIRfJN8Zjxx>* zF&0Pl5tk2vk*xj${E%gjE8Ad)TxC6S|Z~V>&C8 zHg`wsZVrFkrH6qVj*JS`rSoA$6MghqjbjjH@8ZLvHQ&1%<*Hj`3H94_fad6F&zMw( zQK##gX{q+EIXMtZN%RB_fg*Y@x-X49##k=2SpGW1MaC%#d1^?*ds^ayX-H%T+ z&;x{UF1QF>gxP!?-ZAwcv)@kbn{3^L>ukJiTkwRc|I;R+QoeR{0l&by5Ojz6aC0T@ zeJ(;GTA|SEm?0LjJxwM>am6>mYv3c7TUT@d4GVdX%MW&~-@hTkXY6Y^A3k-C842qG zE@e(zRLad9y1EvW7aoy~J_L+pc2IUR0+aC(=#7Eg2j1h!XWP&jAF+V|fwUh{^OOWW zi9N_O*Qzhff?8PXm$LE2!jHA{4=AfbV3Yo^?BsZGka?O5KV+=>mgbbXFQkVngg!TI z(Lx-Ym41OAUAn-0f)KyCVR^-Oph>QY+hE}lmj;KM_~IsJqw>WQA-jb}vSMX;3V zjV}2Lka3O)rZe(~nox6w9->iZ@7ar&H#c;i5Kos5uFW?yKyGb$Om)z5OD~8XFxlXx z7#0&B>Z}3bhC$}58&~TZngeyd*^bjauqV&^Vp!3pv(Fny*I`uLn|8TciQmY+^9SEJ z7+l~-(~@maCKrQdm(#JiE5kbuLQcoPzGy6u$ z&X-lUy5j4J;Cjor=c?U1aXn=d>RQ9kpAxh9W{9}@%WQX}6&@|E~pC+BJ_CLt*z5`{&BvHIOSCi)X3&XUub%)p@$``KS# z%c{Cs&8m1Q(x$t*sTo{TCa?)lrLuVuPj?Z|@_Iq{sR8PwfFG z``K@V?K}zm$`9x_R5N{`%1ZTaCmIAdCSD)zOI0Wr zvGEhcYj*q$^s69<+inzAPtooKqVgW~0zItP(2lK#Q(Szls6E|P)GttC#Ark5izmo- zy-{C6J>)wVLU?m``at6PP89gDdjrC4gloKr9R+T(gDl6a;aMzj$u*s_au2`F4*TVLA zxsptgRB3o-H}z{(%E^mxo0S#p%d-pXsJwY>yK`%`vL;wXFjTgK#y_Qd59VY!6WFKT zaS2rE4V4(SCud}Z_;Rrz_aYBovVdxFvPT92>JYDws~F`nFm@ zO@p3GX)(Z=pSWcH5+pO8P1Z&I3C7T0Q)!nMD;PBRNSX3PK!tE8>(Hml8)}J5tKG&Q z-0REJ$ee!RoMdV8`$|+`mq#9`KJ${emsB^B1=BA2wxLN|=9VI92`MszBF|`RMRzfb zNjsUNgW3EX>Q)wV#-Je44!Y%ygvrA~)`5+we4Prm1c~BFVcs$BC%DYl>I~kH5>Bq= zG3om?LBHM%miG>s!nk6@>gVdfsth{gh3O?}W}UJpiWYeCNkWdmj@_!}r2dWaX#%8j zp;#*O{ztNQsBy{sex(kiME5#EJCf;Z=1>3-bJw7&>ABSlqL&)so@e~RjS*h_Xc4k# z&+=BS!??XqA#B7V!?aV#J-4brtHQ}zLyS9>^6HM8S1@xd31f)!?anEtR2Ks>>h24g z6OAC`dvPQpl+%hd%3W}4w{3%!k_dxD4@cIdlrs++656DLQG92po4Vac;{FPKpteC& z5;iO!VIB^kC=!t})ze}N2j{F#!`0yQ7sExDjy4;EQx8xW6I^g{&{&g^<`G9Qza3j6 z$ASTIyE3DKv1H5zzrifZZ^^_cR8tSxXa*qSyaa;rNXv`Fc#~RsFK_Zo94gm^JxJX= zW)7K3HH#O@)+v()5jtXQ#t`d%?X+YXD(>o5?J5^*;N9Ac1_73?s(sY1id{2K-&nm9 zcF1dj0!rNL0W7qZSiLVie=vVE2hiIKcpbX>oo?SsK(mJgi~z8g8*@BZFZ7tXEUe$L(? zLY$0(H7Je|@t1;IAx<1_WF_7at&(>cZ)ChB|l~ z)YZtc+wFJEwf2s;cW~^^s>~(_>V0ee8LimvsyTLIQ#offW@Ga$ZMs)X%4hJ;Ir3Bk z+BGD%eWO`i(K!G?ubb*t56e^CE#7)|k~C?by%$RM%~0~JF*A3gTh9a;sMmbK9-gcz z<4xf|?qbq^y4R~T&{^N!))dPJC~{<++Z~?U1bUim8-yNv2Dg{kK01ifXB@uKasA?aL0D4p4JOTO7mH{(o{w}nqMyDP#v{Sm1#Zl zEqvG|n7e8&Cd^fJ0IGbi3xxee90@WGuF|x-$rd+0{%IM41Imy4X}^Hz7&>-0G_1eJ z(g=bUPDR%9xypC_d=lM>DP%et4P14)ze7rGSR5Uk*zNlI8R1K4Oh3BpeQ0Gm^d5e2 zlC=$f1Q|)m@K>u#IEHJ7efTK#>lyk_U``H{cE!Ea0OUhQ^{=3VX15Nu!Wvc+$o+}1|qQn%Mguqh#xO5bv{ z8lY63&N)CzW>^^(a4d|j3BUxZUSXG-N$)zO$`Y)iEDI-E&hv%CVN$$ zzx<(C@300bt`^A8!oCvGkg*3z2&1oNQPxW61YN+k6fB(G_pD*1 zA>b=*A<{XR7_%}e+bOJK0YsH_KNN+Q_GW#70`is%0v}*ZO^E$Co^CEP*aLN!7uHSR zyokDWgrYE}i4(iMjpCWwg@CnY!+gIIFv85cgS~RaE`j#xYL?n7@!Xn2ARZmV>V&U; z5m5)(B{xgX)*|IHP1QqanN{cOz;@Gnm_q#gvjK7mq_bG3BGk~;_3`V(a;0R~+XPYs z4kx+6X)u=Yh1Vs1{iVz({yWf_REEoUd1b8McLGtr+gM!DgRj6~?V%+ZvNWw8iVWV`Tk%blJ&~_Su z1QqZR`F-?ZZmUXzjwa;$TU|iEhq(KUHKNVZBD0%Q}(`p|sa8uTh4sFrI z$ZPPg8@wyKR{5-?6rGp{Ci*}R8i7Tu3OLoGBKY*6*Xn#8_8i~=#6OYxu%JP)`T-EB zO7YGh&?0sIfxcq)!4MCj!)H4bXw9JzLgsw2+*T-dmE4vfQC5N^#O$6RQkllxO%@<7 zjOZ75S|heUdq;`eU=|!WggOIPJCMZgmAW(ZW@uh0%w`rRPe)cFmS;`T1KLwjZzVK! zrBc*-TjfOmTl-X1?#sn$XE6T{ibgA zhh4b)sp_J=r#f;zkLR7sl+RhU2giIR)ZNd^4DwsBb=(+cLi2)96}JnKlLZU)N#D1Q zRCy|SBQzsMmPs2+(F?9eb+u_3v^?Mnn5_ZJw%>4LlBfrj+MX+R1Q+C3Hgrj zm&sVg-T$208{V4tBcO0YKl3v%zL2ux$pQZ>ZMkWP@U72Mb103~ei9B2zb) zrPxbpM(z$dE{?CSiFYvdLT`Bs!L_46cEC__tl&$@J?-@cNgb-hfq2e67SlO5xOq_H zkjNFblsFIM)<~`&|1^}(pbuM;7_>2+@sMcs3NAhQq(b{LuP`rx)yFqnqlkSIPU;&; zCZ`oU-Lk0t%nW{k^r~VU4`I$RG0yv|#tZ2eql;({Y$V5p7k z3qeE7?u#!=VNf7a4ZVYVM~5&kq1y(n)G?S)fBds92Z3(f^%*Pu}r3J^lm&G6M$!BK!{mE@NV2 z=jib_vDCZLx+uceH2fF@A`;iGhGt?<-On}~@ubXPpkWD#&tDkQ`S3&&VTcJdYP>AF za;JWClgwmE(+|%_ev+kLiTNPXrWf6v-J9KZz3V*Pug{O$Ks|wl1{iKC-i3rg9Rv@|G=Q1=}} zY-3F{T95cs{w0f+kU138fhYC>_hlE-v~3zY+C0eojmFGWb$tjqQfiolDkb8)zOKas zky?ert5AjJcJ6To*iOT?RBg8u#Cp4COtQP`f%c%LQ)VpD#wA&F_W=j^#=b;^Qx;pN zKB>i@wosv`_@d(b@;Y2u>d>l`LR1&l?9c28fVU zteDDQ*;%;+P3`6PyL3<-Z=DfnF<>GpxvyN^J1#yalG?P~t*a)%^ex?I zV}`!~F!vUH=C4H+Wo0WwNe%n$WPawa2fYI)Ror4DVK3pd&peZrF9IX1uF4-}`{W;0 zMj(YSRry9}`&7Pj^-;LEa{6UuXD=uvk(qF@xD7{ESJQPX?##@wa*7Z7s22(zr4ao_jifq!wH3VU@Dw6WL82o1GoAVA5uY2n;{? zt-_q*0ZNfa$&{3-ZmaE1{ z*@Q1l`|-Pqvtn_9&DPlqR-sniv891H5bE~0yXmnq5=aM$Wn^6X>ngjWlT)(XVVv^IGo-e86Hb1$2p9ddLh3_P6% z!YAH}Rw;(*;X>if@d5?UBb2Oi02u(^#H1q1#Mn9ZL{4BSQiL;t52%E+@RGf8sWLk6 z+H;cq@c3+U4`{U$QoLcG4T2cFmwhAey>-R*q~!X%S&R~g6I1Z|KoqTVzB#tdfG4O= zzwC&vaEesIUT}vuoYB#w05`UVY@b+4P8!K$a>`N!#iE$_BeKs@h6SEy{5vIM;nav( zgL-LaW~^l{j2B!@%QuKLM#O%^Pw1d^u{NCEc>!-`!Ga)Ra5b}ok2PsCS2HirZjK~>g_P|y9QZ|k!g zS6JHj`$rVeR-j+UT(ktHk}Xbu6L#2bt_a0|7qxwwd(TYF^S&(`ufNxZ_7-*MyYa?# zXy6zWH$g5w%0L*%DIyzhH2BcXU$4kGNH~1AePs5tVIlXG7%_Wk03RhB@8NK%!$yqQ zEoS|5I(A$Xb=+P6I6%k0BdWU~w_NyTlHpTP^i0GtC8iwYa2|Tp0%I=P3{!s~9fYY~ zvV&?MbxmBXWoWQf=g6W&GpWt+9*rSgDvr%*%d{qG)dubbJ}1H!_7x@!{3GrO2Gm2a z;N`B4(C(AGd>~4^Jmzc?rIJkMPfPCV6y%zw;Byci3XC&|r)gqjH6*0eXE3hLpn$KM z+#8%4mKITml+1?=Fw_FF21IE^Hktm+tBBL;eVS0T7%N1k5^^n1Xr*_!+D_NOM21Eg z!S+@M{y^eiaq-R$Y;psqgLS44l>-hkqw~a;-ux_vhB@ob=|=U*!HX@w!gIyT#uAhh z(*3(h|IEW#Bs57B{BrAAdyB@usnkZ$Z6#>AC$g=R;e8ugRX>hbiqUm z2}Xs!-OHhj>iCm`qX&7V?8#orqGM`qx+box@tk3Kpf8?Ncae4ra}5qJ(gbN>B{|H2 zsh3!1uFNEq`CC>n$&JtjONygb8y_$4ayfU`LD1rLF;`OEa$wWyRFypqDFmSz!pYVS z(&8wcmYYC?1VGt=5MEi1@|=^<=J`TDQ;5V(is3xm)FvXAY!x*P9SYrD=tdS{M`Y%* zd7+K(4Os-0^0tmrfpZ}ja#J(RBK>xq|uK>R&1DBtKj3i{vi z-{paMyT^>a`8#PZDS>J?R&Vlsb~jp()t|#n2sUT1mLvtmim;YKigK)tFq!q@LAbQA z3PN%>1g)5JNhPkM8&x}QFWrHBJ8_1rLQ0asxq*DUGf2sXVNjpnL?|zn!J(BRVLP8} zpvY}XecQM2kjpc!itqO0%zwf)D1IzD<(ukjlDwC+UQyOfSxq!upHZ$=Q|rK}+aFVK zPLwt>CM;Uzlb{DwWVjjv?o$Eu)M#f0uB+oSY-we4F}2IzRKl;9Eo-sVj3ro?lO$oS zl?gsG#j?`oz#T^+8Po7OeG7l?5jqQubBJcv6uH49fwj88_MdP zI|w{;djt`o(+=Qj2DuX6H8(oJ2U?L0FyoE(lfO?Z9j5qu^c8w@j9@u{IyqVlA-4h! zPXw{uP_pp29GLQu*5(AD`rc8Km1yf)DT5;KV9;#b#C^@G3pss&Q+i6G?B8=rchMTH zK4iC@?0bL?m!it5%E$HVN5%+)&R|lRqda7sz@$Pr4^ug@>qZr~r`*CNFGd(23{l_e z@q&~O(46;FFZyg!BW+r+I8cWG`U1qJod`u8hD#Fr<72fs!<*tv)Dp`>Ub1&To+rM7 zR1%}RX`P)eiBv`d`8-b`)M4#OT=dOIlcW#2}6>-1N-ifY$&6(&l7mBjAuhRDrg=iE<{nD zy+Q6v%*OR=CF)wg)QyysRmB{aM_sF3Hm{pZ@kTs|6oZ)9qgIuPBIY6#*7!n) z!f%HKIJu^|Ai=Z~!_Nk715ZlByCNXwyEw)cQLgsXZhYFX#a@QIVV-a9A9S<(eJIU&9{7B? ze&W+ zWyw#CHo!!*O#9xKss-zU%8Nef-StF$+;~UF!DSS^q<4^SgGj9Rju&5lNftTLg-kTv zy#_4l^@H^iqccpC#SurvNQ?DWa_%{p?4qtV1Xe=s3QnUU3TwQ4$j04sw=Xq9S&J1u zUJx+57Q847-BQSSgb(vgcKLu%TUheM4v3{$_em)744>Vw8*oXClS8aQcI-$@Ftjoi z`FspPq(rqEUqw5$Ok{2v6sxuYx&QU3{kYMA!Tj;!80!!k6V#v;sc1zru2%?EvwA}& zf=-L}U2G9M_RTct+^y?#vWBGXjftJqvc-~)S)8I)(ro7V_a*-QyV zxdtf@_<)?j6IAET@82m9D?crk2QUy&3J4Go{eMs(C36>NV>>rnF#~54Hvj*>8lQvB{CXJzeEoAO({&#D*_95fbBVLaN2c)__kf z;?!+jqvjaW+-&XJo;XvJ0W-E}(=)*Yd-aD&ffeOkdb`l_oUfm-K`UYrB$_Nu-vI zhSPI)O@~t-P0FtbAA@Gnp{&ydQn2t#Q{84Zwh(g5^T6e@OQn}kGpT-+#h(Ue+;$bI z+KM`A_^<=+ ztxhd#iFmCCV?Ob_pM4*l$QBriy&&FNC;Wxg^vFS58t6h*w8r@zGytT#Cro(#il0y) zsq{Y{CxCb!5^#)l`Vo-y+0hHSjgzdKN+3K$Aqstdvx6}u@`Rmy9&p9Zatcoqj1j35 ztchfoCgHUip`e?Wnh8L9dRHLj5H*R#%!K3QU$e<&yY7H8cRg@K|E4>TUrN_2XrSGX zyv7}3uTlK6GJZyO%+WNmtTEUH?NIWiGOdpAdL zLE-_G)Fx6?RFCg=O#bgM-fIZAX!!vIH2W8n$^RF|Chi6{_SPm&bOOf47XRS!0`~US z7DfjD{{o8;g=?z?c@&;nnKi7+Y1_&L8J*{!Yc?1Ny)wa+6d`5!R6)CG-6b` ztIpp^dLM>;0e=2|$?m#X>5~DF?aw`4GaS!bU3$HKA0Rg<>v(7G>LbMPsOR`z_I0*0 zrLnL9YfiZ5A0hnB>s%=Ix}?OU_T;nmo^G7DRQCdK#HN}8NaH8+ST-{z^J8+(WObND z$1`L-Z$zaQMU3D2%=7KCdqK=G&J;FnbhGg$n!hif-n)4}$!!|6`RQv+sHFE0MM@Qs z7p{C^$nK=y5~4@G2-rCBuF;7lnM@aktj0G4ODytDrjrrXGv%H{kIovo@liAix&$08 z1s^rnNmXkwpCeLi3j(l#bG~UudhhM9C$_8&CnN#dvTwfZ3WRF9S72&ML$0=|Qb~ma z>lFjMDiB=Y)gwuAbSIlRn1gI!s2fGqsAI6q7FkX}w!5r6*GSN$ucUJ-o@OqWFXyON zu6;Ur%ThHlR|P76Z~=@J_VJ{EG|v%XE$?^WltRpkKG5vKP0Wo}uWg0S>>3f)*?lao zKkKA^*R{4bJW}yc=0rKnq^g=e>1zfv^{2n{wwU|KmSDx{Qp8P=wfPW z;wWZfYvO3&Z0GpTrJw&bG_AE+4iTh8E%yfvNflJ-E1zzc^iKVQvRF~Q>kHyG66*tfs z2n}t3gRR`{PRzGvnylEZFE+>`JcRZjBoW%;?Dbq_fOjXLeO1v>d4}*iY?HmoB-E|R zrYG6*EDa>I;;3Oid(IT}-I=0QaZFv_+eD_#xxqt4?#tL`s){ZeBH!Y=hT8UCuxkzB zMyQGBzUimc&;ohn=d9uB$3UAvtsJ0hTRp6?*%}VVyU&7w{vJl=xoU$6xf}7y=3xO% zV|~rc0!l*vuENjuOtWo*0_(ndG?BXC+LeCgy0Xd-4#1pL92kqmn17!?#}ysS{J~^= zd_|XSaN1i5mB+_MA5#^8p3ZWzkTg5lcInVatodWW+vDzFK|rq~)ofjv-L~}_L97;) z;S^juQU2|kRlcCtEb@zWP0ZZC0pyUGEr+8c0|j&F;0Wj#KQ{}QDwMNWS>_G$15F(6S9OoTfYWt-ZI|CTFJ zs0d5QJm$yzPvaiJZ{SaCyl9!Da2VyWK{S>Xvidce0TF;fh9MX=+d>T`$ZT@5L-i4+ z*JggZTlt-EGsWMrwBs~57W@~6(t-YE5%K?H$-~*iNzVRXk;L_Ax6BI$21X1f>k8)T z3I-<%c2)2valU}NbUg8op(q$@j5+y~Ik_lU&CbVs!GUt?WVW&>7-3zNpMa%;p@eUO zuBxF15pj^SK(R!suce5gt$?g$J#QqEDAQkb z#`>oEKk6~m14xj6V*ky;gkFy%6Zo%m$oVTBF#qEhL~LDb82>j3uI=Xa5qxzu%_(uM zf3@v?Fyz#jNSZk05lkF4kkZDGxF!O8J^La_ivc0DZ+aZ5?1gg|jeljqRxC!iKHtXS zR_Ag1an1Ajb88+@i@c1`a7U<^dzHRKUnm%ED1AvJXiTDsLVYZ%&AUE!CKYuo$qQhOLdpZ5(!(p;jj-C9{pVjUNLI2Gf-Z&+pgT zq1l(9SCwVUjsr$pcc&r|eKF54B?fMHLZHPuAxw?gx;GbV)e( z8Ps+^b8(|h=&oHj8HU0MsYZ#DUGO+laV}l*odF6GvM~h6=B=lZofycfc*Q(QyRbJ4 zSw5)st9LLaxnR!qunuA1-YM z>B#d~)ivvQiIJCM2G2gc7=KJJ-7}k8w2GnUPOFMGQRi?7@{oP&Ne+}mMX@|k_;V;d z`E|Y5ATFSh)Z%%SgwsS=QIAO~==PS{e6b^5Sd{#=zakG%jem$s|41D>TQ*S_pqyUd z9EFOcQV~B$xjQ-|RXqL+PBqqf~UGl>Q;3WP71HR!@ZNxO`y%!CBOyG!~xGZ zfR8Lf#2t>{794H|;`-*w5sfy)COu2Xa8obzf%v}pn}Q|Xq>V@UuM$f87kH@tD{w^q zZ-Yd__OH=Y=pPvRCkIr}kpfmg;89~!TV`QnfwKNTMc2z0LAJfADp{qiLRw_ zJ?<;~S%pV*l8-+d7irr1TQ|DR{wlkj$?Ry_+qZ5t5^Q@=q;pB-<>gnBPLuMb_9S5CgWxc(sBd_)$8d`X8K-& zoLLRTWJ`3(rJZ0#hLRZPE~ZfPm)J(1zd(H#SD$`NZHdVt8Csu9%kJ zh@=^>D4hcl7$MA&6D>Aw7K5jrAKQ0=gcoy(C~Im^CIG^pU@Qh21L0YOS#x7Tz{E^G_afhi`~SbK^5DMZY0~B8B1%GRHn=D@uN-lCz7i0{vsKc zi_@$xN-l1SdRMcR%APZSTE#c%szvzez^R+`?zqi!ym7C4J)@`R2jm_o3C6gjl3bTh zP`VQUh1vljilvbfX%euad=oFQX2*)DNEeIk`^Rh+&U+j3iJKRPQz*2+#l35jDJQPW z?}S50$sen*h`B%+Ggi#!sqLI$(v5(lFuX&PZmSwjA zozP+_O*{-6!!Ug94xN%iS06mlACwXu6h&4oY-DzF_901Yi@I**?Q2-V0hbq@{U)}+ zO{53RlTqI}3pWs2P|~5zmC7uDz4}m-s%RlJCm5%NBaFlrwwv-p*v)kMt&0a7E)uN} zzZQd)4DS;6x{9>ViqqId&k!=@Vh7nj(EZ5qT4=3>$e5VgVt6pn$*30SQaqsV(Sx8x za>)rwk>zcvBp*bn!!NibdkYu92Y2WsbOg|XhLQG31+0O)XiyG0x?u(JW&8#3^5a>R z$Kp93VKw?H?jx5NPT&?B#q5rE(Y8cO<7}gLcahGq515R1ZsSEtnYm1LNqlMw5wuv^ z#2yxa)=;lo-6ExcUt4)F3aQUm-5KI^~VBDV#zL7w#T-IPShP=bGg=HJO^I|x^ky;0s9xgqgBetc0lup(ZNc3w<+G1-M? ze15%t0Mqy9#;|I~(!|=t;)1=aT4lr-B^<*{^CfVM&?kHNQXfrg$#g@V> zvHPUL@aS21=#d3X<_aSy4L&Z&4=N>44txiy-d2=-_V8(Ci{*MFj{#EK%{b*O9R-rO zOH9zlu}!Gnq}mt6Cqsf%R$zjL;gpS>bpa1i&j`Yd0F2O>yWP7j@Tmq*Kb!&)d^5O0 zsls?2Vpmm%Uru*Fo@9T4hE>VnDeEUV=fo>T)dlDlpQ^bc>)g>2M(KtzXMc=8?))t^ z#iMHg(AVd3{fHsKIJ_=_QBc#)4rrl4fx zgGFNOdGM5Vx^&a4h7Uu*4*ZQ+%GN3>aLlYvo7+|UbFSCS1@~vWABcUh4*I>iTf&eP zxE)THOM=wC+ftWX-L8I8{*3^Y1uvtq)w@mThR25zJpfEGg09e4?8L4M8Bq~xk2Y>i zTMC;mh{K6A3}(Gg-+SYJh;9pC*zkmHeqdD}VT_iIQMFWg{aBGf*^)01ulLAzQ01gG zI+coBhRg{UX?YylV$G4_p?MbO(D{#f-9F%fxBlP+%Ah zAU<0ID>Me(ggJR`zqVd^7|Gger5+^fJ880x8Fl{RL3o)SWlFq~08S`zv8&5iy&Cje zm9#YhAFBrTl2fdA&KHxulUE~-D36;sk1yT-Q+B>~6b;E9UZg=+(_aLTkT=38IWx6= zN@_f+b{+4*|Frj`Jqq#;86|K)i(RP$2^Bc(LHo@4Nuf_dP)B0DKG0DyWrWJWwa{|D z^XYFvy$dg5g#rfx3Pb#tllFfP>c3V%RS!R86%-z(4C8qVb1Cy6YefV?8E8@(uX_F{ zb4#g3Lxvqzi_*(v3#3K@6?tXl4YdsibTt%Z&H6L&8=)70k5BC@FD4IMGRS$pBb(=3 zr|s_J&4)NY-&b6IaNj=&LlIcadkRk;lko4gsnxO-?%F|?d3^bM60lgZW$eX-!N>yK zL&K%3$CytXI#Fh~+6Xh1?gGT{4c-jlX@Tw>;cAw*RJzc6F04KIyD(eEd9BJ_)qA+G zh+UQWnAaBS5zj&F6QVsY{x*JPd&H0d3=4Il`$$E>Ad9-zC`>KeYML&yUoM|XZwlR; z^9%czLLu21c-NG%F_`VN2Ew!fIi_m>XQ>+ztx!^m?hFMXL7(0X)+h8*MbRt&+y-;< zjXcb`Y6Bj#hUUg{6tc;#;KR?=Q<*}j`NjfbChh&t5 zfci#L{syh%;Z#Xu@p|>`$`v54AnOAi2K6C<_@`%TXZee|;8Z+E?h^D6PJR_Nd<|55 zQoJk_WhVO{#nWk5Q3{iJLW5q3{&91=iA5`dN6GjYBP3SU@yA6U+}x??W6RX4G8W}z zpIAw&yL%^aqo`ochB}Yu1IIV!O5X_sHR+&o38Ub};#3&q(?Fx*c+`PXMU7Uh)RY8- zd(osFa2-^pFk?VjU&m!$Sle^USC_t0-Gn( zyp&tb9#+R;ZjYcl1LXtWQRR}=JR;>K@o!SF)j zV6;I&bCW#QD)OfBZiZl3^U_$xZ~gYK3*N@Ijgcl%jj;TqZ%AH~ycxk4oZ~X+;2Y2$ z5>zJ%7W~NrfaK%+*-V(_X-KK;>YRY(!;r?3cTP?YRS(pzRW@-Bj5NG-?R>||9X9HA zs6F0x$PEvzxOtT_-gQe7-;rZNu{rts2Y*9&<7m`{(?w*u_3Ql4o`wtsrTIn;VwqMg zz4}&V2`p6G_IRnGeSJcuaUr9oy~3x{!a|iT{Kt`DNxB^U`?n^h;w^6)M7I=Lnp{j4 zqNq@VO)9L9=jiT@LTeWGVc)nnD(=M+$<|B5xTc1>Q!(VqliqNOPqw|VNf2qK2#fSM z5+mM3{nxgHI4KKJ&r&<`YCQiq)8A(+fTHwoqyzOJSI8c9;$$g^!J}VfW{6$EnNh~1 zM#y3ivutpn=kh{TkAoP5H7dbvn9^rBh+1_i_rvv51{26x()%6XCQ&qn10Lh&r+xqg zcaB4}DpD*_?D~9MO;#gF!frz@SN0!Gw=DTo+fWxk+pD`!I%xTB2BrJ$8P^M2B60h) z=Ea&LRveCxFFN&S?>`rp1!1mU{leYzb6bf_-)@PwkQShj5ZclU5G~Qxg*qM=9#l3Z4kiK)g4IQwbvS?4Uqjf&JBIw*6C;I}->mz|{ zVjkhQPWWJTdE6v0d#>PDw3R_RdKR<= zZHo-$G?^*Es1!pe7qIhRxgieOt3=wu5B+FprEHlUt)P4FF5ood;&g7xfcpsgeO;YxS740i4aRoF5LyJ0B9cEt zaE(;Zw77X%KcwV4`n(&}b10-35)f0zh(jF(jth9|S2v2AG%#_5n;J9WGg<_w(&Y`o zzAw-0uU+55BHdxm)nf`la_uR{PB}6sjVYGkPXT`?Oc$KoJhIm>H#7F%WNK`%2Q_cK zYk+Yw%=i+`VfZMeaFUM26(yeKrYl{C030}e@5=j$Vo-EYj33AcximMc6iwWb{ zBHs9V`P=To4R<02@?Yd=_{YBpEdE|cMQoiNJ>=~y{`dabkpCf@DA{Qx7SYw!kNZ0U zG`eeeFerABX^L|BP?CN7WV9|qnaReLPkN{2zy3jtB04P)Zht=0Uw3;jHBR(&^YsFs z7)%IYo})73VszLl*GbgL3Ih%(^Z#5$Lzwcl?*?~}8}eq$c<1Ue@F_|!NCBh8s8c;? z$Cq@WV&)&p9E;^o;I9~C-A>{HvIGDNuOdW-9@1qo?KpSuh9WCAZoY1h#Hl9ZJKAK8 z%sXa+m92ilc%c`uDhBz2E3aJPbF5U{w1q5!K4F4dGBp)N zmhGKDJcs9~A*!-h0}ARoP<0iYWj&$I{TmYlK<}W#kHEzvc3(kszV8cqGb%BEo2f6{ zBf*pY)uU}c{>2*X@4c#K;cWgtmO%ep?UDb7b0tc4adC(uDB?p&ygLw*jUjisc^5_m zSqvV%z3vHEi-x9Rb_>01DYsjzL9TPWGMkcyzAKz|2Aok&!Anpp0^cC7) z|A(({jPHa=wvKJvp4iUBwr$%^Ci=&=CpIV6#I|kQ#yh+B?%v(!?tMS>_v-$g>Z&?* zy4tM3&N#FuBjQAX4DK8-+!|%#)nm}G>I-qli~uIQk{aKlYq7>hNTXEl$GclTg-mXt8Hv9bRv3rOzZt*Eti~0xSfB^M3w$|Y2_<;C;aA2@B4J_<5Yz-`|Zq{!K!SMm` zWRu+pe<1^rHlvtL{+RCkXA%7GE!er3I++<7n+pDQ8*Xk8YHkqhe_<{CraBVl^Bem2 zRm400wjnWy@Ia|V+K>5&{`J7Dc_ZIoUeX{Si4Z?IBfYjRe|6{84Uw6@xl=$DID;PqR zTofD#5&A1vxaZ01r=(`t2)RuZTojxP98vdh?{N2cH;55aEr=uq1p8l1r<-F>f<(wMyF;Gk!*MW+-}%c=O}eh=b^1E=2u+qSFOK(`+Yq_Z_|&`u-c68(NqsO;&bfZ z6vN z(F!RgyJc&OuQMHZLIFDEi^t+>T)cQ>g?)CaV#sy&qA#G2NNXZfft3={dmpgW9xTfk zr0RJqm}uqY^T7B?^JrlkTP#`j?-QjGbDjEj-TF$sVk|k{VGJ)>j}zxm^%AkG9ppm# z;-IBE4s)Kzy;TW|o~#--Tu*nCsWYcynp^46*=ULKwqwMETT2YY`+Rw)uaRQWco(n1 z$idrjasp}PKlFnQZJSb+{N?+c$*?fQ={hRdM6==aG7j-wH<9$x4MiEoZ?6a^tfn4B zBJIT5rAJq~8GqU0C1?|PQg}o{F(qs?w|tW&KSADrq$cKqV4@P?+NQf@bdI@kM>v5l z3R{e)j4aMvphSGaBhc90Ltfg833YAz!r4a!Np#Lor)lI*Li6ENa8jiR(d z_C<$*E3T`%#lOxeKU0^`8v6r2N$~$f-~8>hp@Ng;-yrzk^o@$P(%<@B_!Ke{05YL4 zB7OxV3M2^?G;k4M@R=Vf1HD4OxecL$3==X=h-e4UpDP6N;%G3p`PYM0`R&}l6G7iO z+CTYbzh!*h9Id$nxyM731UAI*2tLOl#P91t?X@tBfR<&(MT92E9D5WRGlwC~3W70_ zk2J&#!Bx#Q8AoCRFj(pae@>*b(*h(0-71qu(e zrZcIpV(m*R`MQ+pwe)C=y^R(c@eNy*(JBm}snZ|o1B1Zd2BPtM{~&gw7(^j<-h%?Oi18Qqf=e)Xfznp$~QM-p$Nhb+eH!N zu#b$Pgvc>Uv0`Bi^#q-|Uhf9MfbZx=nJ7U{$UEwFjk92Y??{N7c^y$|7>s8Qp5lln zHuzc^m=L*Rn{IAtHoAarT)HX^ifQW~7x@(e6E-_ym($E?%gm5UXC6ds$&NiUAiZG9 zzB>~B%GaYCka;gIM)VtlkqbH>-$fePy#J#Q(m?-YTbOc2jr?3Zs#KBXJLYabHhn>6 zWubF&i#|WH_;nTGPgBo*B%Hp{W8!zX+-ccRHGm}DN`Y#gI?rSu`l;Pa zS?;9avma@`@%Nz%eKJB7SA@LiYm}jA4RSQXn`Pv{GjS+IzCqws7E;w<`8O0b>?lQk z>b4Tq;+qV@14bcUAy3LOn3YAQgVfi*2zX7Q%e_MW;6LZj!uwx-@PD=H{u!h%lYG7c z=kxm$#@~6xAk^UHG4=~F&iYKSvBAFI{>S*y|M&RO+YMB#89?&=?E5a<`Cr_)oM4}R2NVbh0qQ@s zuK!{FzlD%&RatVW^W|Dj4xNDOlVnES#blj9WnUCwG6^q4`lk- ztV@^ztZ%9tVm!F(;*qaa%Ry47bSaS<&aE~MjM9;v;k--gQ|oN{EE_m7k2fnj)d;sAt12;O%N2om156WbefE<3JI*xRsH?d99i(U- zTEx?C=Vi>d$fU*#nr&JRW5q~oI-`eox;#+%s$8T3X zoU~o)*u#R?ER3nSRuCruhu1&T7$U`wRW#Ylrq&xmwXyIh-@~;T-O(mE)U3B@ch1zh zeU}Rjs8dd`N^GpsbKmvBE3ZB@h#7?Negp^9(lVfK1w8?{wHlMaWBx`(uyXOQ|7{ozoHQ zCQ|E`eJIYUD|QUe)gHKdyBq|MvL&@61SnU|fcM?Ioxithg51c}hw5)ch^PQsU-3#&x#TU)6BlJv%Zg>||t98q{9%|Yyr(2&_$lw4> zaCO?s$!hp(7iqNyim+fRRNNdtQSyt4g#SJKnE2jgJ9&hO{&_}^dgT>;jp-C5l!ZtFNZrBf>l4BY3EIz%TK9&jj!AL4xtrc#)U)jab*!sUie3JIxBrQWvO5c**i%eHQ ze7D8QDaeG5)kyE64>jkujIuP6bJRV?2<}rVw3+4$1 z-w7@UvI&-X<={OQuH3?Pq8g4_I9EVaSmn@g%T!f;O zNY@u{jh@7gPU~W5o$p~&vgjb~Gly1~Mm7v-?Yl~NX!761|WM?skHJ4?f?vV z2r*C$FVzA{=A!_vf1DN|3$M;7MWW-7hyTLxh~Obvx>*G!Tz{wIk}TFLvugq~KqQ#M zaC&QX5f+)MH0%}{*J~D;ecCe9q^8ptovrjds_ z@zDTd2!vcOsfrUik|>@=`%_;4eC{{N{VCdxyaNC5F*448J6x2oAP#kXgBdUVPW^sj z7Qc0n28r67fDp8NE-{K&97{N7F&X7T)B{&~)68NM{~Pv=yT5+~%eQ)w1TXXK&3Gkt5kl@TUnPay8&M`HToI(FfFhWH$fLm7BGNjPY;-;(h4X07l{$a_Y-c% zGH&?>P1yZU8MoL)u7}9G9CGpFHSiJ}|y1 zIt}&uU^W>${POZ_-9+569J;--JhuY@>Iy<-O@q3lKWlHjIW@da#tziiG}BYm**XvDzbm?x)@dL$+8L1H*i z^)(c&st-Sw*uJy6@ddkC_p-4Ri&L)x$7)Y7ve>jF3Hu%+br+wIai8QBQ&A7ICmre- zvf%P5sMzC@YNr(zg#*1H9In1!Kr}m?OUID$Y2;gIdx$SJH$~@^F7x4)CEVB-n+W_W znsBhiMT4kv0`Z35RZ`Ak#^n+h?Emn%RRK2 znOoQx=D-S>P2^>hEH4kSKmTqwv)y$1Ag!7LlxY>9lfxh6-|_bCfoj|K57xpV{u4E! zsv`F94RVzUJ6uyVK9-yt&N)ezvOU)Q(BMynHA^E2;UC3<Qn+O8@#U3+m-Fe{ zt1})b@$j|VY|FKR60{Lu;CZv~JUKrrJ%IXQU1ppvIr`@7#(JH9=l;Rp)8^evEaq>1 z@Y+ClTrL&b4rAe=cGB8fqdgS0gS(s%HrOj`#!klj7}lsOn@i77y`l?Ot>jb1{Tl0G zJ!o#h`*?s%T|-q*%VOu1HP#v&_h)ucD7-+WcQm`aoUL1?p7$Co=n7@1SOe}otBn06 zVb!JT`&38ho9%D)R_))8e0nTBBYJ4ex&CiYSNR*4zRAZFtqt#`Zb*QQ6wH_25Y88) z6E?V5Pj(~upd3|r)Q;D}HQ4-Z1w$L}3(!2(N-3DC9>Pu?uMHBb+#zU!jc5Y0_7Nlf zMJ+Ul>_sn2aU_t>{>MUKVg7TywX24FU3vuY2)nla=tYv6mlIoi{fCiWPXeX_8kUiC zx=R&R1Qu$^)Ig=0iktp(ttTRzR}PSq$K{GHdRRnUAfw|TjwwJfnf{bKq(!&PoXz4 zAH2D}>p7)Rxi>kF7a6m{VJRp5MLn5;gI!<_!BqUpc#kG5&vbZg7I@Q`0p?(L!!#bf zLiw0Io}tKS_FK@e%zzdnYG~Nb*VJOG>)re$J|oK{-HaeET(S^(+}R3tePR#Jlr~-6 zbC22LJcyIg#X5SY9iBTri$O8n0M%3~W^o6f}r*Ft( zffe%!c^Yst5xS`yN=Kk&V`o@OlTx`(7)l2^GReAl#Z{HDpZBnWYFuqF|~ zp8S4le0nKv-XwClZysX1l_YY~S&5tsOr(y2$ON-DjK!`#(E5uv=)5F1rTxaI2Nm1| z>hJPY`;y@$Nw+nUKDv@gWI1Zr*x~jPyx(qQi*~uf?N6TByGU<3`rQ{Ez{4@gyC`pb z!wDtc6?VzPQ`tK5cf>&+T6->YHv2%r&kG$_hnE2?IoP*4W7u2dwfVoclX&}X5U|}= zl01?@IzMzbKKf7u4xNd7NQj{O+aVAx$J}y&@d>b!JY)@Ahxz1- z>Nir1J>6kNGPDt(MR;Hfio44z57Isk%8=*q*~M&>xzvYoABp=K(FjlqHiK1BmXqk@ zuF}S36z3<;IWCMQu{f^S@aNb%0>_B#usAwlbhoPytv=W9V?eW;6lpHBa6aQxw5Tnq zH1}e5l66nZbD2BCdrP}l2e{wfA?wBAmo`XhFOP5Wnoi<`?_uUOJNgG{=ZYtb2T=GT z#%13v)wk2D6_OV|QeZ^)NN&&eR&t-rmE?Jy< z90)c%Gzo?>^|aR2gLv9UgKY!tJ0(DhHD?viVY&{vm2atn+nrf#5qFiPq;}&sHplfK zKF28IKx|zK8Lz@%WpGOLp{Z+Uc3Hh5?g59rXYkGG>*B!Q`}c*kl&U6T7GdL^R{$ zKWBFnNHyvX5yIEpc-w{#xL;QTo;agA4Izu)sV2yyeh;BH&4h4%{@_TFLLCUOyQ;nSTp-oJIY}SBCm$i$aCe~LjYqgGY*h9BTm0r0Y2q|I z)It%H0Jo-*q9q#xhLe%m@{liLvb2(As=bENa$K`Y^Nk@=a7?Am!qUCoI?8CqMXY9J zfh|;K?>ksudvWVwf8|}L^8{0-pf)%q3$HFK;w*T8w)Jv3pBs18e&8Z{%fXjZypp37 zJYD77;Nd}XXf}OqSF7oZ?yAOJfMgnT$9r(b;Kf3hs~py4oM*WWx96`xt7F(;w}nz^ zeNp|=*8Q(wM;`X+B?FO`H>(yc&4ozsd`gX1n)yisC#x~_$A){DjR?%9<9~H~lQEXOg+$Ks&Yti`dwvrio zERlnlb%uiS^?6!T1sNV6iHKorboEVc@r|xa4JjRR)@PWSYqiT-ZAP|&zbZ#p0v~@P zuVE>e-<(KS>h{-Mh}s;qhN5qTCRbXaerpS7lQkC5u1QVi>%QZMBaJjZ?xT7&H2#sh z#x=g}*YQKJBapa5GN#!P>1qdwpX`GauSeez&A((`;2QiLbx4#?E-d*uUjtFtW-!b` zRL_R=10gDs@BLr}QF@<|8ey;S+8`2JCY*9R%sgBmHs1rf-DJ}qQ zh|BGXsV!~TnbDV@39tsjzi=1jm6lVs2KxDZ^8Gmqv3C>duuFAY*Q;X= z@;IY9<}w}Pi>oAf*Ym;;e`V}#3RC^Ema}6XfdiqfTw{V$C~xhrKtO*Am(QaTmgp|V zt5B@HEGwm!*-XZA#&Z65?wR86-XsOxw&7FAaf$rq%``VJ#c~R4%Lt0Ry+{NFFO#eVwnd(3`-D`rZ+rMCPVe>(T0-t#a5)WYod1DyV_M|!K=O_c8>nlU9(bN6d!YEBw z!)iwE&tT*wzu+x%5Oy;?D8kGgg?6|??8D>jzg4Px=ouFDnsMEjAe~(cp^KLka?wph z&iaUFq@w8|>|)%M0M~@|T#ep)17|^RzX890H>;s}R}h}|VznmUd!rblg?Hs6tqa%C zCP~4wJDV!L*uPeThrO7o3udz-|3#8Cqu@U*EfcM^LR!Z8gp2qPC2NOci7Pu$C8E8x zU(DAzi4pQ1YWt0tSz5}(Z6szHTdJQKdrY}lx`_%E53d>3(Y!d1oOcLcJ-;X`e{+Bw zKGN&Fu+iPeWm@Vg5}qO}vjkui{!!OJcekD&TaCw8p4YSw1Rr_yjggtF2w=2h8;>xm z{;<|JqT$9X8*dRlM54TSB5?u^ISQj5tL0RsV{@;;-3e&s*>?oxGbC>eB}%V;VX`Pi7y}P&(JQL8)&WXRZTUhpv)(B32s+V*i8P}iS*r_1dptD= zAh?W<=K8M4S>YgOSkg-nq8CC)-C$|)5G?lM^47c}$+MQXBhu19B`_W#PM7*^i_~mF z&?2A$J)WQf15;>zcH1C&q^$7oAT_uipvo|F5cEca-Q*o*4lCUZHz@(%p3x)ki}j!2 zVrmoM-*HGG{DSUaezRTkyzA{CLOdF!NZv|O?CoL;fcp36O1Wl5h3^H;zOQ^otAel- z&J0+q+uV-qnCYglIlNP(FoCF!2vxtq1YNpT2Xw5MCdjZTFY0()m}nu}vR`vVd2ms) zj+GRa>R^Ap$S^U8Rg>b-^eGXz_7BNL082o$zsAa}MqnyzF%r6YcEhH4DLZC@WlnNJ z46n%&sAMJ1Al0(??gc?-nihq&YY&$u*Lmm|EGuUkp> zXXe_%rE)0u+JWCSI{S<+-7wpy-WGKfj;noE5OZi=9mHA}^k;Wi5@-8)1_0J&mZnIv zeTuDu`;10ZW)C3)wTJVU`HtePTo8t{tx6JXg}9!OnZ|KMMZ=Y>`-;9_7Z*;;4!ahi za23P^4cM1<%b6-8fM2U-x(gA&5?Jq!hkDB5-L>L+oOvk!6kumR>^P_*pee%fyftpA zr*?;~crI&?$=&Clho*E;p={W`H-RXjwt4qW>eKSFj2PR12QKVfM@Y z({F8@(3ZzT4Cotl0sT4I_9Bv*yCJ4wZOZi8MEq{6w1*PM0RjrI?y_(zB(#J&y;y0R z(~-p$>T%vW^pGCIfgHwpHvKZ`52xD189gtVYwPU!6Vhp`1|H~Jexb#AJIe7ltUPN@ zfV@)Q&>Q|hCoQyq{S(qvOX8#5G%0oL5752#*4@p@pl_F&AQn%-z9FcW-?D0dI|xC_ zJ6sU_Onxg{2nFZBiulnlZ%FjzE@51?{wnbNr#K3cw_MG5e+HIKd~mOeqJcrw6DJ*L z{^s+_eY$)Ib4I+Gjwql_y?l2c(q$xTqI*x0BC1tV%WHJ1tbxqrtE_>|^r37KlV$}| zq33)L&hm{c1%dQF|)X!?vAzi<&b^E!L*uX8&sgY zM6%(S4fkYk9>jb($x6bp73fB5^2WMz16Q`KNozb)J?I?VdnfZGdHMVME!RLG|Adi8 zJh*>^yJt9u=RoTY#b$magDQZ$O4*4opm_{_>Met zoF^~*zFuBnyJ@9BibZ6Ecg9$dS<(&`osr=*k(oNe5*`v5YQVMs5zXasHQr-6@YFOYlw8F}A;)DVy)hn5UR2H({O-mq@ zEPtEbmfCh&mCN-N-}lUS*=scTnXF8vNcTL34_@#L$*x1`KUPyo6J6uG$L%&P)Q)Kh z9_gN2p5+|27EaVq0Nayy^tFItkAd8xs688^kD*-d@aw~%>>j5w{_q3rXuDi@vq4|g zO(IFqlY!eftjj`+Mkx>Auaid$1X?>RQJkwDxaNOp$=?CAYLpi`@#z8l{4fg*GTAvv zsW_xd5o+uz)eY$EH}Z%80l@c&8pB4lITA6V+F?!3 z!N1!WNgKTK=D>g?^l&a`5l!Zvwd2^p!h)d=M#j0oJWt_}xsZ8<@3nEMV~DDpgE?dE zXyfLyA~PPCZ-5Yim!uLb4z-L;Tw5_1sHW+uC7xm#H;vRSj&6uwc=EGnQ>!Hq8zry^ z_9VZ(!1g#9ynyU(a;FaJS2}Er_QxHB!2ad)^8axM zSyNl1f2x3#WbNm^Bl(`SxY_EU%tDNsYj^E(79pq(085FeLYJvhNtDs_*zRfdC>p5` z3f>Di0)<0_gYJ&U3XR*XBjG)5uVlwQxovv<@rN7KZVDWVL&vEffVW;=#t~jt#gM83 z3OsRf-68Iv%{{^w$9<+G?cLKy&t=y&Nxa|Pj|f?x>$cDGsMk@ z?T7}U61;^X_rbpf)~^zP7J&RZqkKDROF7rdVa~QfDCCT^yZDKFYl4-iUAJ`rLNhn3 z@9x!=It-aCAN>`YdRTfyy_mfcST0s?B`F~ z)%*+ENP?4js;(bnCXxG{MG+v`qDMSw$^t>VmAQG>$wDwvD8&h`BkW<9Ou!tDAtV*X zSO=KEo~ZXSoBcsB8u;nTx4pRbu3|z%llPF% zr0>L?7JNst!Ev;5l!?xhIZr;h=35_c6Z$nkI-+z@TRW8QeZDRV z9FJfCj?~;Ufm1H&=8D$*qYO9KrVX1Atn{+#or`vM9_ape9j~#Ni<4s877*Ve-??%$ zxSB&1u1 zX*2iHwg(Y+SwA~!;r1$%J@X@yip*kLFJ};e!Zz*W#pkF&@dx@0)@Z#OD+*bs`|}vv zjK=S2o!{g%zgOVseVUZtsI1yw3!5~q)>X%=th(2f*l0r?Q*QJ%T{tb~Yc{$nVdGi# zaa1v$_{TnWsXOb~i>A=Cm-@7NS98o9xV3xsXV${CpDs{&&&-W&VcxgtkO`*Os<}LF z)Ynf)%(UAiRl2Se&S)tk6X!zAcX@V6YiUY*GB%a?hl=QcZ;WmjRF-RNbm%Vi_m=e5Ba%1lXC2G$zm(J z7{|=wR}u|c5K#~Qh|6Lmekt5HoXr_}Gs!J10U;p+ChQ83+6BJ;|U$^;0#Hs(Bm<(z|d*t z@M+@LFU}ZlIY+^*Fr>qoP;IPyZGSXR8gD_fwzVjLWg!rCQ5i9D5u7BS((WOs((WTV z^$k#<(k`;Y17^IfYC7p8Q((Hl%+|_ob`wTWReyt>Rp(l3&D={@jq~J^=7o#j_L zvrW}p>+#t;wC~TNUkaA=yogui`Dfx-R1V-ZhxH9-JrID!`k*ohBkO>((J~GXd(|=H@S%&%hQ|qBZtN z3$H}>22t&eK}-)Zs^wmi4HxO@lfpn0I8GNXM|&2PFEqmo9LWyjD=c&Z{-WV~tL|R& z=d44N0fd3fyh@H^-iU5onGmLL{nk2Ri3hS=oeW>#|8svd{kcD;AH(b7{{$mnuz%^9 zI{kBuCWDNUs<5H4#Xowc${TX1f@q(`8*X$>_mI>IBFKu;s;7u#Qpm(b5=B-<(hOk$ znKfjL6uOm-gn9Ry$$ngW85I=S4z`<3@oR7&puKcUOKBx~x5=r$eNmOgcXzhcObB#o z=m+FCZV{Ur0DLR~KHNd=+w8Kw$TWteJyBIxE*PgaD%2dlMOr_n7m#S@- zo(aEGH zQZ}2ef57qV1Mfqb!i7XPO!XKbldn{$QIe7iAF<==8}5?1j$J(6amzy?{v$G;Fbl>Jt(0mlw>-AFFtCnY$!W zV0ELtWVe~6e7kk)PVhH&G|odeak8fT$gdagG#~Z#k>+bXwm*+t<#u7fDx+={u=zsvMuwdfpKmZMfe7YQTub4-Z;Kym?f)M`@{2o=96EPpt!(a9T zHcoy(_z15kcZkulM-6Kiib5KLo%Co4fzTB?_k!_ zYdV(r1LQfVe|gybFJS(!XeeUpZ0uy|@Xy}j-{YbxmY-Y~JM@@1b>5&-VdH`ik`i?Y z`*&5LJdG~eB&iwh%-U|FHt5XgsHa!Tj{8Ae`c>=z?Cg`{Xw1#$NVHsM6l&R&nUQht zP3K46DbL)aj`o+mt6Y6xtie=87h{S*x$)8K?*J}r{%~gAq?Hap2vu)dxDm26vv$hP z93#PqqiCNMT5KIfA1Zt<^u1P-HFuMumx&tQFRRt+9W7JspWc2;&#S{JaNa!ayWAy&C1O|&{kWlA?B z&6W2Qm@xb%9UT@m^-?&foOj4N!;>nW5FCeorllZ$NJC*&&F;wXfIU$8Ku%v%*C}{j z2vk&jUBSdzWaC{xGKYD~y@C34xp1UsL{; z?6S{9&3LWvJ2*DY6v2c~Xe{-!V#UJEEpeBF{i7nW_k}(b9)$TNvZj5Kk1S&Pxm&a14ji5K>un zp768MiSV;>P$*O(b0}1?QqajblThfi9!*3=)WvTYRh0r z88_4B0gMU-EE-KB`Kt;=Ga=y5QHwvN)Jig!& z=W-gwK5HWEU{H;i@TVeO@qm!Pwiz$h9Kdd_Sowpz27s(Qx-^WvJCERj; zl;-Ovzo~c_*7v~rzTxSgWTucWQ_NybtU@i($Z*VwO_#Y^O*K&{G>L~VDk+L7q6WrL zni)Qc6{y#_ab%S{pzwV-c9vK(4w;HbpP1G2>hcn(Hmkx7aTlA%VIcDHNB2dR+OHfg zez8Blh#pWI?rV{PZ|2dQ+#i`%0eQzY;IJ2>7eqHopF%b7yfL?+t+k0+Ph}7xCNwx{D<3 z-0ZFYQSD``L;I)<;D2uFF|_k0IDkpW7=TiZPymbeqs&@pd@qpvDG3({Wye0XA2MOL z9@J-5wYa#dRa1`MXl2P(B}g)M)>3r^MtrfRhXMi1IYf#qMfyEBF&s}Bx6abpZ*8<-h`HQFjw4Go*yuII{ z8LNvDKxg#ah2{5imjKbpp(cU=ad0P;;2r8Id?9uN&uie6GH~gGrhH&Xo{v)%X}r5L z@J$VopqpLr^(PhqN}n6keYjz0gdd_R4pORHgkkFTZ)G^d3XlHijpf9U&cKlZ8iKo@ zXtCZ30hB85w6Jd~yeB<6yu@~&>(M3G4M^{>=)YAM>zeY4imTmjv?j+U^=4ZZe^-`i z1}Ie+R0^QP41n%7}A!Q1k#Ayn;so9Dv$`%WEW=A;7dL&6NRipDm zfIMxKd69I~36E2FRoGnE?)qlh!ecZzAVSKp7XkGReX3NWz+j>d*K%aa zNjdGOBxeFs$D`z;a`>H^JF!G%XG4j{76EF;MB z1%1niGbd}ObLhzG(k!3Z{QByvcDQsZcA7o(1ZR+a3wPW#ig&@`e`9j!^-JS?{`jEL zTf7$U$lp;tU%IBmo4z)$^OCLn4E*SKDm5B)A{{LT*y-34JEheJ9+RYy!TS%K;i&EK z!kcf)z&CPA_u)Qg5)oB!+@wS_vvmZyv(fe_!Lw-;?%b6wDujUJ$n~Ar#F7dwUc0{w z_X*tO02p?d;ji}eVMb$m+G+TwukGJMnQ~U|!~%OnSj(F|qia)R40%0+e2~T$ul0R~ z*=c3{5+w77@Z!6AEZd!j-eb7oUIktX9E-|k zMCS|288*iTU}^n6Z{udPg(f8 zQ5vMK?z0AVu|5-}_yHP0U5c!SwOJG?8kBZ@%w6WVS}CmTk?J~=VNO$!CCJ(YY;94T zz(xSDRSU5k(ITX>B-m?Gj?(%xjLy7XNw@48r_ZXmab(ahBKlN zg4~h5+mBs#BOk_7MN$c1N%|3Z7DQua6ge&!H*CZ?t?=OAK{W+GeQYquzzdsZMHLc7 z<0T{z9r$91fBRf3l8;A8nrrY?&3ll&0RPkx~nf-~ntDu=5^YYNj-4 z*Jaw|Wq0YBBb9x#!ojt|om7akHw`30`3S-lwnu|akP(@6%izh6hbNiEgu$&BfkYrP zVl}vB6o8ALK_?IGWLWhpIM*Z*>yIr^ii8>=SY`H3EATwHC5K~%)k^hKBteSEOZy#b ze(l~7;p2k<`}1dlK0EEc|Dt=VSzRiNbZ6EIu*6a7*#H*9D#+4HzP|$jY^((dFHffa zle3#cwS&qFL^qCxK0i=q-~`ovZ?ypf9z6Qmb_b;9V>v;K7Xm(Kbt)!a>7E7<=Fr@T z2{H_|69hPce1IvM;uI?32~1ud@R&g;?pz(7J`o$Z(uG)=5q8G3Ovuzu>fj2m^-hOS zx%C&Co#;TTGGfAVgB7{0fi$bF0za8`@*Cv&Leopk?Z#d9<4ZNKdX6(}4&9XVgf9xq zDCa{@BetE={WijzPiU+d{P#vfQN(KGqk{wrYnBLz->*9aa!YCy?tGa{jXn-+j&Z9? zu}pz4mK5>q?-rGu8I}a5_F{rB+cfTJ^qvT7y2JFN&N>QoY!mgWPu;#n(kEOlMl(p4 z*&+lY3t~Fi{qwRalt@z`MqG&&uVMP&Hx*cl^ApBuq#kNZNf zzVi@J4wcj*Oc4biArHcaZIj>npevpXu|U^)&{UZXL&p+Ic*$2z4&j_Mg6?+$a8^8| zgNw5>I-_7)oD_azQo91X2d8%OvUoNWdNdS!G$aIF`!Ml_QmLpStIQ1g!+ib7&(dUp z?P!e&GBFbil6%CQg<`?UXkp0DQ!Zmz)0K5_CpJGo*jM8odlaN=gWq#AT$jo<+Uj{S zM0C2%?&6oBIveVLLCE*#;9ZLw3SusrO};uGDqLeE-m>%uivgBtVj?V7X(J2uQO^;+ zrrOGIk|wX`vinJaTQ)#pvJ*MJC7tG-YJ@@GH_{SFoq;y) zN1*`8vP!-|>xAvpsMc$*Y0;8k&+J|kSmv8Si`<{z2aw)1fF>lqkjWv<*&pvJZc19aY+ zGNz9nqPMTiiuRU<1~)(>y-KK31hsA?IlWzYp+y5?+CVs5l6(&@Bq5bRapa^gxuJ-< zm9lJKkLHCc*(XYh-w%sC#JVzBCC)&Slio4s)SV^M#Mx6eY$Mc%iR;y&BW>)ql{+sFG6ZDS zcNi75*(_W$EBU%oX*9D9zUwBT%LkK2En?stKkb+K@pa+;jt;?P_lP%5)2Q0a#1ER} zy9Z7t?7<2wR0G)hI&iMKT({p*I-k=yQR>Gp>P)fDk|6Q!!-8X_`9&$#mqZW)TAcjeX0Oc~@7Q zAVRnnK|jL4XA)ABWTj8=TazrAWlLntg_yx%X!I2N-yTK|BKfbamg+-HmwGt}eV!Pe zaeYvJJ5$gF6rT!Dn&x&st-P&mtt4)JeI5Y+_;?lz4HrZg<_0k3@4@NSh@wwM>||k0 zM(zYgDB?F_%{wTLrvR9k%=-=1g6QCeT=yA?-??Zg*mO{Y)ccYU5J_=-;633}UC|Wg z|6gTq0T<=kJbu%SNH<7#cZ+mOcXxL;64Kq$-5@0?-JR0XDc$+U-#O3YarOATFD!g^ zVduNo%r!IDwfEll45E%&Cn~P8tyFhd$UaQ*hf+Tkw~0nQ63xLulDl_)>zfheV~qL~ zvsEpx`2{%n>}?p6KWaD+4QRl0E6Ca_>1fv)p_D?iaCY)UW|wCS&Vhrc3am`}u9F6Y zjl`O|3YRt7_Eu|bH>RBy}& zwnuGBFTm6}*InRR?;m8UaV|w;6`G*%?72wWNVc!`Y;q_K?VSjw1fmGac_my;=i!bX zCZ}Un!-CEvnZtg$58Hg93NCUW(IHB|ih_Co_keO>J39gvaYW*7><7KFPjfwMB+aO( z43bF$l9|~ktC9Z&tU*#Au0hK(%=`U8Yg45Jb09ve)bwu z8{kNA&f?3T1>?l4#AZ3Ni%}zGxU7_pDO*d^^N4^I+F#_uY6UBJn8bXuE)+eHLag%g zZj7puVb9jWPMpmXa$z0w6EK+bPL%CVg;?&8Lbb+rfjq)85sdZy_C;5RES;qiw#juA zW)&&ogVk4_MZ&tq;mx~GrmuW)Vd_3W&GB0@(H*gnnB~W^zA_M0UGOFNV^%Zt;!&hoG5u&wY7G{RG9~PV|0lg*keprR@ z5~(ecgq2?QxFS%-a#GxXRo~Mg&aO;P_MDPyrz+p1uC=O2?(;&8T!!3RiG^-Qi2&h= zb88-2v(2+Q6U$*LbtQu;wCk+*=f(ko`i0N0lGFNzNWM~B!MUjF@K(D%G~a8Q(&bbs zrx_JVK6{J|e@5@bNqL7}N_U+~Id{@=(Er|OOq-%vtK@aNS&?80gjbKVK)_h;)G0#w zm#)Oyi&LLtxX+2CC?k}5YpSP2ILLJJBOV{@wo>Em-=z0e1y%dCXejG4T@4%FDHd)u zf{R7mk>8}>L2V}@U1bvQ$+AT5-Mt^mBCRrhG2XSzQ5d|%VG;~JmEHi!Q)h@OHlCMW4ojlLKwxt*nm?5R>shl zvoFL0h~7xSF>>a_c)^zW4(xaoY4)fKNFyZsBw}aQ?DLCi$Q7^l*JQBcYNjkTdu-vT*RQ zP)zq5uAV62!1so>!0U^fHlWA{{V!TKZ@DB_MXr;&4THDYPd@W|efDELrXw-6!=aec ziD9dW^^@JJkMhHfiGh`HP=7_TU;vILIz%g058f=kV0POg5R(u6MBU;Xo51psm$Vpf zA}v2#q?-b@g;Kg?XT5N2T)WiMb^LbI9^)GF=R?d%-GLwafPsL@K!3j={OCX?WMyn> zWk}5M*Md+crzwN={ldn+fsZ{$N@d!$HNg0dmhu_&-0xWl_@%(aZl~Y|q2BG6CTohG zE7y7!ib>sq-sXfkqfKIr-L8=`T^**`kMh`WKepUFK(>)JSL?3E2W-M6RO%7=Dq@?K zs;blMP6sY!=&uP+K1 zPFjc}YmgT~fInK};(pD3c6zM~U5glW6Ys(PN+Gq+9zQhB&`d&L|LVOJhc-rk2_9}@ z!z`lz$drNlNnru>?Ki%$Z_%-yh>m_U>z97ng>H()$G13gC5+m#kbZjqDz2C(v$Aqk z2M04Sp!b}^<=qG17DxJU%)s!1j0iC9FW`X}1u?srJo0ss1 z6NLTD3aqZQswwngmRi0KugH9TAZSl9gf_&4ye+N337xp(fVp=fnJFrl4z2VYlzH=tL<`+_OmyL zE$!APAOQi{)BXMf2S9K5%NJ@@VO?;h9eItltJT7(KP#OMRCFVh2F-$44AxOU-!Ne} zy6Y_Fbf8L=_026!m6tVdz=^X+n4=(3$Z2Yrb>=xOvDRKk-FJ+U`uf*o5lD$m=9&t` z$|0G|=hKitA9^1SMJP4p1jS#T6(lz?U8TE^ct4(fT324?bB=yrzKQ^Dg58oe&IfKn z`#}GiFWZ}z0KEk>(>tpkw)<-uv@3h6X}(A;-Gh*Uxw*<25^e_ zlcM;O)KS93)0n&6O@r`fs4>0UJdNeq9i+oB7W?GQHJHj&9xV4PpgZOn5ax`BFR2nB zIA7($FMwz^7{d#|jw^9GT#GgG`$uQ((>`+*52^n+(re~YJ&xBevdx5BOadE7h`rK3sK$BGg@02w{LJc0W^#h zrGXPY$jCQ4_&~|Mb)&aDZPy?irb3IPbnTjVZ!+c*-%|owleVlwNZP`a%si3mrk@-Z ztxFl)yT&7IyyoBD^-Z#MdCC`5IHjm~UFSfw)azC1PFbRH@P_a0iwi3n`Q3=Rv?eD( z>u-%&E%By}9JOqL6l@1vCSBExLeR@Gd*9GZ<6Y&QE_1_GoJA<{_Pd*7M!I|rsJWs7 zK}0?nvpAmPYui@xS{@t)90*dXu2kvIk|Ju~8r_CXNwy$h6LgUdCLQHGiL%Eu&1G7PT@XAX7&M zOpK25t)>&Nj>I=3>#eV%s=huaF4u^HW}2~$F)Mu7?4f0atzT@FEd7ztLe-~ny6lsY zmM0s#Ty$#HuO80XqoPLi2w07m9qr35)(2hC`KGRusMz%CGT1hP6N_<0dbzc|clKzb z)EWy4ROAVcM*1fL>iQ36SRm@_ELax=WB&fg-lBuftST#)N(^)1M&NPLn zX_sG?oS^}q6a^%-6FpQpl#aIPHD_p9P#B%(M^J_d4uk7c8G(Gx)=AAjLGS;lGk$mq zp(?X3Z16V!T|jzrl#WQzI#=b;THp;1!<;dEF%ByWneKFr?#{6BYsX!%nmVvHjmEAT zYr?>J2~yKce2C&*N%(0YFS>|jEJ&+3s6iTnbQMl`X*aj<+{G9jRFfrR(N;Nm=Ek6+ zA`_}{Cy=J(sg!hiH+4MAM1>=o5m^>#jV-WAxS;+qwj8qvv{A+E`Cw?4=@de%s7+(c@`Ry%MM|`jya%7mZ3XOe7|J;( z9U}yjh!fek6lo!^!tqfCo>J>YSU7`Ob#v93XnMKAHhe7*CbRXPwqOjsoZydexdg<@ zj%o$XjQseMKIk?sBc^53kS3{;OtF!LX&F>(4Z>f1P^s0D9B@aMa(vO{K9zb8(blxG zNX1-?5uQ4x&1m^Skd<7)#M7}VhfcXeFoFkzYnv70QG(mFlUDRzk7owVzt z*d~tlfSJve4>y2FQ}3IRp6{E$5ttdmvAij(;JDz1;I)TFE|x1bfHAxiZjJ+Uyg= z$To6={l%~l` z`*5YdDTlG=pV{(eP41strX3b6AgYq$>87;`SL~kS20l3ZfIAQY$8z1glY~9XBf9@6 z-C>H}jx7%h3g>y`0{n4j4{={I^l%)^#nLQHCFUX38WHs` zG75+0)3E96MbB<9Kkl3PyW?zvHaoA++72T`DAp~Y6DhR3aJ2?k9j#sPo-h%E+)oiPHM$S%I7c1~fW-U^E5 z+JVU~m4Rx1B;8@J@WY(qWx68I9+h_oskD$|>*#4~s|in)

    {?qV|zhOl;*XLkttE z%PlWMFK^DbLML*T8_2-^8sQYq9s|kn`Q$oIOTFop{yv0_7n^ogfoalY*1O?c$es@^ zfnv%Twl1$`LT~tk?5HY-3!U_i_Nvr#Wo71%HEsOm?eXn>p_%m?C&b{%g1}4Ca@v;o zqG-k{KDlB@98dXe@a_e{7fe+j*d@z^x8?3CTE-SGV zQTIj50pa+d2GaS!NZHB+a+z!t{_Z&VGOJZEpDG?m`Y>$FroZcYQb;)}v<@*kNIdtg z9?;LsLxiMMraW6i%Xu~Wn)&}w3^i#dVCQ(&T+Q-2A& z;Tb4CBLB6AfHJKdK~5F-5yDiwk4toO+jK9S;OM24JHZAAe6UuEFjGH5a&~uoWt0i5 zgK2N-ewHHSvUO)znF7D;jx%Uk!8h?vBNJJEmMH<~p#a{5az;+kptwhxDGAT0Ef5GX z@n~$OSy?{%voYO4@iR@VBSsg-AdSSzH07L9VNfjG@ts29-uEn=TA>XlCpO?Onh~A( zlnz!+mQ+(lrgorRM<(F%MT(?^nUb|~b#mA?z`0SwTcfO_D6wlYx7_?Z3TQp43VDP{ z2(-^p6a}>ic?yM&ABk@^#`CiJOgC-$^Lz(vzIAxHlfy+ScP?BUUx{D4voyE7Xk7-f zw)WFnsLVzdb!gZ3n|QIKBIW1+xdh$3XNu_g@`$-lRT@7V0TiMAn)a6E09@Sy?rN-J zatWrB=Nz$7ME8r(XJ87e01XUh`5qC}TgE3X_eakp8j}%{6*`K{@m}WJ>pPV39b^pq z*WQU5N4gVZ7dg{1d}9mLC8scX~F^=-h@c_A17+O?{E z-|86LlmAFe?d2($=Pa`sfn}5t9d0g1BnSt5U;N^?b@{|rXYuTmoajkPNvtR=J6|=} zJ1EvBopo*%RoE*LojJwpB_P65Unz6`aL03ofZ1WF16NV9&!Ax}qN)(q%b==;P+xm} zd=m2sua7$Vf*fg4r~DS(up2HYeN@A}lwF6xcRv^Saf*oA2QfR*Zx_TbGg6^1!}hyF ztD90bM;1f%Uf2dM-W`d$=f87y<@I8vx@Xd>iq;hfO#d52ubhuvgjK@iV%Xpl}KA2Uvp~7rY<*6|G)9#ECYUhMBXrwKY z8HHu)Znp2s<8-(sB;F=xWDZ2ldL|TW#@s0^3#g8nBbEv%5S3_Us}dj(PFzq>2C-w# z9`Xv&H2BFHqrtIRP4f3M_Py_ONmdoaVkv2ZGTv8Y84ui{RFtx#uW#!Fdy?|K$_MUB z_Oal7RTR*DQR&L^0PT0nRG1}0WX=nk%^h0l^}^@H`1TH5S{|%Z6kkBpJ@XD}T%%6Y zs-IL3tP+NdZz*4(2Ak78+~X3orjcHrfh?Vq_lL(gURXPPrS|nIgIr5Yq>Zh6_ir~c z`mLXiVNNytKC+tP&+gb$!he{7LS4i+tNU;pX17~ZYI4a14a>*h4VyOm3fgzd*5jxCCLnZ9z4!O&G9H~gQSx~l` z16(<*r@J|1qVS+ijZJGqol0`AmYQA4HDG*G* z_H{F0VODaHGdHY-QPVT59k-iPr3z6V=Xh(#o|tUM$4Qu&Tzy!pj(SC6p)>A>iaR1f z3j5hS=mxyhEuB_P>_BQU0MStgOwU{jC3@N=aE2G+B}CCK!zrT*qmbRB%^v#IP~hNf z(Vbo@*kDxldoP~?zOHKgb}{R<2d<VAoH zM^^UlH>d0h;JHWYNW3j-UqPQF;dJdWi||LB-Mi4#Gr8-;xC`yNHC(+0$Y0ZC=^C&wq9PU1smpz~7E+xg&Ka8OQil! z{doDR7De5h1<8xPH!|E(paCgRDLP|2M{YQuF)qL*EpmnSZVrf&%)(KMardfy$yk@`( zd~?{$v7Fi5^DIK*vRExDY8hlA%f_dMN(4JwAd0%XJj8+3@lIUoa zY4?FY|I939FkGQiswQmK_ZZs*Il`lL};oT6WS$Eh_UedEO~wRIlBvK7~{Sc&0F z;k0U8`w^!hjzI>-+xL5m4kfO~nd(fbkaVKQrjAO-Sx0QCkTeyp?8a;e+X`o2>$dm| zM!m7lx?4hrw%RkcmpdM_%*r-btSO%!Yf%rHvH}hit+oc|-3n zOxBn7QP&UCC~q#8vEE>t$7bjvQ$&<))%%ywT~WQU?29c(X6_;Ju@}VJOzj`8mxFfh zY^fhdi6DSjuL-$YU@;Y6cJ^0V2vp(3Ca*Dr|3?NFSo632PID12&^=aon(}gMFTsTk>W((@82vo84kFQyjBi`@kwcE&ioMu=yS+5eG$5gIe1p3krWxTzILNJ_1 zBIoiO3GB2&shz@bX@y#*wRf!3BQB>eYGp$|ZZ!o_f7p^hmQNOusg9K0C1YIdy{AF* z3VIwN9QNE3cp(XD8s!49%ee@-dn?la1m`Gvi*_~)B2U0{!%l9)6*7qX{(!3}hR*^W z4Vc_mE?#P$@|HGxEKistNLdX(LeP@$gsxCd5&0JHp(StDw1m4{LEO@jGGbgdbbWie zhxco2NqGJ?uNbcv)N&L49-j})>8rA@L1z)cosPD7mVM$)wB*E@&aS{69*Nw9q=uzZ`8kr#1RgPVgpI)QrH5EXAGuMk-S1HLH(zdW^90aN=N`{6RSpaY=nnGtr>`lv+87F0TR9op{kLSXr=q1imNJSL?W>ag z>>fdEDLE-^>Jagi`CJOtqF3aBiT*-Zu{B2RiVOzIy5pyzoRhmjMlqC@#+(m}D^sR! zBZfqSx$?%Bi}va6Ble?%7pJp)pC5p~lB!V(R~73a^#f1H!OSRtRhAQm!-HQblJ^z% z5%#KSb%Sy%yv6W`&=f*U#Zi+xfyb-*B6pmGNiv6wCZp;xxWv<=)K1!isjuD5_UNN35-unTE}=8$y(<4t^a24j#)kxVGMi%R#O39&6I9oLyAmb@e8(ytfpEPQV zM>K)GUFwKmihkiDl{GF-NH%XJPrb;NEk|TbsiG1tGngu|Xwqq$t6(*OF5pnwo!W*P z@@InVsX{EnoY7*|=FN=Jt3G<;P^ihPbNKWHt8yqdf?s|TH|Lbw9B06ut8);I(o`q z-#GJGa%`om&Q=C_f-5?)j~(N%fYW`+^yq0?4))9`PZg1xEPh!KE&wiaAanxH4wHfv zVcXGn1v2}1g=dl^AlS(r3XT>^olR}^YC^tqVY#S@Wno9dc`gq{s(wFyO$uv?;b^;P zKw>q{SFmq`Qdt1tLb4_UpB?++q*F5%`f zTn^~d1g_%GpHf>)2##qHlwU2ZS z;;b>kw$T|)6P@M`{}g%P+*QHMmvufo);sRJqn=|O+qxur37JdeO=uI+%Ias(-6tnK zzS0Av3X3aYsUKBJ@satd0_tY0=OwT}v^-or!$eQO!TspH#~@N`F5&CXa=eQL5XMDC z9F&j12`4YFGWFkirF|q{MSP&p_B@ zcOUKw?hKms)mOIrdDq075>X9knD zwS#2lp7}r-L!afX<%o0_8b7IY zlONqTjhzKpRNMFWXXp;;?v#`UX(R*%1VjXBq#0u996~}Q1w>L(8U#T?8tEc#)QZAF9+MBA~Iuq13F-;#N?*y6SD{IHFfra*yB|2WIll$lu6+^8*E*sCxOOprGi zdxnKugrOXuy7UDllAbeH5?xkpS@$k%CI1_yuc1NCtLj+8=Peeu!+wx)8sBg}elT9R zh&5uJrrcc5TIfqFPBC_L6Dn-AI$o~qez;S?%}MWwC9Xj3NewkqwI>fB?qT1G0!k+Y?L4J z^zpu#vk^;MHZJ_AFg2diUno&uZ6i;=%d-ZRr|1#3%UK%g2>V07NA6oF-JtWV!zHN% z9u7jgD%-akobjd|WnZTImOF*s%PKH*vyKrN+!Y^a$Kg%#6dz&5E1Rn$AM*DwGMIUD z{g7L5#ngc#->`mS=J`%GKfjrQa&)7DkCWJN&y$5OGpE%wBt7Bx>+m%&JcEbaa6C-g zhjKUzDYR+&x&_c!^$a+GZ+;4VLn~Gk*p8{}6qmWmBby}0xNIYw^*z`$pOa1wnT)W` z4Ammrnon$f?=>2B)wJ}Y^2oatyUP;U%Sow|5 z(ge!6WX{%J5ajl)3$s7NqV*Nc*X$4k9tta-YAJ;6KlEF}Z|{nK(otXNWkIKlkaWp9X%8R9*nDNQ#&&!QtWXi%%j$54QIrq9n@uOX?&D2OY4VYHcFbE=EqD}+BnSuCrbwc>D!RVd9D5sFJ6T)l zCVeeHm1ZH|4c$wh=qpRQ?%~vKy&N+k3sXm)viQ7daQvnt^l8ZgdJ)dGvV2n1Q9q}* zIq|$1PU4Um!p)bW6z?KvI)o;`mi#iGc2{^Mz`PkFuT`8=1V87dNI_jNT)VyO|;vT*pZXt7WifDYio|GtO<^=dl)rt&}PeL>Jr7V*a zHXpV+S%iLwBVOcA$dmXzzwH4QLihIKQk||#6Q)y3ALSDkb^ffHuEWSr{!*eREUGu2 zL3}M+qjqb$88zL<%uoZqGlJzCEt>UU1SP@pqnJt~s!c@c{q~teB2+Vcn~9*;b{}62 zIUHB^wz6DXam3vd(s5V zXc9&xe$}CmzIr)7=(xMBohjYU{4RI8=^iOxptia1mk`~U4J&jyZ;vhJLNmcOCazb| zAkkbMiYBLuPlZzcU(vdC(6=3N`W-v6S&0HeEtz>!(j!%AR;_N)%GMDuxiKW&=?V8i ziilgXZqi^2@0c-LT`Zd>PEhM7ekpoKB(s>bbOs+~s*#}!8VL*AN14hA-|FZ6_V{2GsbGf#Oliz&Ki2=U}O7M*=9RwHhqlPaD z>pME|+_n0}MTOK5!O*0El(*A5kNatKc!2Ldo*Io55qT5`25!$;geW1;wy7nguf)_R zyjZ2Qeow#pq>BU3)18uxiN9~bfJ`$t8%tNqmycWOP}#KNH~FOn*$nx%3V&0$DJo7W`~w*( zjDEj^67A?1HWD_^9tYg+w_-uiF$vnAx^3+?^f3^{AUeC0rp(A+>f>y*! zddH+prky?9y2lpN_`ZP8WBqk69=?{@%<{*i+D$NCkJ=+;9OT>-xEBfKm*sEEyIfeU zv324^;oaVB#xHeRHQ>j1i;g+Km#o~~YpwY#u*gRgSFCG5FSJ2_U?lT0({9*adYt*y zY|;Bi{)C~>#BNH<$mjTqs1Lsesu$}kCC{7wkcu?5{*p4a7K--{R`V47C9N{EpW^hc zQu`vFV_iWIx)b&PZZiiPZ5q-jrpLH|6 z+a1?2IM%&JzTU~Um7M-zj^uHp19=J0T9UL{Vtnq6H`aIudwzNbC`6J^PcZe@iOjn( zVQsqHPq#83VV!YVMvLxxM{c1X-q4k@-RiSuTizn8YnWIddiDg%_vWOagQdlzNtBK@ z)b_hh4hP@l2D)_b*>n$lJIEtA(1)`5Xv5);c!X=9pB!i#%XxVC<|U!-T(he$or3EZ-B z-#*K0>Av)U*cg4j>W-7?^6VHi#c$guU!K0$UXY4gzG?S~#4DSLLMmUS-g=s^i3SL+ z=boD0j38c18WG+^Hlz{K0f}Io^>#;Q(=^fdPXiCE#^7=M?}Ii+xedM4IlVp3`9HlG z-j(dK!q~9l%W)Ho3uVkL*_n7)pg3R6P<3Cs;c4**XE+BT%2eEo(*`ajA%~kplh+up zPsOdh<<6PRDpQk`-ZhyqdmebLuIW9KmjznEj~xL2IzI4;Zr~h>e_C!w_A?%09cJ6PI?FY>lsy`rv>S zSxEcgWJ)`@m&Idg9I4S-Ntn8mC&GHyc6N2bEd+feRke#)sqj&*kolnje-#aO7BR=I z!Z(8G#j4#O}v!p#GV%(i|_R3_UlFUEF@<5j|v~_evgWw z$xXqNlx2#!gM*vs*gLsPxR*0Va#VRi z$n%VolPT&mN9KmA!s1lKU~cC^etR6=fhXf_J50mXZh3PPp;0lA;#|LFZe;o`Wo-&Q zeka_W8r|Tp5S{R?@pdwwea5J$L83KWdJM%kJ{ZfG=c28T9VUWZN;|_EP*i*fUSX=J zl;E^5cCh0;)6KFHBOl4}=-4m&u(Ru;HLrT`eO=BxbMq9tRCm^CgPiUS_Whw%dV`ld zO}NWDbm!N)OaeYLEvJ4Uf82>TlL8}Oe$gxrU-V+0sF)SO7?3XjyEqzqnQRThc1pM)Gi#^W!@E*22L+< z1;1iC^w&uoGRNlZ>?G@Kr|m!S8k?>hqoN)E z2q(C6cOAE6{Ca|f0i6KH;`l5xZqJlzq7Tm6Y+Q50+DgNzsHxS2H-m|2jh~HwcS3n$ zx+=R?c(%>(OJuB^z`kRLhYWtDYtszX?!avz&K8>O{*jlThYMn>{Td=G-`9Sql@;=L zC$;EFpt|+dFUJWyfuqChvLBMAq8-k?ez>FH_Jd5Nc6x*{d6(8uz@;xv=gFJ5TegKP z*4lZY9RucH3O|uj*Z2snJLO%|-dMke0I5C;ROi~WjPKN??lhQF>pI_Zl$)=$CeMg; zagCTrh8Q4m)Tz4&iUe~=;if%i-H@Gd7J9C*LXP)9Je5@(uVs@hdtkfVugzeBZxL$+ z(o9vx?wQ1QJ&0H+VbD2w9#8}qNx1+KXpA>}k>=8divmbr@YQWDpb z+*SyL)O@os$3s_yTjkQNgn|_#-d}(2Hb7uA&MlHO)cE1Gp=z)9$jg$G1;Ms*yzBi( zcLcF|-sXNm_VKmFAo@0H=Tbab6K~}Cb_u`kZ z#KdbxMqh*^92W%hURes;Gi`bd4{@u!e?aNyx4(ZD7U~c#yvhf5<1!#JU#nb-8cjvE zTej)ZAK2xtVvr5X!W%UpR~0*^*(Bu5^4_C1NN_7j!W=dUCBcy`yYJ}~2LkIcxV z-v2&!n$a$Jm;g2I_?ArHQKd$a+zb4iRxrf_* z=)|dYq93o)u%J~o#FO3Ec{=P)qZ0^KeTK}ne~Z#*RN>tx?^b*iF`dK*6vZXgW2$n5 zi`J#DdMle#zR%_-x7GLSrRieo$G7XJ1V~9~gWe~A&}6~qewiq%2)yT@^CfIbjE!w? zURdM#oPSO*@sV|VPRJT9dH|f^WD@h)*^->)^q~3;f4J;1I_qT8dPdg0vk}u)x$jt2 z_gekEoOTeir|`vogxKRfwpcRh@ZvKig}LdGdy_rQa6~8<7oRbDm~?k5oVge6%!MUa z8l&+{MFagvWHJo&@nbJcbf#bt z)5IxtPLSk#T=OoVG$bUww6QiIq~KLToBv%|G!!umaSbl%Z~ei$y4hKDnQZNIfqu~o zZYn&@!5}2gAY`Q0bIhPX^stKRHz&R9O-c-@OGLzyMUVG4D$S)|sf8gFF)1n6#2<== zt%{S2f2-@Eng;Gm`$E!>-al$C8`_Q^ukFUhgyS^8P>|%#F$)z=M6{LXfM>yKpGAe) ztVq&$D%wd7N}mi0C?4at8)YA-)i%i=RZz`YNY8ZB;t^cC^W%E(?hgWL#riZ^4SIS% zb{Pw`2mY|L*{

    ?uV0uTcNj5P=fb~TEq4r?Q;|7@6RNZkTVx8X*|jiHuFF5&BD&P z5czBHQnh_4tnSa1q;uo}0epUpQyk}NC-G7(#{E_3gZDHCqxRO>Zbx-&v-=+=)*rXl zbPtYH$^^8C*&T~)X4nUVJGMD6-MBK4W@)k?rOz=6W$~kYkbHn_ic6fb=WxMx- zX0gTcb1J4%=3H~&)2LGu539i?Q>l8)(aNS*=ghpT`!x+!uYxMyrX5EkuLR%}OziLk zS`sNsDPevegs-1uC^%Sd8ka!MshZtcghR0HZ=|!KVi7xfP0GWO9W6-7oQFlKj@{W} z`?01~ee)VnP@y&ICWy?iZ{llk{E^P;d%?w)BkpbU2nDR@BlI;$+Zp}~N?BQ-lf9Gs zc*x}aE1HN9-zMQ5^#TE-b7qP|vAB?7Fhc{Tzx*-|4fSTR`F>wI^4brk@Ffo#x7`ir zQ{w=Rmh0z`u!f+tf`r{Bb=Smvt++3hMX7Cw1Cwoh1wYbxq`orCZR7l+ZTg)_IzN1_ zh-!sK@B3VIn0%}xSpMMYDPMIG|IF}K0zH4PbV2(rBi2%^m_y%!W8IJLDZHz`!qT3k z!XPvQsdF2D*txZ~;H1jL@*u4S#@^&xI%%0)*QUVY^pcg=+s5`kl=75uH;#@f^J-)K zU0=gn+O5#c;ghlmWiD@@=M+Ba5BJCd1Z2n2l+T*v8vJu8x+!=pZjOHNb0S_bl$rJy zKRzQQ=j}!_K@{6{&ZPzTfix9CFn`Mi>F++JcJZhg@vIzZnMM0|k-sRerUTV5uoAv`+>_O^0d{pkYT8t_QelnKeuZgPxyH?<=aM@T=!*!aZv-I=g3 zY2Ha4B3-me&%Uxn3i+uF_)!=Fhv4MHa97t5UM2}K{Z5guZWBU)9uvbcv6a^kTqjOG zjre3nPM}nFCl%@Y%|=0uW|Ff_7W%tgJ;UFPgl$xwWOxv#+bf+$W}RUY`J7Ai&6TW$ zb)?s*xgo5^z+JhoKd*=QKW+^GKc8Xp%mWj7Ps$M+5gPCTR|kU4kOeSGG-V zh~4EM9?X;>Xa-kHo*|un8$j$QkN&7F<{OkH55{=WzL)_IIYuXL98o2>Q>6>ub?oNQY!_aJxJ*3Osv;j~=aD>>ECg*b0OnaUBg*4}pVNO^}hm-cCqw zKvXDjgufNp(bIkqCDZvlOFXd>jcMlTG8x?F5t=0AF{lro9MDayoFp8aW)%lSASG&@ zwJ>|0L;5lzt|K9Vuh*Y}wp7os&T7PY0}7E7(T|j>df~UvYfQlb+ReRv{u(C4aFGL} zC~Z*c{Ap0F$^P{{k^ED@Qho3g3>8L5?qooeTNoOpbx)4#kvGLQy>8ABtRlb~s$e8y zWMU8qgaKkLN|AjTIBeDhyh9-efiQvopOyo=c*UcjWuyg#I9vSN3|yKb8;>@50~t60 z3>-8UXa3!CVB5f}TS`jYDt8stdCe^xEgZ}(9L#{v)2$tBU3j1#(Emb_{KEnI92hr1 z|A}&dK)JoF9l2dh9$!)O>XV`lIuZ!vfC>T;UP74t8=;~4KUu#juqrFb8A1VpROmpU z3(x%1a$w289Dgd%QrEdBe@p9kbEuy)bwvlvfbl`3|Cj@P4lD)ZUj!(`(a6ri&BE?i zM!7Y!UCeYsuT_RWzjt{tx1s?h@G8<8Ppo$ z;PM+4Mv|L`v=kr^hcO5QbU#}TjDhXXluKfs6{g6#?8`AFbAUj%3xAD6H<^E9EKQ&m z?k1kU5u&zvIHxHAoLd0`UCt$0o8J>IiMWUaM**!ULm&kBxj~@IVc3!TH^R{w0)+q` z`Hg^gIsK>lKn~dm{<@r+RadPq4So>~59r|z7x@DaehH-+ph*1{4Q>|BF2FMUX6(~_ zMwS4;Fb_B`E2r(gVti$2!02cCT~6V?f2^SvE>M^MNLi3Sd`3*g@^2Zq%1 zw}LB-D?+5lsML6YLNEfP6zx9*pwEHjU51!?LM@D3pw89~mPQxY-?$|5g`2qskX_V3 zaRQ3l&z1ut?7t-FH>@kd-khE?qXIA#AWx}(qArA`4_$)&OxQoky4$UYEdWT;qW^XG z_&oaGq${EnmSGi0fT%M7cX=aHowHo=2?z z-?U5OE>e7ofFErKSi}vZU!$V`=qm2-YV#*;%r6$j#aJ8Ku{pLFe;Y*ao0R&tyD*T66r{BP0~ zQ8n-SLk58u3juIcm)y@Ha}{N8;`k>`$!*Ho7rRw1P@XQ&j$7eBL02S^wLE0FxBz*U`xK7iM6`_2kMF}6+ffT%3u&E3@E8gmlki-@Dl4H zxQu>gqJ@Wvy`!DQZ)&QncNv#20J{cwjrkJL!1(_Qcts+mPR3y$K)MCU1g<)Mwj3B0 zaQ*tPp#MwapOx7Y!{2)V3{F7Z<)EN5|1ZuZLGOSpq*mR)NcIAQ+~UE%T+R+zfI%)+ mCe9YTf62P|!^b1SV`Jj1p^6Hu7zo4-{OAE8kTwsf0R2DCBM9vP diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index f108c70f215f5..7e54421b52f82 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -942,12 +942,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, val conf = new SparkConf() .setMaster(master) .setAppName("Spark shell") - .setSparkHome(System.getenv("SPARK_HOME")) .setJars(jars) .set("spark.repl.class.uri", intp.classServer.uri) if (execUri != null) { conf.set("spark.executor.uri", execUri) } + if (System.getenv("SPARK_HOME") != null) { + conf.setSparkHome(System.getenv("SPARK_HOME")) + } sparkContext = new SparkContext(conf) echo("Created spark context..") sparkContext From bc311bb826b5548b9c4c55320711f3b18dc19397 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 14:52:37 +0530 Subject: [PATCH 1158/1571] Restored the previously removed test --- .../scala/org/apache/spark/FileServerSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index a15c3751c21be..506f7484fb731 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -55,7 +55,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { in.close() jar.close() stream.close() - testJarFile = tmpFile2.getAbsolutePath + testJarFile = tmpFile2.toURI.toURL.toString } override def beforeEach() { @@ -142,4 +142,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } } + test ("Dynamically adding JARS on a standalone cluster using local: URL") { + sc = new SparkContext("local-cluster[1,1,512]", "test") + sc.addJar(testJarFile.replace("file", "local")) + val testData = Array((1,1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } + } + } From 74ba97fcf75ae61833295a896dcdf775036ae1d8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 15:08:01 +0530 Subject: [PATCH 1159/1571] sbin/spark-class* -> bin/spark-class* --- bin/run-example | 2 +- bin/run-example2.cmd | 2 +- {sbin => bin}/spark-class | 0 {sbin => bin}/spark-class.cmd | 0 {sbin => bin}/spark-class2.cmd | 0 bin/spark-shell | 4 ++-- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 4 ++-- .../main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala | 4 ++-- core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 +- docs/running-on-yarn.md | 4 ++-- docs/spark-standalone.md | 2 +- python/pyspark/java_gateway.py | 2 +- sbin/spark-daemon.sh | 2 +- sbin/spark-executor | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) rename {sbin => bin}/spark-class (100%) rename {sbin => bin}/spark-class.cmd (100%) rename {sbin => bin}/spark-class2.cmd (100%) diff --git a/bin/run-example b/bin/run-example index a3976beb12141..f2699c38a9a0a 100755 --- a/bin/run-example +++ b/bin/run-example @@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then fi if [ -z "$1" ]; then - echo "Usage: bin/run-example []" >&2 + echo "Usage: run-example []" >&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index d9c10545e7490..6861334cb07e5 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: bin/run-example ^ [^] + echo Usage: run-example ^ [^] goto exit :arg_given diff --git a/sbin/spark-class b/bin/spark-class similarity index 100% rename from sbin/spark-class rename to bin/spark-class diff --git a/sbin/spark-class.cmd b/bin/spark-class.cmd similarity index 100% rename from sbin/spark-class.cmd rename to bin/spark-class.cmd diff --git a/sbin/spark-class2.cmd b/bin/spark-class2.cmd similarity index 100% rename from sbin/spark-class2.cmd rename to bin/spark-class2.cmd diff --git a/bin/spark-shell b/bin/spark-shell index bc7386db4d824..e6885b51ef567 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -90,10 +90,10 @@ if $cygwin; then # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - $FWDIR/sbin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@" stty icanon echo > /dev/null 2>&1 else - $FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@" fi # record the exit status lest it be overwritten: diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 0494ca87266e2..d46fceba8918a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -127,7 +127,7 @@ private[spark] class CoarseMesosSchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) val uri = conf.get("spark.executor.uri", null) if (uri == null) { - val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath + val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) @@ -136,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend( // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - "cd %s*; ./sbin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d" + "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d" .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index f207627e52dd6..6ba15187d9f63 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -27,7 +27,7 @@ import org.apache.spark.scheduler.SchedulingMode /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * - * Usage: ./run spark.ui.UIWorkloadGenerator [master] + * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR] */ private[spark] object UIWorkloadGenerator { @@ -36,7 +36,7 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./sbin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index c37fd9ab623e5..605588f7f67b2 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(60 seconds) { - Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index e2b21f9cde05a..e21812378c94a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -54,7 +54,7 @@ There are two scheduler mode that can be used to launch spark application on YAR The command to launch the YARN Client is as follows: - SPARK_JAR= ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ + SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ --args \ @@ -79,7 +79,7 @@ For example: # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ + ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-standalone \ diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index baa0a062f704c..c851833a18dd9 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./sbin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT + ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 7243ee68616e4..c15add5237507 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -31,7 +31,7 @@ def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and SPARK_MEM settings from spark-env.sh on_windows = platform.system() == "Windows" - script = "./sbin/spark-class.cmd" if on_windows else "./sbin/spark-class" + script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] if not on_windows: diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index ca6b893b9b9a4..2be2b3d7c0933 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -147,7 +147,7 @@ case $startStop in spark_rotate_log "$log" echo starting $command, logging to $log cd "$SPARK_PREFIX" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & newpid=$! echo $newpid > $pid sleep 2 diff --git a/sbin/spark-executor b/sbin/spark-executor index 214e00f6f8070..de5bfab563125 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -20,4 +20,4 @@ FWDIR="$(cd `dirname $0`/..; pwd)" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/sbin/spark-class org.apache.spark.executor.MesosExecutorBackend +exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend From 9ae382c363202eac8ef0b8e0fe1a7eab3de545b1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 15:12:09 +0530 Subject: [PATCH 1160/1571] sbin/compute-classpath* bin/compute-classpath* --- {sbin => bin}/compute-classpath.cmd | 0 {sbin => bin}/compute-classpath.sh | 0 bin/run-example | 2 +- bin/spark-class | 2 +- .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +- 5 files changed, 3 insertions(+), 3 deletions(-) rename {sbin => bin}/compute-classpath.cmd (100%) rename {sbin => bin}/compute-classpath.sh (100%) diff --git a/sbin/compute-classpath.cmd b/bin/compute-classpath.cmd similarity index 100% rename from sbin/compute-classpath.cmd rename to bin/compute-classpath.cmd diff --git a/sbin/compute-classpath.sh b/bin/compute-classpath.sh similarity index 100% rename from sbin/compute-classpath.sh rename to bin/compute-classpath.sh diff --git a/bin/run-example b/bin/run-example index f2699c38a9a0a..6c5d4a6a8f364 100755 --- a/bin/run-example +++ b/bin/run-example @@ -61,7 +61,7 @@ fi # Since the examples JAR ideally shouldn't include spark-core (that dependency should be # "provided"), also add our standard Spark classpath, built using compute-classpath.sh. -CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" if $cygwin; then diff --git a/bin/spark-class b/bin/spark-class index 4e440d8729420..c4225a392d6da 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -128,7 +128,7 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then fi # Compute classpath using external script -CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index d4084820f68e0..fff9cb60c7849 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -122,7 +122,7 @@ private[spark] class ExecutorRunner( // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( - Seq(sparkHome + "/sbin/compute-classpath" + ext), + Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts From 0b6db8c186183704feafd26b454fff58a0e31861 Mon Sep 17 00:00:00 2001 From: Luca Rosellini Date: Fri, 3 Jan 2014 12:57:06 +0100 Subject: [PATCH 1161/1571] =?UTF-8?q?Added=20=E2=80=98-i=E2=80=99=20comman?= =?UTF-8?q?d=20line=20option=20to=20spark=20REPL.=20We=20had=20to=20create?= =?UTF-8?q?=20a=20new=20implementation=20of=20both=20scala.tools.nsc.Compi?= =?UTF-8?q?lerCommand=20and=20scala.tools.nsc.Settings,=20because=20using?= =?UTF-8?q?=20scala.tools.nsc.GenericRunnerSettings=20would=20bring=20in?= =?UTF-8?q?=20other=20options=20(-howtorun,=20-save=20and=20-execute)=20wh?= =?UTF-8?q?ich=20don=E2=80=99t=20make=20sense=20in=20Spark.=20Any=20new=20?= =?UTF-8?q?Spark=20specific=20command=20line=20option=20could=20now=20be?= =?UTF-8?q?=20added=20to=20org.apache.spark.repl.SparkRunnerSettings=20cla?= =?UTF-8?q?ss.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Since the behavior of loading a script from the command line should be the same as loading it using the “:load” command inside the shell, the script should be loaded when the SparkContext is available, that’s why we had to move the call to ‘loadfiles(settings)’ _after_ the call to postInitialization(). This still doesn’t work if ‘isAsync = true’. --- .../apache/spark/repl/SparkCommandLine.scala | 22 +++++++++++++++++++ .../org/apache/spark/repl/SparkILoop.scala | 7 +++--- .../spark/repl/SparkRunnerSettings.scala | 17 ++++++++++++++ 3 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala create mode 100644 repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala new file mode 100644 index 0000000000000..acb1e4ceda8fd --- /dev/null +++ b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala @@ -0,0 +1,22 @@ +package org.apache.spark.repl + +import scala.tools.nsc.{Settings, CompilerCommand} +import scala.Predef._ + +/** + * Command class enabling Spark-specific command line options (provided by + * org.apache.spark.repl.SparkRunnerSettings). + * + * @author Luca Rosellini + */ +class SparkCommandLine(args: List[String], override val settings: Settings) + extends CompilerCommand(args, settings) { + + def this(args: List[String], error: String => Unit) { + this(args, new SparkRunnerSettings(error)) + } + + def this(args: List[String]) { + this(args, str => Console.println("Error: " + str)) + } +} diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index f108c70f215f5..14c3feb55c2c9 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -835,7 +835,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // runs :load `file` on any files passed via -i def loadFiles(settings: Settings) = settings match { - case settings: GenericRunnerSettings => + case settings: SparkRunnerSettings => for (filename <- settings.loadfiles.value) { val cmd = ":load " + filename command(cmd) @@ -902,7 +902,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, addThunk(printWelcome()) addThunk(initializeSpark()) - loadFiles(settings) // it is broken on startup; go ahead and exit if (intp.reporter.hasErrors) return false @@ -922,6 +921,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } // printWelcome() + loadFiles(settings) + try loop() catch AbstractOrMissingHandler() finally closeInterpreter() @@ -955,7 +956,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, /** process command-line arguments and do as they request */ def process(args: Array[String]): Boolean = { - val command = new CommandLine(args.toList, echo) + val command = new SparkCommandLine(args.toList, msg => echo(msg)) def neededHelp(): String = (if (command.settings.help.value) command.usageMsg + "\n" else "") + (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala new file mode 100644 index 0000000000000..235a688334c37 --- /dev/null +++ b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala @@ -0,0 +1,17 @@ +package org.apache.spark.repl + +import scala.tools.nsc.Settings + +/** + * scala.tools.nsc.Settings implementation adding Spark-specific REPL + * command line options. + * + * @author Luca Rosellini + */ +class SparkRunnerSettings(error: String => Unit) extends Settings(error){ + + val loadfiles = MultiStringSetting( + "-i", + "file", + "load a file (assumes the code is given interactively)") +} From 67f937ec222c5a7db5286c0af0ec6f9c482d2af6 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Fri, 3 Jan 2014 15:34:16 -0800 Subject: [PATCH 1162/1571] Added a method to enable bulk prediction --- .../MatrixFactorizationModel.scala | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index af43d89c70f05..bc13a66dbea6a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -20,7 +20,9 @@ package org.apache.spark.mllib.recommendation import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ + import org.jblas._ +import java.nio.{ByteOrder, ByteBuffer} /** * Model representing the result of matrix factorization. @@ -44,6 +46,26 @@ class MatrixFactorizationModel( userVector.dot(productVector) } - // TODO: Figure out what good bulk prediction methods would look like. + /** + * Predict the rating of many users for many products. + * The output RDD has an element per each element in the input RDD (including all duplicates) + * unless a user or product is missing in the training set. + * + * @param usersProducts RDD of (user, product) pairs. + * @return RDD of Ratings. + */ + def predict(usersProducts: RDD[(Int, Int)]): RDD[Rating] = { + val users = userFeatures.join(usersProducts).map{ + case (user, (uFeatures, product)) => (product, (user, uFeatures)) + } + users.join(productFeatures).map { + case (product, ((user, uFeatures), pFeatures)) => + val userVector = new DoubleMatrix(uFeatures) + val productVector = new DoubleMatrix(pFeatures) + Rating(user, product, userVector.dot(productVector)) + } + } + + // TODO: Figure out what other good bulk prediction methods would look like. // Probably want a way to get the top users for a product or vice-versa. } From 2c1cba851c2954bacf10006c0d5dad67aba77ab5 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Fri, 3 Jan 2014 15:35:20 -0800 Subject: [PATCH 1163/1571] Added unit tests for bulk prediction in MatrixFactorizationModel --- .../spark/mllib/recommendation/ALSSuite.scala | 33 +++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index fafc5ec5f2245..e683a90f57aba 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -90,18 +90,34 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll { testALS(50, 100, 1, 15, 0.7, 0.3) } + test("rank-1 matrices bulk") { + testALS(50, 100, 1, 15, 0.7, 0.3, false, true) + } + test("rank-2 matrices") { testALS(100, 200, 2, 15, 0.7, 0.3) } + test("rank-2 matrices bulk") { + testALS(100, 200, 2, 15, 0.7, 0.3, false, true) + } + test("rank-1 matrices implicit") { testALS(80, 160, 1, 15, 0.7, 0.4, true) } + test("rank-1 matrices implicit bulk") { + testALS(80, 160, 1, 15, 0.7, 0.4, true, true) + } + test("rank-2 matrices implicit") { testALS(100, 200, 2, 15, 0.7, 0.4, true) } + test("rank-2 matrices implicit bulk") { + testALS(100, 200, 2, 15, 0.7, 0.4, true, true) + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -111,9 +127,12 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll { * @param iterations number of iterations to run * @param samplingRate what fraction of the user-product pairs are known * @param matchThreshold max difference allowed to consider a predicted rating correct + * @param implicitPrefs flag to test implicit feedback + * @param bulkPredict flag to test bulk prediciton */ def testALS(users: Int, products: Int, features: Int, iterations: Int, - samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false) + samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false, + bulkPredict: Boolean = false) { val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs) @@ -130,7 +149,17 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll { for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) { predictedP.put(p, i, vec(i)) } - val predictedRatings = predictedU.mmul(predictedP.transpose) + val predictedRatings = bulkPredict match { + case false => predictedU.mmul(predictedP.transpose) + case true => + val allRatings = new DoubleMatrix(users, products) + val usersProducts = for (u <- 0 until users; p <- 0 until products) yield (u, p) + val userProductsRDD = sc.parallelize(usersProducts) + model.predict(userProductsRDD).collect().foreach { elem => + allRatings.put(elem.user, elem.product, elem.rating) + } + allRatings + } if (!implicitPrefs) { for (u <- 0 until users; p <- 0 until products) { From dfe57fa84cea9d8bbca9a89a293efcaa95eae9e7 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Fri, 3 Jan 2014 15:40:53 -0800 Subject: [PATCH 1164/1571] Removed unnecessary blank line --- .../spark/mllib/recommendation/MatrixFactorizationModel.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index bc13a66dbea6a..8caecf0fa1ba8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -20,7 +20,6 @@ package org.apache.spark.mllib.recommendation import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ - import org.jblas._ import java.nio.{ByteOrder, ByteBuffer} From 838b0e7d154699291f9915d400c59a3580173d01 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 3 Jan 2014 16:13:40 -0800 Subject: [PATCH 1165/1571] Refactor using SparkConf --- .../main/scala/org/apache/spark/Aggregator.scala | 5 +++-- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 5 +++-- .../util/collection/ExternalAppendOnlyMap.scala | 15 +++++++++------ .../collection/ExternalAppendOnlyMapSuite.scala | 15 ++++++--------- 4 files changed, 21 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c408d5f145667..c9e3e8ec5a773 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -33,8 +33,10 @@ case class Aggregator[K, V, C: ClassTag] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { + private val sparkConf = new SparkConf() + private val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean + def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kv: Product2[K, V] = null @@ -58,7 +60,6 @@ case class Aggregator[K, V, C: ClassTag] ( } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 44494c7e0a993..7dc7094aac3ca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -22,7 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} -import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency, SparkConf} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,6 +66,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] + private val sparkConf = new SparkConf() private var serializerClass: String = null def setSerializer(cls: String): CoGroupedRDD[K] = { @@ -106,7 +107,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean + val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 9e147feec42d7..68a23192c0655 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,7 +24,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.{SparkConf, Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} @@ -57,14 +57,16 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskIterator] + + private val sparkConf = new SparkConf() private val memoryThresholdMB = { // TODO: Turn this into a fraction of memory per reducer - val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong - val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat + val bufferSize = sparkConf.getLong("spark.shuffle.buffer.mb", 1024) + val bufferPercent = sparkConf.getDouble("spark.shuffle.buffer.fraction", 0.8) bufferSize * bufferPercent } - private val fileBufferSize = - System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 + private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() private var spillCount = 0 @@ -84,7 +86,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") val (blockId, file) = diskBlockManager.createTempBlock() - val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity) + val writer = + new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) try { val it = currentMap.destructiveSortedIterator(comparator) while (it.hasNext) { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index a18d466baaff5..6c93b1f5a0c24 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -4,20 +4,17 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{HashPartitioner, SparkContext, SparkEnv, LocalSparkContext} +import org.apache.spark._ import org.apache.spark.SparkContext.rddToPairRDDFunctions class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { override def beforeEach() { - sc = new SparkContext("local", "test") - System.setProperty("spark.shuffle.externalSorting", "true") - } - - after { - System.setProperty("spark.shuffle.externalSorting", "false") - System.setProperty("spark.shuffle.buffer.mb", "1024") - System.setProperty("spark.shuffle.buffer.fraction", "0.8") + val conf = new SparkConf(false) + conf.set("spark.shuffle.externalSorting", "true") + conf.set("spark.shuffle.buffer.mb", "1024") + conf.set("spark.shuffle.buffer.fraction", "0.8") + sc = new SparkContext("local", "test", conf) } val createCombiner: (Int => ArrayBuffer[Int]) = i => ArrayBuffer[Int](i) From 8b5be0675245e206943574b8c6f6b77018b3561a Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Fri, 3 Jan 2014 16:38:33 -0800 Subject: [PATCH 1166/1571] Added table of contents and minor fixes --- docs/mllib-guide.md | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 0bebc41137c0c..3fd3c91e2a110 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,6 +3,9 @@ layout: global title: Machine Learning Library (MLlib) --- +* Table of contests +{:toc} + MLlib is a Spark implementation of some common machine learning (ML) functionality, as well associated tests and data generators. MLlib currently supports four common types of machine learning problem settings, @@ -297,9 +300,17 @@ val numIterations = 20 val model = ALS.train(ratings, 1, 20, 0.01) // Evaluate the model on rating data -//val ratesAndPreds = ratings.map{ case Rating(user, item, rate) => (rate, model.predict(user, item))} -//val MSE = ratesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/ratesAndPreds.count -//println("Mean Squared Error = " + MSE) +val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)} +val predictions = model.predict(usersProducts).map{ + case Rating(user, product, rate) => ((user, product), rate) +} +val ratesAndPreds = ratings.map{ + case Rating(user, product, rate) => ((user, product), rate) +}.join(predictions) +val MSE = ratesAndPreds.map{ + case ((user, product), (r1, r2)) => math.pow((r1- r2), 2) +}.reduce(_ + _)/ratesAndPreds.count +println("Mean Squared Error = " + MSE) {% endhighlight %} If the rating matrix is derived from other source of information (i.e., it is inferred from @@ -393,7 +404,7 @@ Errors. ## Collaborative Filtering In the following example we load rating data. Each row consists of a user, a product and a rating. We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation -model by measuring the Mean Squared Error of rating prediction. +on one example. {% highlight python %} from pyspark.mllib.recommendation import ALS @@ -407,10 +418,7 @@ ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) model = ALS.train(sc, ratings, 1, 20) # Evaluate the model on training data -#ratesAndPreds = ratings.map(lambda p: (p[2], model.predict(int(p[0]), int(p[1])))) -#MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count() -#print("Mean Squared Error = " + str(MSE)) - +print("predicted rating of user {0} for item {1} is {2:.6}".format(1, 2, model.predict(1, 2))) {% endhighlight %} If the rating matrix is derived from other source of information (i.e., it is inferred from other From cfab8f2062fad9ea400716afce28d200dd714c2b Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 17:55:58 -0700 Subject: [PATCH 1167/1571] Revert "Create VertexPlacement on demand in VTableReplicated" This reverts commit 32f957f3317bd62768b415da0c0cd9114f59782c. Conflicts: graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala --- .../apache/spark/graph/impl/GraphImpl.scala | 55 +++++++++++----- .../graph/impl/ReplicatedVertexView.scala | 65 ++----------------- .../spark/graph/impl/RoutingTable.scala | 64 ++++++++++++++++++ 3 files changed, 109 insertions(+), 75 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e7c4b5db82603..16d73820f07df 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -19,19 +19,28 @@ import org.apache.spark.util.ClosureCleaner * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` * contains edge attributes, which are edge-partitioned. For operations on vertex neighborhoods, * vertex attributes are replicated to the edge partitions where they appear as sources or - * destinations. `replicatedVertexView` stores a view of the replicated vertex attributes, which are - * co-partitioned with the relevant edges. + * destinations. `routingTable` stores the routing information for shipping vertex attributes to + * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created + * using the routing table. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], + @transient val routingTable: RoutingTable, @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] { + def this( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable) = { + this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) + } + def this( vertices: VertexRDD[VD], edges: EdgeRDD[ED]) = { - this(vertices, edges, new ReplicatedVertexView(vertices, edges)) + this(vertices, edges, new RoutingTable(edges, vertices)) } /** Return a RDD that brings edges together with their source and destination vertices. */ @@ -76,8 +85,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } override def statistics: Map[String, Any] = { + // Get the total number of vertices after replication, used to compute the replication ratio. + def numReplicatedVertices(vid2pids: RDD[Array[Array[Vid]]]): Double = { + vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble + } + val numVertices = this.ops.numVertices val numEdges = this.ops.numEdges + val replicationRatioBoth = numReplicatedVertices(routingTable.bothAttrs) / numVertices + val replicationRatioSrcOnly = numReplicatedVertices(routingTable.srcAttrOnly) / numVertices + val replicationRatioDstOnly = numReplicatedVertices(routingTable.dstAttrOnly) / numVertices // One entry for each partition, indicate the total number of edges on that partition. val loadArray = edges.partitionsRDD.map(_._2.size).collect().map(_.toDouble / numEdges) val minLoad = loadArray.min @@ -85,6 +102,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Map( "Num Vertices" -> numVertices, "Num Edges" -> numEdges, + "Replication (both)" -> replicationRatioBoth, + "Replication (src only)" -> replicationRatioSrcOnly, + "Replication (dest only)" -> replicationRatioDstOnly, "Load Array" -> loadArray, "Min Load" -> minLoad, "Max Load" -> maxLoad) @@ -121,13 +141,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("\n\nvertices ------------------------------------------") traverseLineage(vertices, " ", visited) visited += (vertices.id -> "vertices") + println("\n\nroutingTable.bothAttrs -------------------------------") + traverseLineage(routingTable.bothAttrs, " ", visited) + visited += (routingTable.bothAttrs.id -> "routingTable.bothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) } // end of printLineage override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), replicatedVertexView) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), routingTable, replicatedVertexView) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { @@ -135,16 +158,17 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, newReplicatedVertexView) + changedVerts, edges, routingTable, + Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges) + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) } } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), replicatedVertexView) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), routingTable, replicatedVertexView) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit @@ -162,7 +186,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), replicatedVertexView) + new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) } override def subgraph( @@ -185,7 +209,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, replicatedVertexView) + new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( @@ -195,13 +219,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, replicatedVertexView) + new GraphImpl(newVerts, newEdges, routingTable, replicatedVertexView) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newEdges = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newEdges, replicatedVertexView) + new GraphImpl(vertices, newEdges, routingTable, replicatedVertexView) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -281,12 +305,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, newReplicatedVertexView) + changedVerts, edges, routingTable, + Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVerts, edges) + new GraphImpl(newVerts, edges, routingTable) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index bef99810bd089..175586b87eb16 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -1,13 +1,11 @@ package org.apache.spark.graph.impl -import org.apache.spark.Partitioner -import scala.collection.mutable - import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} +import org.apache.spark.graph._ + /** * A view of the vertices after they are shipped to the join sites specified in * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevViewOpt` is @@ -22,6 +20,7 @@ private[impl] class ReplicatedVertexView[VD: ClassManifest]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], + routingTable: RoutingTable, prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { /** @@ -50,9 +49,6 @@ class ReplicatedVertexView[VD: ClassManifest]( private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) - private val routingTables: mutable.Map[(Boolean, Boolean), RDD[Array[Array[Vid]]]] = - new mutable.HashMap[(Boolean, Boolean), RDD[Array[Array[Vid]]]] - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs @@ -70,7 +66,7 @@ class ReplicatedVertexView[VD: ClassManifest]( // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is // also shipped there. - val shippedActives = getRoutingTable(true, true) + val shippedActives = routingTable.get(true, true) .zipPartitions(actives.partitionsRDD)(ReplicatedVertexView.buildActiveBuffer(_, _)) .partitionBy(edges.partitioner.get) // Update the view with shippedActives, setting activeness flags in the resulting @@ -88,7 +84,7 @@ class ReplicatedVertexView[VD: ClassManifest]( // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD - val shippedVerts = getRoutingTable(includeSrc, includeDst) + val shippedVerts = routingTable.get(includeSrc, includeDst) .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdManifest)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. @@ -126,19 +122,6 @@ class ReplicatedVertexView[VD: ClassManifest]( }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) } } - - /** - * Returns an RDD with the locations of edge-partition join sites for each vertex attribute in - * `vertices`; that is, the routing information for shipping vertex attributes to edge - * partitions. The routing information is stored as a compressed bitmap for each vertex partition. - */ - private def getRoutingTable( - includeSrc: Boolean, includeDst: Boolean): RDD[Array[Array[Vid]]] = { - routingTables.getOrElseUpdate( - (includeSrc, includeDst), - ReplicatedVertexView.createRoutingTable( - edges, updatedVerts.partitioner.get, includeSrc, includeDst)) - } } object ReplicatedVertexView { @@ -188,44 +171,6 @@ object ReplicatedVertexView { (pid, actives.trim().array) } } - - private def createRoutingTable( - edges: EdgeRDD[_], - vertexPartitioner: Partitioner, - includeSrc: Boolean, - includeDst: Boolean): RDD[Array[Array[Vid]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() - val numEdges = edgePartition.size - val vSet = new VertexSet - if (includeSrc) { // Add src vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.srcIds(i)) - i += 1 - } - } - if (includeDst) { // Add dst vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.dstIds(i)) - i += 1 - } - } - vSet.iterator.map { vid => (vid, pid) } - } - - val numPartitions = vertexPartitioner.numPartitions - vid2pid.partitionBy(vertexPartitioner).mapPartitions { iter => - val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("VertexPlacement %s %s".format(includeSrc, includeDst)) - } } class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala new file mode 100644 index 0000000000000..b6cd048b33bb1 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -0,0 +1,64 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.collection.PrimitiveVector + +/** + * Stores the locations of edge-partition join sites for each vertex attribute; that is, the routing + * information for shipping vertex attributes to edge partitions. This is always cached because it + * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and + * (possibly) once to ship the active-set information. + */ +class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { + + val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) + val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) + val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) + val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) + + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } + + private def createPid2Vid( + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => + val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val numEdges = edgePartition.size + val vSet = new VertexSet + if (includeSrcAttr) { // Add src vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.srcIds(i)) + i += 1 + } + } + if (includeDstAttr) { // Add dst vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.dstIds(i)) + i += 1 + } + } + vSet.iterator.map { vid => (vid, pid) } + } + + val numPartitions = vertices.partitions.size + vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + for ((vid, pid) <- iter) { + pid2vid(pid) += vid + } + + Iterator(pid2vid.map(_.trim().array)) + }.cache().setName("RoutingTable %s %s".format(includeSrcAttr, includeDstAttr)) + } +} From 333d58df8676b30adc86e479579e2659e24d01a3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 3 Jan 2014 17:55:26 -0800 Subject: [PATCH 1168/1571] Remove unnecessary ClassTag's --- core/src/main/scala/org/apache/spark/Aggregator.scala | 4 +--- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 7 +++---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c9e3e8ec5a773..bb488f4ad82ae 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.reflect.ClassTag - import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** @@ -28,7 +26,7 @@ import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ -case class Aggregator[K, V, C: ClassTag] ( +case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7b1759ebbc192..f8cd3627952dd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -72,7 +72,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C: ClassTag](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -110,7 +110,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C: ClassTag](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -338,8 +338,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C: ClassTag]( - createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } From 9e6f3bdcda1ab48159afa4f54b64d05e42a8688e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 3 Jan 2014 17:32:25 -0800 Subject: [PATCH 1169/1571] Changes on top of Prashant's patch. Closes #316 --- README.md | 19 +---- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../scala/org/apache/spark/DriverSuite.scala | 3 +- .../org/apache/spark/FileServerSuite.scala | 77 +++++++----------- .../deploy/worker/ExecutorRunnerTest.scala | 6 +- make-distribution.sh | 3 +- project/SparkBuild.scala | 1 + python/lib/py4j-0.8.1-src.zip | Bin 0 -> 37662 bytes python/lib/py4j-0.8.1.zip | Bin 809541 -> 0 bytes python/pyspark/__init__.py | 2 +- 10 files changed, 42 insertions(+), 72 deletions(-) create mode 100644 python/lib/py4j-0.8.1-src.zip delete mode 100644 python/lib/py4j-0.8.1.zip diff --git a/README.md b/README.md index 8f686743f042b..873ec9882f9a8 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained from [here](http://www.scala-sbt.org). To build Spark and its example programs, run: +which can be obtained [here](http://www.scala-sbt.org). To build Spark and its example programs, run: sbt assembly @@ -38,24 +38,11 @@ locally with one thread, or "local[N]" to run locally with N threads. ## Running tests -### With sbt (Much faster to run compared to maven) -Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows using sbt. +Testing first requires [Building](#Building) Spark. Once Spark is built, tests +can be run using: `sbt test` -### With maven. -1. Export these necessary environment variables as follows. - - `export SCALA_HOME=` - - `export MAVEN_OPTS="-Xmx1512m -XX:MaxPermSize=512m"` - -2. Build assembly by -`mvn package -DskipTests` - -3. Run tests -`mvn test` - ## A Note About Hadoop Versions Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c6f6883b01152..e80e43af6d4e7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -173,7 +173,8 @@ class SparkContext( value <- Option(System.getenv(key))) { executorEnvs(key) = value } - // A workaround for SPARK_TESTING and SPARK_HOME + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 89c5631ad8f07..7e1e55fa3b299 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -30,8 +30,7 @@ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - val sparkHome = Option(System.getenv("SPARK_HOME")) - .orElse(Option(System.getProperty("spark.home"))).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 506f7484fb731..a2eb9a4e84696 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -27,54 +27,39 @@ import org.scalatest.FunSuite class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ - @transient var testJarFile: String = _ - + @transient var tmpJarUrl: String = _ override def beforeAll() { super.beforeAll() - val buffer = new Array[Byte](10240) - val tmpdir = new File(Files.createTempDir(), "test") - tmpdir.mkdir() - val tmpJarEntry = new File(tmpdir, "FileServerSuite2.txt") - val pw = new PrintWriter(tmpJarEntry) - pw.println("test String in the file named FileServerSuite2.txt") + val tmpDir = new File(Files.createTempDir(), "test") + tmpDir.mkdir() + + val textFile = new File(tmpDir, "FileServerSuite.txt") + val pw = new PrintWriter(textFile) + pw.println("100") pw.close() - // The ugliest code possible, was translated from java. - val tmpFile2 = new File(tmpdir, "test.jar") - val stream = new FileOutputStream(tmpFile2) - val jar = new JarOutputStream(stream, new java.util.jar.Manifest()) - val jarAdd = new JarEntry(tmpJarEntry.getName) - jarAdd.setTime(tmpJarEntry.lastModified) - jar.putNextEntry(jarAdd) - val in = new FileInputStream(tmpJarEntry) + + val jarFile = new File(tmpDir, "test.jar") + val jarStream = new FileOutputStream(jarFile) + val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) + + val jarEntry = new JarEntry(textFile.getName) + jar.putNextEntry(jarEntry) + + val in = new FileInputStream(textFile) + val buffer = new Array[Byte](10240) var nRead = 0 - while (nRead <= 0) { + while (nRead <= 0) { nRead = in.read(buffer, 0, buffer.length) jar.write(buffer, 0, nRead) } + in.close() jar.close() - stream.close() - testJarFile = tmpFile2.toURI.toURL.toString - } - - override def beforeEach() { - super.beforeEach() - // Create a sample text file - val tmpdir = new File(Files.createTempDir(), "test") - tmpdir.mkdir() - tmpFile = new File(tmpdir, "FileServerSuite.txt") - val pw = new PrintWriter(tmpFile) - pw.println("100") - pw.close() - } + jarStream.close() - override def afterEach() { - super.afterEach() - // Clean up downloaded file - if (tmpFile.exists) { - tmpFile.delete() - } + tmpFile = textFile + tmpJarUrl = jarFile.toURI.toURL.toString } test("Distributing files locally") { @@ -108,10 +93,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS locally") { sc = new SparkContext("local[4]", "test") - sc.addJar(testJarFile) + sc.addJar(tmpJarUrl) val testData = Array((1, 1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } @@ -133,10 +118,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS on a standalone cluster") { sc = new SparkContext("local-cluster[1,1,512]", "test") - sc.addJar(testJarFile) + sc.addJar(tmpJarUrl) val testData = Array((1,1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } @@ -144,10 +129,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS on a standalone cluster using local: URL") { sc = new SparkContext("local-cluster[1,1,512]", "test") - sc.addJar(testJarFile.replace("file", "local")) + sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 894a72284b3fe..f58b1ee05a64b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,18 +19,14 @@ package org.apache.spark.deploy.worker import java.io.File -import scala.util.Try - import org.scalatest.FunSuite import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} - class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = Try(sys.env("SPARK_HOME")).toOption - .orElse(Option(System.getProperty("spark.home"))).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.env.get("spark.home")).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" diff --git a/make-distribution.sh b/make-distribution.sh index a2c8e64597143..8ae8a4cf742a1 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -45,7 +45,8 @@ DISTDIR="$FWDIR/dist" export TERM=dumb # Prevents color codes in SBT output if ! test `which sbt` ;then - echo -e "You need sbt installed and available on path, please follow the instructions here: http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html" + echo -e "You need sbt installed and available on your path." + echo -e "Download sbt from http://www.scala-sbt.org/" exit -1; fi diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8290e7cf438c7..5f57c964bde61 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -72,6 +72,7 @@ object SparkBuild extends Build { val sparkHome = System.getProperty("user.dir") System.setProperty("spark.home", sparkHome) System.setProperty("spark.testing", "1") + // Allows build configuration to be set through environment variables lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..2069a328d1f2e6a94df057c6a3930048ae3f3832 GIT binary patch literal 37662 zcmb4qW00lml5Uw@R(IL9ZQHhO+qP|2mu=g&ZCkhZnK^f6pF0yXcSYnE>(7_#ja+X& zPf}I_7z7I7uTNqGf#5&B{QCnG029F0jnSM&Nf8nNc*I~>{`HSIxk3W~f}8*Y0Q}=2 z``^9}{m<7OjT{^u{zHka;eV+F`OmLH08DZd@;A3k22KCb9|R2mfbn-F=6cS0I#znN zI)CY;wsrf*$>`sYWdD+XIoYN1woL&my!UbS&1{Hoq5$NkR~hx(5#pM4b#ocBQj`e}AF+Qsz!IK%djA7H>1%Ch;U{;tu|W`SF9aXjcT zl;^-?i^*str1(K=^1}5Z765nW{w5InPGw{M`jK;zH~e|@G6vbQ6Y@gPeFyMyqV~$< z;S`V+a}(ml@^Uw+;%P@1)!NL)RppgRHb%d~sPpF{)u*|W^6TiRu(vNeuv$r6fmz&f zW1?oEd#1=siOQIaM6fMPaYl8nsc!}!#*uZT`tR}L4W()~lF4`BT8-2owFt^^SyI0Q z=zDi!QJ))ZH;|xv?=joZkcKjoDz#Px48PH#T^}znus&CaTh@vn{LSf5{EO)1$fhqf zo^w6$L8`cz=`iy$GB6l*a%Cdmv#7aqK=Gi^PGVwGQb;-BW)_bJDOIUwhZX8!0*oN$ zdgn%Hs!i%~Q!xa9q|)bDcN_&44`wzu#a9?iq#G$n;fX}@qkk#d2@4|WdA&^%KS81k z*yFptZ$T*2{l2JvLLwcC1Q04N+GEHLz(L{9%PjuL-kJ<8Y?MksFRuzBAkzSLT{uX- zoeV~`MeCfo2r#9FIZACrZ1681Mk+!=}aULvzNXL^eYp#FUxJL19RH+ z3UgA;mOiBb-GH{}^N5!t&y0KSGAYD+E1a|gXHVI08CxKv6o(2lLo6UAJL=gXX7O;a za#soIbnV2FxmN+QXpJpbs^}^)EV0=N-&)@^DTeEmFPh*Wv41Zy7L_ny3I%Jr9`Y+0iU!4K%Ayw0uV)2sUK;hXe7Tbn4$YSEHi&>au>e%jr`iC(k;U+;_A zzS@0}4UUFu=i)wj@(?@AbP3KlvaEZr4g99G4?M~g7clMHF-GLN*_(O@JNN}Ej5z8r zeXovwqXHin%7sn^ucGmCkMITub(APK`6{4t$V=FL8qtk@PLqI57Pt_IlOcEQ0_Ihh zU_%q98TAetPtpB8X%>WXBF`$10-xcydo5cX$heJ=3i$Gs}C1hyVZ%dH?`8 zf7ka9%rVh(G;+~%`y1{wakt3-CI5;$oafH#L$lwn$QY;KJ&F9Fv2`d}1=-~EDC3Kl zNMV^EMlKD1<3vT{Q z7>^DVqb)d8jYmb^`&A0Om}ugV+qfiQF|~l7*ES-No$y1owPBr%I92Ki?C zA$!e5cOASf)j>1{YPJG!ujp-y43K7CQDBA%mKW9)^Dl&k{HjRm9o92(An~ zvBe}D^`aO?ZzRe}BPNyQQq+pqGXmdzIJ!~kr|G4lE(raJA)7VwytF<*oJS64GKF9VETBXd z2n4hHIk`S%aFVkSkae~`h6feunlA7^uqoPNS1edh(~ZL4ksPTP#)%}n zPKA#U1LqSwEo>>#iMk@)dotg^xR^^Swft8Zv=K~(Xp`Ot00nP2QR(k^3HZHEw??*U z<=>5j;Y!xzTZHU*>`(j;Um?H4an%PteW=Syt~-pf(IWTcUY>Cucw-6>28N2JS=%n_ z!AVO!i%lFed{l)CsY1TT9VSchYG-`Kz!eh_OnyfI<6@xOV~L8V88n8^)(f&9{1BH( z*3VxrTLBx;Nt3^OPy#iZ8i^6{hT^>2#SvVlY>y0}kc5%rNo!l(Eoc*9%%jYyfI8uw zSK8^fnP)AHBQOB?;Oq7OjHCG#>52A*Ll6^(1Bo3!Is%6<6EX|QS@;3!9`(kcmpw)P z!($xtXbTM!3&~i@Ea43Rc|rJqu4zW9NJ)w(EcG0-tat{B6c`vs8FvjTvFg{vaBEl< z)&A*L-jcQNP*O9@Uh+raZ7Aa=7*zWXCwz)7EI)52YXolFO{jH7WST$<{D5Yv2`|9C zZ8QL2G=FHu&kRsIyfUk0&>M#H9igP7Zc-Mu3GLYZK?18?SH6L}0DOkNaJDwi-+dcA z@D3dY7f#89@9Kz3)hhyYHC3&K{uO&?KcTwEY*(fFj68q=XT-rO(Z2>Vz~%fww^rd{ zf@YRuledQA0&m;!gsX#_;K53sR$`j)XZ6UG#b&dJ)5pdzd=}2yd7Z_zYCr^UYln{r zZ8MF2)`cbWQx7z-Q#DQHJ+(ov;G>^*!7I;`_FsaIDk145L=GTkUb{v7Z)#W1!&vHV~VM(xxk^d33`o{LP|9 z9C1*kmuv-?g?T{SiH;6mKRq1JL0~Ig!2N;lo&QXWexmb!Cwn&=@sR{*1rDM)pC9th zW#p^XQNT$*AHBlSctV0ouRZc0RikE@wYqou*`e#! z)ODeo0gR5e?cI$nYY#il-`E#!FFcnSO5}?a6U`75do&YQEDiI*;up4$%l1Th)kF&J2P;8k+P&4GSm_S0`Tc>5m1%Y{Qr= z)^$`V&ClY~q9(#(4{V_+HFuj>lB(DpNN7n(5kOUpN2{xft*$^92~@cfh#X0pzZ;s= zIXhysrlVARUheYZw7Zp}?69*)YMDi1*RVnQ$9C&ME-^@4?XQ7)kd5R#q|qKtZOjr`-nf zSfqzZn|q`)^kFo0LMo0EN1X#N#9{T~WN_fe-lSHiWEM%oX^N~X%dW3k<9k#sDeU(p ziwO)Z5l;c=Nklst5SVzj#&xQ>7*_9~7A2p=erUE$BqbOWR)BKActViP)?0~wS1)m* z`T4k9p}}TwR~jlU5#Od-ZX!3P_%%&zop0$qD_z&H!RlzrAz09|S<-mie55z#7Zh5B+r z3$k~pB50ZuK>2Hswl9 zT{?G_k+TPE)b90IoU`9F1{1f3%X81Yh^E6H25jThCZ9W58O>zcL!vO^&0QCWA`t28 z!bHstmoaBYCdnk`T)K~O7T5nB2nT7B)Tc&1=#M{yI~sUsi`eCaj;XO2s+^ZE*tz3) zwu=7)n>yB_NX&6{S}0XUoRyy8OeS`bmM7PtI1w|sg;+~2DAH^wTNzR)-Pm;NeV}`R z&?+f_lnuSLeTHqh@4mOY6*_CLIZ$JijcQRCNTJpTlFdTG9&}!vMo)uVv!^GA`9!Vp zcSE*QB6{SBm=$<|5~0h;6{EUdTkrwbpor^Z>5!Jle zIdf2PtjRDdAGb)5tN4Zp_XggC8hPNwy`_rHY8h|ztjK&7!qC)0ERcMPSe?@+!sn1& z+H%5C_n)EqVqW7{H^CI$h7l#NvVmF--zBg-OVP(@qy3SuAM|8PV%T&0U>H2Tv8sGbyj;) zI2pNbhl0pCR)?N{1K|iC&V>csn3ca@tE>`|F2+WLFfm>pW<@bjJFcT%o7QahX*rmZ z;ZS#HCO?IZXF0MbMc*S;Q^yi$T!5Ti6((F-+<~F4@6yPWu_-_qO;M`4Bl)wd*w_i@ zy9r<0tpfMmVZjW$ifLC(*H4%QX1Z>G#>!91O(SOA!>X{GEh1cKFh34R?Hxn%DHAl8 zIjH>e$KQB7}kHJ zbvTkF^7Nq`Zg?94#i~T4<1>K2unW$}L!_VW(oHJ{{pKv+Ps0$D06Y_{AE>KP;Sia& z1qi$duLzl+j$!+GuMowqtLf3QGtQIxD)t3HLLa??OseWy6RxjpY`9pAS!};VfQ`Ya z`Em=-`92q<3)mm>QB+fzAs1lg)F@45V!iK#Ie_F7BA>&$NAQS7X%Mi@nhYwk%R7)r-T9Jfu)uHVwC}2Yk2w9GX9=a-;U^;AaY;z}}K+ zrTl#+zi<&-y4&zdyt^=HA4Rqh@R-IYlr~5nGfX}YH{vKvBtt3Tn2c!(M*@yX zRA-nT0q6l~e4q1P+(Z##51SC zQ&)GUjpt;dv$I!a*hY=sEZnFXnD4R39a|z@UaBPXesbknx?OVrR*U8?3z;xePV;9( zgayTPIa@}9%+Mjv#r%0i@6Ca^X;ILj#t~JQN-~f~`xqh<>G2T8Jg?G8NFKC0(sk@O z?hddwE_dIC<-P|`%;0+yv30}E3AqoNLCUu0!ZOHtQ`f|DU9?0V#e49xx;3e*=!$NR z^Sd=QNQ8maSAp_{qu$Y@BN66a{Fg9RqA;i&1|wVDiYy=y>%7%g?J=!$^YUj|VF5kV z1~r}Zh(YvN&vds&e8B2mWtY73BUi+pzZ)0@3DWRjvoLvK`B3#oq8FwoS%)a1snvCY zrvuP4k^HSaP4Nhg;Z98R*3)j-x)mDkMp-I;<0#zl0G0fRb0|g=vhfe{PkGn)({#?& zvqp_}x0X4DK7KwY33#bMB>zg83j6u*WyAQ3bmGutH$KODG!|ns%;OZXZY6Gi(!TIs zsOEvv5GAOt7nEIZ(N3Ws!kT=semfN_J@gz$+#im*OO;RcS;GzCsi8x03Eh_$7v;PK z?J!XdE&*(h^qUq+&eW$E2wD3+CBof@;&M>gSv zhT<9RuScF-Iq)!Dmj1QqJ&tUWi0TKfZMhQLX|*-I658o8C3S-#cX!>6kpl4|dGS=S z=%9xdf$>spy0^$nIYdGGF&|0?)$aFMcMeucdkvZs&#F!<%-UJowU=y8Uh7*3d%u;U zD`)1v!T+Opf(4fcsw*uMG4oE%o&bEdEx#%Jn;z{Y(C}dZlJ% zv(60vb*0nWfQsQ}ljQjN-^4p+<4Va5z{KYeB$-*xVZ`8%Xn{)l^AeDPSl7|B|fN>C)!feoT& zZG%hNW5PvmqSpwD;Q~4T*6co8PLiP$l&;<@Oj(UR&GF2h_WnI3jI3h-tsfFClhb4{ zQDuFy>he3`DiuoKpL+vbzdSKpjsr&JFp6A-Hyq>aa;W!(3w9HJiL+TkPy&dEK?DSD z!W*wvqeE0NNk)hC&%A9Q7pGe-cwQ0RD9q9vhgOhfDX=S;C~o|SC;Y)V0gTQ2_4^rAsrr<3Jkar1t!QJB*apGLP9)W#1VfRBY8!#e~#l_7(VL+*=k!5;ja1fN8 zRp5Gx_+)bRy?+)&YHCJjxTM6?o{|6fdOEn^_9s?%Tuj2OE6`j)Z6c1(Pgs!x;4)G^ zFDwMqpIJy-)d9WKfmm$DC^C_}oa}|0>ud|L{`+utNvvq>#GY7TANq(o0r_q zSDEmslK+imq4B&Yt?IV@QM7MX^s5IpS*dC)QmI1N@&IX;%FmSdgFC{Y6hsJ*QPyb_ zfIRb@Tr>^Y!AQ!Y0Awry(28!AOiJyYCL5CtVT1lLktk3a9-ikA5DHTmDY-0ZoN5~C z2!lHm3wpr1axy7zbHxWT4X#b)D2s4kS=UCiaudMKivKObz~}RU?S@9E1vJChn6y~) zeAQ7vQ=#XkjZwY`s*emmBZ0_-vTo8UO_sTpg325})SfIFE17$hb#D^B1i#>S?6G<| zQ1@Er7{ZXQUF~grTDZM^%x`H7Cl1@ya3y;JEx$W+&*d&kD~CMsaAa|cSJ5s_qZJa# z&V0U)ku*dL-#A+8Qgbq?hc#_L&g5IN+Eqz?TSeHixJejOrHkh1z>$;}B%{1-f5`-U zFheC46$_4G`J11SR_S7P-iGS&D8R6hOi2W#)tGA;qxilGF5!i3SxcXfd7S%?DK8es z>&sWo$-pPVKlflaYI0IH!ig}{q-0f;S(S*-Sv&gzZ{Wg??is9T3LWII-|vJS(Pigk z4ml*UAJUzbmH>rH4uS)dv+ZfuzHM8`LcG2q5Je!$8xB$hD23lDJ7F`$T9*NyReE&$dd%<)Mdv7M4|XLpsZb^Mk9 ziOM9?B*v;{?fLXUCKnk)^A0t^JpD)+Q+}X$#qH;k-P$H23YwVGbGq=rM3WRHGEuWN zJSpRqMELgg1E$w*NcG52KYe1Ul`C}YHsKOZFyvR(&ER81evR6iD)kptQjL2TK#1kd zc-O!}gaWhYSe;FI%^(Edo9tJ$M5djxn0xRci>S38Ih zoG~NVuyoD?!BlC@5;Iv!p;Kfb0tSl#GnH~>3swoO(md8ppNKNodk03qf1mR85z6cO zxblIRtm0j@R!C{T+#_vq7~x$nk;;g|l+qJj3L=7Te0dvXK?=qM22A{XTSSKos_fTI z?4(e>G8~Cc>j(S5lXqWHa+B+4(2yi!VvWSlk_D`$nFXLhGRZCe=Ehj_Z^U=>_<}RC` zI-SU6&%0qQfu~D~%@#@iB1-Pk2x0S)2m@%T{W$*NoWx*7)F;1+LHZ(j;Q+N-hX;9l zb=7-Zu6B20WU??2KtMq@d=xvWgZyUXbdSo-)b z+9vr|!+-2iNxq={2buoWS&FB+G5GjHrsre;f2j1|JK+D;!v04m$kNQg@oz->-?g9B zwHz{7k-eU)WwXF-BYs6*Ixp34tuor(*QOCGv}auzaz^SO{6r%_6-4}gPZ|gTh1hFS z+X$JI91?xbt|+J=zyk@-Z;%E8fvf|3>LFA2-Qs{r4ZhW=cqEA&fdVJ61qZIrs zaea2TPYZ4hi>NI8DeU6i70imUu+}IB0X0AKMDd;_Td+%O(vo8hiC z*H+&mep3j8)X`%tY{k)Hn4N&z<2MxXA@55|uU7>bp=5;cg{`TcGOQC_nX$jx3dY|e zsN51B8imuTTwzo|1OX}49F)B?^c?jXR9eU04e0Xwet_jT5a z+A%W|HE5{$G_ZGH8h_aZe?GPCL0q88owcZdqBIqcXc6o;dzJBi`MriogN2*`Zvz3EK8{f$vj<7OkCvu}X*Zp}@=) zvX>M32^1HuiWFW+!gJ2_nD5&qv9DGk>i>5+?q5AhPAMS$!X=UiHWpH5-vxC zJ0$@eRR6+zLrQ6cBH}%$)BX^0w}oMN5MU*WlV;X+w)qPN27c9DL(wV%lBW2C zeX1xL{4&3ffjpR@aApni-Sa@d!^4BwP2NtgEzbvnvtMvge{*s!{z!nj>>lfMdXN!B z<4y{>bfnk%q$y^nW2(klV#mnOF!bUH^ktPV5fjX=BJ*Bqc=*RJMXn6sq=gHlyr1zhk6HIgXh<4ncxSJz zzJD?YeHMazex9?{LAL8wkh*6&QK3VYl1ezAUvkTnj80o(o6n;+Q8bxigY zQh<)3<5Y2_xl_)kE1d-?Y+2|5O@ z>cT3SJzNO`Mk{?-?OEsIqdtWq9m9M|j-QPM1A(fTey37!aGoizn}p9%P*Frg27BC{ zm7KM?d00umu`)tJIy%7;4opR;m=uh$rD_U8SuLIYkg04XLs%J^J`Z!F|D`o`gqJ79 zxZRL7B$kR&F$HETvuZCmh$>87P>In)11~qAPA&$Xwq)8HHe4ClGVkQA70 zHRbR<0Dy2vGupkjpx2rU#xxMtO@={SSQE5hE;CCmHFKu)n(UeF=W8KwS54B$H53Qf zA+S!m#YAS6w;IFV=l(ihq7Jfdg=FgJsgifrTj&l{S_{Y1hX+kLY|qDohk_z9fsYlb zO^v3MM`!S#%I;7J&h%qqWWkHRE72-}MTa1&8yPi^@_7^`@*8Q1FlOehdaC@QKj`w; z_fhLNLMP@Hw1iwZ!Q^15u8id%l8!Ljst2{IcvB|X9Ggo9JF9&qF4#I!(TjAjN``2* zG5SLxRa``m>}rJgo@=X(LmoSBDr(N2>^n?vRrHLHLKnB5wX>J6M*3p0RFTQ+9|iW* ze@ium&l$uH!SXpXkaxOuB}@>tRyL52<`OvoV&}qm^WGUBHVIh9Br zRoeR0c4c;d>oTcze19jK;tR;oX{B}-4k8uoM`%&?H(W5~at*}mOyJqExn0&V40?%h zgJ)-++gpx};k2>z`3McLsUDEk7KLwMnqgQ$MTKvZ6UKcd4FoGRI+?v;Y;3CVg_aV|=oTs|QIw?1gy; zNg!^sQo}fLO;>@J@%8HAIyfSiA| zDqttI+EF?%x{c`&6pTttP3JNn>w?eP_XgoC`kdg{dVK*vOuRARIPlHKK&3x0T*FL=(1LVU6S)*n}HC$psrvOC_0LwN$t zxeveF9UQjgJ{WgUlC{w37rWvr-D=GpJ^A3?HdWv}HYaL@)fGs$A&B{~HuM_Md$W*n zokzo$fk051YtIBNKApM&|3*H|_^HzLu~keaKV|>;shLF9Yee4lhIn-_`~?ihZAfmO z)a5Wk?d;USTjJPkeavy7!R_h2Ih#pTrDK5-6)&ePAjKmJ{xk?SZ5jG(o(atSb5)!R<_<=x9%Rpuc3P&X7RG+lh1ucicBI=F#M;e>I2eVZxJ7GL#u)o2hwH$}lI zJlgGOYd=?ACc%fXTW1&7AaA-iKo)uNi7clKgis3C*_BM`b*Rf5%ipiz-kP1b*PpS% z!5;tsSbx8UR!)|VW{#%zMtX+-i^0N0m38ZNR(P*vm71#n!X6ZR4aeCaKPN0Ev$F-h z-^gmnfW4vyA!I~Qgog`0-2!rTMl4e4Lg7}RE2yJv4r~nRabDZc!ceJxkt6<+d4%oN z?}j!eF2na<#!0PlXE3eCV;jD{o~N%6sbpzL;wDLD$g<+D+Tuo{N;iteyr3+(sWV=E zRgK6=fMi7ECOwE<)49ySeMO9T8fFaB*C)U9&%G_lv>URAOH%qX; zfZG~1sNt*&6px4)sFZ8;DfM64?a%pT8R+{~Mm`92(M`DKE0 zf*&kXF+H+VdLvB(IOSi)?&JX$b5V2sZ=ytQ)I)w1Nrvt?OrKpnid`$I?@br;Z&Y=f zI-Zg`v-H;?a~cJY=ewY6StKVBSy47N>BNtZOW3pG;gCSOv9R%kBgIROggo!liA5XO zJ9GaUoF{YpuF@vHW#l_`RUen35iVfm{*97kI8#i+Ti`$&@c!q-wu(8R)8c=D===%Hn)_h?KJm)F8tkff(VI zx5DZGiM`SfA}y1HTLqe|0x|`Cda;l$_mYB~3M20^!I6?YRN5fAH7}o%abJmwI3L$# z-ul#0>eloNLQtc1blu95RKVM!1fNXsUB$BL?<N2#3(@Jt1Zu_s z&V+)_O#0~TAQrS}MiTwwOQo^*Y@`PhTPt=!T2`9ZpiXdCLF;u04qrp2UaA2wIi7wlUfewHq zIb&{qH9Oveab0DS#<$xpr3Mli*739^{SDE^<1bl?I=p5NZ};Z(j+McWND|937dg90 z&9{skFv_{JjcN|o4onGOzz@oI_^!CjD)q;-z`s|Kn+PB=RLRP`#qZ#}I}bcIV5|HOpM3gsdEFC_BGD3`(p9FPsC>%4 z`(y%XqM$M2uAjqf`pb11tP0Q_tUfBx8_f5VQPf}mC0P?Z@T&#-G~aiLaiq&6fApio0-C+JPf*LDnh+Qebi|1u{3h#Em*KN;u_47(d0 zxq^<5D%#&SkrXPxe@d=Y1|Tj5Ph2ZY809b20Wt;*>Th=n|3q$(O`glvH^R`LSb>Xh z3sm?czfprTnK9;djx?FxOA3~N9HMt+SrN)mEbC%wd()*!&oQdgYPE{6b9hAxyByD{ zn6pO#t32_8y}RT&AP4x3^~MIKU$rk>G)N)uCL{^}! z+!{Hle&*nXrJ->}m{XkT4~!k>H2S-$92@Sv6??bwd?Jx@L^dT?{66|MbEWkHqgHoJ zl3*RbMh>k}K*aV-ZAMXXm8bLIfdg|`sMMc>>!p~3c_56xAp#)gFZKtjJS7(8kA0|1 zl_kpk;7N1kew)g&XdVZPhG9wV;mW%cVC!o9Y*dak>w+~##Y7Kv7`#17LLa%pyQ7wu zQY{9c0^xV-Rv5a>7GM>0)-G)obJ}r(S+PR#!VyyM)SkA7x~#wY067%`JfC8}M>k3e z|5S~ntC%IuznwgUgk(ScCe*}Ef$vQ=wxd++>OPy~IX*qQFT0bTtlPhMeKwLnI%FA@ zqsUV+{em=2J@rCrd$HG{XBa(J{ls%(8;>Ewha*q)!ZB3xzZ3`AvaK{8^f_2^Vgs36 z=A07|Ru^Mun(G^OLMhon!`ZEW(p+DIP?k*hd8Q$fLdp5KKSQK1&-7#+<-ffONrfi-;G;V@vcVu>j~O&1-QuGTI?V0kJ%C%nYaqp!PEQvyF`GL_pH3<1J>-SG!j1whTto*-1< zzu*+C6PFgmlRg<<#=Yr8QlceaDWFw{bS9s`ZsIHtVIQ; z(>>j*T4{ZuT_=KRlStF8$*e-lo1r7{-(wDRrJSdUKV}sAV@CAqx2`$xz zQaA)IgsA?29A5n}wQ*%?30t%>);OK5*zE_kRA(4R2v$+)Y6v6LzjLsx>wu)$5SkoMq~ zvX7KE!SRv)P2FXC;+UX5&sPa2eC12A0VLI2(uf9>Hi3;b6i}yyvw5TB6Px#VmYrhp>xagj z>a`Wyq+b=QrrCRNGHs&y>N=q2CU`;1x|BdX1G#Ii?ooL=QFJ(I4phkY+=OtfD*WbC z0bx#8RGq&m8>Uhw9cCHRMl7u3qD4=$*)=1n8NPegyVSFo+Z2Ay!J88ga}zvvFv+*4 z22w}1rKrpL!2ZhCL&_!c`8k!^&G0Px9gr>eG}GFb8T^s+_dlub{hbi>uYDhc|ED5q zY-X)zY36QZ|G&xJ7F9Ma{$y`H&pLQ*AuE)rE68AeJ>pH=KiYbixT5e3UpLs3DkbC5 z+Pb|-eAXY=3_qU)^v|Wc+ut`sE>>{Tj2HZ2ch@Q67O&++7Eti7nwu<9OgnQ1jW$E#kTCs6|b4cHoKmbjO-3`z+f|!MRDG7 zYH#fg-_@O+Mf7oyM-^0$Ylas`S(QI~x*oZj@6@6credEos&idUG7Me;c<;s2Ygl5F zcVD)5J?amu5IA#nz-)-up3q6Ylq7^*rL5=SQkDs?!IvIkr4;o5dK@1V$pz^bmZb6* z>ut}Cbxz#iJH=ZuduG*>lgFC(VM;6d5r6fHWdeq#O@8d@BcgDY9?P}$YR3)4J`DO+ z;gVRgM-)WaSOS|sB`bzl;)l_r|| zr>gMxeE|Q%wbm%6_{k@DHue9wRyyo|z3S#*^!HxF$B%ZozvN%o^-@JDW=|Nt^Iat| z6}V}jY3RZjjshLFAT@$BT zyVF{rdEH4EDyWyJ5YiI?>h0QNru(TH>sTZ%L#mv+M=`IDD3Ei4n{xiQU7y*+`=n$y2J6E9{v36l>L5Tw!uv_zcO%S_+)7yQ_WQ zrC%rb!~5sf6K@2Hedp*Ty_Y!Vu-YC}171Aq7xkXxoPZ*SV%2jm;2UOMeNgP&=TdDE zJJ(5vn&HFZh|a)@ObI~qbeWngb+8(7K@hE6x?`6|<)KjWt=Aw_KME%|cb_X1kBq`kHem+S z^G%8@wc80Q7br1ONW(xkJWwh3)C|W&;sa~`z4glCkqc`=#;aa5tbDf-%_w`Jp^9+) zw5sJ&h_3+qu8pd zxjQ{zn)Tv8bBd~!fy|9+SxRebQx+$y&S+@EVTZ)=7s@A-Kcp(-x76J?uR1)8?=6!3 z<)A)}o_#&LSdI>iq4~OB0JuvW?B_u^-S>H(L>@xQ3qHYjmD65`MQU z(1*t76N%~e5bfjqs$Mymv%#iYTEVI7Rii+I@|x4nV~06_R>*z$k5@P7zWE>)~7 z7I@*mvUTVd;I>A!WDyd1dzHx`x_p;p@RvIc?N^=aF`QXCv-f;loLO24pXD`#kgd42 zi0|Jf7#KFVQn^$4%jv1nAU!#`(zqSsPj+^2-3#i}aaa7+Mre-_VQ5)*;~~F5nF{@X z0i#P574AC7PbL*VgR-D9v4IdX%<&VS5{+r}le*`{sZH>Tx%M-1X7%ic#9#}{^_gps zX7$j`H}U!%LTT55iVsD}kS&)7u2NQ8RoXEFYb%RuloFL7+7h#Z8uv;1PDUX%4yw(B z`WI1=)=)Gud48)Gp-m2(KOS&OE2+t~PMX-Oq?3m87nN+i0gu~7^@>hT(>3lB0=bj+ z6R@u5IfHHRHN%@v2?l6bK>$=Nx%MKif_5I3$`uD1Sy*;lBgD`Zw=pj!Z=e!NrNKKS zYe(*nROFx{(}C=9D{DimqLM;7(Cg2o^Vu#90w*c#<0_(llxUVT>b6B#>=FOiT7nnv zBBd1VAdFypn}CGOR&c4I>#||5=7wFg^-BtR9kckQ+*B^rNix9Mp>ITFyXVOyh8fh; zwPYYOyq|qw$`4g6+c?5YE`e9GCY*}g%0zlcys!lZ?kol4b(06-#DYy)N!-B)zlgaX zR|Hxb9%|kiUt7@lyv~FPyL-+;bI%;&S6pB><8igFjxQyC1j?AvQ)k)aOe?+#abvkB zktRn5!-o;yWz%L2W(6l1Z@7$nmR+cyomQ3q|4ku&UxQDGt zy4&OwC_?jsif-Wp*)e98?igi)be+orDCx4;&%rN3)0#J{EdM)LceXtDl8e;m1-_{E z_l0?aWheZlGnV^JkF=BuC69xaxGYbPbkg-$aneCCr) zc?7BFoQr~H!5+-$?ie;2OptN2rWdFg`;*p{Sff_4m91IJ+7s;@uQFSsdKy`~M?sdJ zZ+XRtUKM0c0t6(s=nZ0uW{tU%dQaP9-(R#}F3Z(A%M*2|Mu zOK{dp*u@w4qf!uib26M537bCH8`PF4>~{SXJUwOGV(ymlIR57Pje_>BON3rP??TcTuG6SXCiz4#%;~ovTgKy?AN*>fZX5uxZ+xrUeYXvk zR{@`VG=qnpi$20hRn=8hRYgUGjv%-WmSBQ@`k_n6?BvO0FV9{^=jZ2Vy?$GIL@9MX zWwlV6^w|kn#dX2nQ$gwiN=MY(Nyr0T)RvJ^r}*)#8hh)sJ;LOeK0=X4%6Z$!XUy%^ zR(r$5+4!5#4=aUZ40s)RWSlD z7ed0x^ykTW3eC5f%{8so;WziU@vDP-?K$hKNQO=mF%S?tt9Xq4Q^sRsDDzD zS2bzMM1+;gE}s+SDwgvn@y5QX(7C}gsZ=O8kJqJ!inBPYQ3`FvP|MJSwDqYo%WwDR z;^vk15xCFBRBZuCDt(9*!WE@nk_~jd{_H=Y*2_2Yqo2GMeY$;4O~$iZAx`^b`wLe&WfL zE20WHiWPgN(6)7bdD+?Jc5m#Oq?z-z4sY&t zf0y&+eQo%1baHe_%Er#timt8Ge?C=F{f*}h`Tbm8(5~3lxkT21NHNg9X;dB;J{Hf zz!%^HVpqhi4IZV3H#isvHpMKB8h`N%Z>D0V8@_Mei@BA=ddx7@zTPL^b>PP!<2d}! z#;VZkJ#HsP?Zs72?+hg;IJsu`d@(s5=f>Zm-Dhb>i=i{!@!s*HoWRmh{Z%*7NY6+~ z!beaKkb}Nr-5gk7AoDKH&W_&knK)T61w4D_2M<3+iCVkdF1*|mYF(O;IY`N;^O>|eha#2vm3Gg8>uoo@F${U5^4X-O0yx}sy-wr$_B zZQHhO+qP}nwr$+8J(HV-)6m?vJwK6uT@B>Xw*62JD@;w zV==ZU6!m*ZRo9h~KH3Ar1wh}13n0_z3WMC*hlw7|dY$5j0V(Y-QdP{W+t-YYb3G2( zk;YoETfX%BY*V5D4-l5W1K9IQ!Gu+9%K&#Z_io#x=_>r{n$=*EDuX8B;bVkKxEEw0 z!pSW>)no+$m8fD17cj~kbWl(!gOl~9`L2LV z&?^FGJz^@r_W~iJ0TKqrX3PP}Xfd!G(F&ks7eG7c)$qukvQ1b}6?L-Y&&iJL&vIC? zutnD~C`yBbvDTWY9GLmn-j!1;0j z8VlVj3qqm5HR5d`u$t=1S#@Y_{ol}J@It>S6!=tZo6bRLr`-c#b?t<39uk9fqHjL3o(}-kG1{!GjG zJ*>w=IVY@kFyNz8^E>>6Y7wN&5}?b6e2K)^GKb&1vV06B-@rQKrt0|N1xx|`dA0qq z`CODkj=Hsw3*YdT!`4WPs3u#QcsMxx4Z!LfZ6@~w4?!laD4%edl@#T0!#XD9>LPJv zhpsMbB4iP!H#uZdjXWx1*9EGRfys;vnGA5oz7=vl47z0_m3aBAqI`|nKdVhnftO|h zWb9UH(}pwJ3HMW~*6iqplQadIv@K9?gGa@vMM((7fSNx#!nCH+@@R4rF@HHHB?i%G zCJ2?C^*81e?t*k@cLRNe>yS{cL<_Yc>pzBz^h2^G<8}YUjF!G}B#xHp;}H_Zo@nA8 z4CVqRAR(J_GHA_MJc}9CuR@QSG6-3aiKJ#`2fp*%BZ6I8JeK}B1shI$gtYG^s#~H_ zi;l^xx5E1BAo?kJ19w9qD@^`&dkfP*v6Bt6lk+~w)bnL0+3_s0w6Tvv&Iis$*P39% z@J+?L^52u7RGm@?0C;>O;>6DF0c%od9h&Aq9ByjYzWttpzxJ+4LU-gausGOe5?!dP z``J{0p1DZAaOpD%CNY7|N2}5)eN-j&9WsCVqr1Y!Jd2cIRk7Kc`hj?0kh*y?HR;9m$K@~#msfOp6hesZ8X2pg5ipdVHdqR+EsnTz7(;qI739eWX7vmlsjmO+1HRU)Hj`~ za2Y@~CVn!(1-7+**2L4=uPu$rLULOi!U!OU9%Ya2Tcy^0aYBJbq7O+#R9j9mboBll zvG!v{=Zq>;YKJLt(A3UR`l@OaB?jQ}{Vb{zvq|%g7P5+(dA_7J28^ys)^?d2A}yPw z^F&ekDGp?Fac5d{xQJ%N?b&JJ1&C^dmA2w^7n1^ctztthyhZQK0-^Fj#*p5C5dM?p z%7BCxa!ojZ`r0vTDp(cZRa~El;R3du%E|7#NDU%qy^G!o6M=)S{8f%IkYKuujI(PsK$_QK}tee0+4q4d?#m z<4|=TprS9Zo;?j0Dlb>YOQgRw(89D3`nVLOXCkDXZ+hE)EXu$1Z>ZwIy=upqXGD=# z3z5K|;BEbp%tKO39h(LUTF-NB^`owZh*3!R_4M6m0c&a$c5Tv zeO%7+S8Hr`z?Ihlv8?AzVf)nO+}v#ldpVXD$bL5RnEnVsA=i;8hbn9t{uR29`jN|& z=~50ERjqx-RoqioE<+wpcHclU;$Vv31aQ`=E)45-H%w#xqDqN)xq)eeI0KsDNR@6R zSG5LeiJOMp06y|Z1ktxL<)`UaQ@0Tk)#L}C%0~YIr9zP-;QUYwkvij7$M91KqC$5` zp`E}_%fB>17R+Iz8!0>C$ujP5EuLzKH!Z^n-eRTn2~T{O$Gc=yt9G_lk8fGzc66V? zae3N&AcI!Ju$|3_*c)`}af}u#iiZ;B(4a-LiYz-vYmFyM41CbYAwdb8hxj93vIto* z!6bX&3YV`h4pyg0JswjxBcCL!k1*dliu}+xso1YBU?_zS`qgfuoRp*5Bmy&f&7Q%q z(l}lPBZ0xb;D29?0c|j&T0(;*RpXo*NOZp3&Jyp3uiW3Ckxr{lsZNND`|HGjdi>2g zk47{@-K6HE?~PsjB0Wz#s{j(U(mk7sAuKX;5NM3+5r8f07PwVuuLOY$f1Yku*GUk4 zyh>6{?!U4MZ-h~9&jv8E-{*rO8c}?}xCv$Da!ZBQt%l-;|D&Yp#h}Twf?j8L9v`Q! zdsfjR_rVppF>uTIL4G9lth6DLpWTF$=dRHB;}{lSo{uu)HK>*<>vc(aSW zxON%WyLgub+hj`gr@tbek;qisoEaKckAb*wnZ*jsa4u|T+kOXV1qdw*O?H?j#G?Pc zL?9d+HbcO|dU~TuqJ~6bP?Z4kb=d4^GF5MB^UgRf)gBG+=MDdF&m1^m>grFvqo1BAHcw+`>;k;d!z*W+10IECnyt;>Y5sK~bFh_KOESB+Ln%6swh zPLKUX3BKugg4d(X*zHDjw@mfM@qQn;w^}D&*wtWGhsV7YDz_Scn+GK<)VYR7h*dVf z#}zcs3pf4fR)9}*^f_g%;wH_nV$_ zCTqSu*pR2Kiyx&&X15da6u9Ipp#$ z-}f6hp-3OhcFxcNfuQ#F&j;9qQ7tB8oSueiB~-&8hl`@{cXKK>ns#Q3t0-76RtpeZ z>BW@ho5>XgirR70FOigEl`s>VB^Mqkp!c|f4XpnCa^P7#S6NDOMledfFnXrIGQ*SE{fu&4%l3xpwY+oq|co52*I`t=V(N`5SH*KjcCc>5xVB4 zxVHtW0PzSqKu|)qt0?9a2C^w&Fc+z)EJ`b5%PKO86&JugKtK5RFYC|ysYB2F0FU|7 z6+iGdu1e!*PD0wcEjmQqO!2DR8A=8%H)Z{l;Egrl5ttC=$0*|E<#YE}BkqyrMm`Ytb6wibrgEU`c&sU)6nKoZaN$csA$~lT8iYv1z*T+(I=Q z(bPS`U@v#gnNPrx*|Lh9Lyl9vY!EeUVwuW zk#=nr!{tk1i{i3w>FB0roLs#tb|qM4d#mLQp=_%LOT^IQm^4_Qw4Td5Ms6p%>b8k} z){A+IHs)K-|M~Hyh1mxCBBJh->R?5qiM@V?(UvM=6|`x-Ey%r-FEq(Vdn6V>sg^uA zFdfpEGu=)%F0j%&+NKzj>pZdq$P43y&fXwL=gTG6%Z&x=$K}W_YZNckez~Qw(iG%( z!jbcN}+u6|UdTYU(t zji?bv@#oek_%{t>{3amOsLd%1dzL{SJL|YW6YMYlh-l!NLb0rPaLab}5X3D!ysnRZ zRaXy(Blyg9NHo#4htz@BgO+NwZz&Fl%xJd$k|Wg2l7A54_$9F0FMlLm>hoq0DYp}K zzJys@&>M=)a?+iPT1ss22%ed~kAbjt*%C{hrjNrLi($*LKEeTlUu`302pYoaXRf-b zQSSYEN=)-RK9(kbn0#WU=*TIZX!yNw!S&XLgw#4znNjO{4afAU>SJQII_p^ZVnLlP zT9mrT+RL_$9P%hzY!Umst}c(oN?JC6w#?1r!w(tM5<3RLl%2|W`A9ECHX3im_!@;` zDRlCp1$#R@S#H6M=m~`qXnylx&ezZwnj2YUA;uaYpgo_t{uTI9-wqKOjhQ|cOcvJ1 zy2pNf#|jB}{@zffQR{wdq0u7j>DMItP-I7Cw-kyfSQbtzL_D zSrFL0-LgLOHs!if(PzociS^P1!j>mAsox+Y#`~+r4RakeE&|>nb(5@(ru2+Bs zCyb{CW3;wsYn5uiPy4L2tNw}IJ0(4cH3tq~$p+@^b94Yt=pRlwrW*s50KXcZ#l|2t zIm&}Uh<1kRFu&`nJm;)xr4T^cx$)d*1r!BvoN8nBZDqFro~q$j(hPI?X|->J$Q@)|wG-eGpOtk5rWS^g@X#U0NuRK9raI@)LZa-mfLty+C!) zX{6uNOt9}~p}Ut3Ch%_gIb7O+M9pq!(5maCELo7e(TqyA47jdTSdG9-k_2C2kt3EQ zg(=?XAXIu7qB6N=%?WW@#~GrP75h4}pZ^yCMJ+V`mnw-6WGsE1jeC&CG>*#_bbeuz z-EqJDMOi_wiw_Yjh2T#0p`B0IOvhjnJS?-Hjr8GkzqvxQp!M?4573Nmz40YX$3f@q zZExnhEW<)2mOB}S&5H1`MFy&?Avzdaz=a5QCs8ODt*Y|_SaMhyQ&kIfLZR3RXW}|* za*l#e+|G~xMMZSM7#o6IJbRi96*~fU*kv~0e#FeQ$0)-Jhq;II*dq!__KcXH4p_aD zB0yeEDEuccIVFEoX~U<3p11E&f}YiQ*YQpBquIfdGEQE}w)II=KtakQ=5T^*z|P8L z@ceIK1|(;;w(THqd4M0(K;W&1_dSiinu*qx;>~&5QZvB?1%xz%?P7&k9 zuKdP~1pbwa?C4UrA=$fLVA+q%>P*$e(r9ZRjGvWaI?2N;sPx!HRmR;bKPMrr(SPym zNZgJ*=~p^WtP~^!YI~jcOF{zr%)%G_u{n09DG0E~^F^w6rXU2|m=SP^zT@@BL(+H% zJw(5x&BAS_pW&_Q06XKykvMJ4aQK+iYdckY!TG(d$>aIJsl9uqXP%ot!~MzTNzxRq zf3JTbp-N{D2Oqw!4!bge<|z^?2UcuPFPo$&k3OuRpXImxkCdNmwfi3~tQwEM zSxBTghWSk&TR^CL)ohfA3m@jrTP8$y_Wi6T-ZY3BI8#I)bj%QI{*<4WzMM z<~2q27MVp3>kAA*niCGG8}?ryWX3>1&M$<~V<|~)%5~#8uhrD&t|qq(cgwe$1iptd z#!v@TIjY+2h=s>GGk(z`&L|)L^?ISm-h^pI4{wBG+ip8i& z!_<`*n@fZ(LlIdfdfyQtq*K$mrN1h`i6$J{jyxA1luerRL*8xOoS+OTkK6ZIP zyamOL8gMVZqh6$VI7X69Vs|W_g;efyp^)vOf+NlFpm^pA8@@c^%uuK8S~E+DM(M@E z@b7>%g!t-5uP~4gFr}|1oZVJQn1c}fRuTiRCGB;t>1-Nx$^4{B%E3E@0MamQaBdhVGOah$x_YNsdP6?(Rz1`rzg=zXH`steA zBow7H#oi6;E|%A@zNoDHRWOJY1XmcE?DXA|#0gs4B4e2r`{%;=LoBJ6!SdKs(*ea$ zx2v_mHfJZXkcXtePx2N*fyxd$?OV;`@2y19WF@(c;VC%f^dkkr##!<(5rUPTWUnX@ zVboo~5Yd6XccMt(5ptBL{D7aLTS!ZWATlmqMuq6owGd>O zWkd!PoyDNDm8e-PLJB&8gkJK~rkU*&AH;LPlS^U~GLf2OfBoy4R7#j-uWG@?C+tk} zjl=qy9@2zY+v7ehWGX7aYtT2_Q6nr!sa(Vd5g!^sJq$yGG?%#pUTH|v$5uON1)e3o zcN%+kJ6mpYoBG>#mofyLqm3ADv^=Uw0k|GRgKv%itpFUZFEWvrbO{2gv z=d2az$JXy7Iw7Zxy9c5js=B-+TkSL#Gv=eZgUUkwS&!Wmc&bvEbOLFR)z1;Y7NkpL z`mzh#VCj>0#q4vQ9Bdyn8Np?QfG<(Xe`%LXT`~llHHo51Q&$&!L>Y^?>W4O+j+;x` zW0$~{LggDeIm@U)O`EI*B?TYjbL5i4yf=l=_t)e4!08 zaB@fKl~<1~Q%OH%c_2#QSmw`57SK z8V8#yyIBR9TUpdp>&Y}nKFW!>yRR4%o@j3+sG!r~5k8AC^I-(yZ4(E zv+k34OA?rrL~pb0RC*p&8}G*#)R}E~qR73G)@Dn|M|A$-lPzy(mG^kqa-aYaTE-XUlrt(=GhT`uFwv*76( zp+yxC)g71b(r23-<;{VHq1V7RFB8im*4OWXU+{(+rS9Mrb(^hTL1Q4ES*tZt4&k%| z;IG$j6G-bZ!zi6??Jh~#K&>g4477M2+5&^eUlqW2!d_bE1VqkTQWzrxHshcH89EyN zwu?=@J-+{Ypd0P&WCIWq^tt6q>m3O#y>UwTlwet`yGc1P7Ip4Ucel7`qFgNr@Fnh+ zvWQ58Bh(Uyu-`tH4fMoOS@lLUi0y35Ch)d;54W8-s9tf|*B@By1t+mrtd)|x?2|=c z9xDx)pjM@b5h1k-H`4S_6CQ;UM|9$hjixdW#+?a6AV-_C+Bt!MFKINl!023(ukm*; z0*Ag`U=8)6X7Nhr4z%rIlFBiJa=GGy&ccLZxS$FPqXMZ7DAG)1!z$8V7ST@t87MV2 zhg??&4-zZPT_ELTn}4XBO0wgSLiz!6qdRYlCV z=eER-TjAIp@3t`kQ*C@}3e4V?Ac|ZP!a4(|k1LGWULv+;;rbXCw8uPq*n|<1LoBEx z5aX0(et_@DuY;{s)0&!*wxCwZ*&rsH;;cd$=6RB~PWy1p5idL|@-*+d&JeI^N_?xe z-XTu&sf~9RVvER~pX*Yv;WDh&*5pXQoY*#AgqAXsH}P^6mc^xP)xV+!b|7X5jJc zd1?l|uan!*Q9z|H5X`VQ&}3Bx9d7=Qf3YqE+^aY?iH`@L40;Rx!sWJSxTHyQ!cO5P zH@gU$K0V$>_=CENYMKnV*xLkyuS`~xL$i33&BMg>9yNWj;97y=!W(ZaYeOUnoH(bZ znPGYETGS;$L%l8)5hLx2zE>FW8J>k0JuXNSnb>GN*CZl(oy-L9sj%+@Zgq0RW*P1z zCSY&2R7i2gR-^F7M%yfS4R|vv2^gAuZT_kWxsc+-p5eudUiPayo#yy}uQczTO1b-$ zFs1VcgG4{tch^iM(L+Wn>PACOL(p?j_K9>KMj%Hn0h z;BS;TrMFg8xO;P0X4KGF zGYsA0MF1jEiI`b!Y(TL^z9<%Wh6jqy3^wlzoQPeV0pMlpH&M)->W$owSxpS~QC2OMPP zvchHndw;z~eXRh{qromXiK^aBC9Sm3({(Sh8=9Ek4ish03w(Dkz-~Dx@ye6EN94+3 zs-fGoIy*;j;iUgIM&JFC-<#x6hz@Ydsl6ZTVizTOkUGKi3A6qK?|Sk7mD?Im7kU&i zQ6@UDyMFd6_ZDPG@-6p`rSY;~shW?1oym22rq0C}6j0_wtRT2ky3rE*a70^~%5=t; zk8MV7MqAPD6#0b-E$QGD73@SyqP02b+RNPjy0jy~A@VYogcyZWUJd=Uz$&>UZdXPG z;JQQ!hpoYqmhUt0Z>X6){2i88KpEu7y_^(00o4Q|(alOQ0KmqNz{}I67MwAsBf-o$ zM+)Z|i+qsu)N!nJ0_oeBlnyQ|wW})860uEkM%$JKB8WZSA1_pG7T}9id6A6^7BU85 zOimqN6)9P%8W7QV$Y%w-rI8PM|5&zHKr9epR_^fY2E0+>@biN@x# z5GMwGr$EI+g2PU^RN=PMU$MHDD zqPMonw8D+tWHivzojcefxyZrB``3jg4L1j1T;;y6BO_{aN=@r>@C3W`Y+T|rtKnKd z*4Zqe)NH?@p(yzU6mxwcFB9rTb`gkSo0B^#8D2xd7Hifk>Nl zgs~$0R<9f9%IZku3Dxb(VQ62hhs#5WEsTBtKpoaE&`V97cCUMuz74p#BWrgD-r=Sl zmz$o=4DeO6dsYB$y^!q}eR6NEDdREs_hH804BolL=bFyJ#6un`ABkx3#}y_`+TT2G z!p19p)5%1QMGg=E@j`T}9p^ye`{u<;1I=WQ06K8j+v+938N(=rMycd1st%R>y0W8Em=8mgZs;IEPe# z9+UU#bJyj0VMlB8RSq8q;QFp~cf&3}NHT5H{Rc-^As~j0e~8RwDh>3^)s}3R+cev} z)%^AIm9yO&K?oU&E8t?W#%B0lXZP2n^?Kk}pXoMGkfE{%*b%n0Pu8rGi76e+Au^#?)>c*z)=lYcXf!k1k)34K%awI(iFN&+DQz28qvYs9U+%_H zrksS)nIAc~cTy*=ZtxgepLm(o=X_o_W6P#9J=-ITwzrHX8@8g1PfTckCryZ*l!ZTq zEuSM)XMyAxg z$|@zng@(V7XJ=*5P{;L?C^uJFO>H-9Cv!5#)tS`ejHrkq>=MTxXbqvHEg}scA2{eA zr&`W;1v)ZS)O)x(J-i>b5Z~@m71XXbD)DH6q-BP-PyMgkYw(1_o6FdwYaV;IWPan< z&~xMF7CnLHTc~QM%bu|Wr>pOn^fmTXi-ouluIB2R8&G0p3w5!*7#&9wP-%MO*q6ts zb^%q#_Zug=TuOBIHt7cYXevzbYRwbI7x=tQ<(W6%GPR*aSyhk6*6u#A$|X`-mrU%_ zn_G0SJABkF1aXMh<3ZzoS|YA=*1jt?8pbo*Ztd@T#kxMg@~_lj>rj)?a%MdSW4!f% zc1MPKEC*DBTJ%dcR3qLJsEXytE>puayb%+UIkgZv=j-lVi#l@Tjy>t^R?A23rgN&Lm^2-H6TgP%JNMtyz+T>hu=f3NMEV2&zb1V2vD z-e87L0GP~c;6B6(fiVrbiHu-KgmHntuk+4fIwn~P+q5)^zGBNKGc!uF%|wIv<@?tM zWgVpjRGXX)ZI&r8UZsQo24@Bb-4s!labPxa7T)d^%0W%8H?yW9dIl$RZ3rEh=qhWV zx}3;}GWALWF4a?TNE>QzNX!fCqvY(WR&7ZxQ`y>N%2)zRAg`ytLEObV@L?61EBAT5 z1zLmT`_sk5d63S8Hb?+XRHN`25-K}{b4RxsYD*;D1Jq2?Qg!?Fx-OMMKBwLE+v}bk zCUrL*m1vx=3(@Lg0rrV0bwVUc2rm-G2pU<<#u~1SEL|+ulu%z3se6|%_OgMV*QOh1 z=P2P)sw@TtaCr_J~@=Dhd6KNz3(7jORr2*Mj3UWUWoS^rgS21R^%0oK zE}p%kd6(LFjMGg^s^Ah&D9ZQeH0=0eY)Q%lFIzWTuWCyw5~@08th$(=L=j~=>vbR# zU!NrR4#FQuphQ%SeQ^|ESE#JrfQEaZC?Q|+i~N5)s{e$u?U|Kgb^ke3jQ_Ce|Aw>e z9qpX$jO?uc3#a}+?AjJAE4wX@hb~`G`cHw?Q7aOg`mQ@0Gnl~I2PSV($o)U>|v`$sf|RMmsNMa7fs{~b$(`X$gd(2qr65vG?W9DBx@n& zZIddMeQ=uDajFC38N*SFw<*`LHi^mVaFZ-!32&^>{z@6Nu4o>{UUGsAQE#|LSV&4e zW~AFf&?8C<+H^Sq256EUvPLum;d@1<6EIiJ-m=Wix~Z zxQXYM?SSTp{kB41vI+$xvWyUM#Ph&nOmN={4$RMEJM(U^UfPn{bkS;=uXrG<_gCZ- zK%L;}!Leo;^;0)_c%NBwuyhG&(f+?r)iS!-*}3YfX*d2Tw~5e`0*Ub=UXBt2JS^h0 z*gU2qN#X9hmTA` z-v=M$pKjfrFeWMY3q;<1{=9LHl=Jcj52n;UoZd6B=sl}&0+5FDk)AYL@;`J2Q+LpH zd8+z_`EyquVQ-u~aT#Z8=XK{rgRU@bWVkm*Ds#*LX3-NR_?ko!IF+aQjkiufp23|L zw_IzxJz!#E6{2Mi+b?wUQNBXO?+>t~=GHW|uvOr;lht82AXX_)DG2#meiPgdy!(%0 zsc!TezC9wo3*1^L7Ja4|>`}YX*DmT5s9e;k5=Tkc|J&2yu(!&_cUbyadmD=#vT?{( z-V|mKRooSlcd7Ew8T!f4(^VePL^qs)2OuFu5QN?CCc|$4$YLKBLs0Gf5CMH^9*l$* z79_n+W$cT_-%6DSc1;N9#QtR!?rKxmFhxsFkV4J|C>n@qj}2mBdWVR|h$|B;3HLcY z>~kbrYKx4p#;<7=BdOHOe)Wpwp#bX?mCyl5w1~hS#vh1O)MY?CE_v}yrGKy}AJXkR z8G!XICMcans3`K5Dgw!_qok87 z;Q@f4nA6Ic9eWRwFASO)new4&wo_*j%A#3a@u5QIMTbe+@U+<_htzKpRXE^IcnB9+ z{IE_ILWI+f^B89epx4MXCbm{OGhF|B9Vt#0wn00G<)HHL3Bj{N+)^+s@)p5T!Jd_H zGH_!_LPmsMi0FN0)B#La+=-fY; z?$L*go9bK(X+&Lc6!M9`JHP0zQ)?eNLA$v2e8V^?05`ihpWg$_#SvZHVi1XeFXU|o z(^-k}#hvG|7&Z#nt)0hws<*Fg%O3q;jIPeHW76P$cisUINh*HbHQDo0Rq>GZoE?2z z*-o~gyY|Tm6!*`kaFGoJU4yjiygwS3a;It|WmIu~0#kg2`AE6eG(Hq7mgeZUww;XU zYEzVxBj~89p#m{~di@BLEQ$Bkx6`O*)j*`boH>_{&`$k@Q9Y%L(@x|ZHqkTz8RjR9 zaHWy?d+c0<9d78%hS1W33lULfmNzTiFBx6kp>N-G%A*u=GChB6Y4&nEe8qz_0xN*q z03%?#Yh{;bSv*mnY$|VdM8U@^QmA>ltNhjQ6(k*x+47!0J1FJ7#pFEdXyG$CeU&td z8;R9z;|hJ>5Ng$B2~@V85a$!c$Yscz3U4LS>Ne%V@=>U&Pi7)9YXk_oUILhsa{3oB z9w;F>pDHb^#5_;C{xl**zeb!mOSFms4LEBJyyx(uCdXkU-8|j1gMw6&8=hl8;BrzE zw%t^c+-GVyHH@Gt;tkzBBT~}b87#7cp3sCtsbc`6qIpvO5SLB%iw%WsZ>gLz7HuBF zBxg2;qTo{=I0hUsn%PCRFw9uWzfD5Z${1SS#vCBh1l>%(4MxM9(KzS0j%*(C;>0f| z-g=Qn70MtO1jY^pv2b`rO5uOw`6eQqbLDuqOovhX@{z5IRr(g*OB;tWO>!%9{^)0- zW^WZ+6WC>u9CVa_HwS!3z_>9}<0@W2u;l8)B|u^zL7P!R$cYR=_}MkM4KH^m9JF%P zkFpY*6lg9)Tgm_Za*mCBCUwGjAT#c53R3d=$vI6*Z{blvRg(};_Kepz#i*}l&!~DJ zP-q7TwpZ3P$c9R!Yr+;Xn!tdb?{kWeW*p#VJNnn4_KpalPK0e>F!$q|0;f8MFP1AV z;hz=r#Qvf1XPo{r0IW~2Cus8#=?y|cXMvGZ8dn#s~S!dqy@(r*1B`DdqlH}aEsIGm9@kQ7dkF?* zFOPKhV(xs^>3M#mo-|3^Rj)BEG(F^V2HJTgZV ziwepzE)cKTXlnnkxdd!_iN)ad22Y>1d7|PbN z65f*}6u!Yt_Ni)rGC;Wbb2GvO?E}zn^-sOS%2RZK_ZWXKGK~sqM-uo0{dZ?8o2R(} z{$?ui$BN%9tKSSQkDm*6IYl1F_YILdz%&r2IM_G#;SvOZwQaXW*)E+R^k1r?y z9e9&V8f`v8ZD4T{r%iu@^v?z!lo23krI->EOO<_^AG7@MhojUnhQmKL`LIfdM$b61 z&Wq_|{8->&Qk;dV*-&66xU_^!)Hfbfxj6W_w;dHM4xw`bC#$2QzOs~sUhz@;%w!B^ zPSQ&y4B3m7o`w?f(AeR;av*5E`L#`&X1~wk07}_*zYRMSC6FfH;EsLQclzC_b1G3g z*y-uO1ABQH6*#MQe)``&;Wr^~ttUj)KPIWNkjHbJ2(Uuc49na7$sl|Ug$>2Z>vX?R zaA6Z>IU3Vo3lWT?i4aN+v#E;Z!ub!zGS2MXAnY zW^`S4cD1b(cV;yp<*)LGMloxZRW9c)ll#VFlzm>uSXE)qKY>&{qVyBZ{t9?7at`B7 z{;Fc4eymYKf;rmFwDD}}Fd8q}cVZI~k6&zM>W3*!Gj8|Pk-@iSXjKN;<-Mfj&pCya zt_{B`Z2j;&(C6Di#n1Q5Bi2JnXXzvk3I<1Qaq$F3jsE)wM&DDZfvMd*q05UxtU=ok zP77^8;9ib;I99d5Def`ED5wT5J*FQ=iU@OlB$#tsGW~FZU;Yhr2Gp=b>&S=#Pu5Dq zs3~Golp;*`;GqO_L*n}$HOV2=6jMtsS?Pyq9}dA6{HtS#E>)1r!NE%a2>pYXA3)9r~fCfUTa_4VROR$-t-Bk`QEj= z#1wbo!wEF;vt90%q=_MRLDO`OD7T)tZ{A1S;YoOl{Js)bjEO%+9+7O($ODU5M-)>; zenljkenj_p|0K#BX&~s_G`$|N)Bi@$=GeAvamJW=P^c3vs^F9ifb`BCfrJ-PCm8ql z?>;&OS0=J9Fg}3E#nJ3hZj(HK^5nT&>*f+LkrVIbo1)nDsr7XobtutX;ADQWD`6 zvxImc(P!MwJ<`Yp-!|!YL?%U_C;%v-`C=Fo9T;qyc_)q<;w;1b&pxt5_4|21k>LL=8D>}`{K+}hXx)GCg?M1>Dhwv2l^ht%$jSc0pHoIVQC z&$E8d>mM3q=mkR$q$6E`VCH%-A4O6(vS*bIQB%|8LE&tz)mg9S#sxHu}SpU^k59f}SH z#!p+1dsD0jULC>L5xR?xF=B(%A2nb=UwTF56l4hFlirpF1%!+$TjDXu#n6WP5Z%?_nJkMUBmhx2D!1@L zc$;>>m4b^F zTbhlfLo>Bnr=?oOW_6YtM0|^1{pZUy98fhebkwm1vRn92%7b?rrH^AC==@;g-?dmH zPkx~x&^#L32*AWspmV_pt*a%YQEH7;Z1?#ug{KEVj3NhwlYa}+KD&(q}e< z70Z9-^}cd}j9R+@Xk0)c5o(_~g73oLuzEp8Ce%k%cU$s>Mw^^n1b$*=9 zOh3@kN~I7tUB$XmKR0+J3ol?VM)ywl-#9?`AY8a^Trx;KZh$iAHmGjIY{Z`6wgh!B zdQ=>&S|2hJdjVCrkq2r0!L5WJLdx>Dbcky2AP)k9<%a!{W{hg*3|ePR+od8gIdq&# z)GEWwy9@N{-kL#faS8g7a1BOBEa#KrFJWnqT*(rBf^Z1SFeo0VwP-oY4lUAwe{-m) zDQfb)0|&J-rIK{VCeWF9;`*^JQI{V#HU0juZ~@k9E4<^)49Dcm-))@n%*eWD;&ul} z?g8RJ@bPmW6}2ASG_HYD0JH*-RJrJ#xB7#mlYG-{S zOEhzfN{M%GCA6@!)nj4HZ3Ul9+bZzQQU34yItpeb?|KP!aWgfVAc&ZzxxE zO!VZ14k+7s>KnZ!XihaGM2L}(i4B3XFopIylT34z$kx8L&;mRB^q>`#CZ==Pj#LVMEbCa*x?Sl3Lt2<}8gk+EO` zFHoM7d<9oxK%PmC7Yq>5>D$7nhc`6lCbE>tD$A> zrY?RA95_Fb>^BQmS1Sp9umt%QIVHdv;gjm)%4eCzXKUKeTdT}HPS^x=gFwI$+LNt@ zCzc61mWWGU7MY4M{Rp6_eWJoA=HGEUS}KqCn^JoK06c%2C?7Y(+sYRi> zLpp$4QJXHNhTdQ529TA@TeI|{FJsDZw+LJT+W>X9V>Moy9a%4)^ z2UrbkaKN+RB~*I)=!^&HXiUwStU}JNUV!?$?%fGMlkf0mjW#}oNnvY$wzSf?oD{k8 zjx#N_?0}>Vh6pc#fqC zJfd$?VZqvsD|Hs_kaKI`~K&B&pGq^wH=KrsF2cf48opJC;VytWLh^`OXej~ zW47;s8G(N>67H+^U7DrXwmLbkeR;#d-ZB(Wm`m@Ssa<)#Q$kc{}kr#*RuV+;c;AD(*ni(O&4zieLm-!z(i#ziIERHl! zk#gs5$Da?hdgI=k>7dDY7E_-T_d>TLk)bZ?VCy*03;d7Xh<|UB!Vjpt9*omqSP4?~ zb*j4mgTjpFdw*|9t@fk#7hjh|>HVA8OI%JCEC3fH^sWD{E=aw)k7 zwhr7VEwna*51W@h(z$v~DeM+4WRx%ML=f%EGE<)DGlFwC3)rMUbq+J=kbqM_#;}tCT z!B2$96AjB3N5$Xw{JxeK$>2zNVfMc~d+D3fM%>nsx{W0Xv|7@_bR4PHh;Qb!PH#gT zsXROMtXGIgW~+_@ex{B8pm1?V%HJk=c+Mn3wYR=%HKn93<#KUbjQ<{lqfBg*CRM6s zjts{;FUm5vKr9_4QO%USk2YoWi2;(h%u{e>3v|}?v@fvrS=srdBvhc8C!6pQ7Q1{M z7At2cw_QsEsiD1{7rMhz(im`=1Qcb{;V{^DEAY+)ypNOL3i=nG1KzP|h)vTHsu>qm zoP_;~woxYYo)N@keR>m!k(Z9_<{f^5ppM~!FV@&{IfoPvyG_w#aI{pW_ zZIS=o)8Jtc4rO&h^+UcbEyyp-*?L$0prYMlz(@U9|Km&PV?2$=x6Jk4xD&?Zfl@on z;mYayzUZA++Xp_Gsy@$Q&#Vc_i9v37?$*zt-Xrt>-5WzE(=UpjPkwk)F?UZT;@-}? z7X+O21Ng%q+G+o`KW?TF`AToh3vd)x37Z=g)(QMi#a2EtQ8J_5UZnCt_-s=PO@=h7 zke)t1Q*ZZHg;FqL@hZaCwq7}7f|9GV+4&jj6p#M#RJrOH`rv6x?a~;(+LX`T1B2W6;my^UZg`ZYb9=oplJo z=1rKNn9|pp7#lcj^AqRC?4xrdC_V_3@d^~nR%~NHQQ%^381o@_S$RW56QDX6!N(K} zWyJ=JbXexaa46uCmKfg2=l`O-K*Y>Z&o-aH5wlC2iLFM=B*+G=Tb3HniWnk$wh=Zf z_$EN~>h@P zJwJe^tYe7^W?`lY>JSHJ*(AG`1?m^~^<*&_ppH3U1{)O!!@B-C-^Ky8LjiNN{Q%)u z&m863D4^~PV2T2D2*px&uz59nonoy{J(z-93qrAhYCXWTj$%D~uS2EW!AzFgYni~p z?q6>Sv(KSY=wN(;&RYD{vHS0dpmN?|B2irk(W;;+WEy_4j*yP^_XSjc6-=PAdzoN0_@!U{jk{{73!w^r^E9IVR< zD!AsY#Ji60BYPU4Z#{vL%e^npSDS^)(u0lyR|9B}<)u1m#WyxcI@LKO4cKX^Z4^8f$< literal 0 HcmV?d00001 diff --git a/python/lib/py4j-0.8.1.zip b/python/lib/py4j-0.8.1.zip deleted file mode 100644 index 3231e31164e51daad4254e8f09a49336205563b8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 809541 zcmZs?V~j3N@GdyEZJ)6{^Nwv>XKdTHZQHhO+cwVF+TZSexc9$zJ6%bq(_Kj=^*mLn z>Q;~j1w#V@0)hgX;Ug0QC+QD>1_c6Y0|x@a_;1(VgVmCbk)D&Di9tqESX53~l-}9h zIYoKKc8e3K_mK*Mm(BlD)@y4Am2QC@SaUg1MTZbQN)AfHP+$T2;RPT}QNBgjdOlLh zm3?bBS`8;(x4_Rs{^s5uz|hv_=lkLK`R)Wf#aAls=6FP-@*R}r$uGtu_g`w`V3 zf7erpJ$SD>NeuUP%|j7R|4f;1_wjirQ}SwE8viwbuJOnMAVvY!QI;J-Jn;NWL@q2t`g3uIj8Sl?-798C=!XnFTYTNTo@m-)SgMUfV`Bzh zol@yIFLn;lgn8sxo;+^mCWSGs&klzMDH>$XP0L9@!%2!k*K(^#sq7Fu14M)SI9b;? zc2-WPF%g%%f=OW?-@&fifbAZIYPpjHlUO$syZF8-p{mJKoCOPq~i-0ft(}tzy^*O|)a(?49GxT`Jyq-pwKSd7B*QFiMKjwe2+;ph z)g+oJ%+}Rz4h{kY#P=T#|39h}q{ZnZ<;3JuRJLNX7?FI=Xig!8I&GHKc-SncUD~pu zMBF}k(xKfkx5f!t4{reo##z-$6>q@6d}$f?0q{(zc2y8@zqq6b=rLF9k-Zn0>9=nZmX8s0j>UJvNjpsSCX}M z+I_8~4%vtX@qk{B=uyF>mT?&f6T>|#%E?~$US8+F8v=JQ(DUW_PZz8a8I z5RxpwBAaSD&OT(ko_1kqdz>Yrsm+|o(`CBxyGx^;K`smv!!)|cd$uiaVI>RQ;n@39 zXSa|`jB%S^WIRrN7gv6Om+Jt1+VH0zs35WV;<_vp_+87jU{B)N(6 zx>e7X%6b8=Q8!g2^bYQJOg@)0XK)61KJG@%n-kEiy*}K)wHO2LP~)W#7KEHru_NojoH7oMcd_J zs$DYY9&kNNYU~{a;Vayxk)`q`b4?2489c@SKk=7Ye;8L1@3=mtw3(>Lmb@lmq!)?d zz6~xm@W8&@A0!TdAJNb7-H6!JR=$1N=ZVUniF^2^7Tg%g6VleC&Gw+PyLxK&Ny`bY zHW*@YRf(%8X^3c{Bk$>XU2$KIRlAh2`C}8aqgXi+cSvhAV-2|2DL9crQ(SrCm|==}C8VWXAu< zPx^Sh=91*elo&JC%&hvshkZHox%*YHdwWqisz0dR5C;IPdgD(xXV1OY$b*gv{}**9 z|3}@-P@V=g|AFuHKf(T=AT+i!VxTv4v9LC_ur;;&6Ek2tM1&-A`xBYqSVOV}7Wjx; zv-}7Wr8yBeF0eFbr$Z>t^0*70H%Io4Sv!S6#D(#7r&nSuGX(xulDg13T7=C(R2uHD zV{*&wa;#<@-knsoeCY8rmg-O63QcdE6wYqG4vygkm$yICXRE=P@>impeY67e-!SF7 z{_%wFd)AGS4f7Qqt||()MCP^(9W0!hXJDtL1uRs|v!GIw=-j93Y$d&=ozBlY$p2|$ z!zLz)E;JBO;(y`p|HnoHLpv8|dUIzR>l=-4$E|kc-!0Ebx+SrfWs;g>&lnp?d~G!x zvoBTY*&SK(qX3EUU|AG6AStyhuY9inK7rejP9TZMk9yp*AIq{_84-}E5yRKEKHd7q zuB54-^Rg;n5$cG8%hR{(W<1y*Y2bR=&v_hXI}^#H$<_(;zX)ZWfZU7{Xq*L z&IF~6fU_A>7Pptp6Ot#fwcViA)zzt}vDC^M7f^K09 z%xdlt6&WvaqgxyHOKr#_PaL+UE`-aY} z+&$gxJ>84^!smnhFxdWCS4~@2Q&S58z80N&PHmYVjO_%4-5CL=xu2spYo?!RW@h5* zLjyPOClePV6Z>PJ>G2wiA;VisO#?S~r#D;5*;bw0lr7JWsd}k`tJAjL)UR}vt2o;Gu*biKEVtTZ*hV5cgVCclEa9Fqw zF}<~haTVkgGzCFh0mZ;XgHUhjaoJZt@@ zuB;R0&7bw|5ir1mPi|Rd2kcB%k=p0XYhAD1uSegCt?%YWJ&?sfUmpo^4l_56Nc8dx znY@oI<`w{*uLYf05Lb$@y*FQEK~Uy-%XMjUdz(vyNj(qJB#VZ^GEb znPcqDi28AODCeK?W4$Y_CcmnLssot^4`ISxAepm6dz@Duf?y{$GNt_h=w+jdQbFtm z*g*$=qHcUhmEZ2huBIg6%}%Vkl7-oHP>1jOC73mguMC-zL+E{f|6156nQVoe|HHC>U31k>7N41m5VzwFau955^S^A%g*zlv&or5r{`z?FGUR@4mf zAAYJ#dU4-!m&wi1{&NR=WHXh|XmNP8-j#B%AMhVEJL4HQ>RgD{GdwNc#qFha9n0}b z`N=%p1?0|m-mNAY_nTQvODaI+q1DAPP+pc-W4y&pVT`Y}%;h*B8-wtcFZ_FoH6q~a zQ7BUwS4obO^^J~D0EUiXE;FMbP`?{&DA|(eMmu(zMTGsEV*Vh-x#HgeCz~Nk>l06# z-|hh~7B@K!Ho;7tie!fTo&HI$j3p`)SPFdzQXP8whnGQBA{6xc|MwK9)oot9A zFA+IYXNb#?PvFen&G31HKqq4djg+mhrrK@1=dO$iJgj=j(5U9BIn99Aqq0$?Tv>ct z7IK|d_wGx?Y)d=?Sz0IEl&(SC)|w6eOm}&CZvE%latK00y+8hkQ@&5FFmpK75V@LAYE)t^d+5lle@n3R5uhN8TYObJK|k-`U;b(y^sj zW5r~!Qth|~WXpDI$ki_GD0%tY43i7#q-fro8HN0rB;OfR zDA?Ksr=VL^4Z%1Pk+F-mE~Wx|;>PPw{#xUQrM5ky&$zKuAj((9d5nf&%pD@9A5N!r zFi`w)K3Rk(QR4}HjV=5+(0{sOxUcIYbK(?yk%DGn+ZhQIAU<9XVH8f`WEs`V-_}hR zRKbKvB>HBzv{}X4N`qErMCPH|s!NXnM~1UdKMj{B$usJVt0|tQ>~K@A?8d4)$z59F zFRh&=!MJjbDcS)H`79iW%J5VP+exf=0rxVpAFX8}jg#Ac0b>(S|0+I%Dz3G}hW_B1G1a2e4IySj|r0mJ7O z&02$>QOb0|PdUd_Ls9%8*m}a1;Tqhmkn9iZCkEZ_c>2{}UX{nTprXxl^XQ|5n-ig& z8RRHCL+{NJ(baC5sWZc1jB1@0#OFmI)Cr}WSWS$>5_eSSICZNHMzLZCBsQ3#mXq$W z#3v55k5VM~A0lCRx2ep| z0GoLHREwRQsyCW*ojHZNUWI^fKna1H{>#2xV+6k+P3EK`G zS|qH*dIA4YQ53f%NNxsuf_(tPi_7aYP!J7tqGL85O{@rr)5U+|-+z zWsv-S*{E)p$H(I%+NHB+mmx0q*>kQgx5@9v+2tkiCIfLp)%-mHw=2NTqi1)*V@$uA zCE{p9u+GiP%W9X8Immvy>%Vw6wE9xt`L!O?(picM$mun}Q_6Ewo{Nj6-rVr|dTS_0 z2*Czq+fNN3-Z@MfMH$4<|Hij-i07CE5IXVy)0;8t9Q~dJLq0`no0t2 zHf#9#=y5TMD8p9_ZNQ~5W<0T*m7A@Lt&?Mzg~Y78oBIPW{7o>y@_PI@*?%4xSpoLl zVD{4Px~R&N*ZV8v;W^Ywkhz!TwYg=6$?%t;uagHmTbGBQ*U$H14wSvIJKXG#UUn}J zwob3d{UO?}O$H~I)^}bk17sSeogP0R0HO7e@j&}w$4-8j(#ud_Dmp{7Acy$=J1jWN z>CtETqHb5>5Wo|CO>}I<4(5yu8GxzEmXr{jGMq#dOeH=ExWq#Y0?W+13uiGWA}&-I zHaw6XG(&e78KAX+sL)(-G!SrWY4OGq*1EAuS_S?B$@g~sx>R8c4CYH;INB^VCGL~V z+T_@XAVlVh>sBk7Qw=xWbFoPAgw?gU4_iI{&>8qyS@PkioWe+ z|5-5pyjcZmA8fFJj3BhTX{_NG+v&OO+xd09?fP$3;|4%S_fOkctE*<}b9k?z6)I^J zkFX<++uD`HL=|Fc1%hBRD8+ZAIi(;Nv)j@JIjzZ|P?#RR7ucv73~xOzL7(x#-92n{ zC&>yM6Hvdi_kw*laIPPTZSL*vZ$%<&DJR2ZhNHT!jV+Muf^cg-IEvD#Au#d88+vxkpT z^qS$i{jwzmL%P}C&a`+nS*!s&)1*Yojwes~`KE|PTh^P)A{V&UOtPT;k3bYvv6Tpno$r^N|gpMzFSAds^;T&~WyJ7Htea-;%5y z0EM*fT25_;u3C={Z7-rp_aVe#@^xjlBMWyBrRt7i8QXvqdIQT}`yKgP5;}o+AK;?XbJFI??B13>Tni$J6diLf8d>kz|2J8Q!VsWNI?Zfw7h| z*p{@2Hmx@{?FpQi*mS?b>4x@m2gL*2TVvw(dbc!zmE5>dmAFfwshJz~dtJECH&f5x z1?L8Kff>rQcm+{4d2^WypW?ng&-@*Qykt@wr4$9ASLPz@LnHyMf9+E z&Yr(l+Kkuged85!vb}@OA7j8CRYohkrCh3<^CDD_E$j8V7m6K5+`#V z8Gz!HwBNe;D;KEkDp`}PYI9jvb_-oRc)*=+V^7meF|<#=gUM;j`dIxA{0N$-PD$`9 zq4Sv;;c0>BA4a|KK}Q|=~aA?!rZVgvn>t){hi zKuU9e7|D*{72dzNq)AZH0FXAb2E)3J5FveE{af}wSN+n-799;;Yt`cm11uDJFomzY$g znVfL7(S+@IrFiPnD)*Nu&aZ;6{hK#bx@x0YWyl03_fq|EqL!ae?H6vAGx;ztNN^s5 zWid;@hkb%FW`Ls^0@Wz~z4a@u>`QCoIWDlO3j+Q6tfD@QF|vl2qM^M8-EZ>6o<~Ak zBSR=Pe+RokGUk=*3zAhWw?cN+AAtxL+iMUKPl5I~q}3z~_+v8gcdiAQqo1@E8;#1< zrZG>zM&H(W7*(#v@IpVBU8QCln67bX9mg{pK49+-sa;3B>}vfvyf%N46D~314T@wn zfs1aZ+tLx6H$erX_#;4$kP=BUh*i1{ zsRhil69OY8Xkip49&nyKb9z@t^VUs5a@xU^d6)0Z+Vr*TjqS*kg`)k2AddCgQV^+G zUJP3n=by12Le)9|!(|4&&?59H-jDOr2ew$gE@n>-(ak!^yf|$y_W6S{c^KzkBi#$v zSidZ0FB+s3);Zp**uR{yUrh+CJ->vqKB$B{fC2AHH^@EmazwOjKUC)PoZORm#vI1$ z{(*c~#;peuVX+XS&pady%I(#_xfOsYp;jS74`o;iCI-n$xOi^|vX-!^M$Rb26WV`+ zb5q8=Ad8~OexWHWHx~JX^m9z+)RI1gJ-fDcYIW?k@BwZ~v z%@#tS{$GuX%4Xk|+sxHgCBuojU&->WhzAbUjpzO@ol5nVW^d3)gZ6!e1tRj(5szQ- zm!-w?Noan5J@VbAlD&TWDu0Q;()%DPVbg)U<%$jr4D}MgCrQ=rut?fu2`RrPiE59S>*H=3D5xl?}-R82-Q(bXa zi~rM5(X(d;=9Psz@KeSG32q;_2fyOdtPM(ew_i2i$OU8H$_rnJB~l0f5K^!aC|Kcs zmfu(j54*}*r40@q`9&r-$f)XvAhivI$c&Bu$>RI~f{dfZhts7#vgK07FZD^|a;Ap5 zMMu)rIvZ$FE(Q+*+pfq6lVw6MtC{X_Q1Y&6)v5vze15{N?iw!S4>W(%D171N(rl?b z)NHS^Ry|=x*twLdlIYlkiTfu=i(t^dZO7%MxRksm&_u3lM=<+RSn6vWw~DnfI?!!L zM--^&bANuH1RYjG=JTN(-OE!oLFkZvld^KDNg$N-*k88*0ivRvcx@8*l5hxCcUNX> zMY~;;zIUYl< z*PzT5mLK6Kc-eUhm)cqBv%SDw6RI^ep^oLYD|p-cf@I#RV}Pp|F(zw6nCZX0oa%3C z{#g8!P)%@UW6n35*hg`~9XZ4v&nlzR^YVTYMi!pey|ZT`8e@{i>q)U&%K`d%bLyRR z?5e=>xdVUUVWW8GRh>5$M`&jZo+4QmYyK}tM?%N_(>*wtfTf8`E1TzmYH$fXs^E&L z^HAfw`XcWB4S5Oxj2R}kY7PcNW`zY$RLPUe1_7F#%j3+^=+Du5pyRiSTjmX}n>&cD zq^dW{%4+d|{CkZ4F8}&)EX7LDtMQ;);#`lLXe?BG$W0Iav*i zar&~=14G^^7WQG?ys&vyFug&*H_967jkA51%A#q+|Iw~oA-fF|ol`iiG)gZ#lZo}I z;*|O!g6%Z}@$idgw3a>1KDAC0rc?GMVtH$L!cK)dpH67&iv*9A^QR^jWyt^$H%!+H z!I?j?X4DvX=qFC!7~`8yA2G!lYZB6{uV$C2VFJduIRW)KzK()IZSm*U1bF*#+SZUn zouISlqNOjirDEgFea0$wC9QDwaGWpqN+|MjGR?oCPpwi|G3GwRQsqMk4L~is`IqJ| zYjr8yiL}F5tH}ELP*=_h9v~F**;VJ0sAv|##X3MuEpjz&K%0&DQ}#ovp#nXyj$YRp zT|KrPapu-l)Zd2h#}WeRP67`Vj*Un+DXZ&x-qmOq6NvETd@7bHI|@k(ojvCM<=hvr zQrKP}X>GCBy*02d=8e{HBmRbNwcF0lfy5Jq=amtD-pB*DSXj@1PvDG!AjZ^z;}qr{ z99C-=!syQ!pI{vd6{HNVymkg^YEa=|F1z4|gP)-}ER2~CEmnvcnc0+XG;$Xi=w89Z z%>nEfo`KtBbDre5rs!M~MX$xEH-)HwUI1|w`^203{5OPt6&mb;E(8}m3ZE?ZdZ^I| zxyZ(Z<^Ba9mz=3X2zPJ~Co8cD&9Jo#iO~Q$j5TFk!f9X>cz^903`%@ZSa`NZO+^sx zO#m5nidftU-C7`64QH!juF#k}1UsJCNps&tB;!x~2XC7nH_EsurRxTdME(gUv`;BS zN7W)sKP9W75@kOzE)JVyIIPH@JQSN9jRI+c+uF%1V^a?>G|s~iAgDBdFmdO^)-3iv z4g%%F-Hcsg*ybFwr)M`^-gzOm`Z`ofjiOCcp}>j$P{$%duGGG9|lH=h}VC7zR49yRs)1W49q;tS2LyL0wG9aM- zW(wux^#4-~WTp-#EDJ#V+Ro{0@ER&27^DL=s)mvuiogz9UcY!60o-F@R|qOEgQ$Q= zIm_11ME1XOe-ol^N^AKHyD~=8VXA|0D`1^l!Zd+PtZ_AD%n2^g`m0G?kccivdgx|~ zXn5(f6CD#!5mO9WsK_)>6yvd`38~^3qG6G_m>2wdi!D>T(|iC`jUOF&3Ah_GyHcyA zjQ{#mCK*r#mhDTiGDd)+!+^#`DWtP#kf*gqX+#(*v?Xf(Uc#v$zDdJrub7aq>E{*t$!Hp__;vC=Z25Zv4NOT)9|=W{;er?cSgW6hHqS5p&00|m$yzbO=#715r& zFuQ6&NCR}h<>V85U7!%995^ygpe*u3cFu9!?nwG%a}eT?t!k`6=KCTEDl5`*$8rAX zi+JW*Y80nPfPJuiY@Yj6?~Z)8{$!e)9bpv$31b^OPY7$Y&~@3*m$E!Dipz>(zGd>M zf$F<`U?zUagTXJ`g#9GB=9R3@ORan5_|}cp3h0Dloncprjxr+IfL#l$)K$wBLG9r7 z3*Rt`y z38Dqxc19j$=G5|EK7ng{OL;+xBCy>o+xWt+NEyLU+q4VjbfwDKSKldHbGI={+YNoR zwV-;-4uH7Ou`>tW1PUPb1+g39}IU^eYNI&F4y z@f-j01l_NOy>WN8aGlC%Y(QP7MF%fCM`zK?M+n5dr{2|^UuVx*JGAhzsg=UjeC?q9 z>nbX1f=y%n2PZ45hP^&DW$+}!ZW+wr#<57}a;mEMIa&9!bWzrLbxat;C6lk=t~)JO zLIi?MOkZ)(!y{HJ9m11b&a$1b$Vqtrx7&DP`RJm9YH($wDwX7roX3mgW zi6^aNp6FTmSnSbif?dtpITAbPF|VRB%~Y9M$;6;cO=tck(Vz-7@~aq|i)9;-%loJ$ zjY@54Da=|5gQaSZkX*Xyup04N&43Lm8BKsvVDWcp3AwIr+3yO;(;7ma*8&Cf-2>Y) zD|U)inFrHv4|!*W2m2gXtW2BTA2!k^8IP4c@L{x=kVlZgcoDKl^R4) zDr>Dj9-KlOAzWsE<(WmvTgx9s^@;wpqkhM~tO^KjL8CJ(iYrb!dsBMlven8p)kxf~ zwTbFkDAX+Wp}d{PFjkY6Wk?v*A7b>$2bJfEvGXNh@t_b-D%g_R7!8w*(STPYUvC{L zT1MKBS#*qCCq7BfVi-CZ*xtV}w#Z2%6YK&-Q6m`#_7gOt=@+aq#;d}UR1Oqs7dGrqeCO?r$gRZ(#kF)(iM0jBfv^@L=6%kKIwb|Q zZa8a+J40s@9!svcskjcNqY6celRL5Ju@g$L#lqIerYuP9?!k;|`N&em*SN9P`BK+< z>S28^sCxDC<;ywh;I<_#IPC1mk`t%LI2;{racU9M!I6IGkct^|uwTZl$9#&{HAJ&$ zO#)brt0i$LNf}&3K1mDpXt^J3)G6<|LfI!IY5_!fpf)kdYxOvpI$f~qq1o2zL6>Ju zx{+0ba?6_${*xU!!eF{tkbKAydl0^bAMsUqe|+>_oe$u#|jQ z&<@vGBPgxK-@d1C0VC(sDXpXhA(BGf6!hG^!)@{}KToazxEF|M!E z5?a)|Ohp7%!mI}1D)9M-n;GmaRy=b)#I zq2xU{j>nbs5E~c2B$KAy447$w1>|U6>bN>KrAtE%OuqCc!PKh3Yef zjvCe~5$CcevU^IeNaBsW8w%Xk(;i!d#1l1FJ`yuhODW4n1O=*%x)MuFbsb!WLUfsS8KRq}WYVoag4Sbok31FzUT;BIje3*`Ar0 zY5Pwu@N=T|{Y#<%J2>8;YCp(uTive`JsIJcn#6*Bv$8xxa z)hZuy4yb5=u5YuT+!dWh+pZ?2Eg~|=67f%QVrMgj5dn<=WD^cyDg<>X$ez6T>DY!CL2v!wgq%!_Cyy)8rek8*dA+w@eG+-qN!-d zNlihWyk2%UAD5pibNk0yRnlsiSj9rw$rGZz{~@j0{_5#M^?^mb3n2DrK7^Okt`{ZvI=X_xFiU zl_Xs^=5^w>&DB7>xC*Rq82tg4Oly_4z%x!J8gXHdk|wC|&8D`is5H)Qxtwy;$V#(B z2^*X>;*R$wl~UgGmKvM-yj>Jn&qf4Qt6l=Y$1d%rE{B9)egVCQr%>-Dv#IgSdN{*G zWBV#x`z<}X&eN45L7#$63D$0%=6%JT%1XjbUii@yEaXFtFm$P_BZ9(!TS6Wo#r<(A@GEu#GuR5X z$6#=zzYcbJakl03HvXeWE6#I7Odb&fgAH;6-Bw3RXG1S1Oy-hFG?Ql;vWjF{nJ%VE zs$>Db(%BD0*VW7q{pl zHiS9{X`T{*sN6hD4+*O4PlR|{DQLey1#W6CVU|#`ibT3dMJ7F}fR}tqU(k_qFkGQh zK$|<)&MqBnB{(=g!_F~AUTOtz{(Sh?_8nkRJKK*sW(cNx=@in>70cry7l2T-(!;3K zWO5pszz6e=H99*Ka5b0c9`Bm=+NGi6?IRv_8Twca8;2_U!VFcLvCSI6I_=pi*)s0D z95b#hi7%H$yX9V@BbGN(@g$5gzS{0MOtMJcd*#7r2Y14AKY5_7!X2Aq4eg|Uc` z07p=L#6F1y=`5BJO>t~lHae4o(tRGD_OT_&z>L|Z`v_-AD(_XVE)q!*K(e546%h|; z8rA$)%IE!#YO%J#fXDAg`rRV82on=FH8QHs$zA$Sp+7WLxd)htGigx#DHQMcq7NJBmfRra*&F$_ zX>dVq# zh8L1njBvKWyHm!a4rijolV}9*fP%YF?VVJii=v}JD%UGrNfJ!M zA*8<=VN%ZnVE>jXXmWeO5xUFlyG-zMTvKCL)TKg(YO(_Z;$V~++D(mJof0OP+4%f2 z)MlhLQH%gz7P;$>w}hPw&msz?NOe&DmbHvk8KO>!X;!%hy~?K7RgoxeYrmZRf7w}` zE;OF z@8e|P3u_s(w{*slsIUwhnK^7z`ERr;#!19qpwUXp5E&5pT8m@p2!cAQa77}u=Jn~% zMFvK>Zv+d-cCaloUlKg}1x|9$;W?7_m9B0hB!!@;q$aWb-+ssL2RztJ!qi<&qJool* ztt0nu1MNag2Kiba{ezDy@<|!Ti&qcjBPyR1Qe%;7%15*rEJKM^???UaaNS?2_ptN` zd2n$y3X=Mv^Z+@A=&a@`{~YP@6z(9ZC*!*yo`m`#|3S4%++hb6x+ELdC#>O-&6n@a2VStnwh>CCT&`k?vqD^A8B3}me&AHn%I^zC4-0gNY3lI(lp?OrSaE)8XAtgaL z%ilRR%0>GZZNxIez*}uA=;obQ;4YXmG@k0h=@qfc)62P1E5Tb`2EE|w6V)9wx7X^2 zA7ZG&8GnKr&irqF>X0>RluLxs@W<&rq}zJOAhO9=P|8kasKY0Tjt55;PosG85H_5J zh91hCvT&R@XF7e1Z{F0%)RCi}Nr`#oyck-*?n*3HG7?3M0{>s40x5)~kfBHZG)Dtw zhaVmLHidWa;f+8r2>3BgHK;LbRCMVAWL<6;vM&5KUN?;U?0W^it5{AGJ*rKlK9-Fv z39#nf=Q_FPs&&=1X$@nU`|$pry#v`TdMAC=KA}J^MGf z6`MWRmTVF`Z~0bsOxHQ{Q5(18O+!lO3oFWG_i|`$NPVGLrSS%a2x8$EY+^NA9dSGr zSkmhl@XvT+i#l+>bmLQ^KoYv%uD!)S;(9$&RnpcrVuDc7i)T!{coLc846)XPBv>)Q zdZNn$CM-A^Bs+Ca+6hOI>62svK}nERh6KkGi4Bx3O__x2zz%d~@!J^{$KwxtTRP}L z>uRanVh-G!f2qhhWbhza#Jy^H)3|JkiFuApmtcN64g*c=)$lXi7tZCVf+*!wQ^I#)xuL0c%q$Lj^elnT^oO=+iEZ zvb3NtsxY*hHepPiI^E0WKVrGR$qucEJ#TohnG8`<3%sVz5mdH-^j%OxY$(Xi+~kyY z&QLwNVQe@p>hhYN!5mpz@dvh0%NL1EB2ys!*ArDF@{v3t^i%%@5J0Zjr2e~sI$q+D zoWy45&lUYKx=qE*gH#ni3mWGr$>+EsI!$ zR+cMS3T_ug7qR`-jF??{VI_TAhAr8a?A4T;=qWVWWX+3s3T$_ob);jX?OgI!|E<7( zt$Vh^0wm>jkVy6o-K`(#x0L$Jt-#sb)`PlI_x03704$dX_N`E5UDw_$EMG&WdS;03 zdx>~vPH2vueHhj)5(b>`j=mtO+8tm|BW7 zNulgVR3=oNdU)1+X=WV^s&H}=1`-R0&dl{w4G(sn!&p_`5$+5MejN#>n- zImRNB&8mi;`{XtfPJ2O9ZB28ZCO;Eak`DBE`S(&tH?c4ZSO^MO8)N&zn#KLb1N-TC zxgUzaW76Ek;AlLy?Tci9xhz;=3fk1uGa06C?)9UUrZ(Sb>oguOrY~Itx|Y9^0ZJI# zfStcG8FRI&OW}HA7VH5nE#dy1!J+v4I24z=D4&JzS>icb%uTWM;WA_}`H;rOr6@O& zBZXYk?yk12S=bmOq;#ocYRDQqCE85#ILKAhJOzK8*+BSV|2(r#K=6S(Ytf7cm(#lg zMlRj-bt{#bG_0#=X^M#9tfU|{xWSpB>*JsZDl9ePvs{xgH8_wuyE<`3zIluJ;>PI< zlJZnFVM7pIV5=L4U&$Ys;S8m~S)~q=)OgnpMOh~GuXGLxlLUACG@6mGwDOw`f8{NwK8spRGTd6asp2$&{y1NYt_K980Vy*Hz@ z;*sG4Q^U8q!KpV6K-OWi!DZ%f5Nt$(u0crfr+^hb2k7Sq8p2 z9!6|D&@c=g?~vGG-Q8&!Ma?o&J742UQtogd(>oicigGYMm`CduGn6rp?Kz^xsuuE> z4#EF?G^iBTEqkSK3WU7URj4rlndR)`WW`?G#cMhX*+y$39{Nuoq>E%nG~eA?gvl||e}{#yc7$-MtU=5D2^O&W0;q^#e$b_{uP zzR!TTYcM~_I38{6vyqCbvvpXnxcD1bd1*?7^-NTZf1*E6p3w|KrwzqS5^$Y7g*Qax z(~aaL@_I)yY6R>4BuIrX5pk3j921VXtYnzj1x27ffi3ngbKJ^$H5t`I#6Ze`PQ2LL z>=gE*QABs0q_!=hQJTT|+8{TXGS)SteN;RKgh_PMNc4}eT3|#tu!@Y9Oa%%(Zj!dqk9(kX)ssXDc}A5HZXstt{MY5m_>G z0=WbN9BN!XDj+o3iV^Y5vb7$P+1J=^s&ZY_Aa#}amnT+g_`edyJ_(vs-kAxY%O$yk zSx^lrPA>MQf>HUuKfwc3x*WMR2vQG9HSQ%Q81W{#h(zK$K$4Z-;gL;EY~zdN`X}hN z&Y`YJ<=#*Gz($A!{dtgxj%P3Vitu#n(o0#8VM*fOen=&tCa-$_2*EkuKtk^c)B(t6 zh{NJ@H-AH%THaAF`^P^7ygKE6NACH}=az2@`OM!J=qx3ao$ucBJL4U$HNDcdVGNE* zJ4vyZa3l6nQdR|@Nq8G0XEl_cK!QI0!QT!n8C%YhHxZtrYW|IrP%ogPB?l`Dys)9) z0Cew0mdYK|Gzq}y6HxR#LV1q0DNYba=Puge3CQ0%gmygFI6ocpYl`@HpMZ?-i5>-E zh{#`}MDp-pHw}Yh{7D~GV8$NAaNG>Ch{3m#L3-Mzl$s|+aW>XLntx-9Q)WT9RyHtb zv_iVeM_fb)bdf}8*7C?sNuYoi4(Ak(WpY=dI)Nb>c4h#@)nO&B*zp<3&JAzQeA=dC8V|Q!bmf zLpoRZo3Rw6Q*9#H))ZYIM&QG}uVVV?H(-9Aen#*5%rMUwof_u1oySaIP6LNC@>el<|c6-eDa(R4QzKx+;9@_O_>{yPI z#SusH@f&JO0JP;b>cl}t;X9oCUuQhrN#h);Ne;m>;j!shQ`z;KvEAa9-==L)xhT2N zXPgXlQoDm^Ce1iJb8g5;^{r9mE@{Js8X~|!Sk;Fh#!2}{!@Y}Z=+-3*2-{e~gEFkd zwdJ*96s*{xd981wYScJUn(x03y|1K@2dAR`uCK_KrCtADY@9=wC;^jY%eGzh$~Irw zwr$(CZQHhO+qP|M-akFF=&yU3+ss8q+&K3fpT9#r_EfhlY;s=l(p>Dr!;*af9Lh z4!20%$Gs6@`QSc!cd8HNUORR59USQumOfHhMvazjYPJ<`Z@7Qn;eI}=w&Y#W+np%6 zj5e8^aeiAJc+9XynU%t-^!igW3rE^5|xRdBsM6LclNLjTT3k5WfPXxAo2f$AtI zmeKV~wL#$bN~oKtppaUSmsWDPit73M@y&-3`Pl7CGi`_zaj054 zJmbA*V)+E41?ZX#d@i4J>%D8l8}WLj;E`22(;>{R6r_xZmYyK_RI=attLR6Iuf7l7 zo?w0|Bn6h~uiT(`hcauJZ zG-UWCCEN>B4KIEtL`b9EZNa}xJuQzQcOjwvBRO^O8N?m9%e-&(yFWO({T=EV>$0v0 z;PBA`tf&ygP4 z8DZ^p3-!Hi99n?9c=*Gf)0bG=LwQMd6K1Vg-+IG|Y;S_(t{c_T+A~J+sL5FsiwB*I z72|hK1pwv5sD!PxgdB5+3`WH9HGR2g+V6z+R<>~TvDu%Xo3a$T8+9>D(Q{Taeoe!| z@?X@ubAsE_caI{IG7L%%{(Z^maERkpY!50!}J0Wx>={$K$#z7Ou)-@Nn6yr}QO7`Qy>pKkBd zRXKaKlb zKDo}r)t=WD`G=f+1GA%6e73FHfokli6D3o^-70IQ$Wl3Hce4%P1{)oQr5xqV)3 zK98(^55Dhr_UCtgc30I?M>V1(ad3Jzd)Ytj?Cio!qy0*X6@F8%8zUPJ;Y&L|&GPgc zw3xpddD*teHEsAFqtef*aj5Pem1!H~@6=Z}W6=}H;>VoI81It<^w34h)pkiutdO|a zyp#1O#vfWPHgS^$y>!j`2?$z6T40zF=tn>Fjw$!%x>itFn8=}h*S*`?YM8(5Ojry>4xNHW2>>Ts? z(~4KIttuB(ssve+XE6^iJ!E<8tRQMoR z%<_w8JL=^zQ)*%=$fEEQb}+LacM$Qz?)P|jKAZZT;vQf?W0Z)2(#_)2ap!}V>7L$@ z7Il_=X=$lt+)2z`8jJSqnhg^ZqA)k!TItXmSt z%gEG3)AA7@@6dhX=yQawRLeXXgFAM*C>8V?(rU_vI;MXtKEre;bCjt&nl*UF4_(4J z>RuHBuAavgA;ViQfODRW7Mj>r*B_rscxr+ywlckBiIwRdb{<(gY$e&3g^jNb5| zi*1#MBzk1liBx!%u$_r_TGcYnywF(?R0QWrB}FaKhoi=N?Q{P=-$3Qio;3)#xvJlW zqU;6cPY^z7YwlLIy7pn;)QZ7vCzj;8R<*jgnGo(Y4qRBj?%4pMCfRP<=|%EPzbV(S zb}c{=*`liG+^34i&YSC!vT29_b(g8VW*diK$|}Z7vj;dfbw0Ue)jjqsMaivUzOTVj z&76U&dpM{9_zLtB2O0CV683zlh~2B@kDqB{Hc2Gtq$H@|)0qbo zcTxRWl$%g;qn9evCfZRqT~#%YVzWh zLN^shk+U4lzY?}$OZ>fjLEY^cuAQVUR7yPXL2e?-6^Nw>>z$*VMZnO|-#_wiXK;m? zZdej>R>rB+@Ah6%6?8GVoCQR#_Jgn3)Y1CiIPiis1wpzisc3+HO7B_u>L5n+iVN@T8mIQ=Wxx7Ko@dRFlT zuSFfpw9ZL&(Zv*3<}rPkwHE&2klYhR7>`mN?2V|SL!hSEwOE`NKL?)-VlqDY5r6i; z#o9WyW)%-)dQ*}}$-+ouS`|6ANSs{NxH|jeEt33s)Z#OMK?|Ny9%X-`HkXhn*&4Cc z_rjKTz7m8k6{O;M_~UiDi-yIU(z2ok!%Xe6zkV=c1jSCY?wg+91zKLai?$sZ+Q{!2 z;2BuBi?&iaDRth+I*b>iJtM@w{E=@pCc}u@gjT0!c_@+yt`dFoGm@f z1Veccw(ZFGdlEUqm^l$B0)z^*PApDT@MGVb$@9|&c=v( zdTbD5lexsg&#eqIN0Biou2?ZADO@jvN7CCLWYNy)M!nMTQeyDETH*N)+ z;fP2%PqE&XgNyH{%P6PryU+S8>H^hc9 zo{}bdqC&7c8nmyGOsx;S%v;hT+U&!G0I>^JU}B}R)GN+LPH9V{CAEo}xMe`iT*4q5 z$5gPYie-|5n%-;vujFP`!;(AoW9Is?=TCZ_@=n{wzzp_fRMgvB_&JxfvI)=;6jwVj z(ooLNn}K!UD`WDLGM63%m{jQmM*5RQi+d2W6O4<^THizqR!by^iGl?BJ3~!pIu^Y^ zxHa8(4X0T#^=8WX*sfad!qS*Tf~Y>km>#PgBHBxlVl7OCIz{tbW%U0j{9Q|CR6ag}@ zGwf6lkbtmu0My5a=itz!yZ%8X|A8?i(lUq>T%6l`G(XS)qKA@*3?#?!)F*27GNpvo z=s`&L^v}mC2FaYo^j`;Kld((A_9!AtI}HpeyVRxN^{x_U5rSTMFdq7cC4vV^u_R{+ z{_c(mV=!4x{^Teue90h5<1<|6G$4u-fWld&oy)`eV3vIi*oAoOv~ps+We0Xz!2*4& zv@Uc^O0#wwU#1-6KFm7vwfqvl6-*8lYflh(!Q7}YizWFa*Wo|mJxZGmgHrqJ^bC^j z(6H=%O|gFl2e#W*WFS#@Ccm$O=Kl19gk;8Nl+n2pj_XS#{o>&*M!195t^A{URJsn1 zRXv9z7pN*CDIDE~BW+JmXRweMf@}ohHLSb|rVC;cA+Jr-CrH&&jQa@paUVhqkPIRF z!09V>M!;9*w+yVSC#tK&V?2exc&Yh!d7d219;gz1*z?jf>FF=gl8lKE18Io5i0z2} zJ@^);omORRC30C&a$QY<(BKKXB))&@ISw!AC{bSzt7>jME-X|;T05$3Jla~BGy8f; zdCU39jR}I7`9XN}F9_1iF_Ms3*>^E!NWSMV*rcP*VkJ}&wz@Lu0t0Rn?4DgbW?z97_&zwSi` zUskZYv;>C>bXca04W?McH?g_1Y7}ue-*C`k*TDl^)ygdaJzoxpW>-H_qbEuLFNzr3 zCyp>Cl|aZ+R<@nhY}*9C!=i42I~8fv(b^nCbX-OmRA4v>6?wO*e>R(lB2L0NEXG4O zK+QtQ$cFWO0UT49w=UWB+o4PYlUvr|5NeZ^ zUiIA}Lt@7Mrn+|nus2R|ZJa+fz;%LoQ9>B*my{5~o3}J{i{WFd<*n1de5xB=X-QI} z(u#M|x7=(SQb`t}k^>P<4|)J*Bfe^Qcq4&0SQpP65ScL806KxkdpgFuhdO_N#7PLA z2O1LxL|fez=l&_2nnWbC*f5sRdw8WxWx36DBiQT*GimD|imV2*J`j{l(X7*~JMzpV z=mgiUpr$qzmj=j)bXr~Ems4KlxY{@-Z5=oCx^Y~A>>%q4ogFM{x>UR!!1m#la+Ja% zp2!MHW2tf;v>THygY8zN@W4y2`{34TR1^mgOD&6WCkaki#q}@8&NP#-2rR#9n~GfB z2TVg!2;{Yvt1?OIT9v(cb%}3|h^+E%j;u}&#j};1BsGz>STVvoBf;A z9v%4p%G$jgabE4yd|sVYqPiWFJ@};3lYWk?L91skZ4M1UJeC1XiXE1?*wto^b2*=V z^oRhKDadF`W`q1C(!_7gruk9_(b z96`*Z*$6Yr5!PtXNudAv#&<(G6$~qnoK&+#rnqhxq%mMttkJ>#$D^vD6Ga8{!G3h^ z1@09ayJ=Kn(kaY3DrI^r<6yM}z{ZD^{dx(ia>+%0^m0f2AD-dDW8LoX8q5c+q6*3q z>XHMV)PhE{Zfuj@mui$mw32@ctUA*YK2W26(7>#92`ZSZ?eIhpzJi}w-4}I@<%&R* z27ZR1FQwNh4i4F^k#f~u)tRK|M>)bYWk#di4(db~O)W;u?M9WUGuVRF@PtBQTO=uN zt0_+2mvHCSm9lpm0O+T3w^bW2O@oi)Kd|V)d<{UVPce86$Dw3Lzq8c!xjpzH0jjCm zgQYpDnXqg~;iz}nx*5|1E>twR-oe=DwPMq!tZLU*GMz@a7;#)oURZVN(B!BPBHy82 z+)l!aGSBSS1H9aW&3}H5yVlMO_-!*dOytuqSAHH)fQOqKt8NVnD1w}29JQu`mk$W} zPG}{EK~CwV`Ibjr2!)sU{Y^{Sqs4H~;llbo9=gtiE}KLDas=taozAW!Wp*vY#8V%8 zC6DVHRqjk6Bt(1_%~^jaA!jMOMS@99Ix5nbzNma1c)fciBi}gAhv`-j3h^~GCvsI` zHehK>eLIEAoxYIThmRONGk6q8a9wJ9wH(YXV~C43mtroY zq$-CF!09t02GYKpLm1`isP~_c3h|Q z1~(Oomgh_&hFYprrIBLt>C~&eyh(`~_U%`Br;uhq1fOQhH?~Id4$fKc*Ld`0RnV24 zQo{zN5blIHeC_&S7le1=mrWJoXKFsIF#ZgI!D4Nr6NyiG=z?}t(^Y-&@RP`YQ{+SZ ztXxA!xKXI=gk>1Xy_v?P`&%sQUNSxDFdwY#K5IHQ_e@s;22dSnI7GVYq0RjpF_>eL_Gm@v9Mm{lfz2$Rf;)o^8AjUw+Q&c5|3 zpolZW%yBeoX7lNTwKa$X-prncl8wr5;+PU{>K{hWniy{mpt)<^=}Ky~L6@On8CGd+YuE zRNaE0vq?6_0ROnu&k0TKk>`e%*(XbHE?RgcWN&jk_;PXf4CM?#*h>75M`TSLOQo0> zJ9m*YJ#mK~GV=90n|tnz&<(F;t8VH?gQYoiw1Gv$??&FuuWjJxjzBJ~Y08;Js`|Wz zZNOK8Sv|^A#9!FK*fQzn{~ira z@)M|&!2jy#BH_GN*%WK&9amfm+s0rO8%T=_fyvVkm4PdzX{G^Hr&&T8aWus<)7|6g+eYtH-cF$Q1sq@FeYx^D0V(h7sOaeE>F8|w zUA%vn|HMa^zDw`{Kwk|lttO(E+WgVtY`8vrVfgws{9L3^s+KkzrB3oWh>Qbm+2&%> z0l?5n5bA8<#(=5fpGLBuwb9*x52}_JY!qdph;H5*V!5ExxuVK||0YNRXS!3`?e;7U zC8F=^8BLdje_K0+1jf>T=v?aMrH>IxyQ62=91O$WJ8K0d+J_I9#(3$wSEF`!h$Gd0 ztOjfuDVitoLAv`Gw)-??N0ZqYt&dU*PW4U|w}mgW++ET05ZeoFMR?rniyNokTR7xm z=!N)Pa!G>gmg$@DB8Kc6<8w5IXX-$LjIIjxcM2mNw;BPhUw#u z1Wf(#g4t)<0c%ae`YIun!WyBkakO^nZ<)g+V2rn*W-MsroE*2G?XkB_a&rh2-X zDk@DZCw`Em+R(%M@=78^ma{qd_VU=8vPMVBrcdbx`4>=n;0fXhx7-C z1A1DFL@mROaD;?xnv)(2H=fWVRuaypq>ND&O(a_)r!k2NQQkkKi39)mFbMA?2Ff08 zxUK@U8etv2t{41JOz9CM!}N^E{7^b}JO81axAzxLx91|b#Fa{-xb>yjAuwdq4ktcD z+!&a06}Iui#niI&<0MvS(OAyCM?1M#iNBect9;Q{ZqKTgk@6PZ=sP!0tGeMIyS}J&Cp- z`Z2`Y-kh%UpSb&Q=laIG&$s=MX+r4u`yP7(shNMNVcf0Eb}a>f;S6`L?a%Iv`rN}b zLckD!ptavHg}ZGJ*QB~7)Z|p{+qVYBuRWVTFgss8CF;ghs?Wp&J@&2S=AmJHW{HqV z(7p7Y)VCf56`dw7#;+3piLGai3U^e!6U+v0d4;^Uj!r$p+8=qDE{?QG z)%hNtk>U*;o?{DUC;O-j<7zO1{Pk?Wyt2Ay@arV_X1;!=_Q-=Yfe}09qoNYyF`ozd zp78L+QZ^rv>8{Beuj<(C)n)%MU33`o1SH_sQiay?>5g2{qI#z1Ias$iI@4l~w3&~M zJr07OcjB~@m)~op@lO?mjT&OfqBU#t4juzV5%EO8>W+RE;$Q;hnaIm!mQx;n&_AY9 zAe)05TgwYW|<|-YII+}BaOYL!t!3q1O~_kF^&_ZpCK%UpUa%>N28cWWmuYq z7dIhAMK+ce9G09#5Fh{-`W%M?YL)LVrttKH6bIjHYoX2H#>niNFgL2z=+&~1zbm>xe05Q!GkBgq%rgIzmTtlXEX)%&u4jqQdYrtKB1z||Q zMR6J}|3>5VlMm)_%9UjI#0uW3$0cl{$im-ytsd*kOy9RTdIrt}T`8i4g=qQEk;1rF zVcb5@N(LH%#dv3eurFgpM>5|R8Vq-o(UFE5q3>A#I7=i+FO{%s()W$_fFJ?`U+1tf z>sU-Aq7B!w46mdiMzsSi-?KaI>*ra8j3&XJhTmMk za0#ysKO0gy%YekW0%FC&npOP1_HMkz@2PzRzUv@4+L$yBM_cj^P^U6tCYL7#*u!Bd zC0o)k=F*OOP4Dj|((1FaGm+(<@|@1EefVHi`uOigDS;W-q;r;pO=qYRfsv}qiBF_7 zKSrQ*nJnu>e;p#J5k|AQhQ(Gh@Q_yqWs}6YR0>tGl37tGOgzq#eNF|7;Aay!Y^aT- zftW(aG=AlA?n$`z0U*Z)(|l7`R0gh}wa&ON8xB&LhZPQ_T&C&1lEL4z5KWCXJj(6tM zbfy-_G@aL2&P*O~PfCqei5h0WmY~6@)*QmEa3{hqP3~frsm{eK1ua#!`BTa62$iir zz*0!cjTBh+6#7=S?#E*B^Wg05PW|>Rmdk+>pfuP46Rd=mbDo!YMSq%-M%=27RSL?1 zrHY1Ebqt)!4%sy0iAPqvrm_Oolpqe9FK(3xd3xwzGr8f1Rz#}8_;f7IZ%q5uzc1gv zXDJyBY=%}XsSF$siI57_s=ZL;X5`a(o9QG{im@R-S+h?Nx@doSs>|D73)0GkrdYm; zeq5w>D6=Q3fH{o~U1N;b&mt(j-FXCh`PdV-%nz?Rb8FpVl&i#9_CnGaM*!d`&a9pV z)m#&k=8?*IJk|>_(h6P+vVS1M9m)Qu@i+UNDl(mJrZ)Cjv}>? z{e^6rhphP`95DEeI9iYip&PGVz{$5@lZ18-+zSP``BW5O41h0)6{t%$tU$^^05W!b zLKDous8TQJ!yjq3z_;A%3|m?;7ZZuBSzLx$$fcqUN5`olX3NPNkDRszn$`*bA^#&q zQ1_IriIO?tte5uel4q=r)|_T3nk2+5`vw`9v(Q*B8zb3yrB@FQ3{O%j<4*+sYgG9W z?G3Ax_h!9d`=V(O0H{BV9Rb7{OtPmMJ)KhtuY`uwct@67UsE|rGbiOUwLS@s7C^2L zJyq;hzEpS$NPeEMLD~!%`qUz|PXn%$w0o`eR?*ZR>13V-^uJ3IkvthRHBLqXr}-6` z;DZqyTpWk5hJ8g-YDplN^Ohel0Y1RH-+cN?!SmSF0tfYmBiVI&Qv>Zr8>XZ2u}{HU zVWNTtsC`~J&dC|j>sd= zL?#-d%jo?j%~;9%N)=MMMT%lsLt(I(3mgYw3kzY3kb-TPQZ>P|NqCV<*pdss1w-jh zqj)!|LM^mHZK{|>rr0$~QA?^=g-Fp3N|A*makFsI4q8!5ikO7@N@swQbIjye&T_S` z&pVse?w#>i;6b+P@>c>nol>rLU_Qi#V?3R73{CEZ5 z(=%_ta~tZz&SQ-M&qkd`oJoRr`}vj$r2OpJUF@&wZkI|8smvoHOK{TSiZ*6_|ky(3*A zPK@3^X1KdXOVEu(HgoY}Uz|AS&t*~{z8K}&VM zy{GHnIy8AZ?{=2+0>Jt@V!501fohD4Nt+#_aDF;u{aq-QqkQn0mHW+>_Zu@aE2Dv5 zL#Iqa|FzL(#R4arsoFWmWz)Q0=kDOM9uzsE-0A!Rs9}2T_%{0fOyXsOr^C#{j2h9q zzRTZ2EOo9QAT%P-oK(b_%w1=lN7Ie(=$fif<`-|nL*D^4Hxd39JCKo?9HHwi$9;27 z1a^LzljOEgfr2FQY>lhQ!U=O5 z=Ta?oA4`L++=hXMsWLg@jT!?Xn~F z@5|i&-k)_*wgI`obh<}7-y)zy=nFV^kKa6~s%kNIfil_6sE=d|g)Bw!4ffdKz@J}t z^IYrO->2iVws{+`aSe(7Rvb@?nhTWoaa%z(keIB*8ua5?2q2lB}#ju7?ORLOur2xqi~4 zG|6)k-GKwaZIJf(;^(!q=b!W$BCLXCak8Z$)UV~SXk|DB)}NQE)nbJJr8mMz(P}WHbORg2 zYt{2q@N%>D9tHwT+*)2qk<5uV1$i*TFt^IO;-2f~!y|Y#FgNGn`tdVldnYTGLs3(3 ztz7>1F*I(zZmu4j?iQ8d30d~-oIM?OctI=AJAAi!owpnNF~8n??)GssW@up)kR(ge z!XRC@cY>Qi^@JrzL~A%Rr?b|3h!*B>wK`)rG$bq0@iQw*E3JTNlh5giTx_yGx7JHi3u1jGWkcRNgT{)4m`yuZZ#C9Ig$~J zCj!We4{KPGBFIlV$G@hv3A{#A7l49gPc0n7uZM_Hkf~08?rVH^uy?k4JXc3qHs{^T&wF`$Fn6?e zbaG$pfHjLTnHP@H3)@qbSzIuQtbs5(+;4v$4tobhgH-6TIbd(5lq zb|BoOM&gL|)>@9UL_e{|atuj?)m)6?YRK2zjD*fL3(X=%m3Rvn%*{h^(xzyK6@6zn zW%k7W3G;&whTyP9D(d3^0<_eS(%L$g*|^@QH~LokqdXjcH15SW0L_>mIq1Fhy##hn zOf2jVjVKp7oKSKR6oXhA9e3qNE%5#5V(3{M6YIZ^b^`oWAQNiG^C58hmB)LR~H zSCI>Ge{NM$MH=2W>{gND_PopTnP!&URVyHPa^2C!dW$=qjf`+@j}NgCo^XtP`z(Zw z!BXKCHW$6~a7M?t*JRz3jACyRzdp2aWPZUR{j~`Dd{KbmMvWbV>+|m_kIF zXzM|y4I)={$rzat)tE>TnJA&cFK8s~_!wFiyR1If)NCQFM`m1*bVE2#g1nt1#iC4W zq*#g&jaLFq?x`RopUl{VLL4cBwx1^SQPI9DN*{p%GUL*LoTg7F(93?TLT3b}a;vm& zPAvk*pt`i-Ohv+}XH7uvjLmr{a|~Dd!ZAgx!vM<;(&K$W)hhAAhGu$w>Y>K6FoPkn zpjmANsO68^@eWjRMn*%18H=X)xI{Hf@t6X&bt_j_A>OjYd@}C|Ui7e<9j!@})*iRP2FP=m9b7LV)iID=}Vwn1v zdWO)XJwMADvnb&lO4$lwpo?0T(R1c*0LE=b&9h%kL+^g*N`bYHg1ku~q~J2ee)@^> z{U#+MG!Ry5=R4x_ZRl7_hM)#r48K_^`E+8yC;^ZLLNvm=)T~2Hbj929x$SnIK7RU>EevG6!W7k{nH{DM{%uYDFo@ zs^ZL~vgUyxREl%b(2@?aohkRoNDO2VsoN?Lv%i4V5y)zpzh9#LD>PeF_dZCNZ^R*b z;s2CSW5khyd26b0^|*>bk3?RE6&_tEhe+&-9hZHgeo}KX>-NQM%G#KYA(MDjlIF9K z%3cMOvwmRQoF^e}sF?~pyZ-MLg>O*++uq`!ie8ImFcJ3rcjak)q>Q3LH1Y1vyJS=1 zQm+Ph*B)A1o2rkMJ?#;~9nT7pCBCdf5z3efWF-^Bc%N{Kw9)FWQ+1FI@qy7xoHWI3 z_7#n>&hO_$qDB&xOp9FHk$Oj3HUjqvot0z4_JOWvLzkyaCEB3fStekaZmIhg_I7QP zq~dTS9Sjhap9CCKLa{znC2M%AFP}WR{+V8&VChH{L|q#-5d1m=bW|C~l|J2mDL0?! z=!*%qr#J;80x2M223M!NDrFnyY@HiGms~oR!8e*!DU%&QfNn zSUIN!?M0ucfXw%lQDcORk)+}c;seqh_0}A;xt3sI7)W22M}V_CUUyVD>Tj<&BX{I_f%f~>oiA{8&P`zT>73e=c3k1|>M^<)=w`f6 zHTx!M4*>~mqsGE!0X4%vubo|0y3S)^*`}gsO6fhUJmK!XeX3&1gK^R=2{{ z>g-ScEH|$Hd<0)PUj2Cr*7hCxP3=AX{SErv^&D9BXGarQ_Ly4j5j(MFAMPcsu50b| zT-(~TzP{~){EPYh0sZ@&I*v1Qw~m2KIw|u{WtoeCvb|*y_fQLvCG;kcHUiFx#AM9% zPYO8G5W5`u)q0q++hVOheG%yS3-sT|mW|IF$0%$7fHNNe0QCQP7ckMc|F4Nf`%-07 zqTvUZSJ?Jc3Z}6>%73fPjgEj=pAHCyowHF@1+9{%*&0;6x{L(pmg$l25$F_frfaRd z!e%^y275g&D4ppjfabZlqk~pgxmBZ+xBD-r)vu4w(8qkA;pr{BObonw^NyL<8kZIG zzq@{YZ;y2^r@pjhcC+9O6`MoxY`4haS(Ao-)kbWAUgtapTQYwB!VYGj$@`Ug6uB-52>! zMEj!8FXRZyCYQFJ%VJ*9SZf1tjh|dfgd35CHs8X@EKZUg?PJkg^ByMktAConS-KH7 zRaXb}eZzwV3Q#>6?bmKJLCBv_ae*~NxtS;7YJxA2jifZHaR*!h{VZR^okHpb}O{}Z_G`+IZN z)V<5Y!};$mjq0Xa#eTO^46c1fCRRo^Homa>l&eQ?Xn0(<7Dh^v%etyrZ+qyoFuB6d z%E-LT(#6*7WOx7N@pgV%wO%>SK{`aLjg^h5q22BE@rHS|K$}r4`PuxQtek9oMGqC7 zpO$q5mEzNCz1GE_Mi4{^2 zrNT_DXfN!Wg1g(alYK@pcrC5+d_bj5mX_3i?HX3mfGUEd_@?4fAN>-&!u%vv!_AM* zmgZ*V7p5{D*@+yo3!JD&2qC1ZNF3(8(%FiZM9h?le=IQ}6R|VB#p4de|7D45d^hwb zpxhupWr~TYS5*n4;DpebO^QcX(-^CMJ})LNf<%a1U9rRacdx}vgYfaL|K7kha@X|f zgV5vC->*yJ-h2{PwTRVSSmojd)hr@c5f$Vk@Ic_4#m%LG1(re3D zkhy+lRxA1(zJ%5Dt(bU_y=ato1M$mh+Ie~}IW!%3E(5}Q*65Z-ZeE$+HUF&vS#Cr8 z%DHRAGFv)|_CvIf#XYRHD8c)(+eCxfS~9Ek(6W|4-Ub6lK4^a&qrBqASO#C9An3ch z%62l4MM;>)n&S@kZtSI5{w%sCXJld;_0h1-x9<>qO=LAq3C|E8aBmJCnNGe(LH7$% z*^m$Et!{N7Xb8qXmiX&%=9$1*a!2*(>;f9-uB6eQqrGy*3%7|iF$cJTdhXTi3{aJ$ ziwG+|8~tG)S|t=38b9c{q}E()J7q&~n67a&PDoUa?&4lKD?DcMWgseM&pUf=34MmSyvzyzB1ge|w1 zTHmbJ#F^=-lfCM%4wZ$xmx|!IDdsu&8Pz!T-RWAguYga5`sDz;-oqI2OjkEz2oRA!5xU=SibjHNEI)ST=CkEwb)*cpVouYT1X;{oyI!D@=7 z1M((re%iPL^Tl0BRKs60zid>3{duOWCc2*q8s-h0yuS?Oi4Y@2FB=ha#rUriDvM6+ z9;fR?ge$1a(}2OKgy!W`wlStDaIY3l)Qu&CNHwk1EG&xii-IG9M7<$S+zq;9J(0FW zdQ9YB*s=E&Hu2=uerz-+bDXCWM(=|xei+1ppr{LpHTyFWA|*>~sEQy6?sRE-(ogze zCBPz+p#&)E@;JGsqpg)rfC*f=aG9;jLm~D&(r3B2f;L1%K-QBqc@8;v^45_&f;@$` z3TQ^Qj?Jz@aENUgoOvTQw#G*8p496=F6EcUAkTG+Y-}e03-q?V^nHaMskk@@Pyk7N zVs_x{BYv^{*nT=l;{GjBg#67+A!ZMl{vysTi)^4gjgS>c0@WN9Av?prjeWLY(V5jz zy0mxW6w%(ym~v1!q4=_HKag%N#YQ**?zT>U-XaNhXi9WP2OFrkW#KBTB~kh$D>z_# z%VR}YW$5pQoB=W+@HW0y~SQej4LSc;wI zjsLWW^4K zp;HUssy9sQ)cl%9B#WAvPW?wsE+-YD*X}gcRI)iNDNgEt)?6|+G42b8uo(Ql9{;OR z&MjkH-C{uzx0d|8obAK@BA$ZRKXxQXS)N`rHJpZ4vuKBLir5qm?1N0l%^q5PCK+=mR^V32B7iP z`)f!TYQm!(AA|*y$eP32>OoD32jdo2IE$&t*SmzPH!BDDN$1Rv5EC`72!PK$5xpgd z!2i@XLkd>4OBP4ZZBxwc^QkxB^zPI!X^SG4c9T=Q?OmBS;Pk|GeU=G|R#J*?2+H5P{@-np~?!Xn3JlF+Emn94jGKJ&;B$vzM36j6=5$eGlS z#($dsXUQ}0nn2=-VoKfc2u{>!fvQtJ36wJa(*M<&NUB~Cr4VS(si6x&pwzZ62=@I< zA&8Bzsg7vzHodn2>I`xMSX_Qd0L0bS+O_k2q#7#16&DegvQWTTt%)rHzdp&>Ew``- z8|Vxq%G}DGe{_q zu=d>=&T&uSJG?SlHqbUw4ePlqm2A%t;|4!fKJT&muum!j14yp_E01( ze*oY)->?<{Ls$(ZAd-a#o*W3+i+~kNa@rmTAty_@;jyQ&sD=2=4n2|QPn+7?eG71L zn7gJkJT0)7llN;`2;+e@D@=83b>UU-GxAb=Cdz;CtG_E{qPHh=?+I;YRhz2Ype%@y zyoREChOp_pa~Lscv8~8+LLs$u4Uofd^c`A0nmbeJ%v8^dtc(r5a$wQu8}1{O87`S+ zm1TJ_AH}s_$$QtKe=;crIAkVS3swX*X`3X}Y!CzWaDiqJSzG2E-}AC(QuccRQ{+l9 z$K%D>-go9LBlQitZTqcTr1Zc7aAy?2ExgUSz4J5L_gXN3%18H#W}kl^;EATibaMiq zd2p|D3hR8K4LLaETg;JGV{g>YQ`5d~996C@81xF?zHZ&b15bI7WP$m=rqpqr^L&6A zg;}?<&3+R|1Fb|cHC)v$RXs`%jwH~Z34aVH0DuWvn2|BGMMy{797A-a-|<6q?CDcZ z0B-|$xsOzgv#Sn24hr}QT44rkMacEA;w!HiflJt=peiKbb@uFRA0eBo1<;8?SEwQqa8iL{9LCT3e%1d5W zg3I7-e}EU50B}oST2V@*NR@^|1lQIUeWzP3c&jbztx5RVnif1J4 z{zISH1tIkmXZRF-2T4VW%)6_XYFU!1{-c9Y+$%mvg1DQMHpYg+e1L1`DeVw~8%o*v zF2?F8*bgY&dOZ-|%QlP2iM67p;woc8hy?1dAA~hNV(s~cw}Fm!kfd2g*ZVZi**=AR zozjhj?AbhcCX4Fnh$Vd6o-N3OnA438)ebvLJx!eF>|paCQFZaKv0KU+E<1`&Q(6nJ z$wM(xE(rjeK8C_DPXxY67i)w$70#jYlGtQ@-M0+(Zmm$Gbk^T(y{jrNfe`axpmtTK zZeSVv`K8{l4bL*g((?h zsOmE3_*O8Fn1-(>fv|dbT1bWzc(Rc4>U2J}Rz0xjF{=x|T5xEzG`86ADmy*R#9fE* zZ0dI@l|N}co_O!TtQcuYYRt7r!AvU~Hrk}el2!39eo1Lg38pNM`m;lV`||d89)JK0 z>bU*cey`0Kz`ECIaLlInH_S6tW?q-}GT{c8irQQxje90J3;vqc*0frO=WPCkpkFyxdxrDQMP`N~ z!Lm%2R?`*rYf5B(-_>_q<2+;Zxuf5%dqdQ_KXhD2>V&+8524E`npPL|;+~Y=42jC= zFgLHJQ&ajgRr6X>XoNTft)PVHEsSxL2KjY}6!Ql>Nf9co(3Kl{AWW4p?J9#7k#PPT zP=e`KBO5t8s*v%23KwNCaui8AsfR}0PYtFu>o&Fu!6X1%QjKGY(?T~+c#@jfoeIG% zM`CHBUSC_`XTwEW?wuSR69GzaU<`Jf5sgn=fiJQHwp=OOH8g{p%c!^JnrU2c&aWA`H71{7 zp%MT`VE+ko>*h8~-tQ1`l`5){XJ`uOPZTT3lkJX!Z@On7-k*7!@*xzw!*Zkn3?g2o z+OikuemaPPE=;MGW-{%2W@1yKA53(@Pb|^gALJ$tdJ!oH!ndqn-Yk-nTr`y{$d~%; zLo|CK^CT)Y-D}gFznn3VlWu&G3LM*B7rTMEH|NDgU%VpspIp=HQy{HL&+vdTPZUH6 zlOX#8Wq}TJ<3lLC(G9GFbZ%)`&YlU#GJ-gIa0w0#5(U~BITDorf7m+5?o6OH+rCLE zwr$(CZQHhO+qP|0Y}oG={6+W%e+e7KX%;|hx*d2-*ZGL!bdJ@o3YF3Bf%0HdSES$sYJfb=Ayk3g-4kyII z2~O0zNfbfWo=lgV)WlmUvN{((chgT>mfBSmDJ)>)@g7Uxy@u0%e=0D7RX~pZxd_-- z8A(UXHoblfUO>Xnm6PFp$2c%wtj<%%JQBtUsQqB=g#^FQcn9#9P(=#`0KbmY*@6%I z-eWHOL6sLOAq33EAhl;YCQOhV6?ZCUjJq(o4Co{N9#&NC-*)F=6lk9;mVf~sjGk}3 z0Kw@eM!N6NCR%zlMS=7DRmYG>*{v+;p)l`x_;~Z!0Hnj>0nxe4X_qwW<^(%8KRniu zuT7fnn_x(!?hQ{V`$|rU1zMra!hNPjLO0h8(s_=MnUhs%gJ(do*JDt}kC% z!mL0)_1icQfRHbuI~U?UDEk=J{}R0MC}y_wbhG$6kDPmZDx9hFetT)9QPYi}Qu3q{ zx&js^W)UsycWl~FLnV`U6img6nIxR5)gEs;+yu~b$yVTEt@wOAND|oUWaWPFsl(3Q zrC-XXgd;Des#txZ4)a;p#p7r3&m`re*c&tOY31L~VnMXE<2}Ucp_=zKmf(>_zXju0 z=eSJUf;p$+(G%v~ZLjw$&%fth${gBr+|R4;?_jrl?VjeBq@TLzx#mVa!Q806h>MaQ z6NFe9U5B--T$+Jz2LBqQs(ISSWcGo~C{r_kPLjfDXR}v9<@7Xt;^>X`)*^8Z&5&%5 zB)1||4ZT1zOIq)|y`n>u;!JfSgm{{Ph)faQ74uh16i#9Mwj&}3kh+D{b*xfY zzk8u`f-06gXo;t7a883u=XE)f2u-LR7>upP_(B(~*mRU02_mq-Pgw#QpDko8lgPMI(`6}=evuJE3jU6Y-V!v9 z%jO#J?FsPEFnnYY2vJ!*!qYOJYmw@p!SleNZb-N1G`6@=Kkl5O|X`gZ}7FZ#%YHC8jRlkJ$?| zgy5x-!e}m!Mmz3a-kv|G4TobckFHgv2(1S}i`zxC1v+OHElKPtXmGWD>+c*93&z`% zMwo(gE2m5Xaq$ous+5uw-=K9mp4dtTl*|4%xvF$r=F%ge9@42;k%0NpO(Z0Heqn`> z9uU&0`^8}C1V&crZ-#^4?+W47=`Wz2IXxkQiz(_9K$Tw|fq4y;rvoh~jQ|7`-!Gd5 za0~n!^VryCaOyn>{IP&h5DYQEwFkj-%YvsN$p0G3r@P=-8S*u5s;aGl!5i!shRuN7 zP_Ved?39ljIGB(L7DkCD%i$DeG=Src(F8VaHDW?Af6&qEsT3gh(Y_AO=sdRxqee=Y z%sRo$^#!T92%)-riAK_yvWaM~TAcwH-GR@b5Y`*ZHp zLc!`~i^WYfp^?`^#Cc%98Pw>t?nQAvqKP@YkMoxg{R<&nS%FkJea~1_j7P8YGHN%s z?b{jES3HP?i8l-b*TMN2GZTkd0Hm70l}-xQHd|3r|A4Azb}yT(GUXTO9?Y|b1$U6e z2J-LJAYEY-d&XF}H5ZL!YWbx_R-r%|pcW*dWfA*UsTNFHe=oYCwjRk7=mq2u5ZsCf zWwUK9peJ(T>O048%$;{wZG?Y;{Yq*uxSL`_%h;;>!@ouPYjR%Q-Ih}4#T66S<7Y4_ z916WA)KAOL%!B-)=B2eU`(6F{Xs`|Y5A?AQaB}hVt*f&Y$821j9=!T(1 ztJfCaQ9pif{{cQ6+??A!&LXn!ZHvLk_vCXC^b@cFVtygi4eFyYuTu z{qoWb{~`6g5^RX^ZH`U1=hPp~PE*mJ3}_{5uLssB_LR}&Ps$sr(D9EixkJ5|oYV|O zE286AeGqrg`8$>B=-W!W>19C^sAtX35bpX|+T^aYmE-j09N*A7eNtN0Z89|F8U&78 zP^N);=Zjom65Mps=hJonP?NZKrQ0)!z7?RyiKvKi+9v{$+RL;mT3Nl$ZVuKO~ddz zep&i04Sj#jJq>%iI>xAx?>G3{{2z<}!lMG=ZB@us4i@i+$A2d00q7p91*j5Q(w>2? z&CU{O5znQ~h6k21{Kq|~jOK;)uk)jW>aXW-ac9z_@8(^6-mcNp_THAXH77rC>qYFM z1@xb!zcm>2HT%Bm#Lns07jiW6uRRs;>VDszSfRgl+k{5=I`e7%WQrd8uIg6u&JH|o zQdtWQjc|x9Ju)MJjzbRwMq3!b^9zd5;og5AzA|jUrryTu{rxb3BKho5`s6%DC#wn& zoLDBISFF!!ubA`hd1^T&VX1X_4IATpLl`D{b_4DGzzR1M8@D=By$cf5uf6|eDkxNZ zf1Zq1(imM@KLH&~ovQ2g<$d`B_|F5L0b72{#=m0Ev*!QF9hsZjS=yPHdi>87569Yh z>-T;07qx?ocT=`ROR}|hqp7Khrd*1n&05;+=7b3g17u_iWe}Obl+~@Dg1`8WhC5L2 zV-bnAUx_j~@nItXE!r38mwV6q(12v|SNsy0qsbm|Ir@iqEg0iu+1q7Tm?d5d{P-S! zYadSH^(2~Q_%eE(ONTt~k_{gAWe7%G%0QeY99Zz=zHVqZZ?mu2uA%+bI*(kMX`Ssz zTAQM_eti|=?Ry^mbJ=6_*Y}tA4o(1MC0ER5Mu17aq*Q}w=8nkCb>6~$uSqXWdpHnt zM5(iPn2G!1FrHiF@w@^<2l|BFn|W|XC;3jUOX-*u?Vg$YpRr!q&}M15aDLyqtWGgu z&pLlup_6M&Avj(#oY4}&oCMdGn79O$&NNII|IVfk)y5Eph+if~I4Zk3RmFPvWdu>) zNNQ(bO!~f}_LX%18a;&VEF=w+==xb>*Ir$&PTQQEBV(K!#6MAn5FH%c7dg2(E$$4; zRT-u1mMio#&CUVptR*d~Zyim!)wVe}xGVB=*}3%O;I&7=CwuM^>0Uvm2;sHGW~ZNr zcMYJfrBHR^bzkN0o~NIG4r>-dfTWmWG$`vneAVX`myxwhW<^5O#D=T0BSMEJQyHvY zZb^XKAVNokpZnti=t)lVGFD|rkyw)8;0-4d|31gOlg1Z#heL`L1WPcA4$`1dqwka)!VKAGj3>S$5g@<6E zQk$qWm5nj9D=u=Rv$+G+1pNhCUtEuos!EC~v=^jp+P2%~MH$KxJP?CiiC(8n?Ag}9 zYo{PmffSg91bX>3{fZI=Bj7av`p>>3%hmP{i~4~bb$>_93bva?#FC zi-^A(FZn)Y_`5;Yd#q)yX~K8sy-iw#w15zR=d$Gn-$4btLNk$5!qUGy^Aze47}_JxCDaEr|&H3of9e!>lT?Tk$V-Chm>ZP6obO;n#|1-XCO0^WXF zU{iRr&HT=C)#0a!dXvEa&xX1Rk+qO zSeb=AkjZ7lK=aa(i{7S!hML>!!2PhU!eZsCO739upv}e=qa*z>_oVYN{6PcX z%?51wH}KZcYOEjJ(akM*-7l&okQJNl#Er!#6$t*C;1?88ICiUV{7V#e(fXs#xk$Pc z8J#7C?WS@rCaW%ES8&dAH6?4MfIrN+?6A9W5owJHS$E3Oq+%~h;L*GLU@N9U;!$BH zLvR8Z(Bu%;VBM)EAp194KQorrENP!~$BLWi{=xy=eaD;_hpP189?@DSS*NZZni59B ztzT3eFmr&lqDzv`w9P&dCyoO^)F|c!`0KYH!tqm&pwB#m_b6<^#Wyn2R%S9wb72JC z2OUdKsa{CZ$SJ#{&X$=UQ`RxVWhcl(SPt=GT$l{a7P5s#P$qWFpkuv`jQ zF-fXjLY)S>Nns4h2H|b{H;3RPnS>MuJw~93J^~(+RLifvg)bvTqbZc$RLGVfMBF5E zB~=BR{GSFzuha)E%Vr`RzV-FZZCqC-GOsvhWGXI3p-~o2Nh;=N`H$Hxs0PYpgk%Qe z$6?-qHYD8A9hmFnU;>*XH^Q~HF&|LH-l%sVD2&JC^LMUSb)m5B+m#CekX)hb206c} zKvt#(F`$0}EjIU4h=s6ja9x*^c|7|fM1(0Vz{l`@b9HHbB8fbVQSA9?_;sew4C z!5HScBhb1fn41>BEiEt?H6YjA;1ty08m2}U;FcJu8xvSNwy5m7OBDv12cqY^D->Ch zZm@3r#pqyrewA=dtAGdjLbbZ007+bp6GJ9^oGj^G^Y1L_uW~VS{kGX$uX?@AJBZd+ zbF34eU&mEJz_nlGC3B8zJFQR2&8ryty~&x3Y87k@#=q}ik0LaBVye6Z%o8X9^*+FfSsfACddkv)_4Lso& zEnd75Q#9mLQHjzQ<8s12>^!F%QVP52P#-=>x>Y4BV9c0^iCk*I$)Qba^3)hMf0koU zjevcS$M;E;H=i23x+(nO3jC+$gN}>ssiUBS#X)10q_RQ&SQ=8{GAw$*WmChhTM2fiu9Oyv=o5I`_JWr!hG74aU=+P#WDvxiY2D8&mSR#CzE^ zH;Yqa_RiMa;ZPhTzCjOx{NL`vz%MbKO7g7{+20bFWbuvVG>ap=WP-v^AIt6BK(K*i zN2GK0v8M{ih&P?n&8#<6oS^4p+HyFm)Go1sa`kATp4J#ln4x~+J(LR?f240Z!k9^Z zk^{>Z_cH&S^2&yL&QpKa4Nm0Y*(^ z7G8G-*iCvzyS$^5F(T=!Sx3;QN%r`KEF2XUdn4hwYaCn|p7Whko~(X2nk_yX)=Ws#H^F_mQhzYV^=?`vNO` z_|*UBhD7edZkCS1HTk9|o0s(tYfbuS*3>-?=oyATzBO6*K}rZ=;9_9TMNa2XYmjJD zPCNRc0xkGJ{J}&xn0I+@9G(phF`iS;XRc6Of{)S0DuXP_4DN+vk=M&FeT=y>(ou$A zumhhnY8F!d-|NRNj4+=6Sz*G_-1?+(e<)0}aS}GyEZ^eQm<*h!d)6?4UK0^%3X;Ew zk!r8n)q4)6Tj667#ce?-v<5sXJrwte6Ho#6lNO5Px?GcG8rP~GfFA#cW_Yw(EqtSi zjLyW@Zwi6chi`hGw$WVU!A~UU7=#Xr&c)b#0lgP1|1)&&J}KQ(+7s#a_PFQ)iOy#{ zq=ZmH-Djhf_;9P9nF`Nxsj-^`m`*6NNQx(UiPKZJ14)d}0QGGNlJ(f{Z?X;(PR~?L zB8>oj?}HCKU7jK19PDS?pHj`~Qo4AkldXv<@8Yz%p8e{QVhR37vGa?YX_BEQAK) zXZ8{x&FtKXeGkK4M;5JYQ(!~Z7w~uMY7?8-!s1yI2TI>HrN!+xlvSxHfI9pr;%9`% zY)}N^yV!uj1{M;ox-2?dUpqt`xHZKGbUl*6F*$1dWJ573(j}*nETe;TUqbJWCv1}l zs~k$8TKb=Iu?VDx_mg1w2#jzFhGAlo!W9#GplD$XmyS{KC7b1U;QsagYpRkr4bj}J&y(i-PK+2R1>eqU%o_uPa>S&fe9nYgdFy3DPoQZrdH|a1Z3hf z?iXlVa(Q2Q1_5g6Fdrgbbvn_AVdfToVg6nO!qH6>>sB=JWR6^lxg#i}{uN`j?9uNd z0L0a2#sv%kR&bv|l1Q+|hp#eTIaXo7V7iB$gy_*qvy(xYKw^iV@mFnZ2gpNItY1Mz zB0IQ5@}KAstkbLtDwq~hh`w+FzQgm zpKum~2 z#Bp2B0pP-6T^Q>#4A}}>aYKiR0b&3C6cbem%&vOH88QJz&KXUNI|;$=6r4=^9x#cZ zl?R*sTXa{73GGsm_YxqAv}~<9{)%UY;`LI}h~ms60~3ywxeomc6ZIuum9BDeT4OI7 zca{T#XkW@pi0sg!s35rY)Q?Tbevf|Nf92ZA}Nhcy0@1{~Gi; z7OGVKqjBIERR39yb&4~>FW^aj4%@eJWu1)Za0e++MIZi0K=|_2@Vr0Th|BqG8T8j+ z8oh9ffY|lo5ib*LWEDh3ehYv=Zzx`*->tiqODb0MC?vx2V?Ls^nd`NdOK)-eP&-Lo+$HP|bwY&W|AHn(O>E5pY z_YQbqnE`DjHPX*ZP1#6h2~hc>1tb9t0L7x-C)O z8AiP$wr-BAR(F2=zAXF1N0k*h{Bjsmdu!$8%3iQS>cD|XrIJKY*p3nhMHmgqXd)eH zLbs-O)HL{RE@u&)Fd)N+EQgp$NLK1J5j_c7Z33RgmRWnfmSU4Kv6E!?ylaNau}CT( z^#2(Nyi5NjMj`+J0@DEi;{VO`nwz?~SlXHEJG&SUvYVVl{>X3 zwPIs#ODzBL5r^AR-tzP4tedx!laB#n5`ywC;e@nSTYL+B1N{A)=$%X;i9{qx-(k5n z_iVoqqJRMdh75X`{vQ4w4({&Hx6cUJ;>5ApJsy8or}2wxxzqCNA-U6$W9a9X^mP7S zqwHB@H%uq}DG924@7&o1%brmUUkuR!xpHQ__}u%p;z7aF2iH@$LR%x}kM%evTU>Uh z%N;G-`1fvX&)!}7fabTW4dx{rD&{tJlWSP8fch$Lb zOI>rseGA3nz1&m)tQZ56F0AB>5T?k+;mc=QcDt)`c%wERQ^qGdKVGgLo=#p)u3m2L zkv_Uj{+H3^KmXcOb@iILc{=xN@qr6gZOfUxcs+dqUuwgs@u{{sv$B`R^To=*+sML7 zyg9v`oIcK%Z}?M3`AEy?{X}(fbu|0f7`ei#gF_b6wLYGXj&4qmr?1B^T$UmRG+6ED ztvXE8F&K+XMrPdSG7+C2LbQWn&-%-qr@-M}YM~wF!dD4_e)L48JUU=zQ%F2vA|zN(e#CFtqH#1k`1zz=-`=<9ZKnCz>}W?!5G294C)=41p;JC+UqQhK=;4x&?#f@oDa&ogc_DmQLz57-uCU+lWAO+&o_ zCR_@ZN70O+oIFv^U>=rhLHSE(V>V{mY@8JV5jV+~kJAKrxerNMu=mI74&9Pmzi@>^ zmCiwbOB90_-Ft&q7RavAX3~3m?8)f#h1kW7E7C@vd+jzPAJ3-qG3}+NPVfzc0iVHP zbA4UMZFIvf>A{W}_?0KcPN2aW5f=0TNcw=+1LlkFGf0A0J%}x>w`l>Oe$b_(a(Wxv z&Rzbk76-u)sP?*rpdbIzY2x2!hF3CfBO`&%5K!8&tK_Yd9N29%yDmmsukJ;p8PD)0Scj!^;NOG zjvQ*?BCb-_X#aev)q^gb;548t6~5$!9=a-Ckd)4MJXnPi`qv8`G*{&wM-&ihS5};| zx|422(v*sSY}ILXatm&uD3P`%%Yal2&LPvyYyb~p6_j?u!9gf z379T@%_13P#ZaKbM`-X`LrDP{x2U13WUG2N9p@jmT0^4Mw8%9nHC?swTe&cx_~NG3 z1=284)3D5S{BOv&W5fc@8xwcV$Bi>3tB{NV+Lj8+##&?vjwzMQ7l^5WJmZ%Ep~Sk# zoUdYi3twe;>VB+nWWlBZVT)pFiy%0R5Z|C*K(Ig4h=oAHtMO@>2Nm<`1tkGc;*_B|J@0v*w&H-q1CTgx=0OF4AR<*j zf~a$&g39r+;sHMOG}3vL=y@kxT_}>DXO3cp{C(~k1nH}t zJ2P~fL1WpgP%Rr4ifx1WA!Z^2aeIABlyJgxZlGp|#p2PVD-lClw75x(%8}QL zQN`$0P{H)2^+kO$D@_edG93iNz|iiU)=ueYD7EYU{RG_KRC*|r4SnN{F2k}N9>@Z? z4C7)IKBZPJ3uYaW3HxfOfiFBN^y?IhI01n8yI>CM+~qWQutVQBkV5K@04|qnzclv6 zs~1V(Iqi!Pi{J7gnech%puE>5tmy3t)FVR`Q!a28L=P1SgO2CsE~+n{w<1tX4;+%i zjco$CB}uT$=zD?6rbH6%^VNO+sf1_9!8Lq7(w@f zJ3|OCy?&(cJ%7zDBjkCA7y!t7ID;l*1q0+Pzy*c`B$x3IVs=Fa>|3qDHb|N)xeaD* znkugTI=-#t6A6_9!j92$D=Dyz?5pTA=9^lD+S06HX@ds&CY~=jZlgA$m5Z=o<2=9` z@`2WhIW2E&J_^tx$QL7^OOPn2g87}+%cv+LKA3N-dK9T?cnudE$2b(p1iXsf?`%N| z!1b5gbe5;q&<`p^DU;)=B%Tl31{Y6^{Y@X7!6Q)Mya=t@*f9B8x%~<3wy9+hta`6Z zxu3djIn|VG;EOp3b1v~qrs#k%7s7~naLVl<02g#A8)+|usenpWZAJzyQST{mZQrwI zopr4VoCMCER3d{U;f{jESYuFC?kVCS=ABW%_UAc? z!lmbO&SC{cUle9bF~e0iR>0^5Gp_9{0U?Z|?Y{2&14}S@HWW-b9c#c8ji(()sq|c; z^t=@MJB~Rjxkz^De)Otk3xkm%|E|J7rx-opbR^+DMP+Dk&DU-4QvfZ*3p2<{Z4YRN_nv9`^V-StFgtS2EQ#vyFr&D-9rjyq_kp} zT%UAGyGWF6b=+T}BSRK59X%M*WW2;<$~YTy)sL{$WoT;{v`cl4M7%WC;-x+A=ztw? zDrp2N0XrL0(q&@2DRm-f~2_Cc3{kZXXN8tXeV z8Bm50MG5BUo`WoSQ6jqSds0g^%GY=z zabNl!6Ukt|-k93n3J1&q?RefHqQxTd2C0fpU{%?o66#GHr-mMb;x>iKLJ++{RG- zyvU@z=3f1X$g(rr;yMV?-Nd-&)GV{8^%a4Utq>N&>*e|bVu}LKuZX4eU<}>mh|xNb zh2$^DyTDSi-?wyjuu7C(+O%Y`;MT##S*($t!lA#`y*hh(dIoexd_b+ifdU$81B135NTXQ7&1@DLV?PZy8?$!TLEa~ zG`0kWOoEZlagJa|XGPw8Kh{u_lfO{C>kKzkl9m@EQkw25!VXXt7p`m0-l7O9R=U0kk~i=@Go-ZHPYaxjkBhY!-*^zteJBEWGO(GkNPkiV>sbA zat{Ynp=ef~NU2giYl8T4)0{mjsmAX^H4mzSl3xdVlL~KSfG z98axJgQTI|BB)gByD8Y#1lX3D*=9F!De~nX>lxt~lfxY5S#M?{Pc2H>Nvan+`7%2s zj}OQeQ9cwY;DO{blxRaG!~~QUV)KO5NrBH_zKEQ2f+_ergxDbSxk9C4uy8Ul)j4!* zV^!TjG^@=jXx53O6nF6>5iZ%S0SrvhiE;&2@xstOLm?up5hWFgjmCewk%}}~Gs6@a zHBw_KmA?yWlY1YmRY1LgF&3)Po{t&a}DmlX$j}$QCX;F~LA(e{CbzrHSC5s;HdZDYD z>k%KD8jBi5*t(rfoX;M{_;Z?etzg(u4o}ML5Ot^7W)rGnmQ6KdJ~lThPMC#0$5#8? zmE=P!F?!k>CfRh5^bTL6^7ASa&(Kk5#5O=#AIgc9U@%DI#!OPcSFp~=fX(&}xXtg{ z<>QJKuJLUPL+Mn0`!UB*k=O8&=@59N@k#$DrKu$B6CpI`%LiVk{c)oK+mGm zerYwg9Mw5;Ck8Cgm6UM64RHPZYrAfozYQEuZKIv*=rtiyhY&OT8<8?>Dvd;`R&t=0 z9uhQgQb_vIC}ex{A!?Qh-w&)&qg!oU1_FpYQbkvALiK}4wgviH$^rdJXgmeTaE#i0 zHMxkZu1#A8VCc&12l(kfD~M=m8qH`mSmBP2Kg@+j@PQuIhL{&CKxL~75Y7IAV9XCl zg7&z8ly7gf{w<|c>`_dW@t5KArJ=JLbn*azw`{!(|S0GlP+9HIiwKE zUG3@H<~`g}=O(p>7fM(^HQnCM4lTT{;*nt`{#cLnj`l>)L>2mcF>$|F_98Nj4P-z) z475WL-IwcwRdo=KTLuKQm9rxT;1ZIw^&Uq1_*Ft!ORd|)CR-sMG*3Wla8ySy1uoT5 zpdwaMuN#V3koznq2M)Gl%yXO-i=3$vjfNe~qG8gU*0h1NUpt=!SL;nl+*375=AbUP zDo0N5MKZtj_Sk9AQ+wBe+O6YI?eHqyReF&N}OUo{Cx@2xLCC}F*T;D0A(FW zLU#U$$0th)7z(}cF7ld0{Rsq&dDG=*1dnSN*q#_xi20R9kl4hFS8k0rk8A<4`a4V0 z)uesqxH+E^@T!&;%W3%TO0*paP94*hf z>vs0eElhOs!jQwk`f`g3j>a(I5C)0P)iR43%w!&#Q%Kxe^*Zk1rJo%SPq{}A$4GD1 zBjWLF1KM)3mh!V4>fsAR#kyH0C`{QjEOoTEnxkCgAq*6eqTIsI?Ziu)1@fX1B&(nD zMt!44E&c-}{3e1sy14n<8TF$s&#pV`^2lxd2(7CCS^+OTNSN*yua*nfHlcb&re|QQ zTU*}_)!L`({&eMpr^ghqt>y5r2hULsMMg?(5^Slvld* zO@K64V;{u!TDCqtdnpt*7PDwrN;b8dV}uZyzd~lJEY0>1 z*X`NWvA$WDPD2{^a{R3wzRVcwl+n6yj4nV2n&ZRj+xqlSLO7DQSJ{q@T z-dt!t!{z?agnPvjXn2kv`w<&Jde(QR(tBzqV@d1GC?HyCSZVA_dKNKcb_^cU7Y+Yg z6Ix;z{+b!I*|U${P1idGg56Ce^8P{W2?ePzfQZUTebY>ZE+NT#zoRzEE3t>R+%y^q z+)JwT6u^9RtiB-z)3CwCJ*#=QCuM)w(1v4j1Ifrz@E~@2dAKf#t-Wu&kaP568@QsN znwQ~GGIaVVm@qHswITzO#3oU@A|no|!2i`p6!CdjKtk$PMYPeJ$#%q2Zj>cr!BUzp z%PV9Z*;a5r{lRj|KBOhvyqj@%z%U96Yg#+Z2{f9eKz)-hZRi zbMK34iVr?*r!T_Ii&f=qf)*o8gUS=W21YTX!#xK5RzRNpH8j_TZ4=@a)KjT&9X&{NuH?{5wYB_qX^O3Rc@Zn@b@Hh0aObEOrma2Z>V?a-Zibub~A&M%ErO2 zw6QuKT$j1aDgz^QFS;mD``McGQyeVCFx&HnX*9zwkur>6z265Eq6~Llg_sM;e~LFU zJteau`dDJWPD3oE*i_y;Ht9-Rj~RzEdEVWKL4qI3(4ASPB+N!wCZ$v-JYY#zEQGjR zJ2t6BpSvatHRhc07MhUiFCol7k?9CCm@n2@^+5s*>vgMXSh2r)VIGvIqJ?eQ0Um02 z|4sB&zPmq(p$fr6ixad@CJ78ruAeD8m6|hWQY1}o57ekM}Ve0zi*C@FYh zH51?9VDX3G2IXheCA0IqSp_Q{FR`{GszBTvewb>nUIvY#z?M)PWkq{;^85SQ>)F%7 z`N0CJUqNflo4VE|Oe#33B~*VMS$@D%2R zmxlKeT~{dU_4d_{Y1^2C{B)YC{Xr@D3XN-mH3(}M<0zgH3RO2#vW%u5?SN6r1i03` z#abR#@>3WXx0c!Bwv#h&WHvl`Kqw- zQRpIn#!b$4<~rx4{d~te6WM5)9BS4EFSz3}qi#YfV9xlVotnN54%)8@tqNU*BztZ+}L+(Gs1;Qo0Vf zlEKGHlK^k?df+}VkD`%)p6c8LQFL<@e~mS>##BGu_J__Gp#<_p0 z%aHqi6M}rd?$@D&k#J=HC;>Sq}- z1UPi}GU4yCu-3PT|3d66Em$(AX!<~^7g%(~?zVHLhq{-)BR>=XXYBXc-#p1rgS#1C_*L&N4J-t|J zd8AdIhlM5Xu3Pt$UbOoH3d^!nADmtzEWIN;VTfXgbvjV{q-3@m8aSDFpyE;UKTAKz z^gEIIB3xB}_LE{rec3Qpf z<>KZ0{G2=u98YXeeryd~CI_F^$I-d{GZ!RFKYyORUOgdl^Z)+M!}*n$o1>T4m_N7M zZN`tw%g_J%9U<^p+AP$LX|&j`;@Aja#2)94)m=Sj=w?+78{Cn6K3$ZLL2%B37IrLb zL=mt}b#$L*oj6~i#PA`42c_TdA7{}BIE&k8@pgbMcYiUconJSVLr)ZMq;rWW6|n)c!BY3y2rU^Y0K7xvI+Q7cM3h($LqkDUrZ)XV)Erqj>U zX;3FGx1RQ9e#3Bg4*qHq;{FsKTEI8JiBxckvGy@}?#YH7`iYIztvQ|VfWUa1ez3?; zsw4M3T68&6~8H4^b0I$xM zr+eE+>9k~<=diZv_Wg|edZ*S_Kv!3J!LZ#!LBZN+7fd;i= zKg94Z{IR*dyJIa6tAU8b*#|^b*%qjBi}koj09zriq?W3m+Ro1;rVd|g@`>kDCicvz z%!=|TdV3CjHh{o=-0svL3mWK_Nh?U2b$Bp=0vzg*5l40=730B?;Fqm;(@;L3`(6A9 z2?XwuZ|4;0a}n-i-^4j>x$fns)Ty#JjcxVpBm>0_LuM-@n033fdB(pu>g_mZrmVN* z^4Y0&8`>)uG{A;;GvUe84Dm{^z*qEg#eTh{$9|uJ*~6}$&i`iEwnp;Tw9UMQh`B*I z%1xh&C!M$UlR_Q2KJgL7-S!y$+1wLEW@9ilne8lftra@SjdGOA1*E$+3(f3>lcwP! z4AsrH=U)y6lz?uf$n(2QK2>0aUih0(tt-vq+)Fl1VZ!K&8ug&5^{H=oxZ(uFVogP9I3gAhfWMV1a3UjQp-Ym3p_Ek5i>s;LT$A-?ngnmi{azOy5dW zzOvUsYUzv^6YKw+wl5u*?ftdsGpWGHWE0UyskO1&;gVr}9PBL3)ht@f0HWMNs{*jW z^fjcbxpE4t24^n2JX=Afh1D6(9cpi4<@{H(l+*0074f0Kh-NDZs&# z#fp}Jj)RVo-o)OR{{O2OXn#3vu_66%dqJgiL|;{kYPoF=w@qXFmpYE&S01 zKqfX&1c3=mb+*ATqyGl~-Mdc9N1=#RsUcaG-o+_zl?-re#P8Fk0kr*Hxj$Zb+S|)1 z(11gJJh?vDI7f8S%6nOQPCUic$1l#;*U!%{Z0m}w8 zwP0})-=aH8boV{xrLeOVwZqqL61faxa`_Z1E8q6~^(U7b{ zHvAJw0)&l3g6YNV_s#3Y2Gbbv@+dvVY&`uMQU)?C*>nrRDHjm|nFtrfd49{yE@UJL zLXXYg2dlXcCd45Tq@jlObyGV`ffdAL#7U0mkTv1-Lfbo5w!&zP&^d+^*teILucL#n zldq?@t<$UM-)VvWWi&?c_*L|;pHtoCu}Fd?#Co|oIlE4K{T{B+-m+>QL8f-s*l*O*X^gc^w5_T}XZ?UUtM@kcKnOu3-aX;L8HR8mbQ%&m zSo#N`XhbZ6mqXvf>ygV0SkGNnE=$|8)PoOw@_9urd zq?=bR?zVX#u0Jd>5EqD?L?odikO8uYa>9?i8J$N!wV~>f&;&@^sB%e4X$YQ&HM*{Dv$cYTB~_ZCZ+=tVi!XW=BID)x7NIXyTYO%1r_*Lf%RxTZzAp) zjtn?JoyPcOG+J^LMCC+Gb6#9D9vRXJ<5-x-*m0j=q&(-=AeG>KNAGoKpKEHNSUf>< zr22D0%f^ke_#m*55{iBmZ!!|cHOJgx4r7X}ZQL`7t}&uYlyV1`QSM4tv`fhemB|y# zg9K4993_aAVfysiWC}el@bqiboC4g&`l#^ugJh+|`VWBp(j!_I&})>L1Vi6Pbdbba zRq(6?v8GRBc+dgtYIbwO89oWlhAN+Dpp2%zX(Tp(UUxEL71WpLz+Px6U`-kBYpVr-P1XSMPJuKv@_qBu)U&2r%ZXJ`VWhVG!kd_!oEE%;Ck_{X{ z-jqv!F&eUv8L7%OOdde`O1yK(n}Ej0mBaj=&zi|nS+qH9~O&MIklaT63Ha@fY+^7x_>842>Tv?)|5$%K)+5OPTf6?_$ z(XoYX*JfpoajqgK_?8l#@Io_k(% z&YI5HH8KtHB)Y(I5SMtbBTi`ybz9t}LIzs}Y;a2bpCZlA<=<8?*8QGor557h9yMzi zgK!IO$iq9XS;g}F!m&?4Kut=;{OT8IlD-}s5YO{K2g-pWWR&*)bmKuN3&fu)s7 z<}1|s=?tdsD1!NYcT+>a^C_cbnEF`T^;Sk|!|c%BhXz=3+TyVvb0X~HCtTnC^#olu}Hh&l_%{#}O3?N8J zwCjt9lY=~uyk)TO8jlTeQ~n%H)h-t&TmNgF{sT$rNcYTmJ}AEsY-PiW7*WP#(+PX4 z{@7gP1yW5~%w&823CX-1J!Sd}|A{W)+6ypc6dmx!45{=MHC+d`3)ePm&*Ds=@Mlh3 zs=yD|Dz6ets#tkoAVqmf9MqUh$$i9gSyr)FSm=YH+pz*%=d}Qag=0EM4T$h{!1@GS zbiZB+BgUzqIw$#L2=B^~=6E6qTksp30&5j3jiq+EdoH%`iHk7AQQbdIFKq#YX}YOxTUZ@fS^ zy8HX()kL)C!r~nKa&LNKrRO%gY`+T4f0C3V@8({@+%94Yy)pcZ6@;nTxNiM@w+zkQ zq7@|!rVU-9YD`r9C=15rq$MOa6Y)i&z$e~A+OtXR39)+7Gz?o&e5kmzqFzYlJ(kPc zqdqtfXBnMM`8*7yzBcnGO37l*q-J}`r}>jl@DZ3)UREu^2-x~w88?RMj-(NYyUNNc z2Qxc|R66R?-31i9FFSD{NB?L)BT}aFPmJYCotds0&3HEXZ{?G3bzYb`dME&)IAYL% z6e_4`2lMHzqn$pR*kJYqo_NpiPVX|5GK#$dsxMLbhy+L!e!37QL|jOi3THAj@5(}X zp!(PnyFx+9McbKt**8|ms;jb=vd*+MnU(yHyzsxvi-!44#zJfV9MJQVR**C<+L%pg zpd8~M%eIx0VM!{*DJ^#kBIvMa)HG+oA4B9-`nm+HP+R6oi>Fg?l?)P}h|(#qYCTq_ zGy4#L+BoJszTvsyoI!{K%vpaTafDld7su)pl`3a?~%^Hr9EXDSKH_X%rM zG-%bx<8)x8S!ecW#sg!|SXzs?F6m!3rPbS-8BoO73{I=>fR zt<*Z+=OW@2FB6VO;}k1X6EextvxBJ|OGUTpv<`tb>Pchaq8$cJ{Mtt&PZBe{ccGV8 zE?ayVqMLuJDisR;aw&pzx&-0l>^O6sv54sjx5yIm3Rd-CVE=u-v6!4LTYWqHgti)- z_fPC!!tZ1U!tJEgF2y9&8>yiaemMRxLVLhNt#J|x1DLd%NeEGLmE%l_H3=#ef9dPJ z=w_MEe8cnvv;l$wk>i?dVw}oxCX30hlqx_!q%1dQ|1OfyKTE=~`ZL&aMB;6&C9|1x z+;$_^WrN?C3*3VC$)aD+Zj$ovoH=$D^_}nnmgx%P3t<^oQfNZ68%l&2w(&?bK+7dk zs=g+X^P!|PR05|ZrWt)GmK#`t3K4>U-WZVGJt17M@O-{r`4l9DyVFRRKk|nG5SK9U ze?4z;Ism*`27IEIm_x`a=AQ4zC9^=RGsC5J*y0c#5)GYZ4J#Ehdl~e9Bj`p+6+HA8 zwn~@xUooqP1%}N0oBl4G*L)hC-YnMjm@b#sK4z=rbAVEo>o?tcB^SSGhV((;#Y9|o zzg@vJ-K#FaRK`f?0<|Uq@pm6C)CUaM0+q4 zwoTF)|J&M+g2E0hTWqLxjru--7L?1P>Gap>z(eF`tD>78&mQ?q@u7YGMm+c;{~b$h zP&=JU<8^mSqr2*gh~H~-se!*it1@YoBZS2Cwf}q&vmIjK7zr$!8ot+`O5!Dek*D=@ zdD-J%d!{DUds4r?f$+M^Ga3%8T*m(;+1=@WId$wgEI|ZTDsFtU|IbYkSV9K%A7mgP z03Q(0PlWRS-W2^1)|Qt4bzc8xTf`CnpKTGG1Li1d#Df?ebFERK;U5W>WA(p(wkDLx z4*kW&1*MT$1T>c}CO|J?H$XSKR-8bIL?jbw-0EDMRLaxBCGuuXS>E=SiB?{&8a(FS z5A$S1Z&%1MmxoRMq3SQ6Jir-3 zD%P`Pw(2=Vk6Xgc(3oSSU3ma*&O$Q2^}_yzk^P~YmfC$$E}M&6T{xRIV*$=I1-(%Hn+BdhN>G?rHSzKMy>X z1|A^yx%c!P6F!NJu(|{C(CYJ@$uEsZJJ7vnr)VfK;spWgl-RJfzXy1RYqHW9+i$FkPH;H!%6KmHiHVvA-ClqLlA^r zh+6;iva$Dj(TRzM%wIfV?a1I@Gf4ir0kEb}L6i6%Mdp;hSW=(co7T79$_i0R* znG{$jZJ$#{a#t2W7vJj3nX+izpe-@1?uRd=r@%+9?HcNL0?0zicu1=Xo@FK=gI&rc9D&}6P@jOk3Rl;r$REa%s?z%m~#fnxmv;B-9j5mZrbsg*1Ix=5f& zc(Lr<-(5LlY!F2Cx?ps}DD{}XaDuEYx?U(MClfoK{MEq;DRiM9b)7n2Z5q$+Czi_o zG=W7_URvT~l)r&b9U12yh(}PB!%Y=LMNkyVzw<{a2OAuj9X)8I1&B;6Hga$&0m~9a zma`Z_LDMW3?Rh7b$hs?mLZwH|T4QI8)qE(1)wg>g71ZYLSqvN#<>53y+~koU}` zf`$Qsc>Xc>VEh@fLwS~$c=!W=vNuOFTFmlEi47CP76^i8LRw_DMytSqr4XsGplj263``P5r%Hl^i)V=~ZY$6P!`QeN z2Rr#(Kgbe6>0dn;S+uy5jQ*IW0i7C7mdJtOM1NFo!bqL0RsW>w-zcNd!(0*$7B@TKN!v34u1ITX`vQPUfCISt7GBNLM&UQYI&I zQ^O;-@%28q=ll|tw_{`gVJ}unYv#v$<{$5pKB-GD^G`K*(`z&b{JLOE$K-Dg9*L!j zGR_$a*l)7Fb6(5T zQE(KtGk8zGZ2oQR8xFH$4`b({mF*jc68UiOem^xW z%&%VzGk^X>>!DhFYR%;j3H0w>y@+@}RO@5zeTKK&^pWr2MDNn6O5Qt}411<*jTd5h z2lreex+ihLI_Aw}I2cJFzeCp2zrPN$XDZ1tCh`Dt+hJDvRK~c<4m;c z1->a^#RobPj)Ig#SG`S~G`6VDV8s-_MuFf)P`%r^defQu*HQV21RsCb9$Ry`{%X+~ z8DW5>E~FbdvAQ6IF;#=GuzuUMgnL5zjSEP0%E?zo+<%fP6YLEgK?+!*hs>Akp`A0v zH^nbn7>pu)Rmj6*{^cjp^MU$sH5UOvF*KMtEIT%-VJ_vEWOR=-uuwx*LE21CLNkd+ zape8=H!E{#%56qDEC!I@#odOmW>zszYdiuBvzk4Xs61TMh#{Ct5}TNI9yp~eBqpw? zqooZ&-_mOTBw&pUw>1W`g*AAKV=HJnNuRad7PJNbGOxD)W{Uh+^;kvB|&etFvWbsYEGY2deheMKJ=&EBNn<+`jJOz>0r=+;E0IMkt|8TAYDNbUY(3SlC}NKoKCc6_*2 zIZU6Ta|l7yfs)0s`HT4?QEsm;$ZW5aadtsoB>8yr>q*A!bVqRcP2q~`fsaa+yfTo5 zp#Y?KKc*YGGMK}Jw96CTTozl*0Na%h)vdO<)QPU_m|^I%RgX^eB^63_6=OjNu-r%} z-AF=rJVEkna`#qZG1aUtAZWMga*vWOvgde>O^fQm7$7g<@v9T7H_a zp8??2^%Si2B+7qqz}cMM-qCgIz^IN4&0oh73=p6|EJ16C7q2*TI815L6wAC0_#7y& zuET((S2~SqH`b;ApUsKQ-et*CqLh|yxlRXk{p|Yy5=29?quN}7X=9Pw zN!~8S>@>FROfjO1XzcjUv z*vq1b2nXi)0x>iD*^+x;#nNk3*6x`~iqC$0@(F~<7x-v#lUtU+nCQA?>-+#5E8yyWsLp>b-h`re_7{AF9>`9u5o^RbXC zUhZqH?V=%idQANqQP4bvF|Ac*8J=piC=*1I%+X~Dt*+OY zkQKPVAUlXO2pg0FS`9WV!;<1OvrZI{sBc}m01~)EyP*LC)jvk!ny8-gTH8orqzo6a zd-~Vq1&G8_feA>hRn^6UfQy}bcjob0DFERsPqlRct`%zW=Dj&Rt8e`k`Ol)?3NZP1 zPhR1&y8GbdW6pJFU8!`z-@J##-IV;#K<7E+^U?45VZ}YEL#x4nXRwN}tYu`#>*z41 zy86AG*S-sXHj8UBS_hLyFk+Y%?J_y)m+4;G7lo5*a zZFu|B=fex`Hp<^Zfldj_UVn@BR1E^wX>#n7k0(w!k+FT|yBbmqzxa0ZIALD3`f-(K zG}oVpq9?FA_dP$`b=I61Ur+8a?N`xBb5LMJp)mYx&8NpsNs>Zd9Gf46|8pke?#RqN3l0P{ z@pIt(@0pB^zPXWsqXV6pwKMPpIP)6`k&>;cnYAm?&j}-IM;m)LB4;Ce2QwRMA|@CT zB6)k8|NNedNZyS_^5?Vaf4}1*`ajb5&k1EyBO-evE4}{+;Pyl|#zc;$W)4KgW|l@o zKko)MR<`y=4h}|!L{1JrmU|+1OEY~s7}vE%jZy1;4y4X^YOo0{BWY01M_Q=4D6j-; z(6rO?9&w|LFj5$_L!&P5@N$g#Xo^xfVzVWsTJ>s-;N(Ukw=DUO@eE~o)g=Ymx@u0h z0G{w}VdZP}YmvBbJu*9+_k_O%hjeS1*I8=d3WxHD3Rhbm?<^RS!}hFf<+TakETE6T_ZI0&6oaNaT?;O}9`ek^ z&gltU(kisKk@=9EnuRpe0)JQ}G~9sx?zbRH)PYhejiB^Wr7z*skYKrh5iZD|{6Y56 zI<<@hQ_3FFwzdj7`N5j@kCAl-2UjeUQwTaw82>UJOi=|o*moHxMRR$Ln^Kl;b>eZd zQ2C8vwATcTc&=;87<+y?p9Q+m)CN9;R z?$^*bi-G*%v&?umD=QK?G8IdB{QAzx6aiP_6gqB>4#J65X*nPSB9Ac`;?V}J3VK-C zUE4gR4mKHcm>t%c%}MgE+ZW2cg;>*D;y(<;Z?KJV{)=8|okhpSfH_pm%QEP2G8_Q< zU>4{osn2k2?PfK?Lrg|IQv}!YnxdnWRQ&vR2c%=S6U|YRPklamJb;skWatItD(Z2o zI8UF8Ma7h$uAjrJK%f2;UBY`Y8jCel*Ixo#B0|k{zrMAMC|CnK%%~;jrh!^D^WPM@ z!K>*+Zl?2YRHlt>7sv#I3j8QWs3}JzV<~djzme`n2Oy(avCiJr-WBc<1Xa;Bz5L;F z94NVGFRV`Ej~5l2m66G(ea^~$8ztybx$md|0#0%;&X|&4X5&>t7ExgA;4?T3v3xA(sI-KB*f{*E7utk2+5t-GW zoLy?0*FULy{W}_oQKdDqRY88bg>?7m`6C{0Bry#J?D`j!Lr|K%FKAgEP`yjKL$wl= z9R*aum@gerU*@v+!38S7;0d0+__ZMSX|a_!U)$z>NnCxj4V0WUB~YX(=|t81}vksm|#c^UFJgU1|kGdt?oO=Nyv zx0xL|vj-%A@hv26)s1?AvXm%JkGS}kR#5BVmeBIRU;Yf!v`7O&WCrJdeO{en`)MDK zq!t1#sRXJAw-}+6OO>^hac?boyo{Z<8ywrujsUP?n<(FDt<6&U%|uW(86$sQ(fZ7k zQ?LfDUjVZlodM%wuhPGwaoV271h`}ZZMZPz3h6wi^!CD z>7~Q&wjxz)&C^PVFJn~Rm{^6^d#6f!ALe%z({%W(X;o)KR7-fxGmL+gj=+my;f_1L z^RA1tnN84jb~hVMs0Ud)=VS8k!vx93!q6MpG!Ju1{r2B$Yxc|>LY8YvGtL=bz_TtY z$LOvw<<*`d#c_!J7EDj;HLd(Aq|l*3ZThEV`|Mt{iZ_6SC;40Fv3BJ?YWnq<=U_wh z3xD#qvr>jEq(M*0sa%%ukMfLtYr6gok?C8*7V|Naf{nh+`upFgQ5z}Z=_6a~6(2|! zwCqL-2g?_UlI-WDraxPr<7jHA>oCXLGOyfg=7pML=}aZiUxXSe8I}L@MzH9y2=V{} z1cU(q0wVn1Hv(HXS}PmF{|C3E`Co2n-8&rK0se3ZP+72SVJU&VnvCj%H{N*YElrm1 zCmt&NGoJ_6T+)8$_lo|ZmVn~(x-1LX=hNCb8*2shYge>Co?z|oSIxc+Z^n%<1v$fZ z?v7qMTtHMeoqXKA_S%XAlwMvxcltS5GC{ElNHL_D2ogq|+A){te922V7PmI7&UgvQ6lgRpURbs|+zgUQ82OnC2Gl_DWJ^eM zR2NRQhpzrCLC5@xGV)@ODgn=OGuMdc6IZk20HG%>&8bTowfp3T)%p*zBrV$D#hE}H zvkUrMA1X-ybPC#$`XU@CX3P10^j>6fGLarS*c>}oVg$_@Q36Gogb_%I-&~O!E6g-4 zW4!J8)OzeHs7|ENpauB6@J7zp^@j-?=7gp<3agOyA$uO^XoQ3B;gtZM{x}kP)}O(C zc*E@fDtvQ-r+rbA1+{f>;_Enjh739Db=LCe&U~x&aEf|xhqwSe<}Ox9<2}vJcA=|o z???Qc-x@S~J{H1C5TD^3)VS{8?&#v=KHtr}ESWE@@Y!x9`1@>ACEM2GM>`+6_PSvs zCMvZ|(GWF7q-bM3FztdZV=`r;*Z9qo0H+rR6I1-~+c2;tHd9O=KZBE0JUCVQb_D%S zQ`KaE*^p3C02td42Mt_3G=1So3zBX$AwEjbp*EZ{lfH5dL>xx6-*5kSlArUVZIdciZtS>blG8(Q2R2<_!LQek&aui=ty)3T4Jaw3suj zs4S{}MG3f|Nex6r0vRbWmmzq+Fj0Ihs>Ho)Rn*wy8fp};5OHoz3TM?lK2+mkx59rz zVTwR!KW%qBz0yC5qajyim3ogr0fbXx?ANm^m<=(xqc$-B7h$qK%;JflL%0Vr6B4-+ z6)un&4CIX^yQO8`M2}Eq0Y#+}gvudl;MS7g|EHMQ;gFBOlM7_`rAm$go;2Zrg-h_ZS- zjmA914%GnU&bAXRpoM4a!H3pyCyvC2GF`$xQ#nf1JUE0!EKFr$+pN#c1B5}j$gMwYtkE9t$5j;L32TfvR53<)YTu~XQ zAJit=~8xaZm5zBauG0|2Ao5*sqnxKdz zV@%ASG>kUQSyQVF(^`E)S7}5y5;7>E8ufrNfb~an2|s-7D=c8BLw&79-Y_4!zwnuK zED`m#I3;l6*-Y!=C}^P_KhJ`4A~b$BGMxS}kTEv#+>47$gLL`Ev~A3n+9p9HK{7d! zoK2LzggODQFC(B@ReDu~P~sl+XQcqUL4(5D-emV&EC)oGx&1@0jRyx(*)7%K$x8cH ztWc>C093vPP^kUB{xN3R@BpXnT3*(5N-K0rBTmn z6E5?B+Jwy#E-c7{0ZDt_!$Uss{CHn3SXd-Iju)<&12;FUy@>en3?K+QDDh+gRVylw zSk%Oqfl(>5ggLLAw_!eiWnvv9>(vkA(z6RT62Vgsf{TY#Z!eI7mXtCoI*WoQMi3AD zRM&3J!zgIG_oRu4BDh(oXbS%jr(HggAb(N(Ua_nJD%4Q6b2BmcJe#&};*XBk<1kjQ z%)Kt=D&b?7)n3-9%2Xt;6aC^8sDjq0o@@_|skxasF9jN{wM0ynS13Fn%IuamL01n} zUl3Iurmih}V;UrbrJ3iohO#`ou60FkGY(9!$1d6Wi%W(GGp^**YF8szsKm6co9g+b zkbl4ugL?p~SKH-FfyVyl^Y9Eqd#bE!aDq%b;>%hc6I(UQ-Gf1k(d)GZIQ1By2o;sQcLF^6t9Z2~f0NSUw>Doc z#-yvj8u3b#XXPZ8)d7jVgGvH;Cy@e#oGd-=(5oNY2|m-(#IZxpi|&>`lu-1Xp3Wi_ z^4{jf8R&a&nTU~0YDxJ9&Z(S&8Pem6vI>qg!ESvjX=hW z?b2{w?s6oFEI}A}c#|1!>SGhM}rLp*LveD75} zUmZzqOZ3J3EB|}^(&6FtV;k85if%XgET_|Qu7d20aG7bxde}|*nneFt;O)BTI5)@8 zY1gT-8cbCuD^}!;Xv0%nsg3DzV*K7RqAC0$j;@JUSn(uE06XW0mmRQ#S)0uNz~pm$K0PjVr}wjPu7d+Fju>d0A=QOieGXFmCU0;|uVPrIqp zdvWU-`YrVsR$3JZ=X^^}C4bp%Vn2tjc=_l!d~kd3U9Wf295-gDBF>fucBWR0nS*tq zh?I(bOLZ>lPs%3_P7|M`3QFBtekf#(?2m!D#;z75%yrYYZcUV@fe7WK4c;-&{Q-m6 zrvy!lV2yF{Q{fQgrTc^K<$L+3d~&&41udF0CF#8klR(Z`Y{i<4*{kU0Y-dO2b5UUb zkq<9Fc9*bQ%;$qoz=jNpv`;wBRnF1-h7q&;p-XY}{M>-NS2X1Oy1nwL(Yx@M4M%mR z2==(&t4}+~gj2>eUsk3w#J`x=3`vBFBL1W;>Dsr0W!ZthP+jJQU&boG3YuY0MbM{N zS*NAb#&XQq>;O2BOatunI#osbY{BVd?<0JiW5&_JEvg`zA^Epj$h?&O~FH|j>$L7xwpPV}lwhc@l!!-btilnJIed#2&PIcx5^ zIbAO-b{8LVrKI%Sv3WxEBJuxG31h2lV^g!*!}qG@s&tM>#Xf!bmZi-({q|zwPd~=~ zRQKF&ZeL&Up_%>qf6**1vHJsHKQzmh+z-+6|MkGkVy#o0xFRdQ4ZTaGh%IoZUSI!RH0l$5v#DjkUS+`8zDVDGQK&JMxOlgyPw z74A;4@ao<0p^=EF1kCwrHD4*!+Zl zoa4d&^`!mW54~EmEWw`VcMu-o`xfosYF>$?AC(BxjBmn}wvs^mM_8!X<7xxY3~!mG z#ipEFc}Y2KURiN*!e=w=P2@uT@?*?-Ut)pO7jzblbaw?#F({(w#ejFlzt7ON&;bhv zFPr@rJOQYBZL*+rF2_5VA-xI}jW69)hsW909AM0Z;~q?!*0i}`XM--b;adkqP`vXo zL4Mu-{-C#K^SbBd9J6kmjvg(q6+0{`CA{P)YSlZu-w79<*lsLH{}1b;gn=z$S!3~7 zkQd&L@U#oP>|EFo3KE7Y@aS?Z@I$&_6?bGLIFb;Q?d?xL1gEzr=MoWj=A4yd3*q#D9yMLbdYzKKP!N(8 zeXt^#=e^ZT^fVQimf09U!mhS=e4Z+z^j_bf8DoM1Nf1#$4R{h3)y0&A(owA)mGK`k zCk3S1H*{#1Z(=gm5HgNJF@!cM6buR8<}R5e*-;`JTh64se zwyiHK)-O-{n_^4uShN-UH{kl@_wP?(K5r)H@a;>6Sv_}hkGA#Q!tFV5nHc5L>m*=h`+6V$k zE?7xI2MKiUzWC|#NYkq|e|MAQ-+*<*Ags3v0btAG9Ry!X{R@7%Y@OAio@(+ z^%T}79FyLe$6H7%TQXFOcmh(|^Hwz*0yv@~xv3GtETJ>hLrC;ZS_TVN(InTeILd3- zuPMw8y~Wq8-_Z?Q^g6Ohv-lwxu~A>3qVhuGqM4}ztb2sxW)}(>f@GwMwP-dL65h)p zJfj-n@J`WmIY z?%8$?3$|L&SJ;Fw5c9o4F>*=$cftz$>uFqPO0F?76D{6%>7t;t2Ldt`!%v1lv-}=& zdC<+M2v0FbY)jIpN^1O)i~dCT0KH^lRw`Im&R;>#SWoQ@qS0l;r5A&-wS<|?M!+#6MuqO&bh}hAtw$!b|x;fxFI^zEYV!&V_oH1ln z$cZ+QQ@{-3?OyQ`d|Ki~>RHp=$V|y6I^Xg03l&P)H`IucY=If`Gs@ z;^53;{cONi{>ZKYclZGbEBJ0#|MhNftzE4U-)n3^M3L>)Ig{Yh)zA^$g2!#}ou=Da zRiG1t@soOH=y#bXliLk8H#4NcH47(#zyHy^Da9xSa_Oy%2G6nQUFW%^dII>VMtkjc zRny3%MZiJ$O(rZ-dfJ%&s(7QYNYBDbb;x6HTJcT=8;fXIN;2P=vtUed{WQH%f(4M9 z9p$d|oFrK2v)R(dI+Uy2?7<*T9e|ZV9C5>oH}7F*riNkiQtmV@PSQ-?U9tkpL%9c5~%G6&=*ui-QTEL3hP|(>hE{*v=NDCXZa1YhZWi_Kq zb*LG8HE@ofp`A%>9^^_d;pZrVU#ErVPCN(B)FQB?jvNXx6dY^>^hMdvUC%aaOt z3_c_}hYLGRx$-STMlCQ{Iz*XaMu+xye1&b9IHeFnk8-=!8m=gsR_?Q;$>=V9C0yR311yDZ~Gn*D)`hmB;CwUYbOigrHFPkb62KJWOHG3J>bjtH&`j* zNXJ^~fgGY!oL#p1<&nVI@+2(@y2npGr|JT$__YF|mhfNq;y&g@jU4Xj4!?YD0~LHp z;F|t@nb-nm+Lh5t87)V zmJHA!eirZ`iRyHu8$OsfN4J9^zNkCOL_-kt0_!^ga$popS5?s$X}MK28XBlTfLdgb zYk(M-`MzgKSQ{p>ozbU4j~@S8aD0#n_L=pTO3?VJtsaWv0D6VKt;XL(VDA1sJ6m0E{U!v91u6iA*v4lC&;Y;B#|RZ z{Y%!9t;MsusZHnK&8AD0JS(|p;!!J902Ju)V?P&J+vED83jPEsA%yZEvXpM=#Zb|V zTP*YX!o7Pyq1L)>wWIBR-7nQ4jd#zLvTKePNUY|T;y=~MAN~$A0h|nQafYn9p6LmG;Yy$V#3dcWavfg4&ykWQ=ZkkprEjT_NsAfuxA>SU3-qDg|?H;ULC#o zrlK&_U#b@OF^)+&gp?fvD2NliAG)FJ`An@hJR}eh=L#b9mCWJsyGYG?ZKC`~Di14E z6>M;>q`h^NA0(Xac}X8jdQ^n$sb zx2cHu?rf*JM$Ee?{)JE1HMPtx*etjd!nv#A)j8cvU<6KsR84u`_^8&{GMS*S(F zqPLEzbff}AskWR~fef#)NB6&sgb`(27+kU=Ip`u&B+H;7!LE->K7kf!boS34`&7aQ zEl?xW$RvL|ZW%>(4^R!zcRby^pof@fnY^^M-FWC|m!#_*Ftu03I_A`KU^%qi(GtJk zY4pqY)7}IV=cIhFs$qGvpTp>=p>|b~eWaH|cW1B$R#QXjs3Cl@s{R~HuwUP5&;+6M z-JR~w7D?}oW0fWLnAers5)o_dY%5zPzDX5?T6Y*Of)qHr$pal&a;t1Cne5>VNvg-K z8N~|`UL8~puI_dp5hh68x?4cM=IlMp32J6_)8Nq;Gt8tPJ13me$F@sNo)&a}y{6V~ za_5=g4GW%jb4+{+qPBtUC2Y3bzx(QR5FHG{zTD?IZXu>HPWYu`zQt|vO%*^W9fvX^ zE;ntZX&(fGls97@p^UX!8z#nICKAq^A8<@6yjQPz;lSHh^n@LNsjwCGCTbOHld|ef zUH4h~54nzjZWvoR^W5cZNZH$tU5r825!ZtjWilS}H663VJ(~_2aN`ACKd8vr+%D}> zB;(AX3Z@0`eP9dlanPP%a|E+)c=Ecuz+_8$s6u$DeMf-WhVur4oH?aO@!3t+f%J$L z1BMo95ZDIt#D(i(g$uch8#R}Xj5KYDCUzACt5sp`=YWKPj?|=$P9Q`j^(Cr682va9 z5fZUI!~V*R6EVUlLlX)mdl*EQ7kO|C2yQq~K?$37N;UHeHLoxOb@cldk)xf6Q-Xz6sZcdeB{K#XoTyuK-M@D`t1!`$4Te?E9=SSR+m z*2+pCM0}oFyGCS<_|1iQ(v&03i4rGh(hIs^%H2q2hsR%eo+>o49b7(BY?@&_xr{}= zQnAH?7b7vGdq}+R?5J-UJ)Uz;>>mAM3&w`U$Y4^#s8v>&mD1rF5U8P5GwRkvpDGU2 zggg%o-Sbn-kURrqA}2*t3V-EiL}B0zR#J`vrcHYUEUw6N(fN( zUito-{&36kI021gVelj7OwMTz)*-DShwAFjWfI0@`9ipQRusn|FArM=RaWYl%nHp|YcRDK;nT#AyHs0|utiY2Q$9-30Pg2d}o(ROdBd{-Fj za0<8hVVl`awafms6i9y~0^J?vLgML#-&g&>6?p6lDu@gXAz0GKIimIhn#iPn*p^vE zT2@teTUl1wSXRdIV!fSYf8Tt+#xN35ftwU*6}?V(MbEYz64Y0;`b%~T^!|r@^R`J_ zY@2CrxWIiX(oSYyBl+eXBCAo#~vh2}uB-Ao>GfC+RNtTDnuNU3u!o6azVY(|_KZp|s-UtQ%XW9!!PC7@^hdU~C2Q6G}YV^;3GzY|?Zb>bQqMw-Ga7 zm6CiM`s{-WwmM@e zE)+dR#fw3%6&UPT0B$=&`9exvH(Q4<<2suKexx

      zk}>zFa-jPa9}n;5P)j8o#JAN(C5k_0q{D3+uivm+G>f-(2l~9b9w`NLsu<4L z-j?(y{_Z?}o%uKo6peU_Ls%HOp9cK$s8A*?ilyEElUK!jFf(Y#NkDxN13apfRV=2ft@nfn@np9@+lZb$lM z1KMCz$!TO5>rW%}Iz(E=+wqS*C~AfLoNWVu%OcvBJ>M_dk67Iskvd=*zvH&ks}>?k z6YXZ+*buRp3m;;3BF7Qgl5R(kf3DXs*rTsw;u{=c;qY8$E1)*Ce;M?}%K2xp=ifpp0(UWXCDhd*nbbhBZOEEx>=Q;R9NLUmY@DSY0TdPj9T zgi=UvY)6`!jY@c2-Aq3sMXo`2V_iHMCrHhUS{hR)6@{RB&VZ88j)`(0Er@PN5=Bmu zOqDIe!dz}dX8mKlai;x;!Fsx%sBN@I#IA`PY>ga}6#6&XEDcQWLm`+?jFRfsg?f_4 zNTP4PQ2rN%>k`*=3?jpRuqhXwW|0I;u(+?D!09*LwWf1YWA?^3%y6zCy8>zE(@3!5 zU^LhiRV1l!E?}jn(x`6_g-JZRR7*9 zRAfSJ!@2s=3d5bSr{a$aff)n6+_?OVAqEI$TD|M^EE0D;|4U-cLbzG192U7oqij9V z(9d9BJGbOSR>s0*He<6*-ZRc#z4!e%|T|c+$?CTId&!*CMaJpn)tef^k)|g<{xwC{|9sPs+$Cu=8k| z%B#w{ol3KfNdgE&$t&jY-+PPg&GaA5X2XUYK+@ZiAjrQKba~V$%8je62>#POZE;{BRfhRAMv8r zhB%?AEG-pe3_Z%vF0mXfgnB*{k?@5_bh!EG;i;1ZQ(ercgE3u~706kFa}t4jMLt-Uo(1(_z+}VbWMsn>46EZvP6A#7 z?tVc9D&%R3S;;5a#)WpuAiQNh3BcA4BSHH<$t&vmdIo+s=~5|v#Dv-6a(<7dAImtJ zKA}t#FO?FlKTzyRNN&xVME$b4gb-f%Z5CJ%EW|YkrvHHV;fXp$#g#cn#5D?5N10}_ z1EmA~6aQK{O$R(~L{ShHT<`;^^7CIe>kwh|^X&hFNcC2j-&}GyrhHNH?&5mNphl

      Zmrz`$u4g6mR|F&@Wx-))mUDAsC7{#t~p9(lu5)2!kh#fk`bmm5`U~Gd4 zf-KNue^Mb=VeDJ@+SJPfUjcuh1-t_?@nirruSn^FEE(b&P?TkGl*3;U;~YV|6kqrz zu;dW;I?BkZ1H;CEbFyn zay{%9zuun3JQ+ZNkM)C9vocR>8Wf(Un z;pF}`U#q6S>{W07weFv3voyESj5r7yD8@=5Cl0oy(pxGvpYnoMYs&sFqckPA(4g?1bFvoI5%Kk;X|9DdIAWmg&B_WJf_s1OR%>r?9Ks$L0%2TWmf zNJE)S%zX<>qB3AL3)ef?jzlnd@V9UH^7#cJ?OV;ml156sH^UFUYGH@%L1e4g=#{{S z4X+|4N`(v1GC)aA?@MjYlH6HOIG76Lx3l*_o1$M?`@K^8c+mXnN zxHK}NQbc1@?l;%nL9*li$@J4~akr^O*&P4yJc{0Y{r%19)|1TNWt_@NJEz6#eE}FF zYtM33cut?`z@;%(kj#5{*h3dhD+m5%#{rk!ZswRK@saoD*T{!sQz5_hvZw!3+GaJa zhh2yO03aw106_Ww**fSM*;!lvX3~En>Q4XNM&M{U9<<$U@`9!$Mjb|pyeAuv!pn@; z>e#%YO6|y4T1tY(#|1Nr%z;?^rp{I1;-wCOIQbnDvq-i9e!JU9wPvcam8(rBhyp|l z=TFJ;Hh#`u@5{bQE0LMP!+6^}IJvFx5G#*4_^k0cXBhi9jpXa6j~hW?Dxhr@D`bkO){Cpo=oOATR429t z7)=rhpU|fEu|X6wQZzrz{jA|Ly3-=TvGZo_SMt(y@E8AhQt0r`-A0P|08fJ%>q7<}HH?NST;-OMPxoCM!FVmcXgd=>+Y#V;>7i0UG$T zO}*0bE?<*1;i-K8KI{O!tq4_SyR5yM*1noQTGu$Ja9QB&<+jzu$^CRleJrD22ib4( za9yz2&dOoSuQ5EhJlgQvGR;8 zG{`mT!I)S{$4#hVfIL$+$2iPTDpeYI19P+ZHS$j;ELm5ZfrdE)06 zqxj>aa+>CM+i=L7WxiF;eUhyIVNfhyxCz!j716?A>-bOlXjy@#{r*9-5N zwL~d#gXsV?m4(4QJ!`DGJPIX!C_HamgOR|C*=+I?2(xBti^a};4*w5??F;!I2wU(P z_r&i~>C+)wWbhulmg6EB6awZ_WM(!uFI&1^J7p>PVzaQEVlV=M|E%wwN>dKgu#7N`NNPaz@LLzp)C%fUeqX0FixEu`7p1d4D zUbfl??6f#pQ`oJyx)I;9y%^y`z$?Xdm`dN%$nX77^YC|WJQ5uflMa@@JR53bRUI{H zu>>cE7Zd2zbFGNoQbRemak<^Kmd!-KW;`agXtq`FRk7J;d8(C(%0Jml0Ef~N^J!!D zeL>*kJYc}nl;7HE3c`bNBB0p$HgjF&9WdfveWU{5#ZP{Q`O;P-i?@W#jyzx`>%Sbf z0iucGX~>QSx9n!r0V;}(@3@DAH+CLNpo>HZ%Q%GtNJtxNHAn!QBaZ?s^ZR*vRV!1= ztJtBZz>AWqo?1i(W-q0lk9Z8rBq|3L5B$C^?+K4J*xCQ4LBn8z#FeM`ZKn@`A z-JZsdZB?23n5}ED9vP+RT5ySM1u)bsHa13E4 zrCI8Xt1!I(@K{AFn4z#H)T*_8k*CYOLRS6M7I5Q~`5|1#K4Ih`H=R&!jGF`jkg8$+ zPn`Wr%BgE(6aYrJPgwf0n*OQ3z3^G*oum-h_vm0ALyiIRzdeDpLnp#W5Ck85bvJ&h zspWuXPK|){-_q4>!mtwk2c#&L;R6}4tq1kE=VQ1uuT8+!2uA-Ygy?i_G-P8)7OR_j zykGGE$Y@UxOco_c%NuB=|@vaC2!#HmRgjl>YC^b84zSP+WPj1z3BMv+ITR>?MD3Ckuk zc8@~d7MCzJnzVc^xP-JNghZXZJ2lJzk}5KfTM}^KoD4f8d-(t?4>S%mFM<9>w3i&C zgdU=f>BLZLib%IXGnzy#k5IAVsh?*)327Jzly{T+!|*$`k~W3-_W$~oNLqsw@TV&> z0j|FKPa*z}xIii!li7ry+m_M>0xiV7V}lX@z^1-`r-8t@^LIqKa|e!myC;J)l?Y5Z zAO<;)lEl*h*$xAhS$LH_jhXiW*=Nxmyl|h^A~Q4w^UM)~&|Xw3KcE8} zX{C3xOX^0AJ4wQBXi}@1R&=UhbK!=sLFRn_=LeeI36*cRfs)eIvJ0wEGSwR?M1f32)cOm>TbK;LXVY=vIO4d4e~7C z{poMum3|M8;i{SKpRoFi;5*HtHMB||<;UCFZ}Gry!zc54H9V$W3-|mF$Hl)y)4xD? zfl?TS6LE4Px(ZWkU+@k^N*Qt9AxHR5cHS(lu^2=9E|NkFwV)D94O-AKT8j4kHv;iD zFMR=QD3#DrAO={#8}U(K-I-A;nGl%*w~3Zip+z$#49x^7FSV+~?0ZtCRA&uY-BVU& zsul^6$3BoSC#C2C!uN$Fy2scgSJQ2mz26w2M;iLW8UuQhR?9|b|EGA6ic8PgJ7aBHSSDb<| zu)}Zy;20enkJ%ThKN6#dnK=}l)zh>8NCkFh-4Kr6?(#1i9rAnV4e}xRvVtZ>(V-3m zrhQ_;9Nrh3m|Ii<%z}jM1D@vl7HF@{p>FeQTYKl11#BSnslW{%pr{&u8nLIO2&!iO z)t1|C?q(%zP;em>dEPn^ndL+7`y6=;0FtP&G%8Ekq&8H1kZiJh?1c0@xY@vi_aSJ)ssFDN>jKyK02=I1LCMt6b+J<5gM}lgLG95Z8BSO zw68>fq=ZL__R;k0!+L%T&xwPON)G}v3t{a9v^_nvu+VCXqmX}^D6ZamIsF)t&}`+s z@D;~r=GZOsUwb2?Y-i8j?^=N zF~x`&aB)+l>;{%k1rO!UBhve_iPA&H*8J}@Zo>!k_)#>kw<@h{3U?I`Yg*$Q#e0Y{ z$3O@>DM+HJ2OLiC6sf*zLg1XtKu`KOzY>Ght}dKJ+c$CdA?GXzvljm5NCqBjmWAQSkK|&$kqLk09ImQWI!FCdtsQ-1m zmT;Di;?j{mS}ym$>a&24**!68kT&rOyF`$*24JioLJu8h$BPD0gZV1L^Gq#CG>3@iICtJr;-9dC{NU>7X0i3Qx2vWS zjK4%*m>%3-7Bx%B#$?(!x;uJ$_z2cuRl7dyV=r`>>m~5%>}V(A%fi)S?PgJ+^g5)R zvW-e}A}B;n1gh1_yHa@#Ifw^;=pa70?omEcPL%8zE)GC4VIvlslL*yyOts;1<(l zFd6FkEqVgP@%s%C3-sV|DlU2+SCH0k+>Nc>t`xA#|wlHvB>`c zJ^+LoQm);v&L?n-WL{o6OeNM_`hlZ*xTa06E~Po!5mR(TOw!;pf%L$b71_9=8p9O=iA341X#F{@N|E_4$Q$)S8qevE zTx2w^*-vGx*l4utK_TTh=x=}lQoWc*LYu(?dy|oXV;QE_||(T4s@p0P1&Ap}{di z0scA!q~?Bk+~GkInh(K{MfF(foB!`h`t*L$^37?aR$<6UU;Yp4+3ceF@rwo8m+)8> zpPE8%Pvr5xb47r0wquB_y_>Lq3{wyizXh6TX=!!5>3QC#`y9Ev76KU@zE2t7D}Z!v zd7{hz7m{*}K|v$`jyQ040RVnE>HpOx{zomZ_a8{AXJ+7R;%4CS-zkUBg(G%r!mZt` z62Vj}VRCqp2^YIgT7n7JJmuy>{h@KIHr;YKaDoxM*=c&d@wG(7GvPDoGw3s*>#lL` z;RUpR{}#ng(cw@$as-Y(7HnSU{I)sEGv9Boz4NzSsoXFHF8KcA)4|E8Si$_YtDlGa z(iJbKAJO;IZyCnyuN9;Dkof03^cHXAXd&3^YJVx=EaqDJxPtUbB-=*+fo!588 z2V(x!zl(DtuI8O9_wvHaRJb*?3>=79KEm!!OPg9i#_mHj+Aoh+vt{0 zhNoIMEHQ<&2?m?oS+WMg_s^@m_=#8xbi&Js@3G%%Iy~t`fsfA6QzjCy)OQO4uh>6a zxQyvMo~s@wy6xyf?8&Rw(x1D2pQRkxJnxzAd++%i+&pM{Q9SIJSM!TsbiGslKz!cx zhL|`k3C?I^#v~>i2!-l!a9HRP=Ow3zV4pSkNd;pi3h22Io}sG7;OSXjaaiWxqRIm zUSF5%j7h9q940Fte7HF}S=n8_E{`Z!<|TK^Y%1$^UR#rNZszg4ktEkNSuMr$o^O1m!@9@xfd&VhS)yF-dN799>Z*0KO!@Z(CndxFB+5 zWC%-+%22W>GB8-VM(>eO81G6C>$I-tlEDr~HlXAXE*40_+i)=6DiD+b1C2a9%7@H8 zMfAHuT>VbDX9BRScE!a?3uDC5vgRpWAO9FH0X;83>s+p$AFACBlga+ejB!SkrdfwY=`zy{1aOW_3TyWg&}= z4JMcfO(@;JyokKTSW<}k(tshmYO;f^t&Jc{+)MbSUrM!9B*>l~;zJ>#x~cSUk8zHQ z+nQD3o~%>GLBAPQ*asbP!t@3_pUSmU0*Pgq&d-c+cMC}mun@k`9Am5nx^e&p1Hf2= zf9<)5c=L)y`n1M`|tGH?MbrH57UGB)820Pp;nmB9$CGdi~*nF?zvEk9mZtq6w#r1 zINu%TV@Lk4RJyJdrlqHoklZJ1sAn(q4b^=^9zgeivxg|vO)TECZim`;92nZ1r@3T_ zz^u3!fh%x7evEv{zYK>tjLy3`_{~RqHE*7`wK3hmRc(=XA8ucSFNFGOEhbR%) z3!Ybp51(|Bo1@D-hpp53$qyZL5Y>OPLX{0q<5fH{T~HuPY2D#n76jDur_tp!{Vl__ z)GfNi;wgxd%|QjiyplEkK zSaJ#jshU3s^aNNjTrX5fuJA?d#4nl0>(!TAFuQW~1(<9!nI5v~T2HUvGnUmgG z*0vpw>>3g$Bj)=|d(8=s(cIF}I{D*Dj*NK%0tlVaQT_*V_-#xMq(*S+FdakDjyS^- zCyE4HJg(P=(i;FR$a(eR@&tCJ@nun2O|(DfU4RIPPb3~VLypNoh}hOBu7$K2hlFhj z&{`2;L(ltrJ1G0to57kLrvvqy((7YR@hAU#f$x$y3G~|udj|rN6bg2mQz4e#b!efQ ziyl%ncL~lbjT!_lP>jG+P^t<&DHyHKN_-Va$TvlhnSh$t+lf_(HOLTv`u+S6nY;Hv z6{oLQFzMzG3H*_9au$qN4Pi$!nmUF7af2QD68L)?YsuJVhq;BwtAV%y^2o2qg|OoW zbCZ-?4G=GaQe}>BhRnc@asGl!nkswuM#-)BoAQJWlKRfK)ijHQc}m}r!5zP@xLA2#`?a6@`ZO?{x1FHRf7W+}-w0k5h>;{5Og zJyc$sY$N*b%1NPh+IEMjb@a{3>~{Z{DYm z8_-@gJdiTrmZdcc3}sjsTw}y99ZG&9NRpHM`1q7ih<~qp#+I8au!w;qyyo5j%3pqA z@LcJ&HP6Gwo}ABn%hc>?1xHBRpA%y*+?eMp@5JkVet?=-6`2u$EQ#X{aee2`(;qoF zZHole{`I@VfHXYWhhZ#F;Vdvf^{r_qF zaa00y6c1MPTyWAaD~TX+PR0VzI|!~Ai$%_}Ioop=HDD;rxMjHgn0s3{Bvv2jFv661 z8ceHHVYa7fLBs6G%2vf-aqbjB-tmGm-K>13GSitpQ|sB;;GxEFdF!d(!hGM4C0F3C zrFN~(i1`=TAkRmmj(q`Yj*1Sf2ey;NYwWUe6%e32f-hkm&0Ck^EJUtcEb|yBfZ92a zNWn^sKvs{&VvX8ZHz!3G0ZpDRYfmNd=ubp22+|A{CqiN_o>-rRYA<)w7#y8^SQXLb*ZN_w zGep<2kyM(IBAts1?%B|dVzzRjfn38Ge$wSe@oYev^oCeUJ|GwVtnjM>Hv15hCVDk> z_0_H>`uqe65nd^Y{H9McdOhv4dn7e(G1)Intn0sy^xz0)z$Y<7%I7BTO6xq&?s)e{ zAh;nA52)FbJ*Fj`j;H{>8WIQNt>J?49|LRO(!rx7XN_CN#+^cxHB3}V99x2q>%l?89P_> z5@@BBp`JvQJdK>%=m}F_gWVSd66H4=J3Y2MEhfedoGII$-kSQ9Y7TZ^;MCXDoa~g| zcc@?;jm_T}JZRZRTSSe0e)|mcdGeP9%+*dTD8K+efiOSTsyxVuFT&n#z|8^(Oi9~m zNi%W(Fa@EA(;D0tNOOjO4Ec-d-k>9T z95gfl3_G7`rZsr$D`!i@WIVCBC55d?I;8}E#6ds-udc``@hO)GLMq)rACGOk@Yf3y z>MHLeT0aGQI5Xom$vA&KWX|vk)|<5t74fyzw<>j*pjgLO{6Ad4;gz*~1ss_$Vx*rK z(U(6^kSncTbJOmjISWyip=dXk`1Vhcncr;f)2A0K=27&i9M;1c;R zcyi2DD8j(d@2sMTL8=hVP%G=s4Y0y8GE+liF9;0@Y#Vh_`b%Rvg9T$gxyytaNgSt< zsE!N(m9W4h6GEDrtCPf16Io;+aoh{N34*l`eO&WvbHbRu#i|Tu&yIt_BbBz{dZ0iK z5M&4taJDMRJ=g`>Apt6^m;8g-$1=>o~QC$5#K)N^X#MEcu%ifeUEbRyc2m;zd1x&zCyO*! zIO{8#{{-ScasqzNFfAlEJ;XEN*TRR z3-I3ps-Hg(D>_sr=zVRkeJ$DbAXjZ(}dyq&{S;=EEmc#;_#dW;QZS zg)@Dad$$BJq3L%N)EUP&w&5p@bELD^ka7*pwkcv&vmUT{EHH*ydio4>3{X&*! zLXlEK-1Y>*ZPo|JOn9w#0#4j3d&sNpx6hFlU@SY}psP0K48$n651hVB8!!_g10Edk zp--HmQPkh7p4t=E$8ajfXr$j-IFhLL*OUxlXrikTGp;kuM;XC#%3GL=rWbew9uzcT zl1_>!fzf7KP)|1-+icrqxpZ5CuBJ`9I zi6&CVA51@2+t14`dK{(Vxf~yBnI(%9jKMeGmT^dF z*c6vC3DIW&4H|o60L=(!e{!6=;7{Tzm^gj}+A~RvkS^^HcoaHj?Af3`lgB-+!EdW? z5mwJP7#~Da<->}tAMAlUq<@xSJ-Q9&m!i98$b%@EVoDKd<}T)wf?Dp&4*iP9&6MwC z%wO`axPm-h;I5K%kk$fyp#}Vyi4K4k1z$8>^pwn_&Qan%A(4T@fYg#&{b-5a!LH6% zL+@1?*6f>%r09yn|8jmPQ8chuOL>`vlHOk~Y zL9zdobN8v#-l*7iCGj=KXG4-sHTeJlMU$*z*snO%&K%>N+)GOOmLD4z|IL!eRtQb1 zg+QoQ=>+|al-(6y2aWwE%VT5l4V02H?f;;8&Yx-rQ;^W4)F5cn3uKJ2QC8w%e-7qV z4W(}7n`;3x03TQe#JdF-zAxV|FivsyUjq9UJ0QSrwVaC(c6cW+kfZ z=q*{Jd1g(N=ZvFynur6`V~=q-25W1@>!k7y!7)QVWWA!aK8l+ILwgsV23h3Spw3(- z#=~!Hu}j3ucU^&T?3!;LW!e9fPc zW@4tP>4yBL7ev4j%0O+ag&wXgy5xR$Z-lT@JT{xm(g!d7~KCs~6>F9^R>4!d_* zI0nP>uQMN}8-V1ViipTOw+S|d(-1`q?kP#{L_NK!8A12PsnadF_>*jy{pq=A>X%V8 zug<@GTmsrI-~il^1=F6Z2N67)k61MGZ-gXdXhAov9N?1`^w~vRO9bvf(h#Pbe-MFH zqvBK~flZYHhwS!4bq%z4U%yIqj4}H)bR9+hKt2fE7gJvb{2QXXBSIm`P zswZa73bX+G7+G%sHzSUm7#(51sflF3VhTvDtR%Dl)&l>*_V6^HcQvzGgjO(3Bsp_! z^92_NO~>v+wgub2x*|~Jd`MpAsX=%nYOQ?G)&$ZyMTB43X0oMER%wq}Yoc6AuXzlp zWNC7M3=F^34>7f4e&r|`D0hI3PI_$Q_{$TlQ$|bxg=|mG`8(Jg?AMGl7=F<#o2#UvWgf_XHUtqK+Fqd zoP~56(tIbN!eavQ6oSHilyFY2^kt?K0X$XXrL@skupvb=0lb8g$3ISawCF%G^Jl8c z@jLG8wAnG3Z-WZ{7XBMJ<&bj~lJk-gCOyT32z%&*&T>EW6)h5i$g$OuBEIOw^IT;# z5Y*cu?J;bgWI3A}koDF{XkwP?u?Sr&?e)dW46rF9iv|874$h1`Yl^i5xO8t&n1Qnb zk~=d=bdhj)b8y{%v@3FAzeMd$e5bL0P{wwluKkofxhA}^qqh zz?OC|ts}EP?0z7_uP^vO2B$I?HnR$A9~PI2i<6w>PWe3R^&InOv(?j2pM2FiHFM&H z?y%$r`n23fq%a2$cQ6&|An^6Lyx%|)U5(ASib|C`>nB7+^o}VJ#^$8T;BO4p$o|AD zKM{qZjVLO?TuCJ!SKW7aPegdaQ#}}aV(MS#Ln+s_*?DwBhd*QU6hnOjQC%=wu}NM$ z@i{+aEg3_iC`!AfuS6&16rHP9HV1DE_u-Y8+<|D%P`sCQILq@luhkd~z7ea9Kz)&( zl2v%)KEBZNN*6lckYj?xzmon1j>V_h_M!N!P6&1cUB>Z^0fg_1!azy!kV+?P>~?(Y zPcQs!#0+)iVv_yQNqYUeMbKI>ezFTX^S*W7&VexT)DT*x3S{UJ-9f}P5qL3kKQX_M za3k8U_M#bmxN-ZQhW{wyP>GJ}!~o>JNYCr{>Jzhm_o0bPVe(I1_#u7Q!QHC!oKuo| zDAmiZuqW>s#l+D^#AJu)3B0xc%B90ZBD36r`Q;Odpy9_NT|hI&J{3Qtg7H&Iw~FJE z8Zt`8N#r+E=g5@Tz>JLmR0%ym#Y*~9I^&}`2BdwPk0!~wPn9Z5sdLNyVu80*eY{9{ zsk@~hnms@!Hhi6U;gJ^hslT!1acgtJ`?*;!v?kM#6;{l`;SSQ`4j@5pKBqGgLswm& zs?sgc^q8_ooi1Gv$EwFo#Ox~M7VcT^;<$>l+0ruGa1fd{3+w^gDB|!)M>-jik9;+R zd%Ne1t@+Kbx0&d^Q*cl=7;Kwn?9ge0TISESPbL6-qJk+Gv^1JXy@dig$z>G9i6?1G zyr)gJ)B{D6931h-Tox0r!qBbVtT*qB-+kY{@6i5oJeTO%Ny`K>oD|P3K7X_y62;~+ zR5m5m?6!2HcgPp8x-3cb!i7dk#>eTTie)A9JTLj{&+fR+ue6w%9iLC|3#-j%oP2Bz ztVEfPs!>aES`KZ~wsO%ajfz4-Sr_k5{iJJ?1lac(2pFxeqCz$KNAf6?G9eQId_5}< z${w=OJp#a$=7WNR^xX(CF*|>2u=D8rxVV0YMSi7B)<8N<${4DCHHaswykT0sBLvA} zjU=7$$pGU&$YIYjb5BsqVbs>BJl_6*a`-A>C`}}iLz;rYHd%aS!gS8qD87AH$M8dv zHAxL&wj!mfR1#xDGvIMF?c|F=S?GjC68nxQqyPsen&@B!k~+8?`T=oa0!>Z~0*Ap% zpz}e@<6e{dJ?7kGSX@WZ)+}D8ETGw-xFrEknBjvK?>{ob2FX$ICF1hy1xe5f4k!f> zE|gX{OZ`6Byl5S&$|}CP{I8zq=V_KU*2yZRii4~sqEgW&Sa!#8pE%C!c-EkO$VfWg zmaDR+t~s+_0ZYLbd8)2-)f{Ba9|Ib%>hx0y9768L!GKthA{2*k@Lvl-Qq1MR%VX5=z9 zSth8l98Q=ntYaV>rL~=9hYEefn38e)0b2U{Lf*cBafi#Wb?e}21)wUETy%L=aAE5z z`h_w%RE)eL!SSIAJz%K3>82wo;6?+ODs^h67%AL z6Wt^vC~DWpBtYm#Gz&VJ!E*x4@c^JJ7l#A*;;Kr^5xjk3d?5z-5gPls%5#rOre(7X zlmDXr`SBoJ%`!PzZacA-Vz?=&>~gb0WY<|o=vGnFsfl$vuW7W-^|MXg{{2pro(W8H zk{>Q3QW`QSBredPKC^=0*PXHUBk;Ht?W(Z$Q**yKKyk&o>_2`-n!ycL+ZCjGGalii zbaV!%q6=D!F}ctmvQ_SAGZN#vFN+i0j?|bJ?o!3j8J5NQF?hN~+)Y6Cp!yj-#MC?w zX`j3)p7mcY)lUOkh6$<^bE0&!Hzif;FfY+sP1v~WexAUtY91d_U{x-6#Sdp#fxWHe zZtSir)tbt!R?-?(ui#8&$NEp57%Cle8uMjHBV~@Nj1^Uvb+X+G&DWm?4$B)7qXSww z7M6ZbXGe=IsF79gp%+!zk)!6B8EQPgE#M;5HOf za~Xs=6S8)gEuf62j?ZjwB~^vlV+i;@Vxi=cp+PEBtnq;#lmKw8LWO00{npD$OcdWfmudASY?ur2OC>X?n=jOB)U)e^ ztLsT-3Nj#tdN$H>DtVP39ojep;E}~ijCH5dnJBl25J||0vJpV%pfuv2Md%VyT7lfM zU*l}VRjjNp{j;n+o>x-USFp<%nNtBg&(2s5=5%50nelWqgP{UHAX$}xS>IyONd}y| z!?7h*S9hix5kYR7#nj*66iK~t88wSLpi4F!1o5C=H+PJm1*vw-%lj>zDkED!LE)fZ z-qjbf>z1^7i&B{-n2a4x3V*HIUhlt_?O^P{U$*Z^qb%%W!$!3?^eB1~;?HO=N--p@ z7ZhbC!8FRih`w$=L%5uMN80!94IfR=Z8d|G4!VM)J4WP=3NcZFh$W*I(9iftBg#S< z;h?uxHm@w@@H>1fublnUMuyC^Z9#nRwn0)b+)1f5;phTGp~#K8?;6*b#h255O};oO zT(Uv&{OOy~zva?|&H=Uax}(*|Q>z#>nO7Xy0%h{r#hhM3i#0s4;7hRGX7cxuBc6$8 zd~W#$gsEtyWrD6J^F5P`x@1SoGBpn%O`&!-JoQ@i!9%sRhp4r>OG$AEh^*BXB(Kcz zH=zd_t`c5F=gGYRW2CDV7+m&{=ul62vfKszGj-aQqlQt`IRQ8Y8EwOkr~LPw*NXPf zz|@vetD@pKb`yfr`_$pfG$rBiejr?fQ!g=8x%f%E58y_11%XG?wKOm#nQ3(w7)AqozC`E@k_xD9ECL;W0*X z4F(yinYu;!yQo{?Mhcz$_Tg{$U2-#Gva)|KrE*h3oWp|2SUy;*E3#+Lx-56~G!UFf z=F%t?mM*(CF*d_J{R?HkBb`tnuaFkUl)BFe({_-w2ehrxxMqe1eviC@6uMtY8t*$4@rg;O-G88uw z_CMV6Vs2a<5>=6vQz)TK+PtxPni6F_{sgTMHZ}n_ECAK@SF(a$D1WE;p7mzm^|S&r zfogbFIA{dh>T1c@?FG+=tU@UjmA~%~vL$@(8y@PMXf8#aybDxD8mKr~% zDWAUuC;PNExhpZsntj4li1T>vP+YT&&xFLaQDdT)D3((8o10e4q84uQicqk2{a4K? zN52Z)uKHPwE%Xe%DvQRh;Q!zNd`qjf*MFcj6}M->GR5$Yf=>Alc+FqU3Aos#Yz*X& z(YXIj*4{Y2`%vn7%UsMbte{S5ElNrbf^Mt>JW{3G9c$7m;)>qTo+zY`vP$1lMPWpn zI@e;H-nLBl&fh@S=dPk=yo9T!p^AAkWMR`?BGWFS5xwx$QiqP7+{v@6nHw=t-0dFz zev@dixrM|Sc;As?-c*4T()kNqB8!Bzqkz1t@*9wYzgwGH&>otVkv&GUQHI(`8G=0j z*Nv(*G=XWr;Y!PQ?%kg3Vo=;j0exq)NAxOFZ;NLgZHiC|{&H3A+LQdDztI%SfqeS+ z1jUnEA`#g1QCnBe#Y?B~rG)9Kl=62m#4K>Zd^Dvyri!|nP(0+^9lcW3Gm%EhzBW}p zQMV`mT8xMD)8YPE9e()FMb7)KYj40_k`RW9?w@Oojh?v-3*`oT3y!A5?uD&et80Y^ zXTkeAWhLuMeiffpu(eYbd1ZsO^%^22TLZVsNM7@`*Q!qy2SYKrD5RWppb%NqA4c>0 zc6=Cjla>|UX?TT8buQ@KYbfYo=0~L_tt7+eDz$({J9yKK{6>3e zNoqVkjfB+1=m*CJUit^2jsz2(9odzWE9P{%nU(pxl>B-gIwhXYfN72DzAGHJkuxww zr^uz^%T6U1ARI7Y49LP!9ND4>Xbp-6l8v_efd<_E1q5LRQ$HJ|9|aHmk~SC#1u0Yn zssaXr0>EnAO^WS?&#`OEwKAr>T0eF5z*}f+C2-|bwmkT@l|55do(WLQd6n+=W%u=A z-f0y5om*&TCX)iJ_N>mqHu{jvw2u>xMexuzC=z`1C1s(+b{Lk!uot7Vvmz$E`<*m6 z%`@)l;^>>+qA9TADAVu)`?2{i8As9Sxom1oUqjbAx`O@@5D~n~E*MzQUUd!cIVDSd zoil~gi}RYhxkBb~NG1%M>dwRWrP87=lF=*6(Eft%dxH?S2Dq=`P!l#5V})TxRffM~ zY`T@)G)|nHTu+|JC?amfpKj+%@&5+F3RGOs46L-S)NlzA5@)S4nNXDX>Y8`zUB3^~ zCJ962W3RW7_jh<+oo;lmWrZ=N=_r|g@KB@ip1)(bI!+zVc@q}~ap&=0iCk*}+_w*6 z{ZBonM!y4~(~rQe!)AW-GtoPZgTFmZ@}1G;9h`co$1G3|QOd{@v5Ea$iMCt#3N)qo z12^>(T=>UYnE#WVktwj)N|m>v;L4ZoG+Kqf_4$!>-8*yMN{=GrnI0|Rq%LQi%)QZui0NTzbPhm0D)g)zuXG|byy^si z4YdQo^5lxF*$lEnzEK};TK?ICEPYgPyp?@R!a30vX!pWQ}$3!{#is%vG;JH{^^x z+saZxO5Q3Zr?tY3gYXi9`{_cp4|E!qXm_P1@6dv>E^Hp$TyA0X`RS_2tnhSKBDT3-qdBk*Q05Ugd@36Cs!K$Hdl!@@n>8%MHN83D)u^p z*I6NyXI3I@i^Adt+U3vVsuhsQ&M#(mJ;=vzIoqpw^}J_nG7O4RVe@j$hq5<80=1I{e%IATmDWx?&OS?fr-Bnz;F zix6|yBG*Xl(Ed+KjQoZyw}M=xN)W;6qF{*pBC@T_)k7anE;f=@&d}pXY;nOq&Z2Nn zOKAIpQt(Gu^-Z^iOth!aOjE_Tpl7Ame+JQ*@=n!l~1~m$G1Nxb+#{Vyjw3PPuIgzl3~-PFtuTyE9?ba zA!*@Fu;xdEFJtO?2=S9_01^Ir_owIU-}lU*%^gMu@B8i#)b~s82!W7}zTT=_(}Gh` zOp!e||M=gGsLxv?Gawcmm%f-YHczw1aQjg&%Lk4O_|IL#3A@(LpY_Z_^01RIyCr0) zhdNYvWvVrKjy8)k=gf9e_+5}qC>lk1VBay?bMv;Qx-B#0e)Nwmaq_r52HgI{+8!!u z{%`)KvJLDX8}QL8A!#Qjb5_3Db?9l@dEmZfDwKmqR02>*{Bu%XDkAV#iO{dH#M$`0DfS7AZU*QtUp^R6T23k6JEPR8$WBtjY-fbFBVnlY5xv(=eePBCR2@ z>PaBA=Tb!y4K-qd?8vb`v@gyd6bQkT;IBug$_>H|QhOiQf6tFH1Kacazdw_CbZ*f? z4^2HjK05z&c=5J9P5b)U-JT=@??+mQ%g(sP!o7qqJ7nGH-Ehu)>*s!+b@q=EFEs1L z3rG^pK^eYss}u-olXna*b-)Z#6~+Z!qpbig3!|f1bB$95Mf^jhTmY{*$KAyJM9R> zCnXXC3%-7}0s5KE3p%us|J?08A)ZX3ExYS`nmWYK^uA9mP7W^(Uh8`N+&lelOW;|G ziOr*MZV=-TLd{QZb_j(v7lzMR*8N(XICwAq{JF*#gajm*RhCWoZmJYIFZy4znDr~W z5f3t^VauF6W7<6t$dN_yraE%Ut&u_p=!J73c;IJ*NYBc?I>0E$X13T~MeoCnenhi^ zfU%UJwbd)oCEr&nGayrfV=o?esse$w>#fmfvw0d@Z={3DAg}%UH7S*siHQ29(O4h~ha-GtnRH?l4H4U;b#x?SK+C~HBYdpPB?y(EBUzkz2dj=wY4GF z^46xi0Cc<5#g&ihie8&hO=8_NtVtF!Z-sdrts_fy8($H9RSXyEjKuP}Y(mrU)-Q=hAL9b*Z6DEx8N@;{m_A}pBVz|VR>^SB zI~*aWI6R zy9+4ki0SSROX1?kGF7=MwFqm(9hSQQENo_GW4iH}2yucpzHrZpfM%^OIGVf(CSARN zJ?(!N+ss`s6iT(d%NGS?uwLM)fy8KNfe~>~!31U-ne)ylqWEu)L7x&h zh6xb}>(Tr_%=?v)u*epHgPVy|QRu~FSTpC}R z5?7cOa;BK#eNYJS*1jxpJwTypQ&$^Tgdx3=;ic`Jcb~Q-4xv_rx#J9Kiq-ZS)ffz> z*KfPsC1<4kNwM<0qk}O%KO6sezc`2Bo<9%+b#u{$r*<4|;Ck|RV=x>%zY4uOJPr*9 zBniH*NrKFT7xx#UN==@Z1ulWXNo*z&Xsh`o%ump5tH4h}e-S1c$%b!gr~*I9qGX?%B3NNS?(1i^?Y4T+HQh|2e&V_@v%fU{)w z_zifHK`B9YzlI)uuXJ2w9o_1)crrHEf~FQD)l2iQh)8Ifb3$pCA@Qb0Rd3~MMizA*IiF7LcCEx~YbMQb^@y@~x z6Y`x(8Lftp7$zuYX;?ZWnl)CA0k>RX0Nc2ji}kpXh8Hu~pYZ7%%;4kboir2^ zuttlUI_4>1MG@l(Az-y_3e$v0B5E@mIR4rok~N~=`d{TF2ICE0`2`J!Sq z){zX|;|{9#*6OP?86sn26PTPQW#3Cy{EW_3Yt6qbY%Wy28al}o^DBE)lu~%H8_xHm zJ8Nc?S%+ zbw}i(;?P&~e>0@-Eq(ff9HXp;0_kHVZ*U)>lig$SAt#hQv}j;IF~h zGRA*;Zm6OSfByhkHGI3xtK=Z{B4aU!)#${#M@PBNgS}}PPTxXB z_<^RC1BYskOATj02BoDOLBYg{Ab{UOJQirghYPVIU56vP<^%`8{zz|!3q1zlLE$>< zl%w zcf<#kvP5x?P6XjuseqmXKs69KFluN@_(s{w|4X2CEZA*j(&;b&v#vc87G`!Q0KOJ` z+o2Bw06e^+JMNKl?ohB+rGt;g%X1STjZ!v-^+t}Zq$Fz-Hf&)v-9aF)pMcB|t94T) zzoabvshA%$!2%#2nPChM>n_av`G2TPj-HQ$lwH#?v{1xa0OT8Ko)qb_fOtsDqzk%3O8Gw5b^oc1{R}+ z$7EsU42L58GMLHEGQ`X-9=eWrwX+w&z?c}=C60F|Gzh#=e_M6KVco2%6>J~MQUQl> zyNL=-k_=*_Bmr@)v0%$Z2s*nHjXHeKyh9KWcfJ6|F4V~E{CL5eV8o@)OakIcOiMgK zH1!Ij3jfa5VmE~tDN@_4e?b@q<2=sxDmu~3)MSqI4F&5OZktF=wYgNC!ucw)6(~M| zeQqkzDS>OMnEt>#jf%A`Ant~_Sk3hNZa6in%f{M6{`jMRbUR4KV8uCex4aP3+=9E1 zd_V@>$2JeoY_amOp`qRA{v!GsV(L4RB8>0E;8r!pu76PBW7tqw`HqaEMYeJIFwr|P znoYts`CxnU$vgcAo8p64IK;n6Lwn#=tB4`P%cegOA)!x{-A`VOH6@wPx~ewL^_N>^ zC?J{=%Zb?Tqf${1s-GMs7cJ@YM8^^ZDm10DG+%e;MT=}!XRuHO?&WTF1dprIf z={iA=>%=$Oa(LZ62$b?29GFpSeGHEKCCqN|bbCL3WfUPFtbY^S4<%_-G&JMo=(E!y=LwWiJgh!u8}A03iACn4&kF5!kNS#UOl@G zNg=gG!Fuz&RaBOQNu5lAvRM(OHGK9uJ4=yN$GEd$#nWfO?E^haOzS+0neXm91tb$l zxOiP8k_l)<5=3T@qx3J<{6?oo1+RRTP)*(nj)1qQE@wShw)-kgJXU=X$pez zJbvOQc7J|T)O3zigacVJY;nHv*#%0W&y>ZBGzJxN=NO1gTw8+J$P`4n^xQW|N>^fAzLbrEc?F-Eh>RlK>4kaeG3XFaEb|0& zSUTXLQ?2OLjJijfzK2k{C8BL^JUWl&a8A zkQZa8bmhpf25G{O&4jRwkO~_%>kq>_`B;C`*~qPH=OK)HgZEL1bc!*w&nJbGw|v&C z;5g)-#Zj5RS;=GL`|@8OG5lB@$*jUh{1wJq5zcOVX;<7Y=BZj5Qv1)0oG^iP)oD~U zXuSfSjW%AV<10D0AKS*yBc{z~pgf-8{_)c4at}=LxmX`$&?Q0!2oc+lOnA(shgmn? zEP5EIg6a;jK0$RG$77iHB(q1K6?He<`JN!o1M%SV`V@ocAQ1bqQKordt^?Db=9C>& zwoYJ)x*MdP$Gkf-zPrY98^bSAOva|$7q{XfToLmPFEp<$%6%T@c;k!@>e-=!VEx@Z zKa1o=oWwK*J2P+$d5brjo&aqLLFT@+`iKa=sv~hn~KRB;~Z{Pea+8p ztnNRUI*DES0x<5TVH0{@MuxP`$TyEwOx;&aZ14|7cmQPld8viR?}birBR0>jKXv)1 zYoPzNtO)q}z3zRg=+#mBZT{lhT;t>B+8~1b2m`J6y6yJ!6qIwFh6o!W9NoLilmE9V zAV9uDc$s_Y%@t4Pd$n1pEt0+&r%a5xY&$QyWoh>#0!MmefmJ%TJ=Fg7#+|4nH@lWB zDMAV!FzhpY6@TB&!+`hJHwcL@i(SoE3l*2CG_%kHTNj=rv?aHOmMmQrb*fA04>wv! z+V%+p)9QWw6*~{btHTZcJK=brYUhZm8g<7u9^!}&KH>1;xNGq5qH&MkCf;GY{o;2n zTsIFdsYKWaV)}bK?T*{m=-%xF&lY@`C#Md=$b9?8p4*qfk6n_t2X32;v4SSXZD2_T zI-wQqAS12+=G}73(Hido%EKJ*=5j{~v#Y&wqxo)G`0(mY=(Nc|^y;Iwsnx~zqr{Wv zAoX3b5|`>kP=l;c7>;R%?QsaqPd0BEfQ#Y>c@y@Fy`lsTH(LTl#5qrm`9;+>g$f;i zCuJc;S6d8jf{}+b<={!&keAp+HaXC;6wM8dyrN{47R2f{Z(e`PeMZcWd7lCEFulI< z(EQ&*&CR!?@pm)s$9HbpqhDLvH)DI+H=6n#;4jul9g|&Gm;HfBH#AMgWRX(BlX}gz z>$1L!eocZdR{Qr$PpIGb1sYOXVBLcM3G;mJ;~qgU04qW@k-;NnM~>(-;z<6tW*6(5 z^!;|z$`wo8P|*9gU0wZwjn9MSZ$If%19(eXs8CO^)d2v{R5XLC1GB7Y!RfggqXL5oW$NivpPUnMDCW zz(>kS-|rbke*Z3u3ICS2yJvuxdd(nQZ-~Wv8&4xKeKyhHg%S{37JLj~a?4)QQ;bHN z;r-x*izK5o1bwIc#39oJW(z#O!Ty`gTx0)!&xr72250|Ygs?xgV|x#0i{Ezt*^J>{ zDQ!sJbMg*Vwnj}s$Y8B*fki6~rw?j_HLPAr6e(5Lu{EQa#LP-!-CU_{iEPST!ft|Y zc8~Fmxy(eHyhRD3w6>Q!DQTRaon4=u)x6nXRUEh$eoS~IxsOp1ouo;WR6#J)q&_X! z4b;bWj2xbDZnc#BwX#dHlNH&c*zYbb$Br>7wx1jtx}6W5`$bdAlz9sTPXYhcy z>`%*RLxN0*8_^!SOnw-yW(hpE-cK$9seKV)@6jrdi_?tB>y} z`LEaks;!gjO1E)i%g38KL(P)*_j{ek9I*GAu=K_;Y=f4CZKJ&W#Yv9YjqOLPi|gj6 z#~(NgqE)nmzb@;SE)4Q=TC-{_4vsKiJBvJBav%2vb6^lfWOABPEnmKQ;+AE=TjrCZ zgVQPzgw@Iz@{@@DwE{VkIPG*82ytuXC?Fk@T!@1#EXqlF)agb)?cr26i^UIK@OuK( zXrKTU95FLGG~6QoG;H<$!fZG#r{#$-;H~OQ^nR>+J1?^(GK&R@*mBOl(6|fGZ!bIF z`Jc!1Pem&xf03KiMRmt0BG&-ZW-DYaFQK6bPEu(S-K_Ze5HBduxukB04-yBEhShz` zM=QU365MW5boV;2i~4DMXrGAW7V*PM43*YNiY7dZ-uE|hepl>tRjSC`aHcu?pC0Sv zED(3xt!WfWR^AiA<41cQO>K55L=uBz*Hg3Qqz8m%0uQt}N{#8P^8&G-YqInc?UV&C`;6nQlnc6}r=ksR}`G8;s(KDPyatji#B? z7=h$S@H*FrRJPiORS1Yb%C}(z;$YFQVZ-6WA_A3?yZmny?d-)ldALi*0w#uY+D#pu zW=teF@faM~A-=cyu>--aMoA)trmOW%Qw2zNAm0pisg7|JDadYVP#s zXJA~Wq)h(awIJ8%Fl_Z55-lsV1^9EF*ELPr>+Jd8TDxbL2UuxCiFoH}I1%6<{u= zX|VdCh8RqqrJSHOgIZzu4%d|j@ID(oxTE%8@Q&<2YT!!`djeSJlS zFC^H^Q(#sVM>I-Jn1r`PyT$|8?%uN8iTy^eshyQZxpNd39R@Yg!?7zTEgsblKFHwC z_42|5_23IzO@5L-Cy}MTQ3H^8qtk_(_+**DeXywd&()-am2*KK`S8?8_o+3eLPQ>5tm@{s=7-pMzalh30pgOSHMCB~{ z2Suob@I@Z}G2qU!Nv6)0YL@nbTd-Bk560bN;|kJM=Se@0Odk{ z#X*zPoY+yfLyglYQrEN*Ek_ehf=NNQcC`PKX=_Z_(6U@WGpIYdnhIJAnF2BWs#h>D zCI4GTCubfLwdYnm9~5b?>7P52v2gOP(`l|`8tP2(xD|Q71ry^S2eiEOi!BnNfkcJ}1bd7Wg2XwpZ_oX}pw%e>1bt$tKCNhLJ ztCTQLl#WExTPBXwd?!`SffSWk#zq7bB)w@kC=nFM{e5k^2{EcJR0`2wyEZyp@aBTy zx2H>Cvi%KYwV6htrcw<#q~Z^rLckKQu?%Ane%nAyXrDId$r3aG|6y+l>YSUPJhdz< z&U|@aY^U+Z(gDsZNY-7tcZ)bU)}ewrcuvSJMR{QKMW9=E3l$S%bwh}25)O!hTLC2; zn<<|Gm?yzmP&6gpY;LTi~>V^)GXMFDMWp9-v= zl)Mo$x(}M}9qMlQU|x=maBg+S7X<7ch3FadOJ>aqZnDU?E3$nZ$ceP#Y19)a0mxV5 zNPkr!LU9=E2a2q_^JXa=ob80$l6Zsdk&2}L*5e10^q&MOayb1cUw#3AThatRR*4|UXiw8d}9`tcopEi zc4|>e$oE_bU@}hWEBcz*Uentp9&&OPO%vHBRQijCW)igtf~YRw__duvG3AjHUN#qk z8)GW#jO_VNj*||MaUR~e<<+AG4Rs!j6DT2;uNwBtGebTZPeW6SP3*$U%e&UL$I0u) zLVppmmras3gUp5R8|1K@>b5db@Fd{`~;oWv0B9bxHYifl>HY>5xVg zVG-tGTdX!{=bSpGr#>yYnnKq^NL~oteZ3~$#o?_&Y=-wm`5bGY_4o{8nM&j*(94@0 zrp;413A017nnPvz>m-@2i7%po*x)n(5uy7@dr2)nQ{;*no6qWE&3WGi{`fE^`r)CWyO=V)aK$cdHm0mv_%?-hQ zP#6g~T)&Kqn)4haw8P|(=Fpg_$6rWcO9|J6_0>@eVx3BX{2!EbG_K1S$t0rDc=hHz4E z;dPZ#pmM%Z#f0TDgf*NeV)NpD!-s!)1?-!B-JG&&B@OdF7VoXjcdnc;hlA(iugB(j zV&SpWM}gGzMULIYCA(#g%f?~a^AY%4nTNBU-&QhNcQZlnj|gb*j!VeA&sxBOrwG(Y zlWe#mmf7iLj~a$*y?EVAX&9OV7xm17jI#D$yn}XYl}zzQd#f%j=N@G*(j>Ocq$xvw zbw!OsV)$?`*gk#w1~_hjq7*4A(kkz*6%=+*dPSEpQ|}vTiI>%YTVL~60HbY%3YWkX zBf-{hXYfQzJffKrYd!+2`np^b)iEif4`SNG+9ohm#-!JJZ9^BvFJH+5jy-Vl!-rQaom(%+ zI=n#1tyvV;Jz;hO51=h@r0d?{8Vetmgx_?z;l5x3hwVY*Fu@^>grx`hv9TW6yJn~J zpE|RzwEizFR-skAIQ8O#`E)=d*Ycxv{c5Ii)34o$)h4!ws`ahak4p(9TZpgmbf`wf z_7G%xipg-=5owsy@!X~b?GHAZJ!mitkdIi#U79~%BH%lB^X>M5asB+|R`c-n4Y8B7 zxl7A^;s5a0e(fM*pl}S>vRD7tl!>|5hpDM~M|+1@tf1u0j6k5kHihJ)M8jL$y38WaJ|Qt(w>mkWb&fF6E|6-JUC4)ovFzY=0GK z&}hLcK~w^Di7Mn({HcoYPJxdQr?8lS+_ zGL3Hkyii@tq#x`M$BSYz=DQoEY=}BLj}=+?AoKI22ZZ}a#`lo^82HTSx_Zo6?NWP$ zi%r7`cqo6Z2@BnQyx*Vb=pU@g3kt0*nh2n=JTit{zq5}RrKX^_wx5{yi9QNzyc%^~ zvwoIAB8bGrCo_Q(3#wZ^I|oLBhnRNx)xcS1#vWO_p7{I>Uw->0-;n=3Yrg3n$A3cu z0m=SIfN=i*#~4nghK|M-{~0qizkkL|lz+In1%*0dWrWa?M8E4sKo6@$=(TSsiqf`tOnWFA|^7dO~(RjGcX(@;J{}iSFU9F zv5Gt~f{(Z1MoK%T_#|zqk!{6MdBwu+_sUY{iy_k3Fb-eS%gxTk85Nwu3zyRw2%|_Y zOCGkWZ`PX|d?9^7vV^4zS5zBtY_~NQb&tI5ygg9t2+G)kcxr6L4P0##MHR!-fCJG=T7CkhfNnGm3!R#K6jB>{&@@DsZbY^e*!szw5`)D#MnYj`KhB zD?m86H#!izpO=T{l{rcElvBM_cnGXDPTwDj2nU*tA(c2%bpbDYy&&sVn@JY6@){HnGzS)k(r%;YH z*3m&tuDf@RgR#l>RPl$86qAGr)U)v5Z-7{MluEeZKLESZ-lHM_*1nGJNi=Ym7>+Y~ zS<<96aD*I^ja>Pb#$d!kA!m|1y_h7oBEOZ&?Cmu31G0g`pA7iks^1RT@BCr{v$b{6 z9Ppx+Emh-asAt&Oj3;5DXmuUvv0c8r{q=bK(+=BKw?00a&V)`k&CXKh*mL9mb4(&# z&N59BRxfXAj^I&m)IS^AkDZ!ipL#FXdsb~^l{J+Eiq>EuKxNIpnotIjku8Q0i{g&x z`1Fbu2T!ObJFeom6D)37CCC*GSHCa%w;vhU+h1q4xd={nb>)SdI+N!XN5uR`+Ow+~ zs_7i6(R)cGjH}+BdMsqAUEf&&E&o72Hn0{C@T5D8n$o@0CGc-FO3f3LZ{55n9ZPzQ zk3+sof!+Ym=y<=7Gc{)?;rw-bp-+X7h5+o`E+)X#BEwXUg|mvI3}y59q28qD(PWMZ zwqu*jp5U9y8=h!e)I^m0F4@Z3zFOY#51$J-Nm7!zA8LwG0H=CQ2Ap`@DqMK)r zY5y$w0g4(FlmP^z?UwbY3FKf$fKE=2r5MiQmYaLAy` zh=@-MX-Os1WsZSDYr9qS{df)T%#444o<6Ae=v6{D0Eq1mLwm*tE?N*k7~MaLMt(;8 z@m!M(c?u^lo1l=+m|FymqopFhEgpZAepKzZV|9qA9XON(k zcK1{<%dF)(mMM^Ff0~k}Lz5cUFc)QR6?gkZ#q6$5u4bAF;X-m}Rb-<`pQ>zVad<=& zY797nniucaSO+6D!7iMg+EfrNuMeS?VH&0whOoRNM1L^NAK$FweI%h+DeApQ5|9&xB9P~r6oo*D za->I`0E6Ym0)v(!KMH|5K>38CCq@*yo&=ZxXNrQt^Ka71a9V<7gXmg+^WPk*KYR<(i>cEZB zNfO_hYPrH!BhfHMl^SCC(}z;(NN0g_UHGDzAAzzH3DekOwLm1=#XDr0)g+|+sAd-0 z6t3gM`p^nDWVi+c4TLR?b9ar8jgGTbK>3Hk^fwCO;w1Yl-(|<{194%~kJ}&hiWK3} z`t7NiTAJg(ns2SGBS85G2DyQ4%836`m^e!DCq6TuF!a3y71PlRWtht78;xUAupZre zq~aLY6+10DTwtc;m#d!cLyMsx0)$5kus1PeG&~BjE^z63cO@Ea(JO}f_$T{O`py-w zS@+a_VxNg`-gzL89RGwc?ejk3`MGq?58%%#bP1804l@~iAH!JC6Uri(XnlYokB>GZ zTfFFL7UD_es|0@a2;G=?4*6j#=@=c-d4xuRG+?hUfHFeRt5_1=qL9&$I^?Q){Sm5T zwL7`Bd=8CSjTx?Nt!*z;kSZ25SD0dt0!5itUZV8~D#%Gn4}XFS61;5?5qaFL{a)1L zNW2m0z?BN<00v60qg3mJ(TbCdC~oTxv&hG~d0zdO+TH!xWqF0cN?J=sr{f!bG1*^8 zy2uvi48JG+%N@FSRjo-FFJY56HY8*B-acFL&SKOpn(lmt|Z0 zTxW=n*|OKg4FQwdY&S58nUW)m9bZ6nS`-3o^3@Gwb(?%k=SdJHHtAayvV3V~9yb;3a>Pnp>Hk(f%?ZDj{ALz~K-=9tdCKwYq;^Rd5iEHE^gq_>Ci&mJd%i0d_ z5MMvbXU*7Lm`Du=L1;mu&*tD_8yoHTPhfvrofXQ=$hbS(xFjQeV@)A9Th>lP>6RR^ zDoCm{QCvbTx#=ZPFRS1^nNTM0BwNu#ex;vsqDQSjd>{Qr`u9D*>-SM4gq>_0aJ5wc z?spXK?h0i4Srd!3eR%OWrT8%cqpOQpveoAM+3e*v7|b-f;aI68k7+x6wV6mrm9&6E)(|ZRY>z zPRWpIOE)VgqzwPVPge(@GKY@y;?j36qRVIm&YSs{fZyZGGbQ7TStWTj#LqHeh7SDZ z$GO$5Q?56KZ&9%Sv{`NI86$4UE*yM@D#CFZDG_V`_NLiF->)GmdMtBM#<^Id7XEPY z{C?5aQylT9ASkks+P9STUjIDzUCJZ{pproCq8#W_BA zoa9uuWSajn2kK6SDK@;d#+56V$f18g-n3tS=D!TIxLgZ-PcU|O>v*qV0=l#r7y8^( zdp?RDY)`!O5j%!RTuj@%S^=DA99{C5EoZvz8hpi?NgrDS##%8!Jjqs5D}QKL zt_hmC9+>$!NOR|}nk~Oo8aq_q$8E4B=HO8b$>VI+Z?lT;Tof+Mw9|JVoV>x*_ z)>dyY&c+Zwj&H}z2f9byTJLXJ3}|v_T@kr#ztW8cNA@4C4U^7owRtfKUrTtaa6sM%y68eVOmIy^fSi!=pOS zUkGMQ6+siZo;$i+&jF-6!6%ea9edfAb_*y@G#>iVWGgu{urrhhMFU$amdy5+k3W>H z%t^bTn{uwWcEXM3Gm}-*pp!pY24Qj3wk+0GvJ%|_t#(`=^XlNXEd{WZ3(b$!Bjy_G zA>F`HMgm@ z)BeUYY$%)OI)^&f>$;Ii>6g@Ds_GiCzLiHf%p7pqM56FXsf-*k68m_+j%Lwsi#Hnk zaMEHX3Z2X{PQ2~K7Ysr!zHaAXuJH`|YqQtZA^v*yYE|`5uBSaw?nmCV7k6kq`)IuU zeV&2tDIbbUTbQ3SH8EdgaF9NEC^)#~q$bdAG%iE6WY~n`jho?a>k~lrA}Ay_?y$N8 z<_~cT@z|UIn(0O@<=yX|5p1Ww0_wF#WVOj1PF>DE$yw25_X-X`0G>P?1%^;`pmKD` zr(oQ96r<{*)ZXmSRNLN6}J$=N6QbL#qtDXzu?B)Xc? zj{sz;7~n$OALJUvf)k0D9|9tO`(%VitP3;PYYMx%U+jqspeX6Op_l$jOn0z6acUq3zX~2%0t?+G}E9(s8T#IR}3}!e0&+5TMd*JXS-9dH4n<}fWG=FR1&*qOJBr(?v*GB;#qYD8w0BWu-hnvNdX*^v67l|`4s%l|oP&%YSr z#Zm-9nxmR*!Inl>}UIRd!2BhZ^wKQgYcd|M0Jo#va?b_OFzs(i&O9@3SOy?pTy81 z?iKlt#59=+*qYlKh{%n3k83L+n#Ij|GpvE0dEwuuvmv!vE=9=JX-n#T7%jb_3?zgZ zuGftd8+F=~cSxD`$C`}B6hTFUV}k@7Z+_bj*azXn#XD5bdsZTJYa21s6zo*tYg~@z z_FhQ;1wk2pz_U{uh+}5VU-mYOCJX@gPmw{Ad1rO~72&%FVc(isSifa61`WgA&^yz1 zdOsFHHoL|WPH&9n6MiM^V$BKk@ffcZSDdufOJAGQRKeSlbZ9eb{!i+GvEB&E86}A# zu;0+U5!u>gn~bp5Q}vb|=u9Z5p?z@^5g<~Xhocs5iysOS(9$bBu>vKhTTM3PQsPT^ zs(^(hMQsdnY9*6(Nm^+eY~6<^gdeUYJV=40g^fpg%aXDnVJhO(i-5pX1|9iPnWtUq zgK$!fEq}sE-PZ>6Nzy`w{;GoMOlyKrWTy4N^sK}>rdSBlYt$Ep{ew-vgJVNl4{pV5 zP2t^Ml21iygl`dj^<3e7L3xvE;CF&z1DB~LnDj&RWx@gFC(Qzl21YU@^*4{)I+pGK z$9A{o*Z?auQ_hvczD{$1h5s@kBfdAx&2uAixoVI%%6B_dmVZ@-Oa|3#tWKFaE@ogU zvHKT?r~st_qd|z|ReVB zf2ayA3*?@z5zWzEM!aB=a_EqVD*ei@jM+<=icFOc37RW~jk$uJ*>Prm2(TjA$XGIJ z)KOm}0Z^lzcok2*xO3@hpbQC@ebayofr7jBPt&|;%61k?8@Lk{BXDsjq3^R9=-4=B z{bOras{d1)RSQlBG^a;3c0w||3Jj|TOz)_K4WnGFNNf#+=V(AF3Zk`D_YimlD(9+e zw@qCKVN7efgiw9*%cMJ7h!Q#-OkWW^UF3Aw^pH0+X{L!w56u0ju^ZIdpdFb+qPK(q z4(1f_fC+RK%)*P)d@AUN(`DwBag5+FkxuIs;k8ib8a)PCz!Pyn<8$BLq~T6CE@TNa z9D2YqH@`xu@W)FF4DOHbl8ehg&HEjb0#skC2P+)14djTeMi6;hz~`_fsDd1Z^o{Bc zsN+(n5GD1&GFxGiiK+>A^eEIk8j&(!WGUz-qtD&WqVQ3|N3d+J$ah;CoN`owF5yhFt;p;Cng&jOg7m{O5s~C{vYcZ-l7&TObQPhoKStw zY=~v*JO2Pv+t*4uu)Bl})JqQGZ(Rvs7bPyk7^BNg>j>~`l3uFK$q-rP#*s9= zM#ii%LDij(r2T`#fz{Butd8|rwj`$nqNW-rc6eAAqS~FzK8Apk4ke~ia~KNbgknAU zM2K`OaJst&Ax_TAPO73+6>&%!PPVFPT}g7LT%~?Zo-|&;I#0s>kKyEH5+Y`x7hVFo znEu9?g^TCD`Z40e%{&+6odoiI7OeyXOgjH^Ml(xf%sMD+j33FuH_0(4Do^~SJuC*u z`X5;Y-HeBr@=7Q=F+aazCbGRt_O{ppCCyUbO>_ud9isx1A%(X*MRPp1lA1r>1lRV? zX`#0h0%H&_nCh{Knc=%aw#t>=2qe4M&4cL$4xe_Zj)5uhK-lDET{AUhhtkvc$F&O- z>{S+lR!P}4bqZUlneqO-hb48bTr#1%+uh*5Q6Nu!JhIdI?!osEvYgc*G$iZCRd-m{ zUMMcY(w{uiLon!y;N&NnFF$v4y%#lH%96Uv53Y_Ffv|b4_NWkxmEkwC(fN?Fs_N$AJU} zF=67rdE#^)?)9 za?7+lVCYNwY^s;UMuqnBlSqHGD2-b#&Fg|dflIcqHv_@ja-B1+?euiA1L(Ej8QGi+ z)+D>&5%qF^EFH`E!q_$IY{yLb(`Il|q2Pt{PJOJR0BvIywSdO8SCQMgTS5X8rK3 z&9(KE#T#?teQBngrH$s~r_!TeVJ8NoSYKvt>+)88;6~x9J)Qq;u57%COYD$kG}byN zc^x$X<0UhNs10+ue@PBILX=SlCWh5ojDZ!VXURKUlImSr(oM6iZWk3#m?14%k|}Ez zNhEc)Tzr1jWJj@38mXL>%b3-Ii46zSit8VV+GkQ{-H21xQc3rQQY#MKM)3VJTK)(Our$ zIj34WOsp&fmiW5TnFd@M1C|}+?+RZbpU;?E;`s4yKw^eEIMDBS>3L*qV+W!g?|ZS* z+or{!_T6cpfe3pDS@vLCC^SCK1UKR>agZntCo8K2;ZXCNngW5OV!6)+Q{6)T(|4kg zkYxK*xLhkvD$B@v9^shMLz&%6A2{-DzPiDC;q!uw%Ip;hrgzQw)N6)kwe{ZU*_lvZ zUZczXV*l7L6g1VHUtD%osGH!nhDM2XPo zrL}1qU*!WO#8uv?3;(9mNY?JLj@gL*GPc7`@i?ytsnettY}%*eAlO2wDi^~cl7|I9 zZn{JlThk^bRnwZyYhL=4*}p3t>sxyXG2m+*_pimWHqnZo!wAPjeI3ncTxa&+()F}2rUuWM> zU_ebZ1j>L$%Eu~vqHB(oU9w{Mq%yMv*CJ+;5POSb{a4V-%ZeF6NO2)8yzQ(U&U(Uh zkTE_x`KbSESv>TV>hwu=Ta`ntJ`+C2ij5eaU)i$N_(@tO1SS|Bpam`%EhyrKRP>@W z<&zFDQNUp2`Fq}}F$kgD=*V%(AXvbd{sr56=z(M#EG_`0oPKzo4OWM|9PT`|>9?Fd zSn~J-P;QgH%__stgFes)yVG*Q#8$bL@(;Jfu1j~P1H$hPtt$4rHT1wW* zgVHN*O}J~^;N&h~OO8%unS_BMQ!yoG*Y_lsTETG`5~zbGmSt8PWmgi33jm<3l|* zl_ZJf)(oBQODyvdZS$w##Eii<%7F16J|ee~L8^$MeI;=V$lQ`eV@SnA?+;&MZ|Q42 zJiV^b$t5%}`T%$I*Qwc(04&rRYO}Rr*RMpW-(u`qR{J|}faPSG`**s(0Z|(k14Vya zz=?DGemTn~B!JPxMT5vM!w6kXE*qwC-E23W*y(ZkB)(4#lX=3I9dc=1Y>z~TFwytt z(6o3BUjp9uA$hPh_btIr>1rtBIwq zKx2^4Yzm80sunn*^LeDj7%$CqF;?99STlRIGh#3a-Iq<+mwyi%O2GO*B*R7x>Bct) zk*#i&rRtZM>J&Pmzx;E)7k8Pc6H8q;RGv081LQw?FzUWxJBUhy>;rh-y$Kg_5w%aB zg|#?}cPx-b4x12da>@_v26>W9b6}gUKPYHH*i)v+6BVLQ7_8YV=yE|s;_u&RR_)d- z;xT@yV{dmdgWEHg_u3PbD@o%(ZK1Bj&(CJH6QKjq5(bw`mkuw5`y>)*our+~B#Y=^ znlY>HV3P>v=(1W!nd(&47qihNAgwB)3xy_OOF1q0tS9fIN*u!sx`7FuiM5+2@}Ycs z?#Umk3k$Qt{tGw<ckTWl(ooJW+|Di?=fB|F8Y-`Lbns#^(w-7&(X5TT98H&)#QmHEcjKFnlp_!KTCbe zXZ_cP(b{2HmD1H!CB>b7ZU+TmJ?yDHR;7_&LH|bHr2+%x8erL$1f7*ufDOMG9TVdM z)<&(;0?$Lc3PI^L1w$cAiwH73<^stMxfL?FQdsI@Tq93YZl#ha;7+YaHTY4)g^7~e zf4@;EeY1L9@3GvjeZ*fT+P}m&b4|3DUf(@vxG*V{@I2{--B9F8R3|UZlMrD6Ym43T zdzp0$woaX4a5Hbt+hGFBVP1z@{1YWKiC8}moJ6}ee@YrgojZIXS9Wv#k zGZ*-H+Ojr#<2pahi3@_N69gHTKLmc|#_rE>NBKv6nM2jw+NE2A-Oymg;+|piMOUIo zJEZRfpKH1>MXTD-XGM0k=^t=v?~3Xns;aNfekv6BI57agpV5F`E5jwPHvh-uZcP$W zfN(x#&4FLNn6aq3tr}8DBn4?sKwWjg7DV|cBh*SJaY{_NjZ!v%p<1R= z^>EU?N%AXx%H-&fYdNtkDz_p=lY|L2on5I4Ccy!#>CUdQNrLVrQRexcY?)Gc_HZHA z#y>v+^urY1-<53nqGG-u- zAsY-uU98?(kNegHl?OvX)zbh`g`_eoHe%buu4Er53|{lF&bZ0^A}uKMVrrg~MS%*f z%%A+kXAafSM!(7u={VGcy?-tFQLp!)+)@uDnDT-77(Keo*K&(#AUGh!j<7lttSp;c z3chC^=4sdbToz2xJ&v&%<&YqbU~!gxHE5KX>p8f{{x@9+`pd`Mk=Jh2{`Ji0;EcD< zC#sMvrWsXK+m6wl5_ZsUmQ$Za{Z2=iNptIsXC`3SD<-2(x*9yR&Q;S&U`Y$13!GMI z4QlOd!yf_w!&aiGNCweN>1GU8Y}w%gmuV=-?UnJ<2AM&gunkaL05`K~bms&Su?YQu zg5&-Ozy;0hn7m6mhRv|!^I|WbeawYwnxjE|dHw7=SD%WVE{9}+u z^jdo^W|rtBG7Berj2;JfnEaf?z(cK`zHN} zZJqRbQf@I4WNL;F8fgDfiGQ{O-_e1UJ0jlt8)KeVmd%TVO_20Krege(8UK!TCOZCG zPv&KWkAH$=fOEfMALsjv!no17CfuBSqJRCo>T*#(cZ_-EK4x9#s@UU5vDExjmN|x8 zE*#Gv6=%_spMW8s5YDbkds~iU&8eD)!=k1m-!ZNj{EEWN%s2FD!#*t z`rsezCwzV*9Zb+aKcyz6#}oL;yZ)fH<%f_FtQ;epz8UnJ>b-}Wy}uXkgLj|V{y$~$ zlfw_Yt|GgHOluJ>_;x=Q+v3NQ)2rXfv-Uru8fhBu;m^DLS#7HNcoX=8lx%s;v{{*| z9Bm)q=_Y0f`goJShnoHxKi@@=FE7W6KKzA!%%1s}&#wyag80NtHf_!hbR9zzfMLVCh$qrD`NB34`kQ1efU)}W2V zy-<;B-$?FxIoD{vN&eC`yG2wa^qu)VIV4;GpUy9#{-dqQ2lKtYuk6+n`T1|%QOS>k-(Sa>cO2O7 zpG93(34WRSBjP^F8`7~=>aS6{-22gs-tubhpLtv7-o}WJkUpov{(E0^%I_3gC#s+4 zTo|HGu;6RuYqyo+A)DcyU#eK_{tn{5qwScX;=S&76F7R=uljvOcM`s~R!n+}FEvw{ zRI6Kg#)e*gnO#?$&PcMwrN9L-@1*3BwdpfkC)KR}8S6}9_(#X+ivmYtA2S0_@71w1 zhXgwn$eYw`{UoQRD-ZxH{s?eWPiEW?C$=epP_G0`DBK42W}4(Wgjou)`UXZ z%`1&nKhl^7o6!1KXgFlI#xSHEE8(5&z!z z!1*F}dVc1y2uT0f+q2n<20;Rm>)UK>V1Kn1$lGIkJu(EZYE6*Eu4j}JX8r_)wtO&_ z<*}aHF3e>N1Z#z)YISx~FGe9V}}2& zb#t)VC^d41f_2pXeh#($vJJ6aYAwa*>JuZCQcZFQLd5C;pgyMP@sWJN-?`2EmFXvg zWrnO}T+61R_LkpH`4ht5L=@Vd*}&#K%uBQE%Vn(%QCnkUOq@MqcY(N>#e%K>*d{Bi z$kHR*kV`WAd9Dt8qg)E1H8iK|6?bpUH(y=Rz2*rJAI4tVX0=5VIV(K-3{N}tj@3w zxm4Fp+qz`as!{d=C*#q7_0sWf!rUBS4JwIWVjEL%{kfyHYQ}$z=;eO1_%7YG5>jvY zzLib|8THoj@x1E)&Hul|ZBz7&&?-l1BrY%jzzZk<0O9|y_tST_cXcv0b*48ovUhc% zbMbI7RoRaFXUWkst2}j`nD!J#4I-|UCb|KIz(s(dUKYnJ+vRFw9G`tne2>$Gk)LaK zCPYtVYBcNPpL{uEUJze)x_PqvVV$K!kX(LBtc~ARb|cccN;UEJ82ZwzU=qm{^~ECFJ9VDt^E;6u5&?Cn`=yauz|2FGGf>ivyiB+2oXxQB{0Qgg2^;E^A24 z%$%XftbsqhBe3%8@es5)Z++uz(xJGbhQNvX7wDd9fSyb$EaEm5FIM~Rbm8jnq-KAM zWtO{GBXDEyHj8B8d4qw28}I0b-|NRBHiW|M->0Nm#YeoUHmC|A3G7HI`+@WK@6u>L zIUL3D8wKfEq2YQ@cX#K{{Y6`9cqJCV9BXQvTqNhCi)Vbp&X{diBK#ovzVP7AYoTkU zhfgi4D0J;wv`p{#nHlJNVC%Z7rkRcu9s;%K1pOW5hY*o!b{zUfqHIcyyfsPAj|`!z ztpvoUi!o7?=t&6pi8As?kT;6SmOs&v@f0z^V1!IRs}kQm?Bc`&$0)4q(1Ko1HV+WP zJC{15n2heN-yTrCxgGExR=-b^eLLZ9DdQ_9JiL~;0^Je+KK2Dm zgCg;8VCCLwZ-J-Xk@Lv<$=F>GvCImov(fp=q;-&Vwzrwfbahji6mOoWo|2y;E1Jd*W zI$0S;znobxfoP_v^#sDJ_NK?$`+tcd{!?2qy#otw*Z=^O$^Zb=|E;!6+zjoEO-=M& z>>VtPo&Qs1+d8(+n`4PTIA2kmwRZVg1j#OHmD>ElbbohWs!cb@s z3?7M7)5pE~{M)?2fD+>`Cv^C7sx3qPGV=2Bz3##2>#|xkHfB32oj%JV@?rdnF7;aL zrQ?&a@5WxqS*ofiZp$aC>lsZ`GFq$WJ&czf8(Hd~FO^h9^!%^nQE-B*E5*9nwu@}~ zC8G80n=Y-yESplTm@@WMi+UxtOHColTIP0g&5f>=${ow8lIQM;wGLUEl|5`MrvEy@ zkAdZ5SL6ouWT%l^GuzwRY__{kyMzgNdR1+4$G_@hxL_0>um zZT_cz$Ps8;AB?}=X0L0S#$?xbP-gL8txr3{lPV(HEYBe4@dkVM&)S{Q+w(`FE^z*R zOQe0nt!7WeUrKweK>zjT^M*P<>~F~Z$0P5)!*_rW>#9EQv+F$tLKmJ+a1LI)z8)EO z;kEP$?NI~c2j05#dw`c{G#|a?k(>{H{bg0{`GG}78~(#Q*e#rmWBoxvMI3kx(Tq%z z-VWOs)HpkWZ)v0ns>xMO%3#Sv{p37`W?5)N6EKF{%+0EtAzphJK;&^*>`5gZ_Z1P5 zM4!%7Ylc`S1!D}I%|rm!da9CYQpHITLROKBf#O1h`iGZ^v{K{a2woK$dz)yAbGWPtm=DHK8A@!;Kx1qt`YQ%MCm zIUf?jIy&L;YyVhYN(REi@8r$$ zESsI&cY@nZa{>}@Jkn>@F_9Sb{(M-o3-yiyd_3##@heF?aJqRr81~ztm~gVPw|9rq z3(ME`<@hW(C%IBb(dgDnQjVcD8TUqR5;hJOE9l*i!7xM;HAv5&1&t!@>fhlnoJ66$ zUE02l;KA~DK)VD#br!JN3@MwR0zuu7WW@d*A_(}Tne1!8|Sy0vO!qtOMgT6pa z7%)z2@IqiBZ28!}!rUpW^1Qh+|GNJ4eQn5CEZBUe{rI!zzFc+Z1ox{yI{yne{alGU zW=ymYKflTii*6A`dgh)B*eRxHiI^JO($;qIEMZtCheB z3S3*6wAWC*4JOA$cKPu5Hn=obz_^q#ZtOEOk)1<=!%<>$6vJ_435cK5{eb|9F=Mse zuucjNAfj_(s)fAbS`sKl;ux8I2K$|J6lG?6;Q}2GD3;q+H3NyAB)k(cn6YGHDIH%~ zLpB8v!#ZC>=1*@w9r9u`?#9yejt2acB9eCWII#j)quel9dx6tPUEq3Aeo&{Os#`Fp zoOHR_)vc7?50N^ZB!h|*B+&&IupnJAZcP}RFR0n~L~HA9ecl#G477A#zgo%>Tut0P zVNf|(P8A7Qvk;S8$=Xh3kkN8b@uNa75!88|RxXEiwA7^myGCD62wD=d>4KwEbrP9{@v9VEB9Q?OoEZr4qK8jtnnNWpV$VSz6HY@uf#^>(g z115{BO3fT2Rr1YQJJq}xsMRxjckM)mt=+117`*|Rnqwhwv1~lZ%~XlJcQg+Vct@Jx zA+3{9&b$dtr6SEXD_9nq6DUWjkRp*>#Ybdz4176!W}M*FkSsy8kFzq6KMw-a`fRjO zkl;uOKeXI}OZbL?TcE0fXeDoIjbHa+I}!6 zR&#z?&7iEH!b&VbI(qv`>&Y(4 zb}S=iwcu-8&@}5>E9hhev*k4SR>FmdT!%+&!MKot#Q|L-3Azq3*((b&@BpVg;@eKokdz>u<;F8I&)dI zCIWm6Jpi(-$>?8^p_jY;L3@mp8?dHcU&4UAi(#Kpv^7HpNB=okS;%<(c_Yr`&QbC( zAQIecncjU&;WpD&-38H08F>l*{^|wKt57J1sBXR{7F(e=dH?&rR_Nx-(&;=oZQ|Gu zqWE&=WM5wv7XanUi{ZNg7QcP|_ys%I+Q0t9yT5>m;rX-t;b#*|?5cV7C z*(38F1p0T-_tFp?9xVN15*uW2*qUgIajughU~GuL=z`{)?3Yxboqb;7NHSsR0@q2D zcvtBWEyVAk$=Zn)NyM0>#4tm4Wj1rvQApj7(})PE9|-ad-C^TaYqbIEi7QMZXDyx!3% z%uIppvA(Y>lCJWz6Y%dj^Y8Dx+b866XnJd0V|9MumucGr+)@YJfk1DVay*%tTTwt} zBe)wf3o`Ha*OtuFQdkPB$M6a6{BQwlosJkcTQELfoYC(adl$<5bd?lhWfL%mOzP>- zHPaqBhxPYRu>BxP05P$&|C$qkAntESI7N~n<36y0nS_Af6%Ax(|1#V%nnjLfXaZjG=9!`*vo_AAaHBtb{R)^utkeXPW5U4WG4>;3AJhDv?1flCpNgy<=D3( zF!~3P^r@_R&U-8%Px4+a*H5a{vCwN|9Tu@I1v(RuQ2z*(nL~L{T;#yhV^f^Lsc0hNflLBQ?Ue-GhB}L$fj}B639SQ7Eq&X=C4g|B!|~%w?_#v5N9fyq6LkX zwvT6Cz$7AsK*-CtKY@rm}EtO)3$EHo~Ejr>#{IUzj4;#I^Sy5mw#Eyi6N7O)#rp#oG35YWfs|) z;(eJU;y8`hSTi9K4w;dE$=#`~jLOryW%e=b(wtv#M8W_SB`4#M3sI!442iO8JDphN zBW~ktQGC4()D8jO-{MT=k=?BA7CVZ06SmllF2wfvh&<%xwlSYZPmClN0!@hOu^n%r z(h)e5ElHn4aFl8hbJPd^_*4K|osTT&am|}}BbE7})=wMk99&fmkjlOnkj{V+cT~4X zu&x`B8zs4pX7#j$%38wxLBtQLHqJ121`HIk7SmGK%_KD6aFI5YFzn$$YI~9@!-I_p%peTU56_xF%#ucv~AbBa>Z+WU})cCYQNDtu<$7IX;GA`=^bJz zkKtsc8WAb5?nyiGl<9KHl?p0tTEG!jYMtSaU{ZKK<#7m{bpm2&y9#PC0>;U{Rn&E+ z%=hV{fM9c9 zWx9T>6KLJ0UwcE0#haZm@A>lRdO*!#w3B!u>+t!gSavqBlgyG7&uxoGXO>DRKaS3pNDcm;abTN7~_l0SGoYL?qk za=8sd=N6e~aWq3vAL^{6D*fwX0YgwS)nmCMJKA>D^h(-yIKSF!$TNc7mOz9-TVxK(;8cQX-$BZ15mkDSPI_8ja3_eS~s1bkG=5G>QmzVHHTV{--AVR*w4=?rJ!Jfx+LQaxYP_)>|zVNT{E;TSCuK0#H?2IPK zjA4I|bz@w;)o9%d7h4jr#c3`8bpuT|cZ4pN$8nqdIV0Xs$ja0K4&(YL95^}Q)hQZj zQA{#`SM93iP>ZB*3-Z|2o8#0B0zBO?XeY>QRXI>sQ%xCui-=7)pq^`kBhHKT@`BKzXV@m6~ymV~JE?op(jG9cUqYy`m zTw_l*)k^P4i|}iQZ|auoCR)Xjz#LTQ&s>>OT|+ZWC5R0Iw~R2$AGQAk%3g>@Xb+bF ziOnFXxlrMpclyMc!+pPjbUjXp>T|OXUP|_4m~7o?e};z9Y693P+_UOd|AyhSFemxs z0mCQ0Ck3~jpNaL#HOPNjE?7>FI459vAyxqjkis0|JC$@snbU&ZuCg-czv1xsTsh15 zHq(fOj(wTuCxUf!nzdYAnI)3{eFrfP7)ptiqz-g$mrZ~PAWr3i&W^t5i81;vPHMW?!v1}vlzCR^PTB$iLAQ0i1r z8`hH4^;JlvE?$7g<4=eA?h!KisOz z`9Z%tYpT7)WPORn5QGSZ>=bkr?r3P=k}esuB`ObVK6eDg6?`7_iKTmqn225)jb&7U z?=jXO2=tBfYW;~tE&LAh_223qd5%KqQ1lzfi5K6a`q}syC~!aCB*Ibf{S?>Dd?

      &goyhxW*?jd+>Z5k?gy-(wJ;Y7jFLs=~_T9cs20sU$5^lXt zJQpVuMJF-fMgnkT(-@O*vA&Fp73w8R-nv9|9UgDLrlT*;+e%HKhkGGv-nh*9d#0i^ z-Q-&9xU6nh!-1a;RY$AY=(DP>PU>fO(j*hbR@b4)%pHV+GcT)1yGWGggf&60?G=WE zR1+^Zyz*&4IvW2419Yu_@_vqFob#FGc$b-5Uv*b+|(l(Kfyu{%| z9}*)<7=k?!w0m&mtZ@H`_vv~1OER(Nx$3sAu}Kayx0tG~yU2}^Q-?YuDByU+AUT9s z)Fmp=FX(|n;v(TsERric*-J6arP~a|21&vUDeaUKfFZ_Ye8O_V_fiZYJWycBa%#nO zAn$Pvrd~wfU>fu=khH$T>Q>*PBNlf2?o4cH5QjZ}z>`ktzMkDTp0dV)O_TgOuqTp& z^4+dJy(Aqm9lDOXeIhLGDF*$HLbKk{iw|fpdQ_GPilQk)n(5 zVbD4d1=tv71S~GD6jJ;^f!BXYhmu-EnQrvJpB@Ud5#4S>@kV~wmZa>U??B5Hjh0wS zQZto^v~`r(_HEQ`ucXu|y+T!#(7tKfMk9}_XI_862&J0BVo~l zj4}91EU=?|#x-tE>@_blgluG*YeXl?7BzLGtq|_82(LpyHn;c8qBWJ6u)-X>39-E2 zL04v@pGO0!UgoLUAqjWoEmG4>GYqbVNdLMSq{a?5dJO@VA@P;#RZAB$CCp;UutFAs zaI!Mca;y)bL3|vA`ce`W2{JIxGMlkhEHAp9`l4^W-Oe;Y3J(=(APpORh>je8>SHlh>=l8mhP4yI(&SWPQZj6Vixhbar`*S7KSdJC<6T`fM1$896!qWzvG zoI~nXJgU9GQyt=cz?+(z4g8=;UJ!$Jm>r*qi?>Xd^uDwg(>ZU67o)c2UCYN`GyIsJ z-leGo7DzKNzk8k>0TgGG44LVzt~q)8-EG!rh92(&^Hn)v=gEI8kgX&B!0#6vJy@ud z_p06vYIkl<@+NRQPBpMW3%-R7(Xco!XcY`5EndP~Zg><}C!h_({^m`{l8v`t&??OR zYTUce(>;!EbZs**a5lHCcQ(=O(#lbPwTNm1lLy9c80tV8#wdsX7<$@b%#=v6l`0pb zjvn2#9oo^V>X#IHaL19ngR!mS89QOw?8%bp#(em(R#uP-j?(9>iw#iun1=}0>o^L%ne$HGg*T9*k%FwF~Db$Y}e>*Y?bSP93MizV6jPq8cH zB)R;Qa4gcNxCHBcXH@_K=(dvcd4WKwit|HxP57-YD8@Z#|2R#X@d-j_OQq3cdOf?% zT)ySppYu7P`Z!vZlYipg$8Ja6QLCSeLT}!}%0t9NfXy!X#=R zS$8j~qG9PQM_qdjMUgK9>A8D=e9!D#JwMcIm<}xAm^)uP$#^8v>#r12_6j+zO1~%`HU; zapf3UUOJ;8N(}q#lOJC!c?m1Ob^fZVq7tUz20Z^v3LLp8T~W{E6=Z5 zU6g~=+c8}G=7F(%s6*gG%_bX7X}@(>59x0{rNucem1zi`#W$My^YE!3W3s-D`-#9((m z&7ABA_gg<6@(T#bu$&IE%{#KXkcuTjlyK(CZd8#D(F^9YYV4AeCy`$LfT$dBRyo!8 zW3e4b9n@#r2R%6sYM(8CP{Z>jbjJ8{TCWVH_QVT(dnlV-pLicH^H6-Wi35*l9PiM& z((zozI>0MG^4gQO-8NI(ZElJEN)H%D4(UM*lH=Z5E7Ux9{jZ8n%mD|`fsQjTfVb@j zwi|2aJ-tSMZnAq^^x}nF*rEK20O9(Z>oOrb*a%jHY)Ek4Hzn-Muk54}lm#^|2GsDYJUKxp^=?#aL$}dQVtUJIXsd+;o$=z7!}M5t5w!VQQI02O}Ak2yS3wh|xw~ z4e%3O1QgjSr!+TE_m(|&^XX&17QPNEP_zaJtxsh`;GRtr+10xToOlf23;BKGuw8za$(uCL_A+lU^DQK~bHMw}h`-rU ztj-z#Li3z@;pFnM{(|g^ilbPde8c}QHJ&1xCA_-Sb6x}i0PutY06_X*=sRNzLpyU* z8+-Hr5O>uY*G?$ysNc6`3O_UqtWCfeYfhmgeFJDJ0OpQK_!TMQ+rDMuVzkdvR{tR2 z1CVEZj(15O{H99fN&!?%C+T*)^Gvt;w_Ay>_=>xDZ7ILF_$js0^pj9CO16Pj)C32s zMy&sC`%G4wl4U2u!7mr(E{ie5Qf1ac)>uX3^td?cSW+BHRT1&~_v>=}a&pn=@$CG( z@$-(3akAR;T77C0CIo*v${1_$3U!W&PGQ~Bo2x0s+1y{Y2@wx*V9=JExReD~89{bQ zR<+SKZO#6X%NA5@W2Tkj*bB;PpDHwHWcrwB$-$*ieL6Ix)03lyjwSVv!o|oHucD-I z+O|1t@j8<B}A@<+~zM{CC*+217Oxf7SYw=kTqriJct;$>$Lz89kdH!_)-?`S|$9sLK0q$7*3KSNAD6{A>*IDF2Nc&*ww<;G47M zWc)p~LiO0^yMy@X=ptMsa1Nh^biVSE_7ew8-8hBiY?0ru^r%CIw*)HJI1~TquA|uw z=$-1W>t!VD!#!oDn&4W?(3qimxwHT)rS2RWI3mSr0JRy7mcErL)4Oh?i*7l!P7^FX zBy!k(eN3A3gn7X-vuAFw#uW&!v=mgl;IpD%w&M-O;2ed-p zpJlkRQh$+BpX+!*7PC!iC}x(i7r#Yf&zp0pos8*yhMnUk_m&b8%OaAyH;);!($D0t<8AKdykU{$^@@@uDin$DATDEVA?5AJfk||8z<*2( zjoR~Htz|qEji~UsfQNuAzWEdv1Onj1VSzhR1m<(B^l1`cl)tLau>x7~V2empc?d(l zstiotQWZe#AL9x^gaiTWGQIDeri3@bQiUtezNFr5EHr;`ex2@evkCyp=$F!r4lLD( z8*58?7KiB2tn-7!omtbRkCXTHQi?WLJM}y0aY$Uv)fPbt{bja+vhj2$i$K}K@}5jZ zG$dZ{kKcx>6>UwqRvQ$P3C?v3ZWMD+Z6+hDiM_W&y;2QQey?zWNJxf5z-h!4^)ISv zq6Gr)-;*N}%kLhF>jE^tr#BKCz!vzYU(C5Fb6YsEBB-s`1c^PiZ)u)@< z2c3c1#LqQkvaMPH%+~usW5dui-z3s3+FXF&-!f1F9fhm zrg8_EI|UJZ1mfrH2AhWZn~e!QQ%0H9!?}v;yoFVX4Q}!CAn)9_K7Z%by*QnCxf9L6 ztH6>F>4I2QmsNbD1oEgAzD&=-b9r_d*iRW#e;SWRiw19d+=BxBd#7IMPFZDG;pREX zO4Q9G0@w|$^P9b=Cx`!xVtU7$-@9n=)wbZ{L(h3}vLiBvrY^61zj~S!x8Lwd$~nw2 z?x;l!@3PKwD0A%YR!#lFer~8>6@Gn9bpEvJkpW4Zpc{=CMi+jCg-jOcC9vKX8zMXL zty%q6pq4O6>tH^@$fdGvK|x;`JDN(K?||WcI17t-jSL+>zO^%LhG#&gd8tUY^6iBc zTH~l6SY$y*h>4iI^8q2TyWf$+#z~k8Q3h+WG7F0yS}}s<8ZR8JbS@e>*XY%#{|bb& zWJH8NEDS(GJhg}fo}S`XhsGo+c(c?wG`QPde9bEIlVR3Ddrgin?*YmU(x1Hk86oaO zP@a{HrDz$@8{CQZL}!rjiO2d#+1}#fotO35M_-v7Jd5j*XPUWxpG!Ab@9P)0OIjFh z&d5P1-RPhqoJUYM&f;KC^wXk-j~Ubh%&v{*K4*F_4bgk9+?)g2*j7v(Z9*qf_N-CnC^r)4aq+s zDAtAr;(?4DWj0ekcH0M1ibW|lkq^>3aN==QQl;dU@E05^}Pav zT?t9SHBC6WVJqPeYFX)BX)DcZug`tTZ0{vCccQ?dqW7)@`ffKC=bREv z!{;4lg;Nv5pE-xTWu&MLcAu$174R}5$eN@jes+Wwr%udlHLPc0#8OaRO8Kjij5ISZ zZNLp>P8E8ecd~;`*7C0>uz$$Zcc9!)Yl2`;3`oI=k27|VyvyLgPNTUAYP39V6(eoFnJ~pTH611Cj za{4omgUkZ?;XFTx?R$ICJMwO$ukNQ;ILoz}y z`EMx4-p<9;&gFm9Nv3g(aDxO0Al`cY&^jVD2@54ssrY}ObpW8@f@ag+eSH#@OtZH$ zt?-R|F11~?1&#JpBz1oaQ?kgWmmS4f{u#5$1`Zk@T09+3Hl>ogJ&8e*X<<60VJ2`Y z)|H*b=%Kvfjuo`(XggJvvz-jL^xyN`4T8EMc=T&ei2ey^5BtA>8ni%`a6bCH_d6f} z!0taCjlH4Ce{fo?vTXb!148$w`V^vqBm!-;N~#n!n!>9}z4yLK=XkN> z<;Dnp^}{XZW=VEX!dSt14DaJkm!~I#d3B71A@3C4dfENp8a#3VBG$$|=^7kD_Y$|j zet7H>A4JJX-#Z5w_tZ-cvs*0?7i|zv0Tvm<*E-V^gcQgVpxjv(3zm#}0GD}O-|CU++WQo{m<)#Vgk_MWrp zwq$5eDHv+~b%ZV1dNi`0rV-|MsGoTX)2WQ#PL|n>x~WZ8GMEHb+gkB29vZr6S`Nej zA0918GL>e+-eJ9cgvZKa)mxAQKVMzCOiQA$H9P8FpWsUzWzYmDT>p(l*V*%s=oL z4F&#y|L^OEI`*2GhXDYn{HJ}z|1YoK%+T?F(7onASsokGPn>?BHUFEb36-BDQ(5%3 z^g^*CS^JSyYi72bScrszkqiG&Z@z9oG09|{x4Wg%(8kyp- zLXoI*8LWbYSw8v2m$VE@jz$B2UfnR;vQA zg_C2PjlDjhp?mp>^AauHbFYHSh?THvK9x^!M;ck$TjU9*q^P0M<@L=X`ZawvHQ+%R zW6Wb? z@Ho~bThP7%2%>f9hZ{4UALkv${%I4VkdzI@kt_SckDusR$XS5Or0S?0jQzpx=4u!20RgQlMYh6KZ_1A1GD5w0ltvIwGV z6Kp^r7ee{p{&R{VQ1Cd+o$Q2Jh89J;eG)?d{Q5n6?{xQ|206GmoS2qjJ<|~#Bg5LS zCIcl(bOnMDz}?^7s4(omHg?b1nD}|S7^dg1hAe0CEvjtlJ5W2CSk_9tUJM2%goEEb zO2x+g7r^zn7iMX9PM+O7n7>>KBSnZU8}5BEjp?hIg18@Z(_DMDka>WZ7&X^YV~1pZ z@!6mmq5s#8*+P-XTuN8YfIitUIY7{?lwt~LMYJo~Mv21u6@gne@aU&oQ8e#(tPH9E zWt_VDxO`=u!te9a`<pF#X zr3nhL?r4_~-yw^^Z8Jt^dIS{`x!Gm}h#x1|gqU!%H`^L}GCh}LaJMHj<--vF7Wdr} z8Q+pF9dmFn1&D#x=rF2)A-9=Y zH9uq(yBmTnTBLms{4vSHp|Ek_2ySK&4%yJry9jnxa&X9zY~7`n8*m@ID8Rd>!oA(HGOU7wM%r|8`~?thUU*W-rcI}CjX)<7KZU8T0-5AUswvulR8v0{=*`7*FjUD`=IEbvLZ z-!hE@y)q?|K^)d)N9U)zas1wpsTNtfM}C$HW!AX~^zA^xknRjbl$CUEj+!MOmsK5X z&k|0mv*&>nff?_08*R@1^j$v1=NxEzYQu2ci10juW16N6G9f`3}zThAH-ufdNv;&*4t_ z%!R7Ro!n{l;A4T8nPLjw#vw19{Tap+U+@vmwg+DXVQNl8j?7W3<99!6;zN%v1=`QF zC&;_YwsMeyGIc#9AP(tixk0A^33!^R=i6{byj>LreVjp6M)8kUShAn6Z8>8|Gm{lK zZ|rm1EEMZKERi+w2bGk*%PF-gOqWQgysjmjiI)D~s^qc3{Z2p2G*xx=Ype4tSQ4_< z6^%g~wm5YF;7qPUFgvd<;7L|GE5T3HB7Iyrt6>IKKOyhUPmVYQ^LvdeP+)x-`$Vt= z*Uo|WB`>BN*P)bfV6CyrRQ#I_ds(t`K3+pZH@@q7%L%@qHP_7A#o+WnPDVff}ZE@I_q1UBo%+oT{~cy|CVyevO#=8?9@sSb~H6^YW@vn7GwT(Lh>>^ihAC8LC%SmX$h z{k0^Nl?Y=EwAR99NOY@}d_`W(Sg_{nI!|ax&)h_<6pkuQ6Ig33wuR8`TRqrt8isOC zI*Asf2-wL&=^_nNglAdIokr)~z|3SKyA;GR%s8k;P}d>;&WQnbBUZz@;n0cje>?NM z85EY>%2qWp@?tHQNZ-JDV!!lN6_1_1#5vVzV(FcU%p0BT z)yd67n|XzpWKm8ggIiv2Rt@xFKLb?S>+z2ic3up|b1}dX^=w*OR`Q19qT60IiChhm zMre9^aA-!(3F6?rpa;1;h9NM@qTbX*bxq~M6QbBLw8-oGDWBJMQmCeyt2}-W3&5L~ zTJP4{p=r(WZ6n4NBRJw*IX)FiE-s4)2imh{p_vs%afJ1a=$ayL=)VYAkHH*8&$aTs ztTUl33i8&Q4l3CBq-B&`j}?ubPh_cFgtgcS44qVrlxdX()#%;xoMDV>3yOg^t>sz% z8g2z`;TqGs52^RLe#8Ox z>~+pVVG1Q0vj-+y=6U}rr-n|`c%^W;Z*`J%)O3LET6wps{z_4PL1TA46kaxf|KON0TwDmY*qZ73>EuZMAQ_7d z_W#_EFPDq%n+8ik{ z1wp@$3$k&RxHsuzonVfDQf9&)QE!`@{-8AQh1w^u^ai7&AS&a|7Mq`j=3o`KmFmtQ zQl0{&n6w@Ofr-m)^x0UNGE$VMU|H}UMp*=nHOpbKW{+fPI8XVotZX(7YII1zw-RHZ zS7k!ax@*pyB0jgdk)5FG=k*=TE=YTKPwaA{NCe)OCl35+dxorMP@!bn%M5b`@pgL&0P@D<(76roS z3K@OZm);qEL<6=WL|wJ1x)vqfY-xpgy26%~6@MhgAMi&52Zn2>d;)iLTQ#a(oCK^8 zgb3?^TOiM6Gu5w)*P9#Z0AcvvwU*s;r%XZHoW!5jY8yU0{K0jm^)7GBu+(Qf=V?x;OG6gJSY3 zyBYvd_i%iBRv~LkR(w7+xDyPls*Vu5lvdA-MK4s7gF#{9k}@u#Nf=47Rq=rk{CyK; z(YwtLus_yf=qYD~_t2C`XmHA4MP{qMOc)&C1G)@Z#p|np1oKS|+mdlL1+#<7nR#U0 z1C`EbDHgZ*1M+EljQ@)I7b$8^j$W`9x2dJFc&eAS_0L6e3f9UBY(Mrm>?{6udhXFu zK1`O$n<30L$QTyOCbRAoW7Ar@oYTy)Ptm-r<(^CN z!>+`1+P2FK?ygn@^-m?s8yat|WohFkoL%sXzuUc5?^+q0i&+v>5r1znK}=@0&%@Vo z&V1Iq@@DG+${#6_F3DZ1EQub*$K3iaf92&}Hj*Ky9D`{P)Gpdy3)kES7~z?|HuM|P zg!7+C_L~grLH)O}H_)emN!NCyS%@&^B1rxZa{N}@AfK=N|92X~*kMYFMEExeG5+_3 zC39037fU;HePUv%%Zpd2L>^4p0cv+`99&8EKYHI9A$S znI=`RaNlc*zqHjN7s*;jjk7ezcx>0klp-ezBkW2=$Z{LxXjCi>MQG@pJpS6E3}MV= zl*};I(=-DEhAcl4DYx|z@m}yOLMiVECXi8hF z3=s8lMD2tpGBt{6lnhPLWcrp?Hxebbx2YM?IqRxW_(OF-eMN$_eE}_ou`aB&@;E+O zjDoWZMn=*JVy{#P^~G^4(qyrPbG(xd9i#%Ou0YEg7)PX_AlDmNWSTJnK0~##9B66f z_7zTqhLtq-?A6PlGszne?oTsBy%^uOG%)``q*8WH4er6I9?b;uXi0SgXu3nl-;(sk zh{pdv?7d}hW8JbPC^IuNGcz+YGc%W&nX$~w%v`q1%*@QpE>oG=Zh!a1biX$}^QM2! zj~mjK6eo@%q_gv6=1N$J+9PV#5D__+1G->^!Vd-9yU(kE11Y4s>b#9K%WfrncRE;H z$wF357JGJ6bUxX8UGNO*6+=t}?Lk*2jao&KXZ)a5Xtaj$3~mdH78~Mt%E^vw zneT4HC2t>ynd#fIalIywO}>fCP8;8BQ>{7{rTY9UP!yFYwx!5Dc=|e~@Dnc(2LSIHPAFPmm5Z3n zwE2E+p``8^I}m811p?k~6+Nx%uN-Lu#7hF{nRmiO+7RR>9Xf?%f_u>oml6ZTRC$xIrYmwpTKX6sO)6q zAR+%57?Ulg38fgOLkw&8_|7xs9P}eb*kCpyE2T7Z8j(d7c#lql(907=OAz!()2#}i zKcD&+3XCJ9H@tr32pSj2_J(MWJYtU|a8kohc2VVq6~CACggK!CTd{S#EW?SMz@rm* z$60DK4ou_|LrGzA8zrhuzh&IXBWOrh!!K(OZirp;LR5h!L|#IsF;x2A6wLd$86=R1 zZchI%9r3oHrTDU{oM}qn!@XnsS zV1qhe$KwGZ_*q4Iqmn|a^c3#h~=AhmKKmQSRq{F2>3$jdk**( zP%8v_CrRyI4*Kj&0n38tR4&9;z9hSd%g_$E%C$+Xh7-hjK3M-}NK6*J;O*LIS)Lvb zA-pL4uE7>+ZpE!VBpumQZ@i!hG*hFyRaJWMt`qtRZ-jT!{+SDv3y6*Vt?R>0!yL$T z1>|IOH*WHgBr?8E}wC234Sr$(dstP@x*QAlM=S}@Tj z+Gn2J|Z7Subb1@|HH! zn?=4L(X!t5-ovE3bJ!({B^VJG{QUjKc~q95@>m5^lMX*B=}m7FeqgcSLdwm1*l7qm>&Ed(@IEqXJQp{el=d3fhZpdatcysZ%tZ zFVg~qe!6Q|q!@y#I22}Nj>esh+M_*E8CpblglzUb??a*QOa=lepCtt(MY07O^*t8N zc(dMU5ML5mcCz_2705V0wBMyOlnW!n9$|!6rj-Mt0Pn2-3i+igrT6F=kG}8@!uhkS zwG*T~1KOJT%5ZBJFpx63BDk>YO>NWxGs%0}d|4 zfiJo5pYe-1D$)rI(>)~`;C%iL?^RGt0(Nj4D29A;!U62uDeAOLv+eV^5u2YlJv#2q zKk7YI+!>ma_3d{I zy1edQFUOXuJR~H>bP%K=aKaq4!V-a$Fej0fkYx9spPxU`F~#bZENBt5k=8k&bX4~x z(Y->ddCLx%D1f@N5nY|^Nv=@TF;5M%no4PnW|6uqk=K1~$mnKVWY;z;KS}7GLa`!Pw$U>m#@AzW$gQq{P$f743F$8A7~x zUr}O03F@v8Twzn?Q-gMRJDBQK-W@=IOnW{enGelv7OR!U+T%%uD&Vvq7^AJ?@_UNM zh?Ah2u_C*MP9N=968(}csG{Vs7Wf0yLaW9VP|TAxn0rH65YjIIirK5g{m|nq`0&Yh z#Lej;?^L7UT~;XNQc+Xr5VqaGlnG}DwS9{i0sKF!sM_8NnoJ${ySWL71@Vj z8w69N?`_hb1b#TjGi<-sbJB!07t>SeBVI@!)4(v}wbaR;oIx(7tc8jkvHxioiXlL$`YyChDb8D#Fn!pyiCFQ`7WjO!lL z-}9vUxEb<>LLB!f5(s#G+vQ`Sum^>4@^p0|{+`Ic1QN{LJ}n#E3d6fO^MDCJ4|b=` z+6a(<&pf$9?i9-~nKE`wmGORYDE$6$xCATXfPL0;dxggjt`) z5=*WOlGT}^>Du3cm>H)yq$H4Y6Z#*hvbXT)u`Wm!s}F4rETQtW3|6WNK^OqWscf8> z2`-8h33RIU^Qnbq~k=`NMf8iE;~2*&Z}^z@T;PTq&b zjB)QAqzyB`pAWlt0p8En+!{NQXsSMG?;P32o}b9MzNhs#)kk>*Jk*;l)g8kU6fJi5 z!44V&1XeAbU;qgH?egiLKaKcSJ8)6;3HXJfjCzfWS68Mp5XXyhY*Y%~rqE~D8WmoNRr!B(&nMZ5BWpy(wD0A`YOKwSLLdbsH zv1Q1>(j9~{iA2j*MHjFQ=g6V)2};E@L~R&H^?(`&cz1OAK`g5I4qQ8PLcQ)p)U=M< z+Xa9q&;AoxO1hLq#-}9)YELQDS<)=b0KUn%zm42;tKmKhiDMs4nH&HWo@LNFZhI8% zWo^cxQPrO2ok;`f7;{TqyEJ`MjH$mxnd1J6;@o*adJ?wr1cI&XE*NBZIE5M51g4Kb z(Cz2^@V6iz#tiapuNxO6pr+RGS4Bc^xFYXCe(V%oCS|hRDz>I&Gbz;jFkJt1f=y#T z$5ZGQE-^lL8zDJjuoEJd8n|;N7aooq>e3?x!aOdSQC@+X|31 z7`)C(m7(k@a1=+lWKeeX1mdBtL}#$srb#Z?`0dVqTnSHEudsiS=YMWQ*_&$`J~f7R zyV4`?32R1wx4ab*O#mKRo)%oO`2JzbWLoL!ZU-r>gMWw$b!n zZR@{}opwF}oy&ZAsP1ZEBR+Z-jZ;0;8|{v~sZY#g8n;a?PR#m&GI#Ru2za|czTVrq zJbYeeJbV8tif|yg>J++>d3M|HMXn_pd0e{0zgp67&0$JfR7wt%yl~~b_*nY;;)Iqt zj31+1w2D&BamcgDIA}okP;(Pq*LcQ7{5=DC<;ONJJ5Hz6*GnX5F1CaAaAA9TZ{8b>d6W z4d(0X!kM)m4VqR{7H$JH3j;CSMWjJ)0I#Q@2X4)CLAOg3xD&MglY3Ki^YIg(ZSrbA zj5FINWviuAKtN;`-adGp4Py)W&$jH}r{2?`{#QC$6${keh3+AP9TK0JHjl-_{XF!$ z_B@5)xpD2Zk>{|?o~S*#IutfYK3`zmYPj@aIi8DR+M`Mp@g)=CQt$dN)02*m;E@kD zo2y&<7Y|#YJTJCrxZH@?UVB1A;ehs24DS9wGX=N-%x2J*Ypr|6^0+J0_|LEksj!12 z#cG2WV?NbulG<$`_1W+gTIXSOPdn;lGAoUPa!=UWmf8qNJfRbCx3s*yK}d2$aW(Yc zV#1iLCDu}6MSMSUI=lYTebZHfMf$X?%m(o$Kd?2Gw;V9Z-6b(`>>YCEYBO?jhAvcZZAr zv6WhQ8CHDC4=W-oWs%h&T1kpFg891h6rXH2bi;_M8Pgj$cHrzj)0JQ)-BTyOqg-%T zYJ3!uq^(3&Z6czgb>q7J3lWs3t7hk}Kot|K67H($8&#EA)BLS&+wBwiggKGcqvu zd>gLrLfTyyQDZXL6vklOS!LQ zz5X|1`H*y^^l>4rM&zh-OX_wMj}Ujq(dL>5Yqd|}Fk4~q6LB~~_PjT^p^+3OIRbzJ z3|@9@ipvvVygClN0T|fkmkFUROw|RUd}7rOpdN2!gBjV{JT+&hKrDkciej66gjJg> zWv4A{@)C&+S72Z4b+ua8Z~vAtCo=&~^HV#dbQnge0Dbd;!+%41fB;P2_|`UX1$ z-{B!TZSl_}GcPb?S2iI`-aM{Y9K9Dq|8G+Dy|V^W-$$Wl7xuMy?=#|NJ5yeVZO!ll zrm&Lw#|Ux^F(O}8`ajq%q3h`$`(jEVxET$`C{ZG8YQw|*Z{IUqbF%aDB%FCwDt^D1 zgZGtaCRAJhY`lr`igl5!kx__*dcISf0 zpV3y|d%OA!n@4B$;sCV{kqFxCkU8=i=<6ey@KN^4s>P3JiTnUKG-1@ERlto`tyuG0 zBRo%SpGHhCD;NHpg`00aVCBxx+FHa$)R{292r}|6Y(ICvVAuQdFrADYuxYEQVn{qZ zz)cD0K_tu5IZt-$Mom5C@R{Lim6(sV%tPxokoI{%jnc)qIN}~VPQu}lKW8Sy6z5^@ z?y=lq;)^(vL*avLPPvdV5Ax1=wVKMlFmjx4;a4Tyk=k0dOjB#DT`Rd4B1s66iNmc? z)I1GBEe*{h=^k@vbnRGB0x2K8$7{bMd&O#jrE8{A{Ha7cNsU@PPfwgE8UI1q$qF|T z_e=RC`i7IVnPI4XY}dZ}tgGQ`L&omVXWW+f%3rpm;fk_aM|=wnf)mLZx~7MTT6*^P z<>N&C3)vdF7{>N1Qr1FFD%fzfYL^YGCEME_gKR;@mhhKMQ~W_x+MAm<1j9zrhhKbh zZn0Uuor0jwTFZp)uQrec_RnZ;eA|bBRwcxtdBHF3nx`iiGAXAa$HdiQmP3TIBF$%c z73(g|lQx@dc4QFGn2iySpxgU1hlS%Y(xHkKqd%Z$l%PR)o%Ja5rN4O5@`B$(!MJ^Y ztb+Hu;I5z`>b=U+Fu_y7@N@veWYR`WDgT}R z#U!Tl!bu_UmpSq9g^Sl`gR*pw3Vh#^k(Z0N_#pi%_})9SXq?47G=@fNmpWmdO<>oI zelbRdFUv3jd$3UWr&1q$u8x6{T+=FqTU71Nu6UB$eeA4qzT2vr_hYfO|M4Vif3>9t zx}IPDTSwg-T-WR;TD$*5jyObFl8`FjX{Fz1ab;bb+ry1V|=R}0Gf0c_(O-+Nx*dXLjd0C`w zCOwnzx0tSdou6;_^n&Mh7yv!29I(_Dg~)c)n@AN9^tzgAGNj>9qjZkp{Jj(!0dNfV z=Uk|B3aG2=e#xKb7enjG4s3`U)7B^LR3CDncNhnXsh?yG>RnxS3>g@s3I6|yP3xa3 zuM$a5gNbid?)k0CN&cHs9jhPTVQE|2f68^fd+)xLIjZkTU8}u}dX|1tHCT|V;JT?@ zV+~v;xL%+k65Sj+N(QRrcnkMM&$YgC(-9{VvjAtqgw=6HKuV&JVmrP;FjSjdV)C9ZoD{s68v)%0%|qq*yGy<{Sp@`cI+dG~E>x$@%>K+~{L)W*9Z0^Rm2>tE3*^wb zdkZ>~%!l>0A`@0X}7*V9u1VmrBN90JGhMk$;6cn@SI6f>=vWzaNmkx&zLeheZB z^qXrnrO70COS7&bw!mn~=y6IXABV5rXx`A)=i$2y4GnbHkcHE*Bw>E(tK03Q6g3!L z-yl7*O^E;eRa7=#(DH+onvJ07y};$y;o(WZ`+M;2&Cl{53AVpO8^5mH+7lRCMB_<( zT=idJ5x#ZHaK}MbIW0t;eb5t*lh55|atES8LY?`ZM?b;5LkW<&IwbQ8B4HxUp>@e&*YItxF zcq`a?F|}-b#SCZB06Ds2>!=SYJIZ9Z0NbiuL7h2x=(232q?ubbHA)U?^rCica`gj?5ac6heZkpq=afz>Fa+v z2aJVVdXi-i4C(b34&n%Ai+0MNHi05BtI(L!;6C+g-kb6f!=;HdfGq1!eo-HwpR+@F z-2<0t3w}Y1oyFpWPDxkDVeb$+al;diY-^KIxRekn&SCSw!;WuPJtSsO3%I<{(rWbk z*jqhD|3O10-Ls{-8y-1|EM)=4y&~QGf#o%bK7TCZGeBX+0#7vD|0*myz{|<65FtX! z7t@K9XnIwQ?kTmZZxQ$9jD+uw%GA>}sEtkwqRRzHkvt-Q>f{xJ#7s#8BU7-755 zsh$Cm@_bMm^=Nw0V3g>FeX%lD8L>;Ni9};#on-_i@0@b5wwR1~z*$5+1W`@adKLQZ zdsk}e!^S3l3I2gEKJXKJKY-%OFnzTBkgmUuAc@ErWp?6~|E5X-${96Rz)(i zU4n@e15SGvhC@a#PNE1UI4?{td4so$;@H?0;r|BUG+F|8(m4G!Ay8Iw=vqLIeC z@3eSSu=*}C$iywyEcYCD z9&7F>7Fro~8)JtS{H7{QUoz`NU;<}69f-znRN&uSLM0(XPbb04%a_a#p-bZ(>=b-P z#eDEo4R=i>I9OpNaJ^5hDgKTGV{m7pQ(|CNhvsrvP>Lkyv%}}zLITbTGU4H)(C2J!$qkp6*G(kpTfDtj|`V)=Du|PQm`n!D*5jb#Ng{;+~5Xaj}m&?t} zJ~7N-=2%HI6hP2*@M6AP<<=cSwhZZB725vkm+VD#Y6H$LbOa|RIMqVVsG0=Rk;n_; z1@hAs-YH1(x)3q}Ruh31J@=7zC6-ZXe3wl%e@+BnB&l4Z`nNEX(7=+8v1ygX!O$eM zQy!Xbd)C7S>TOg3Y@`s6?Dyo)ApypSmOq7%&1BQbUa(5wQ%z4ZQn{?Fz!2r)$E%S8 zCCV09n-aq0y?t7!ql9f^S*)Ty@%kwT6iTl0{-j-7&3Bg`R4{7zV?CU7pzfH z=x49ibGK{&0!zJ99zHfkYc!2;;_mvO9I;ye070ka#zY>P*@;8FMC5s6Or4OpljPC< zJ?H)7SF{j*y%lyRLz?^Z9_CQhn~sO~I55CO@q>J3dEcH7Xigf+5tTySZXi|#y>p=n z1F@Bw1m?Kh<(rajM&tNYG|8Owp`$d}`;iy5vw#l5DafAP`R@m5_Ml|8^2CnudLvZ! z4p6n1_}g#zOFoBuC3xBqZ(F`d%G0EY4_%OVFDpg4kUZy-xTI?b*SP;B>uZ=yG86dUkA1D2fz|FFU*Q_zQ3;~aL zkMfNf23B%KIx9|k$D&%_tivil(nwetmaEHCo#FhwwC|+jUS~lNvc~xcN@s^!JmJq6 z*xnrxG)MN&8}nl$Y77RK9;R&8wR?qMrG7gA?^W|4|*e@*0bw5n@=v2!rDmiFUGYQPc9dDnxXf*fcU~NI1 zbB~#uHjLcZG3{e1ZB>Tr1MfznUt~-!GL@)d14Ob_Y1g}{j6i7%J4MmKd6ZEmf&*k_ zQowOB5`f6s5L(uS$lup)nd;_bVu0ZqIORg!f|B2&8E8i?dVOqK%0lU39L^N)ZHuaB zniI-qE<{U-!$Fx+s9QCZq&rUY8{ygX>G{|O%|LB{CI~DF?V`!*2%thrNNMLx331>b zN01}AEgNeUv*AJJVaEZ|%74eL3gdspVEM>p&yo6ulls9mc-%#(Z))Wh3yQ)wGRi{t zd%A98LSbuUD;YgztSO}zN9ZOK%P5x(E{Cdm8|56qTS`sY_EF9}31y1G)cf${vke{k zq8w>ZSi!O@AcNHXDXu>A(uZJA(`plE(6uDdLr=|-LGfi&wSBgd3oxDP4 zmnkgAxi@;_bggDQ8)z@n?Re>(AF{rkM1E3+Cgrdd?dVt`H@^?8hLNx7XV|m2SMVr2 zEiyMg+mu>(o8p!$P;AT}C%gtF0wj%r2GO+!z!&u0)kksaS|zq>L2yY>b#uk-&okm1 zP>WSQ$!391m?)#F9tO`vj}%UWoUyQ%J#hh3y4h}4q#J($^N3~u>^iQ$5$Y6c!Fhp>{G5@o^mF6VKnHTxlF3N; z#`sjC&K*Ml)6DnoJop4|nLjF18Oq}k=Sa8TqQc{|I_wK$&A%lJ3MVkggvd5QXrror zZBfm%1sS@6E$e`59Z}zyVBlab48>%2S?#yt+Yp$5t75B%NNXt&2C(V5O_{%8&IKsr z6-4!z_iV(uWqvE~xre{HhToO}IFO^?F_8s2^ZLO^>xty~;7|^=Ra|fD$-py|d`vI~ zp0HhTm1msJ{#&P=m&9&D+w#xI^~;BWJ;x6ljE@iaFm%U)GZlbcnXsz{2f*FFj^Hju z__=g;Z*$bmlVGf?`Bv{!;@61j#`QqihCc*nwcpo%GVmc0h%`+?$lg~oG`uTT~I%jug4I?VnX>)I!!$L{!|uDUhUJRvV zdUFVC@`0S;x1zOf=5B*Vd?B@d*kv0Qe8atcpiiJ`YJCSTZ-BVB#8;OGWoaA4`HS+} zy%yIE6fiAX#;y<<*=o0HG%w2VxCSsVU0$QO@$rjsE6H`kqrl0 z!*&UB1nLN)PUI1oY3(bOD5aDCQ`^czw?#lV{x^*gYiVwJlj^?MC*<6z1TMBE1BXhy z4W{Hcp=ZiljqaPdJ_#qBN)KF+0W+Q!mbK}Y<=kns8>W;y75XDM=3aWAIsxn@FVqjo zUgZ4_zJ}~-2jPZWgKrKaHgP9KS7udvhamwp*l$`5DoHd(Km9N>x*uY`t2?8LJTXMb zsTB+*OeiRxe_28YqETa-5(~EiN%P>)u@i3e6ByXu#B1MNK$T@2ryCwA()oT0T(8@> zR3XKe#Lb|Y!kSC^J*J3v1 z;asj4Hn_GWi*EXMhB}BItNaxY4<(B_O@0Vs<|HwWq9^9PkaLg0Mek<7M&r~W)U&nL z?9ZWb>6ZOiSEdTSh=%^aS2(*fAQ_|+RnkDCIcKkv1K9Kz(IvASjCYWjkh}_ptKJRY zsbeBn(5xp5O6q$%CG7iKPKsVwzVFU0?R6~%{75a{Mq%Wp;*?iT9plD*CKK7s4SVW? z({xGwvS<8PUZq$xBO$Lk`>$o!$%OIKK#_5YEN+{2;`CXJL!aA{u$9=w*qj5Zm1Soh z?kF;&?*R9h3@;o*_N}C>GEXRq!pI&6ulEazun-warFpsrgs+wv$zBtBiBzEEcxozP z)gPG@cqTm*5yPAMw;9M&(=IcDI>|Hi92a!x%5mcOmHz}C{|O_4@?!GSeD6)c_ags) z+8-Aq7b_D6BWoiMx__EQzhR?=mAUhi%ddJ_R6|6==fU2;Utj)xvzaCc9DwAd!fHJ!-ttK&O)4J0?a&8C!r+N9%}IvW}!zPDj!G$6_yAM z#a}z8Ok-{OofPuVqU_twchK8c%KCLku7*ZOR4ybt3ZCJ3!K}Q@f8=4@pFw5xYFjhL z>|;Rs!SBUQHm#{gco%p+dECW3qHf6|){XWxu2Zo?2Y#J6>+L zpT%W#Q?oU}se|UJ39Gk`Vg{Xptw5-`qF=#Uw?fqOIdLZYIiYl`C*W*h~8~FaBANy_%gb#b}~AmtEvw z8C3Foka*(#V~Qo7m;O#rX~t?}Q)SNv9qYE&}_2%ka3q zfa_$s2O&F%<0NEw+Stdvs@~9Acw}pk%U!OtnY$g24KX90(Hn<^VS)%Ea4ifjyf!Bb zk|hjp0Ml@?PTsYhS>mo87h=RHwEG~`hc*nHM!`-KgU}HYQ_tvkDl=6wxfJVEG%Pt>BPzrG;|bThyqOk1x@wTt+Bf7RrC zo(OFFK7G7h&+5ZQD28IC$?s=HM|0FYm}^gR58g<{c-hZ%I zTZK~>WWGg9FgWLT_7BJmmM{krs3&t~e2*IaC8Civb$vb6tM5`7e<6^cVSm4B(q7r- z1_PDEXK5VHJvRroHj1Ye9S(BtW@*y)l#aNJx4ZVPZ}$qp)0Y9-@OrGl>BNpP$vxc| zQo;r)(|rOQoRePgW+@ZOlCRShoNqe@`2oBRKb|@aj<<3U@W#LCrWmRov@Je6Vq$LR zg)RSd#d!7EvLcjp<&kL`9!0^xo`QU3_rQ;SPtr3fpDY0j?A^yrhGOENvtfR%?}yt* zGA$+JNi_#?;3TpuAl+onIsx|HF~!JLJU!DOis ztjK02yg7W0i=)9EGu5yS;??*W1jO+|IiGPmzrm<64vKYBzcd911vu1|3Wdbxku-4gei;4x;Q@fzM=7ZI+hY<-M-ubZCIizvfbcmUh$oHgwN+S`2z!><_K zin?zsaWMplF!=nzNYDW&%HP%9-94||{G65LrCndma2h^m1a#sPgb#1;Q@i?4m^54@ z1kuF-iuW&!DQ;l`eV>`fLbO`Jh2w|-&|@k#{y1^2{_g5kAj40QfRb4BpBwi}H9CR9sFscLveU!^H5%^V!ZqFSO+_Hb^7aP824O1VTIe$b zET-e%2>_+jIRzKR>fz0q-mzOB$R5!Np<99eD<^M}wUt(P+kP~!-LW}^Lz(Pc&*GVFDWYHt+MsRYvze_oTrHD!JDhw%Afgk& zj5pqqBrsrJ-2(Xa3-@z8i_$t^>Vx2dwE0UqK51en?*MH0({W7^Rww5D1tqE4IGZY> z0*)laP2A17*#>l#?Uf}8LrE>8bmfWBZHE?mTDwlvnkBJ=vI2cq;!Nr78H8|x& zYLklX_z1>jig}#)GZ}C}xMTiUdbwXgb5z*;t7U(a7<;x?tH3Jhg6w2yd!MMat`KK~ z5?F(_N-=cKp>PEhAv4B>1j?A(Yy^CY{E*m(5RCNTu2Q7sEZ%zTpL5+)275x72C9Ob z2q}}T!DM;7JMBMOl8g{wNju~zij=eu_MR|=W{cg4g(%8Xgs`{lcPGrP%^rh{In11JK(t0ZjRfegd8wl?F1OA@ec@|&~7Sq{r zdqODbO@%C257CQAtV}mNSpt*DX;o%@H|*4P#HFG+UE5p_)uwVNCH%y4$vCjMwT9M6 z*%B+9QJ6g?ZM(aAUbPu^>o827s~C+6S=&fTaZ!$3Ec0@S83E^xo9ENnU+82Jfx*Q5 zGC5(HtesltJqv$+t;-A!VpZcb)@R0YsmejHjG+v3uA8H}l>HnIp3()oLp)D~EZ{D% zoFs$gn<~-*`{eu{i`lP>1-_(sC>R-bk?^pG0K$I+Y?!Mys%mZx`$1;oE-aLKH?KOs zma|KS?||mO^n(pa`xm-13D#*V=O5e_dxx-PS#LwkiqjBP&G(!;zx2%~jLdp;%c(-f4wz=dWOh<03zXvp zOU|WtC6U*g7z!$mq^Q@?=|WiuPOw=UGi< z{kZLC!i!ze{d_YSIZ*22BMuVZESd>YJ&Nx{CO#OMV`-pU^jn&6$d2N@6gYpcce9<@ z>w&xkq(~y0^&&ADZXkf-WjM`bT4*`bmUOS^_DHE_`{L4()lyS}I`BnYEgmZ+_EM0c z>pV1FE3zg(A$ctaDf&Y(KyHKcz4c|K(_0c9@yV-=u&T?daS7K)@_7R$-E;(H$C2=b z2ClJtX`(8HaD^(AWoDUz(siyvM3Y*Z@_2sh62xGbSAe%R;3G6S6HRB?VCR60o7PVs z0s0IZX@m)*4JlsU8ZuGU#C$ z+eD|9{d70taNZnk1iC+MxxkUJ_^esz_fxMrw=>IGN0!ojPjIy@CQvYbGUGohb27M8 zs+f)kPj$vyF&*k|-{xItlS;$|MdBx024ehiFU5LxkRH@~Y z6R4Lz)7Torc{#~BW}qX3)k!5%_F*35*{yHajcH9QSqR}bdIe{LYLmJ>xA^XQ-LuU# zxNWyZKF}vo0O6*5_3aGmMbR-s{%ebEQ^NZR{=YEEKR)lO>(!qIH~^sfdr|$rJ0T|a zc6Q$sFLcIERu+~n^bS8PfU4nCZX*@tCE%d3p})TfCnYJW{QdqNyaOOYf`7jf+Ve28Z_!=>-F!Ngcj_;xAJo z=MNGfz_TAOFAZ@oL^lA)V*v&d&?}eK%ZMBPh44o>Ll-C@1gO^~fl~%D02LtU6)Ri{ z5EKUr$Vn$x0px=M3@1#C4*-gE07DA@;{`xK&U1!05I`rH2n{Gd0YHdi79k3-d9?eua2m%0mE`D>Lv<$zn19-p# z>^=+0u0SpIVe(y{qRkr-1e*c*XX_ew4*y&mv4Z%vwY8(e!&T`)VI8A!4c||TeuG|( z58D?%-uI`6-R^zTKxVx_3DD=={>gjU65N?Ys1W1L<2Z@;-!NbAcnidXQpRnX)Tjqa z$d0k1nK5_FCB%`!iBv;m^IxX>JG5^g{5x!bdTV|l?*zJXJrj^e845&Am)`v+0N}R6 zzHgBd5+uMfcyGr2>ssJLBA*NpU@n#D2mt7dlG3Y={}LGa_M%1e1F7qUaPIpsX?wwO z`@lB)VBQVcLxf2O`-KsOp-lsDoeZg}gM=x=rt0yi4H*^%@mPD6EW?r<;OYC-TH*N| z5MGTSbNeVe1ED~LMj=s*h}I*3jH5J3Vj(dyLZ6BHBtem*NeRayP?d;f64)e}l%rG$ z)g}e%t7yAof1sRfsIOhi?GciZRH%;A>xHFv!*UgnQ`d(S+h<} z*nBbU1t+srmI&){{30Way{DKEEQ7X;^uq(f4V<)4NhT8w#MQ7>!`Sue)gT>_4<<7W z?x4tlV!cr8pzgwiM)*RKnzEYmtCHn}xTLH|>rfm}AR$Wq6vRp8GEL;|NH2q!#>`CN znv%36%gAvk42a$lB7+>H$dMw<6m9XqQVqqb^T_k!^FNeH&S)#LzPay6lE&6eZ5koC zl4%KPMneyC4hIfd4si~NZgrq@j0GKwpH=oLmB+26v5pXrSdYNX(wT+SiZZ{OCJI!M z*y6N>YYJMF>y*OFQCM9wxu$~ArDRHK3R4!aEio*gtm;l(+L5+W@a5BHHRdPIBu>*# zLGImQ;lj{@MTc~}|62_rZ zq*KUKaHttnD=JGXw<}@P?$jn!<0|x3d~3zkSX2%wpcYfB)GBin&sA_#EGl@)!z(Sy z?NtSe{j@$J!L*N*Jm*GS8!|t{0K>}72{1+@K_y+7WChriNqdhMk z!^p;sl8D}bph$^Pic#8LNJ85jZ_$$YPjQ48t6|*((gV9;yWt{oaB?DY8M!}lSDDIk zZKYJDJhD8pBU6)8g;UEJY*~CvE6gWZCs`|5JHHLJJ+x_C%3E5^q#6&kOSP_AI)8*} zb7_@ms%h)}60garAzSXK=_o@gYm-lsugYDv7^uH#8EdKdja7fN3cgX>V%OqpWMPb9 z@q4It;3}yr$+zA2=La7IT=*J%6`nSppL3HFDz`fKcII^Um4NTied-y5h2{+6%qq)4 z3(5r5pjD1a)`EluhDDom@Lfc7q)gBReHuS~7S8b2XxE{%%Y&odp4a?CPS&ri{>$=> zx-0f8tE=$Myh+wPs$4A==h=Z#kzJ*uIRTiw6h@n!YCi=Lg+Pnu6A}g!h6Ih8mHOW$ z?v+nQ*BJbd{7!iX{0F=o+sxZq29q4Uzej&hC=A%=8iekM_>+8@etCV(0m=f%fGj{F z1BC*0fZv|Ie298t1-LiN9vjy#g<|sCgwTZIf@*?}`f7+QM=#=Qh@MW;rxT{F;6oMt zPM2pPjE;?>2mD|uLc>Ef!USR+A~dn*n9|tX=Qc&0cSK)BQ$@G4ESMCy>gc4|sQD}D zt$Eiu+F9ELhUv|N$88e9t2e zEkm!MN2cXY6_kyT=S==9gh*YE9z%AE;!YL|?b(mlZ$Zp3$37RaDxyn`FMlgZE$5YZ zkL8Al1MJ8#o~{}vANqs zwOMH6=g^Pn-wgOE5MwsK;;X^5=%8DKOHWm`DOac=8ZFE;%zLIH$GK9ga%rY6Cx4A^ z#B*d{=Y3rw_apBmwv>d)Xwczl9(Pq+b2aihdTu};pn=hZse@PZwnnv3+btB2t6MMl-4ZeXOL`L9xL zdK}?|+mH8TY$i8VR1LP7&X~`epB-=05RX)EgCZS6EG8=Icu5l}5wy;przWxtx=YBpMV{xNrj8%mm(EwF=hEj2(r(j|dhC2kKK}HcW7Ta8_YdQD zk9n{#>EElrUG}+ey5n6J?I0)|`Vjh4ep$XWJgCf;y(M8KC4GT{q~E=pu+VLMdzj$H zG7_SIuYVr-U1dq%N1z-ewVVL}7?gh&5Fj%f^ZOu#iR zO{Z&Nu`>hvv;%SxuV?QwH#c9c!_v^gTGm&_-jV!PaeMk)x5Dpbn$SZC(CZ?J!RsKn z;#~WD-y2cbYHx7yX8Y|bHKi#}B*iDFj%?nITZ@~==eE+j{mBIW)*kN*V#d$2I`+6? z?sjqZoV z`L>#7vRmq`RAnnSM??%B;?UMpMheRS#6YmPXT_PUN>o_&Mnz>K-{j)%WK;AS8fqoj z(87J<2s~GiU|JPcU`R|vSx4(_r?JAJoe$wF5jE0wrdYT#V#mXXX!w+NAewIcp}-1? zaxC;({hB3n*Vp~i#plI8F*KZ_Ih-feh$00@&~l2_(1UWZoamxC6_st8hx0DCAEW1X zFzxqeCe7nM&Gz^EY(D0c5n{2ULMAAp357vK(6ygF2PjM99N|Aau>}o2#wiKTFmZ7V z{3@BXn%hFL6O28fJ>eh`#UO!fT6CwgdzjrGJ7AuGW< zo=XpP5Nvi8AjPSYCsn5&R%x+i5%8*fdi?1MhrdzS^nU)`+;(!=RuG6}uYg~ZeRcYA zwrAIM#IyGB`N}4A4)vL`p`Ur-^*AteCb3hQU3%xSu=9K7rl$#ybEom4w~anZ0vce; zeUtYp0HC^3C95K;k@yyVfRvcLXsxhe(0|2!6wqw$U*DMT^?Ol%pY^{L)BH#9`TxRw zCbss@X8(eHuW(@hhJF8regB4i|Au}4hJF8regB4i|Au}4hJF8reg8j(eaI*nMEhBz zTL1tE?SH^NvqS^Q99PT{fthRH9p~6wmMPC$go%`v(r7hiIX5MwvE?Z7MX_r7sJSV0 zR4kl{nrO@kZE-eZe?g^5;!N%w=CZ6z{8Ie5?&~-2ydUuNf1DqV z0#Dv}4W6MruLSP@2Yc@jEK1aEYi`@NeYS1ewr$(Cb+&EWwr$(Ccb!{^s(5kVt5!u+ z<8Q21HnK4qIlf$Tjsf0n^9mHx@O{?mrizk&!S7s%uc+^ntqF5IG^bZc46VoWuT=?x ze|z#925%S4%}@7H@v;WFHHDPbOo(QtnnH^A$Qv4gBzXMRtJa6x#oOxhrbU1gewk3N z8Q$SRp9a=Wnw51GUAwG<~*1fjcp#w=O^V_-N1h{AT?wEWXz1YVEMF{lchp<>Bidl(6;hh^` z84rDj*JO!H=}ox~`M4lF-LZUHurQS2;2|@vMSjI6Ith%!0KjM{08`)L%q1Ts`D-n{ z>EzGPvqR&kT2T{oNkurKy3F>#(#m!6vu>X81LMD>9`67iMZb$G534ZwmoE*lO)E~m z@3T%DB_rtV?bFl0)fJ}b$&=`RgL*$lQl;p=Esjlpeez_%bEKa-}Q8rO< zm(dsO{W1PEt8>uP6M62*;XJqc?*0A*dz(<#aNvkatlu^E2C$lW(rL{Nnia>o7198>*B{5wg>~W5PIp|M0#bioepQsj9iVF1LM^*&uefS?Bne7oYw6 z4b1}y!m)h-@csny!C?(z$w|z=rxVa|aa){Z{?L9i68%7)0!>`q@gv4E5Apm7=DE(2sNC#lQWU3$gel^G#2k~ENF`R znCuUMBI_z{I*?6t#<-?Daidcmu85hC7!T9<0%KW9F1QD_g2oVGY%E}46+$OG)F2EI z0+}exm@p+yCaq5<9STVz;!e#P<-Ovor(K@iE1#7BLy`{hEhDZkkyg%=i;~A*R8#?pjntJ%GW!|(Scvlh5cjTaIRyWPgH zeBRs8qc}XpR8-O3%k7#J0#O+0<2mlQU9-C$*MDE=`giB+iR9T0&-knF>24*Ua!0(x zM(x%149wY}nIaNLhTE5mqxWe)M@NY6Ua#%&IG5{n{BSw0vh?II3VE3rIoId2Z-4yO z4*yr*Q>k=+qxD@6hnJJf{i-vmlrOg&&*O2sJm%Ni=Ooeh)6Kw!hAyXu4KIhMhyLSA z>6PWIbM0F3v=<pB#sJ*t{hSm6JUY|Y1#qKY=!SmW5R31+%xGANA0?dryB6XB>G_AIY zthCZj&fm@b_V)ujLtS3=kqaIJ(q4i&2rxM{zP~QAkTU;|?VJA^d^O~>&}oDO0NBF( zU$X1JWC3S8J8LIeOQ&fJ9VKjzhituk_wp%rcB3Ta?Hd6C5uBD(8Xz3ko0drVWNzZk zzZvB|0?TcIZXFZz)HIk(A1jg0z>XF%BX1}`#u0q3^ zVc_@dokfKTcLQ+?_Ze8T4C!Jb=Gg@CcKreQ!D4al16`^pv0*c(MzPT4#4b!2U*B)j zy3bT;yomJ1$Q}dddEyKP$0v`k&G%!9dS=HQ#{6}>clwXBSUI`SQL{oLFSikxr$?kWcBY?OIBj zGu%e&5=JBA0mt^>;{pH)at>lC+Y{r0BSm15Pg1EMcVz_HCcxvmXz6ytBu7z4xFJUE znY4`qC8S<;$Aoc{M9F;8xz9(rhk;t;xMkAa7KrxJNQe%AZ2b62Sz5DfymNKMwmn0t zdpC)cKE0A;<+eO53?{VHK`;a&*Y~>&`RHLRVpciatskA)p^R~ ztcR_E57I!r+rjflb?~E2WZS|QBb1QA6@7f)9ggzEtcdjV+oWvVo#v%Ylp5+K)ZQMj zAqz@H=cYERao%Q`UjDGG#xDJeP!iEZ#beAy)g@5}hDEgN7kpBsI0+|=ct86JwW(xD z1h=2}2V;nnzd+tl07%1;ILb(`y;o0WJV*%;S*I4iVQU5lTvu;{zz1mi-LTL1S(l*| zM#@@lG-V5W9{do0^SFcHs7l?#(=5uZ)hS+@jWb=AAEQz$4GR1eIl3YMjI_|H#X%hWhLe1196)ZijEj+4IuUg~6DaC{|5PmNO z2L@TZzvCo&*&=h3GWpI1y8#3FN{nBt9i@+r)0CKqe@Tf%0mb#Fx77tG@=pXe#ke}B zFLA~drkuo&S(li5%qf#RcN@Lv3;r_PZ99dV`JY}XhDpYtLCYJMFe~HvlE2x(=!W6F zws2NY1FSB|{|bL!fuDv5%K?`SjVZMoTZ*$pbv6$Y>TN>-(g_C(*;70`kb3NcI7#~- z`qR$D_ZrJSBZx#`2Vk{q4XXwSP!~li48~@Q9>U?dT|`BI%2`_atJkXO_sL5e4|2>l zx`+H*+ISbzs}O`9I!8oV5EE&|*6tuervRN?1P99>Htec?^@-0w&`5d2R3&KA>Usi# znhCY%3{-^cX&W)O(X*46bDa|+UhR|9n)Zt(wpyBt*Z0D8BP(W5eDbzBPzF(4kkuEz zU5PGnIt8M%#2p684ppf=X}mWAF3#H@ug%$Th@y}Ez^aE0&e=$@HvXeA2S#hGD%f4N z@TCdAwkz2Bg<)$%yFE}vOK|hLBGLdOnECRjiC5{}o@Ak28Gl0K>bU-trII-F=$BNb zL<>Ot8)VQ%I$z95Opd2(HBzuH^!kUWqQTVd6D zPS!5=x2+9^1Fg@oST(@H@NZWb#tw1k=g>iR4t$(!oB=S^N_M;yA_v?D$ORr2zUEML z_yutiQD8N*iKZ51cI8r)+?Ie(8~*c-N_+x1_#-TcpLGjVotiuK#Ur0zZ7 zNK+%bdI@O)`4eRUl+)q@j-sqyl1B2!)#BLf-8^bkhP_3GjQuGVGB3YvRgw?9QGnn{dZDU>K_bPyTJqHA3qdiYb4D zz%+e*sjYKPSb&u|LR>X=Y{g|)9Fwn~NVC>-3X(H)C+diDk66 zJPllHB<71DM$7g>2$H$|+@(i6vjpT02JjAyPg2g$nvj_lmt-_=?UA%~+uJ+zQ^WmN zK)YzPt{5_qqerbB;Fw#plHYWmrMn@UwFA^qO%sAO;-!gXZOFFGasH!5ks;DXkefIEF_8*56>{L%0cYOGQp4*C03sYdx3fVJ_(dy+yGMdpG2#^2qN9 zE`wlB#mxgdGc3W-6Ze=foD2|0>o6H2tDtP{E!!15r)F^LJ@5e7-V9P-a7F3Nr{~*b zMjPqk#b~2s;X}umYgSbTqwmtmQ7rUutd*QUxkA{ODMgK|WinA~&s{^+`Ea!5@p`}A zR-gW$=*6;7SGrhkT8C9)KuZCijy{BMt6*tWJOp$S5E+dxjA56m?4!Vg=LC{y+Ay}?(9Q)V$b4vnT-z|ph3f6?T>6t^T zgI*$8ALkfKG}-3qe?ynG56@307vr^t>kK+auOl6HacxvN`nI)ma^vR1eVoVq`@nn& zL^}UoeVoq<5c{!c0WdqC{+Vii>`Axtb{zy(;3Eu{vvXH8d-m;JXpm~%MW++qOiQv+ z>7xZ4q?QPLbDVVwp*~1q1=kR2d^>GFQgt*J|4hmBYUV1jIpPn(0~Q1-af5{3h5Nj@ zt~&+Lkxy8iTj+dc0=#kEHqASozj9!iZboGITaZO@ z`2S3N{x8dC>coGb2>>+BvEcxOFV`ej(ssVtmOL6OErpP}II6l~# zuqd9_=LZ;u4j)&=A-*%QKvn3xyAKNrb1Sau^ozm+vH3p3#NW{Qersej1{ZOQ7L`Vj z>3Oap3A<}-w6r&_eF}PwnQU%oy?l?dR_A|PvjucNH<4MCUfcYV*{hbPcuxWKdiO@N zq4ns4Z`myin{1mm$#-7xZM)Ci@ES9<+FCavAkiL}TXBu=U$qtJ9b8IP?vH9?)f#y- zI(Drqozoj@IAl8;XqGNa1iC6d9l`%*3~4ykPSiZ-cD1z~r;U70g!FJq#;IocWO5b0 zN&f4Bc^&qYAW(r^G#)hN?&j$Ic`2dHyunHGb}|^vUXs~?kGn5cpr`Y1xwq7}LA6XyZJT+42wxR6dD zafX(JOv%B>^cclVNjN^3Ok_HX&D1>{cC9zMe!6CZ_Fcndf*JLxQM((}Ew>`!8(*dM z$n0sO38}~05OL|a?wyrfa@r{M9w&oSv*>C%F}YcT@{Cqj%5=ZUx?yt+E@bQUN_P%g zT{I1qS7KyId3eRb--_efm zfh`?eOdLJ_k#JYHm)oXj1Kr23&ZRFn;$by_X;fmCaV}4uKHjJ+S^Bt@;VcPey{HR#( zPiW^nw;sODw2ZEHZq0&6V#@gZ;^Ob?@BLpF*4hQ`M`RUH_>-G6bUsKt+rxvh$etXs z#xnyp+QbUlhQdt+q*=Q!V`W=ai{oAJiE<)nyb24}C@`~5eibgQZFKk2$UQ=x4-piR z<$DLli3gE~7vH40?{?-?(H)|i17y3jm`LNs$kT}5><8JLEU*5uiQb9Z!~+v;GGBVy z0%PZFWx9$A@!AWj`$c`}6g6eN9_|9xuEZVVRlMi2Ha`j?=1UckZ1<#ojWM@Lca?(| zZQDlA<_EGjY`CcW)A>e2M(P2zLMqBXy%1!rg)-X7<9fT^pRn7Z!?t*8%;+X}b)C5R zr=Dq@!Bi!(%3M4aB21{c>*=_3j{~)_3s)5T5#@7`O9#04yRi!o&z?%#eqPU3jJ03J zn$v;$Ic6#u%aC!-{`W^`QRw1KK~CFvGQBo*%QdX!Y)QSfJ@gvgUYI%c(3*{1UQp%eHouD`1pyQwopJc!sl< zK<>w>$`cE-ET_;ag^xd2>d0R_L%Vgs+o%heQbbwyC3!3k4~THl2Mz51;Kv*c2WE_; zOh`v*)Je`((FS<=Y^ zda_QSKJ--S?vgmUO$3cGuc<|H5_xpMuC<&pCQ{oX&`T#T$@dO=z9Vbuhu}UzT`kOX zFU)Pz*|2>tDHtxsjJ(gM{g%_|_WGlj`be~`&x|qk8QB>EeS=E-#Dmr#zVdF#@b0|g ze2adgYj$j2CE<2{706-q*%<_Vbj5O*Y%_Zad-poa@AMT!9}BimNJD5&{>v@z+_L!d zT`A!45iTVsYj6MEoKe!4f7RiN%=6Tz$BBV3sw-&EM`s~MX_tVc8++UBPl*!f8|Q#V z5x_n@L5o(@SX2H%jt&lxEwYGd(8MHY9C6MbKi}3hHcGmm11|Wr=olIc-=LteU5gZU zlE;PyXgV>e2EEClBJ%|Zg@b>2EBcXr%~~z@ni5uM+RKhwEHPeJ5Z6@R5KYBAMwaI% zI8aebg<|zl%%uX8VT)5%1MpHbp7;LV_nsHlq#8D|`8fjJ0bdE}*|zPeOR;eo% zQ6NqcefwvlaXf2=-KhXAsrK-~v?+G?)2t#bXIAzrx~!A{<74;EU3 zoJ=37H_ctmqh1{90!~zt;21PXPOwDgc0t7DP-bS3i9f+#*7JUx9anS~NL2W;txcAx z%7r5u2F7fVd=0^p|65^r=rpm!5rrort?I`+1^N07D(IQj`759-Q7#A|NMSv1TX@#3 z@*#yU`(O-Ot_7c4;u(r$ekz<8Q35N*5Ogy5=a-T0x{EC(27p z)tzA`tGwQ7A`d=MAm6bsy`+N+aG)2(lY#Weu=@dYyodGkJNqi7yV$yW@#-eFu6Llz ztZy|!V_okYcr+npG5LO5|^1i3kSV|vj+ zC1A%Vm09cOR38N!_`B#j6&ZKb)A#{$$3z-DOpIZsrqq82{vex7@UjGTp$p(5wg30g;h%BOg2Sbc!2Fr7aQVkJn2tMcM?5c z6L<^NbS_gOn^VHsQqjMZZHF5%DifumbjM~n^gMs?l%4ILvGjG&e;2_C7Zy=QJi`vv zHf2Avuh}SaBDN?$zltr~%G%c1VTW$JJp3TSTXptERp705ILQ6+=HO=0(lGc%QxM>K zi|i0wg29ECtB-bGLuLAD+(s(q2VV>pZ+sxrQEeh+bMZKwE=$m1+H*@6|5t0#FNB%G z{%YS7xJVIUQ+R*0N#l-^bsWTmXCX7#H(rs-T%dv?XUIdgf%XV7b)P9c^Y?f-vV*HM zTLJu#6t*Jv;5@M-Ks4ee-lCe=O%q%hhd6gh0rf-HP1KWlsA^uDGrP9@5zXduCA@L0 zl$k-8_pN&131(R3x~^XH%|tEX&gIMM>qkZrvEE)MHnpAQvN0VE#O( z4pT0qIDzk;*Y*^;g!J1iTiG{=)G6eJ6!4}y&YaJbF!m4BT#3M4T~m}A*x{c6$y--t zq-+7bAnpNDyGd}ZbO;=9DH@)w;xQ{2Q%wkD>>j1YjY<276c8Yb6%^$G78MZ7zkgin8a zaiy>+4OwMf{bHz_y+HSjW`;3pOIY4ai;b8LdK_9aqbU#0c$VSlWk(x#TA2l+^eN zd$3jBIq#b6)x(D4xYdVodVbF$DmFcG;qbJ7wuJ&IP6z3{_`C(^?2_=C`_?`=v(Y`$ z>HgH*UvJ)a<+V1Dr=x=4BQC$EvfAmY#pn{azs1CsXA^y7(%MEGc0A%vJ*olU@qo9G zx_2u0pAE4jYA={Qauz3y$=L)_ z`RRL!W6r$Aqr7dR2Ue}_C@HQPVL!Y~>qJztdcFHI?y^7`smw%O1N^fEy6MD>oVnSY zv}D@)y8`1uJOq3`$L;~IbAS21ik;XOTEQ(M=j2e4cC4n3N!=nHlCPoJ0xbXpc7{R9 zO_e<`muI2kr?Me`jf8)d+$}%y7%1~4YgG$qdMU4$nU&aQewFWIubOixfS1hI` z^4LLwe#V%z9QY5!@mr`EDB{oqPd&n)g6Owm?*nU zyOvFkY*yRh!TgT%{#~EhjJR;swv?I7nhTk(?G7J)C4jk>jDM~QvNm>@h)ieFCQZP& zo~*um*fC{SWHF{&b^17N=2@;0xM7=Ft5}n_T#t^B&Hvmi6rWw z@-1h^rl8$o8oRzs>Ct!*Lic!n9JS|VVEXKpTTkp%|HekFyt!mG!7YgvP2z+DD*Yk! zt=f$bAS`jy^x6L-VnW?zDDcXgny# z(5)P%Y&;dXlms?$b(Ui+Mz;@~FuW$f-z)xq=~|ozvxBkfnP+X(NcS!bEDfgyV6klp zIEyxhvz`!YX9SU_7A|teH1O1t;aJYb{UO9;1N%gZJmF1=bs=6q+!jkd7LKbJ`eW$( zd)})QZi%8m>Vo}Avmtdrzv;uR=PugEpb7)y;#a}yym!GXX|t?mVD-S^A7{L#Gb!@! zuN*F<9>3R#vdH^9S!yajp~Z~OLs#9M5tDYej51t1i}S=H+qF^7)wY>I{mV%P>k%O1 zS;uhp&e^hba-oe8e2{eLa&%&wdyWi6nHCS8WOir1*jcd?U9{%RtMli_nMy8le8+dn zIwGZfa&(iEACLc%CVk(3hS#mzuDAF&x>enjVN!m+N~6}5ehoc6+};PR-?-C$zPw+a z#$V)kbF!v#z~lmzo-?5I-u}7`?lpLOdT@Hk>gj6d=;rY4g2~A)W7|5GgqQVDpI<6H z^Kupw@!1VlL(#*1SYpAHueUBuN;O^T=<4R^(liE^ZF+jPhs&kX$@1p%csD?ot+CCQ zvwdTlL{3`2ENu>4SWJ${YT-<5BB$qwk)LUH7sX~*O(rQl({l3Mx2Nx#J&~#CJ=a9& z!Sd7=GHsWm-}L7URPD1|VfRQ%5n(179+YBPHRFBmRND3W{`;cUKDZ}Pb3X%VM*}5B zp}+bf1ea^~02b14lI9~iD3&zYQ5}f4tL{!R(N1u`f0519AXm&*TbK5Pq=WHC)^&yS zp9RQCK4+zMu}|cEvK(d^+jKL}lNwYcifhgFBL!8G zDkjU4KWiR&Z^dKu`m{zaN>4K8L7k*WR94Vai;v*%M50E{I#WOM#VL-0?KghXMdXle z5Y7p^-K1sT%{?e|5G-U*Vw{AF3lI&KH+(PmQ5|>out=L+OWL&NtuCw`n#6}hLY{zR ztP|141ec%z8#8M@+MePdrg8=C2no^nz;a1%7o`%qbr{g8RCY$RA5cc`lCCI5OB%4H z%?i#cE!@u;H7d$!Ncg1xiX0LRO=$?Bjh zh>s6r@?~(^z$RhJoiL|gqr}n=KtH$I zMFJMv8M2%-J`hZMkZXSBCGLLwH4=~Jj(IuKP8BX%2aA5rmJFq~m5gfO1i>G-3vPF{ zMCH>w0(rKt4ue_`ArocVu~vO$^dwrsxJ6*PzJCR20kWIUoUirBc+YtNxV>~=WZRR) zUi*V>kxz5l^b!TM7@SRIpU51yexScfV{g;?sSfbr9&l)1E@{{{aBdx}M+#)$b6lC- z5FGDhzn{*Sh}pxmirdq#u!rbiUK-{?2G1pUbt+7>F)EtZ5ear`B=+f=a#~4Z+8M2) z+)BmChRS30vXkb28(yu`m z2{=sbnU#cmI*8CXkb-@wgdnv+ZKMaB->ZDuzvc_^&0x7InXmdhJ1gBRBDIlGaehr5 z&m*|B_{{QT)!LMCGV|I7N2K$3ys)Dm2_yePEZ>5IfEeX*yZ?vdE?8>8ZSxv`8KDrv zuSTPj_lj#yuo>iq#Z4Pd&s%6uujnyH4(y5XXT)vFF46x5JqBhC$mZ5vir4iCN(&6< z_xedxNe@tZ;t5O?9!LFF5U2~{N*p{U1=L+bh^8QT5LC-8lp$GLnvbh20byhWAt2{* z3p$JK1{_vV8P8>T%e(U_BeltL%FuCI9X)1foDe$ia>v$&94ZubRI88$)T;*1)qdqu zuo7dtR^4G-`o^m3Fr*>biBHDpqHvuVM##vT$d;yNH87}d7$&65;-y_YMNhaERt!P9 zM2DDQ4IPY0U27+{=EAwTZU#`;Ghb+7&3PsMbP-cU@{y%iLZ3+V%ot9}*0W`47cAdN zCK^Tsnkd^`fyvRmqscy(wQsAns=f=#CCG<-h?g5d73LX$>p-n7q0eysB%St0^wU(` zt*CY4b2pfB>Y8mLlcV((b>11Q!kapmXO(*!V%|w$54iNc;&bQkx=oiPx)?XFZ#DQ= zVdp{&j9$}?>(+X{!{UmB=#Yrt=!Eq_Bk|KHuRLhIWFSU3$WkV^lC`_ULjG<`CV`{f$4?x_$)ygj_-R_bG8u^05hYk1rI}OM)zd_Y!xQm)TSMsN76>;Pk za3&y=6tV8`rNB|CDKE%^(aLS)MsD0wsT4@{S_3I)3)feVS_sd-!fy=g(1rOr|_Bc z_{mH}ro{}eNh90RfP}%&oOEapF>V@kNyiTjyj)SUUHT7W4rd8xg5=tA=Dn@`lW)es~dL*mkpE0W(mgG{H z&{;=4s-%*h5TgSMbsMBg{y^1{%U)K=)TQDV*2y^p{lg_0d(k z9K5E>WQd$-l@?15joNFW8ly5{AdW|gb&L{VTtRW|PmeJ%=vbt->K3z8+*pk~_` zT|HJ}zwtAWy%f0MCn+q(Z(26h+2MJZ9GI9u?Cu|#*5v#5XwpZ>_YTScsK>Eu#Qp{G zZ@QK`FUadzS>}2cDMI(ZFQwH4PGfH3YG3fo=H8$9TrnBNmeNEp+vd-oAP_H~2d-fh zyzT71_JhA^S60k}n@_qnA*A@B3Ct)sQxosur;6$b1QPYMIJa9uV=XjG7mr+6$bt12 zv>$M+Fbg?pL)X_#@uuF(NQUj(v}@->`Yk6)SBLvu}s9$iXzxRJCb zwG&%_SnXCEc4kpJohpc(Jli6&JW&k_W3L!KX=D+ibh!4z`c$xsQjP<0x2WXov+d z@D36(E6+aV@Ue3(WuE5hg*v!%q^YjRspzI^pg&})@E4C}D6NQr#prQG$TKo8ce|OX zyOaj<1s``MnS{qgAl1VZg@P2Vky7~5uZ%{bqgb;Ubi+@!(dLfOv=>}v3Nt9aN$Qb9 z_Nhux{YcpTn8FgZq@AvD69QrFtDaH{SSmH8+1?t?$Slz5`COx}5i55a?w2c3Hz6o6 zW6@CgmdPL?w#OjLorJle_$&?slrbL1Cs(`Ix0eG6YTb!iU*$9W*Y!fi5m4>R1Pz-M|6WFMxFyGg5hqBrG8%Rb z(CHWornm;ircb61Tr8js;+A=%ZTKp|s}_HdPv6*6gRqie z`PZFuY$~rYfl=YFqO*+ni<3+_@2mBYZ-W{UA16gTgf z7%T>Wm8n7vt{;;g30PN_G4}~U%gtR81?*fM3(hJ^mU%b;!zJy^x4sMN$sYKlJt4mc zaL+GU_2sD+t=6^7G3%oTXjX|;@rSrTm5OqxQV~DXI1mc0gx+^LQmgv(5e@nDR3YYT=fvg0PKg&kas&(CYif6c49$$1 zr*t42IFeq9mFAr1?;aNcN6EH2#}5N5l|0cNPAM0Am&YNUaPnExnw3idKIm*oX={8aP<30&M%%A ziQ!g*W}22A=F;{}H!8C3(6o&JTz8_eLn@3D>pkWAw;jWf=rM^OFu*@(%`-MBecr?G zJPiSn0vF@`Rf=Vu2|2?TVG(zTse?CiXM@KKax$`9UfAPG@R6BVLozHv&o!x)VMX{3 z%-9#xMOdnDnBU*MMRzfo`Q>u%dFJ>5;@I8_5h=h_aDQE|D_r|D2JxT^|A=W+bMYFEMJn zsCdoJl6&dEK*#PX$Vsk#1HcB8lb^e4wJ_6KM>QjJ8JY6rQ^mmjW|L*N z6Sr>43Bp9b!kV^!rW7}G23~OF5cp-Eq$(j%n`qBu2n~9ZVCDQ?Uy+vx10KdXP+N_n z*htlqex{_=IF;&)PLaX70;Xt|oqi-133#$eUFE`#4FS!#v?*LUZbN~ccHQORDdc1s zkan2lK{dVxn4Phq<-Bohm4Tom$b#rQ&|RP*jV+oiDi1C2xD^a3D1}alnlIDOmqQf( zn=w%>kL6))Y4p--f;Kk_|3ps4~giC^yq=HDU#s z9z>)uqhp$b@%ruh{?=qgPuJlFnQ_OWh2}T=GNwGPD-X)ORdk!Q?B|!H? zJZ~kedr*^h#^z$yBYjt}kmdWqq`9H?B4De>h+aE=&ITi%(WyI!$Y?nY%wWw0^jXuk z|K@%u31c6Hti1b{S;KB1TlB~!x7}vG92+T>Pq;Vy(q~;PLrSs40Kn4B6uS<;LNF$R zSqFZ$g8MavHz_=VL^RG5TO9lca`MRfZw}q_(&Fd|)!Lt=_V8ViR2JFECw+ca)K*D5 z(Yo>jE9ZpGLr9XJ24MJVuFn+YqlqP}eTm)(t$MtRZrZw~%I2&4IDe$8+pk zEz)(1|LnPLXI-bjQ82Osii}_fk@Qg&KQhRQF=Zk9C@Y#pbWtP~d{E~P15V;|fgARH z>;vmKp?3n%T+LC2a+gDW_yIDS^rNR}XCJha>2SM#@z{?TM)`l;%nXK6!*@y{YTv!u zGUJ}nZE0qTD?`Buzm_inANT6-)43n?9z#u<`+9Hf@>}tC`51=OeM*d<@mLc*N6W3e z-;#stZj-9rE?o3!F%jqaJt^-sz*Yhg=gv8y!~7Hiro2Ul>uzWneFGh~j3Uq^As2*$ zy+d~E97(*(==w;cHS3-sUrk;y0Xe4=3bH@UZ+Zilh&s5l|9l)Lh11~ktG>`4l z9iv;9Z%BLW@n;Gl3j>b^SN>yUE+JVuAS*`;tguSGKSy}~ilCwDp5?z9 zF{-r9nFz?3XE9X5PSsO=iOk_6d0S-GX~=0KUOW!{$KLFQTS#eJ@==Mr1AE;M5uU2c ze^8XVZyVNb<^XUfvMeHJf@T!{3b5-5n+`*1P0TPAAF_q!V7Pk< z<=Ws5Tl$Sry#j%*_|B~nW$E^CcB|q9(1rEtzr|P8hPRY1s-vI~))$g4|6Z*BdcI#Y z{@WOOIC%E!;pE`UrIY*l%FX>=9VgHAemF11_xl*f{1!4B2g9Pl*^>-Yx@La|Y=8km$AE%lr+RiX(uDLeQ zn5I16lo8^)97*NBS9xCkIYvWF=+uP;)rJDl5?sEJ%9iZnV?vK;G2X=N>mIX^n$^W+8!aFOLL>x_LJN zgAc)Bb1Xcu5mGuYF=@u*Y)*OM3OB*?`PF!;u= z=bjK9>}-OX7<01@wrSLg8-AtpA{koC^W>PK7@vU&!P)5SUTp4wMF1Is z!BkUWBX5y9t!bjE3#AcY$b`F?@+Pv@^@0=OPTwAM@*?|Aw1v6=5|J6kI8JUVy z^8kG|^YhgZH^FojRZ`&)Gb$Q6d4UYNSpO?tG_W)b$14~1N3roGgo$;Qc~mO%f`9-u z`S&WEMnP83OM0?sN{$=tJ`1NbQ>VtCaFG%`VJ$!PlueZy7a%!h*X&`D)@Mceo)9wd z#~YelmJ9C{hIJ`U1!jZFFpHFko14%mLqpYV{Tt!HMzyZe!O@r6$9(nX76GAK*q4ap zkdXYzJZMuGD>gZT??T^a{u6wX(jP$K4% zAsk;SJ~chcy@Y`x!wJ~oe4o6W-PC8V_(=AF5CK%PE!34{t_&)@w|u)323@GA(LXt6uYA?DUC3$QGe(Uv|divECX)QdJ;u?s9K??epbpLT8JXHYQibk zYc5q%kkH$v*`6R&T|HVlN~3ly(<-bdIl9%YV=lrse@l^9I|?PSTJ3?)@`QM2X)Ww0 zQINB8z2HQ=(0a^QUkuC$c^%#G1RH(S|H?nabv)^GwycniyylkP{u{4W!p2!t{MULp zfkQNcz4Os>zd~Z~4GHC*yW}bp`5FkVg;!~pYQF?g%*43-If9PmB!7MeEXsxflK4qh zFFIbmho-Vr^fwW}94WAN{ba>pj}X;So7-#ARgvrnOotN03B4LM@E&RPk((150c2D8v`5+N510jNtp*~3*@6yh1nr6rYuL{M69mGJ zTc7n=eM_mptf1+%Ag4_W8sjB3iMU!@`7<&H*@n-bkd%A)MH#@W^{AnmdRbSb;R8bz zvEd>(aB(kX$uLFE%+lu}rnWw49`L)abUPxnj2CQ+rWaMlB;+QsMdRDOe$U=7L%r{Z zsMV1D8K0#dR0_PTL50hDO+>tH-~;Bt%LD}gPYm8=-ja(CHVfxwbVGzUp6Y&es2;f@rzH$q59(Vca0(#lgj-llIE61;r{yTxG z=7BKVB|PGKt07gzzkgDaAX+AwHKd&@s^lA#uBDgqfu94p1zhPNsi)w4L({&Hv;qrv zR2mAK&etQ1@5Sl-XC)8+!d1>oaB{;(g<9kh1rMFo--}0|2gi|u%QM7y;3f2lvr+We zTg*v@9AGR+gQc?+K{mz$EOhNrnswtI;iqS6%`2{nzZj>kxJCVM0A4_$zbM$uT^D)1 zf%NzRqNXem@Ep6_xEkli?%W<*IcB=*J*!+fOOs!S<42?Pl7gqi;{)oUSLiv z?i%rPgtXw}Vgs=Hon?$p&gnIsu?kT1c?P{#m;BetA1CmdBkdpSs1sPBkLBmTJ@t~YRoO!u%xbwaSURro4X=a>zJ+ibNLT zaL-6~3n{C>s78-$@c6L5#Sp93B(1X{dNC{~DxHmJz-fCnIy~mlJkCCR1kOvVLQC= zuidsqR^bJcUp3HFM4R$_+T%cUNZskvn!ke;nhp4?25WpUHr?gE>gA{PiLaQ53O%}F zq~j?gun0K-(6@92pOzfSs9>1-IZ{E0>Imtn8cXeU{9e0ld35;|UwUeutX8lxhNXmu zn>TVJZ;TBEzxhswkyS`lv57P4Kz>KXGb)>}|v>memp15 z6uxDb&}5Oav6W4!^;$ph_9YnI1TsB1wHBOO3r?+FoNDkxRQF=Myt4;^-7=Qw;fcX` zT8lg>8+=tt$n`yBm4>|gim+@jk}11~j-;>**=$n1^!?OE)#ID~5ww*Fo;~Y&e>#LkvKWl{duOcD`9J;2gc1sri_cr$H>0*= z1gGMYxnP$F<(0*p{ysJ_S$kr#0~3?c#PlJFy(ILRde#aPrb(T`OpJ}44;wRDdUvY$ zSr=NJU=l5xy!P&(E#F7c_kXG($L)^e+`*l)Skk5E?Y!*2?VlYmf(>Eg21u_uj9WV! zw0qYs7_eG<2fn!JymlY49Vawno<$yA2VjIu($@J%o9RFaK%~WL&mLK% zHlx=t*22~knG$M>WT#o{|55wSVzl4v?O!1}Td5UoUybjG%c>=YPh;T{gDoqko`|Bz zXp^+6Nfw7SIe$fg>Ccy=+0!jYEg$)#VZag2|EA-aUzWiQ>^c3*(dC=rHn_YxRez_? z-X-^^>Mzp3Iw#6ztK;gfldKN5?6+~Mev$&(RY0^={5P+wt5sc~rGG^|i((S0>%=)D z?gIzfH`1lQDUaI)qAH%8AMp(xo43i6b1y*r1+JkJU%*CS(GIC?pFg!wYjxcY{w=gq z>;AvsN~c}w{3gGAVuQ$y3!P0K$I&7}*tUgvLe$g3h6={XAdzEyxrmHqVZb>fB^AAb z^`6a3FBW`tWHZ*+{8nzFx^Ff$ZE@QX+OnmlnQ3jgf+R3nA~QkoPk9MTU+AN>V{^?3 zw;b1Ics1csl!)+)#qR$Cq2eZ%1G9&$dCLnc_J=Xl!z%i#L@vyb_KN-U=g|s*#&05j zC8?0Dy22abk}v|n`*qZT6E?(rKv)UCoLsXF+%@uGfVI0Gfehq$?%VDB@VA?0rwUIv z=k!=y2X~5%%~GY;Gb|*2k*r@6g1b~1wD&J%SAgDSgT-RB=81G)>Nx5&bwnbGo8yg+ zb38)qu|=TPQB3_~f_@!5$Q`0EtnC2EI{xwhXklA4Kv)Cqy|}n`joDp$&)X zjH(KCu9poXVqQN94~PjNYEnD6>zIKMLv^EXIN3-XQB)TR5D;jaXg>)W4Ti+m8(d@O z-$d7XPuhK7HLzbX(C)#%WGc5RNlsXv8~Cr_rO)sWLv$pIR7?7KCJV2F?bV>eZvr!V zodlWCz!0DF(2zUEL`vRm_5i&WK(CQGeAC_}8+Vh`Y7yOVX^t|pXhM=LC-D5106MYw?tQ&=#3s z;L7H*F>+*nPM;)B7*z4o6C3anwmpvG=3|(~Irsl1+p=d%s(Z*@B}c8gBJ77<@sq6d z{c!Ibvzzk1Q6>*qhfH&DAKmMqFJ}s(>zEl@r{u_#yh2F})zpV8Yaw{0VLT8A0~SY0 zC!9%}-XlB4po6ULXN@!AzE8j;a}3=lStxmPWVP?Xp0^%AF3*HBy`uGgD-!@WVVL5~ zS44W5 zT)ba(lzkod6N%EUqc*;#F9|X0J#am|4<=<;y8N5q)pEQVucDPJvDU}vbfvCm@Wx$@ z=g6^Eb~C_0TV~46+i9)lt+ZCKa-yns4UxCDIBo#Agt1h7&DpKqRQYDZf%ZWEt)exp zA1pjI9P=#*t;jd$?#iMiahID)(OGjcQwf0p*y934%jBVz_W}KK2XE2WaNG$Lo|Di7Yzf{MZS{{FgzSN8|t6D0CG3i1~l5!Z2<)s3Zg4< ze{bs(ONG!LL1!S)Q{TY16?;TJW=4z7HNFA21v?E5_Z=6yv!DxW$lo57n6f#02i=I5 zNJDis{KCCLtA5|Bs`+}(i3(*eYPA%H1!KOV6*4=s+iaP>N8+7CZUfg*NYiZkcxV7c zj6r%~g~=GADB6cNk$U^VeeR=ygFAVBEmEYTr;7$`14LEb;b6K(&LPM{?Pe8hW0FH% z0jyr>!5<&DZe6>eHr}8Q*P?XRwxe5VxPg>Fa4=3G7;$uo43i zL+{o2P)oA&>CDMDxaNF=bE3`)FlqB9VNs2>HkmQ7>@7}ba_iqnwjwzh2+-)1B{q}Q z)bdKaAV3Kl3m2{wIc*sY%?WE*%yK8sBr$TSn72yGaJdV1hQxZl?iAxjOv!~N1J>@NCm@Wu28JY}-}S~fpj98APtL>o_}S!R`uoY`8Pd`MkOKxA19bBTx(7L|aIRu+t0*)mU!{=yb;2z7;UzKlzv!N6XtawLF1 zF&7e=G1)1NrNajVdJZ29Gr_DmlZ4?zOXTBe>Abg7AQF4u<~33VU2)8rI+Kc7vbe84 zZHP!d=S_0MJafN}koVOHI=aq(J;j|YiQS4gM@}21oO|MU!+!0<{S7Gn+KlSn@4;6v!{Jy zKWD3qt$`bCs=us>zf_ok;*MvC-zD>09Mxw2aYGkUlI zQ5wL}(qYKL$Leep65Thlm0TkeqJOM>yO9K&li&Bu7&;%SMC&s4DUN?{akZ{Z3fU>^ z7&x`!k@kR@yzd9ICpw>*>l+d@;Yp02gta))IcUvu9prK!7L7%+v_1O8LBF$a`}%74 z?Exh3%!3BT0Vkhpby04058*UqM2W_ry?J6GxUp=q2gT!PzJC{Y%!WD}RnH8urM(-h zy?@)`gRQ;UgYZt4TM}l;WsmwibKqCln#=C0RhBDMuGE|OJ}la}dqP*U9>kFihoV{T zNi>v*IY9rcot$Lm!#-ZXe7hI)qm^8D&!Wn8y>YWv?dQH^;YtB@ZWdMA4O!BPX;#m# zxpvM=R$A2yJMb1ZH<32fL+};^hoPFn)JnAWBgdg2EV8JWzLMr8>yrrFiIk}tSB=zl z({3WHi3?~Rr9}DknXvhin;2&Fhd?fLNtGdXFC*t_+Uj`BlhBXzXPsip3Q2qERX3Y%9n zM^?FzTKTdvnv_03bJEkvc&S}DS+`R0F%W4pX|Qs=QTQ}$xYofeu`UQc{$Oof6qMM- z0Xcmr#9Fg9B2~aAs?PR4>3jeXGjfQXTBUooP7XwhGqvbq0u2X^2O!OG#bK;&BxTQm z+@^cga1UDbY_~9}G5oBAz`X7k&U6OPX(%~6Go%Q;Uo|O)I#!f$ZMQ^-jWxnEDXj`? z2^Un;VmO?c*X}wn35nc>39P^ST4u|&vo!Qe@4;#z!#1KO-(= z<98_+8M(#HPyzy|CGG!ALPD28t< zzwr?GRe5j)(3DOb>@36-l|!esBJN1&mjiJb#?9`Dv`ohEG*E@@ng-JUqXvAZ0TWt9 zi+T>USGw(e(+VPFJ^hB2a^AEOizmp$YojVwlKMxhy#!I`CnrHRO`~#}&tPCg*RyEg z7x=+-AkhJBhjQcm^5_4zul+^J~aHqH2RNr^@8PeQ7HwbvR5;qqRoHADt@`?j5FDo3qJS$8kFj zj2p}H929S=wSfUO^jBId+ZG!}(*qs@F5s8Qu}%I*oyh4z8>K`NY+;b;9Pf65Ym*wY zo)88S2l*rjzv2XwcIumDj0I+_!VYRLxXU03GA^+X8Z?Q}|98Wor4%HLe53r*2ZF31 z9R~nXAXLaDS^)`+078G^Xuc|#mde^Y*_y8aOv9W52d$#z(9klEwShYPmlMKGDp!aQ7$;S%q|$Q}iCp!8)#pI^g&r|_2wb4%|4 z^X>Ip$@jl&I2_jcY98!3whhw}AjJ6s)w5g4^?q$s*EQ<0Hh*inqk6`v z&t&K|V`?1ffABfTj>%ggB1$z6!W%q)esa!MsY~=S$Il$bWxN%xdCUa!`Q+MofGm`0 zBnYpO_E&c!r|F_uHRKB2w$8wtsvV@UY@jk{)%_~2h+Cfmu*GHtUVYoG^9sinN4`G~ z&j^$WL-q%tiYjDogbTy!K%uwBA|a97zEYZitSkA^8xD~lAekf5caDJMgdMV=0k@?t z9OeKe4V$n)#l(8OVK&_&aa7Q5uVJ1-=QCQ9{r>G$WnfJg`edb7(d0^9r zA{q{3X}X}M3u(I5aA6we1Ip&vt^D{o0fO{{FB9t%5}%wK*Ci*Go1rK(6<^Rk#{%mw zV#RL$7o|Ln%yauRv9$%e8xscGoR!>#25>eGJyY25mCw%{2zq|z#jezesmgEu^Xd4mIEuypW zEurDXY`liRZ~gab&26Hn9QXIn&vYQlJ3=Re_cRE?;A{Tz@e{q4?X(Jl z@fsLo9qis%?}2rosJDX}o)c!dsDDGdl5i}jbT0BYa3NY*P%y)sN8beMy-l%;5kr|U z`Nq$K&^d(`me3Yxf&9mM7wTP+@~#jznT?ejPNu2owmHrm=V)^lfy9Ob%#F=*LB}Sx zfE%0jp^k1~_wu`Xn>ZR>Ov9(5^cLq1lmGL_I<14w)aFD8+JO~_ECc26P@9t}hi4)( zFB_xs+M=13Wz0THU1;81HkY-mh6CUYF%OtNf&~))kzMZ@+FmhoJ5#R-(;+jDdczmA{R!Jm zH3$;Iqf1RjZ@2pbRit;`3;kJLWSqv>NNy`QH8j}~#)doH5f<&YtOhk+3AV(tB|E~k zt}vMmnjUt@L~pG~yZpJO-}QO%JQZiTuk>mg!-;8SV3|!*ncIzu9b#S++J5N=>MQ>H zg8zOSdDF|w_s`BQFRRPT;_@=Pyj;#a^>ySS2E--2_@*b9mv^)6^k*t~c1AbvW7@Mr zPyG^o-K^LhBDMU9*6bC1I(_={lkMQYXOk%K)OV40c{%ll-wwT}J}vps3-rw-^oFm@ zr9b`X(whZ-)9%msodr)P!R6)0Tl(}D`gBYG%IEaXQ(r`0s4L|U{XeYf+18)_><2XL z+3e{S#nFxLQ1$fZaJJ=7f~VWj(|}5&HT4-9ffRC^FBaS6x9xScT~*txy4fl~-W)Oj{W%z_98sShr z@A>$tk{^2gW#4Omf3cqr|9ckLuYPmC`XWI5dIatG6McAhc`_RYTmSNo{$7sk6U6O6 zc>6HRt5XEDpK=o%0-s>%>698cMdW%Exb&&?kJ0JP<$|NfpSp2KY^?jOh`6Z4ljxM@ zbTXi+JH1wajUK*y6}HCpCj(J|dim9hH*eZ+;g)LOfAiw&*WK4ss2L-Mmx(SZHvld= zx?lXI{RPT&VE!rGT~Ysww{O1>JCpuhpuT(a`X8^p?Y^M7rK81t`kC@ljN*$gzv_%J z^f?fJ2i#8PoUzUYIt!w{KOGW`1wKPxwrw_?exdK_HDbKD@aai6S^+D1AZZG?*?L3i z^-ldQ+Q+8X{aMoy>b?=Q1Wj~%YooJ-<Htf?dAdhlSh5} zsne%+XJp4vNA!iH$Ej?nawi>dyA_k=p^1Q34I($r9N%iG6LsTXA2KzxutywHG5Sa{ zN*pC0>8y_X?>@5m9=nV4vI*^d@Rjc+cI766a|AqfG9TMe;Z#a}NEI9ak%pO6`_sLM z7Q-{u1p>wxA9=&Y&=Wx55q6%+`0m>YdJ}c;Cg>fTUDE2ML`m;?=r7zp4|_`t&rF%d zOkr@{)%YwB-N4OyiD271*_fInydN=k0ijtEbqE^N5aoKmRz%;t-{15^KxyyLvADQS z6JqMYa+n~-j!7=GdRlPYOE@u`A}O+rO+)j^8*N{3@YBlqlzI_Z{%c3fv*^J%oLb)) zkVKXFr*;HQ$yW)(p!4k(KYx~=a}n#>%?GUcQX=O?d+Nm2Yo01FA`M`S6Xr}DJx7oY zqY_CpjhZi&NL2Xwq^Z^hfs}H35g_vrrVo|4^N*YhmBje$!QLF+vFqiQ1YOG8Vv$Ls z$pQw!e(@y9Jz6m)-O;_hoU|9t*gd<<-z4ca7jhDYHY+rCgkOlmHSh})6SIlWpV@ZZ zYY1g_A^NmNeP*C0nen6-wy1e(BRTXDRN3eM*>Rh3v=6R+7eM216Rj)&FzO<_9ra8T z%|VX%LrxG%RNRiZsaWX7@d?=yg&3D$f&~t#{7;iktUH*%^~Ov^+~A3I>H+g(4~q%z zCl!bI2;C^EnG*;&6t^rSXsfJIP#HZq+)McBnUb^nyE3U4dH9X^WQTuMP4~mUOdfXO z15U{g3Ed|lB%i^21}+O9@CAoS4%quJTZEuA_x-1uP2Sa`%ke(Q8c|&b>pIH?=xl&s zLK`%k&UU*!%e$tvtiLs@M(MtHT1zLXDD@i~hjee1rM~^=zK1-UI$|kRF`1^bu;Sh_ zas2nr4(XqthVDW;jVkJZsX%w<7)AMk;wE<|NIDx`cRE876fO0NkQj#JYjv?gDyLG> z%+{_!kIXQ@BANf~)m_~1SF!iPm# zge}D>A3N+1>K*mGr^iG4xJYi(ye{>j)cW?n4;A95-CBhnJ}`^aw-u3x6Ms4~uw?r2 zEb^1-CrF$%{Zt44>y~*2fsFUxk+>Mr7o5$7E;##&Y9#reVCI>>dbAJinEw16SwX5B zZ{jB$g0RtHLwqUS9CF(ebQYKB?0FDAd+M7Gf;&U%C4EbL8W$)`E{C09SV65e*sA;KY#z!{t*J&vCyu?>-hhEKt+I#I0=PZCL0t zcIjYwoi{jXq1z9vvXu?0I)Fs?pG%{yfa4aSYCL*Cl}9M88MtMT&vv1|*U=8t_Rd6; zhxw4z5qHVrr+uREont~{s%7G%6CI3&1Oew%65#~L0wb_^hbVSEQAtfSbtk)1Z^{nC zMfYTcmPAi%f!o0|ElXkhxUm+9G^s_85+`lC4tIfd7(`xJO?=J>r1(kXO*j`uXx>lY z2BU9(rrV6I>GVe7=sjoh^6CD|=O^d#?umIPNuK@EZZ5O4nl=gHO*rdRIc%ksa;n>! zULw$*uJX)UcMZ?EwxM?dLOlQ`-^Y&~+w8=m!hOWU=3_2hu1)IcoGsX5#S*#N20$Os zRaCm{PX*4g#s09NOUH<2IBaruYsG~*!cQVl06gmSKa5w)X@+>nVb69Rmv<>RS{lck zHh~9Wn-}sW_sR=}bVs_E&hf^Y?LHGCG4PV4vRp?qcC^TqLu}#J-5K$rdMy&z^_&4N zhd{IqQ`3?)7OoSOoWmR!qm^XJInlg2)80?X;z%k8ahx{PE1`f1jD2nPrRV1p8o(6Z zW!N1v<+g>;so&mZ-)Rp*zteOuaTFakUJD0PF=?B7%n#G(tqF+8cy)wu)G$8d#H1nWKOXTFJrM=i|0(b`+4Qu{ zxCUW;w^LlOZ@vE6vtHu#+SjF)kkFOJ5m^PdYNG*y;W6Ubl%I3-|7~;agEnNLngsQg zOgh``f5RhI()=LvF;{koSnl+r-Skd#t7YZy5!5gy4HS;!)@w)PH?2<2Q6~AVTcRN> zr`g5XBpX&?#wCf_+pi7Oj&=qfXBX8t8%EE9#HjS`%+dG6{5|&h`5%0~+8YWwYQ8Tb z+t=~QNdZF^Q%%vT=K6!H84e5TP;V{ZAm?sRV;I5ooC+kl0@T65_Z0%=6Zbg6c7A4J@tDW08 z%F!&MCf26@Tn9el*ujm=G>vv{FkdJi_!_g(eic1w@)-@o2}dY4?iORJt2dC#+T*?Bnr5beCsibs~H z`v|&WdFZI_2@t#U=6h4~(ah?q!Nl9M&G#1E9HevYF`>->WZ~eZBDs`)b_R&q)xk;iF;5k=DtLr8+%FP zE~NOtm`4l3SB2U{MaR&U1twh$MR9=O8@N#pVr`U52WHVD?+hmMXl2>gOcDG%IYvJ2 zPNGmMtLBIR?@iMMtQqD*@1xm~2X})us#p@Lh}m0Wh@ZyN^d_=)qHQhLc8ej$x}@^h z5o8lM#z1dujC^f`aI?RR8*%b1wGqN?@L-LFh^jC4Qoi!=tz@Wj)@sKoeE-(+k4&Q4 zxJEd(O3gEd*#TfSEYshAXUbpu%s~REzHp40z}%KD{FE;IJZ8qgG!_IxX2A;OMISvm z$4-&coyGEl(qtBHfGrDnLK)tgv1I&NOjI>U?uGRYZmwI|)2$m^tVgq?(KKR$%r&d1 zT(gQ`=Oe_W;Y+jXCUUsc-;{CcD5JF-nb4ML$&JJZv9ell$+V{=JUAVLN>V7bl*;-U zod}-fSlbk~?psY9L@hFb9)oFg%t2I>_0(ENs3s(xYs;{M@{!NoDmwi~R;8;ar;11q zHklO2FSn2w(k;K<5(o#usY+31H2pdJ)8+JXq-Ia)yRtGp-cJgp@+x{@gB+?;GG?-f z1vxVnSaJNKA&79tt?)d{cEk|Z2gxLf?S@a;cYxQ~!#T99bDoRr7B^Tzp3c3RgU#iXub5uT_IK8Z#P`OXQGU-{? zCh??+R|8Ce2*~DnGWwOz@Ch`(p)tM}!u^=mgM}NgG{=uXO?eu~bLLWb) zq6haDqI4xjv1mpr3Aw_~eFNr!4Z0R*3xbGcBsbEQjsDsrD}FSKYv%_51W}#vm7a+z_;>0*=g?lHCSclbi zJsutG@6&hLVg5&G?tzGet+h`2pl}L_52)Ju6!g0Vgucv?6=x0!C{q7 z=7rSmh2De07^d&rS0|kY5_4X2JBe$vE)@T~ioE#hs@TRwmCjemHZD`zRdKyY^X=6l z-Ol6eHZHe_oW_4vX<2O%@;+Ua+huw^k68+Y%eqLm%RHwQ+=>Eidvjgn^=7+{i+9^~ zf)`nQyUptgzEZ?ET|+FmEfXH$wyf7w@_viZ@Y`GJM((K_;xLYW{X34{%f*nL`B?D< ze;@DkT5Z6AT$aOAwVrBx!P287wOGqSM2R{72 z!r3sG{``FQG`Ku{etMlkEJBs(H>YZi+Y0$lPE&P_o9*$RCYN`^@u|9(cEfUBq?>9h z3Dwa`KxMDmFGbFwyCvc?`2c0_+Qvs0?9Ml@;K_A)S)M+hX^*DYnlUq4ye;ea;Jx;S z-e}R*WeIuH^yg@{jp*8_en)7C*u>MzQ~%faHr^)lbsR|V=*?T1&8HUw2&<+qU%hzy z;&QsZJh?mxw$sbm<;m>f+3q=&ctUgk4KZ^vh;VwYjIi*?2<)Pd%+*KFsq2lHaHdwt zHSKr#YMCah1+mLgrnv*7MNlM#D#)^=gubNLML2zC-^udmB7Er&Y)4#}{_(Z!SbmQg<}YsecHx8!zNyx$H%XDw zj{hAw?diSgcZ5;>cp<()o^}$Nb>ivnX~peXiVT0HW(g04{+Nh7!*A$pZ5|Nlsqy_i z;(IU4qn$*NqEEZ|;hC)1{Ym&ic=rM{eBQPB={<(y?IR}~Wrd+ymRCDE1>Ex*5( zM_^uB+$?UPG*xHq#yT<#UTA{){lKlA`2N`Q zoK8VEtA(yVq}7d%ZR-%7R3D8Md>)VbQ0t3){dJtBn}&8%gR|D2ZK94FXwhN@_Yrb` z3fJghH5one0sl8U&~anbQkaGd2uwn#P7Z@XUD=Hl{80zb9}9us*9DCh_E3`EvA2W( zk~nh+=mUJc;W3QtPGUCPo&A4dj4d+t|L5_FY>&pv!|EBXXj2Y})O27U zEB`XFeRS9*#8h+ zgl;&*#O28c_bxYmkp3Fn*<$T-v}6ma>4%%NGRaa08@KG>n`-LcRwk71b~_cCGuU$x znn82em?77w$>v_%QRik@kOT?zGLrg3+N{!1RE z<($*64+$6D_Cl#Ox!cOMeDGA zN)Yy_wpb>@dM(LA=O4_kAYm0csMx5b9KmWsKya{ss%}D$g3*;m=RtT&7|5lUAUom4 z4~mO*U6Qp$nFD1{+L9owk)*QJI&_H30vqz0Lo!8(8VE7GT0t-~?{3h6o3@C!-j_fQ z$H#L>uJrsZ-3|CgZQUw~ln^CUCsl?fm3?9AC^iP+}qDX@AQ-j%QQ@KJbFcSC#Wj=-1FZKqA(9l7|Ml z=OFwzp%3*=V*RbGxsBEwtU9993du@|lZ=0j-|HX$p)OV-zYu?F?=P?4dJ2*)NewEp z5KqPBeKrq?&beOYSMds09^1T?9+cCZ9a9Y7vit}&dw3*;U*E^!yoBMg>zmM9$?G6|uE2;J&~Cc%?UB+&F2b zx@#56VWfmyx@H3~T-6b0d7^EFh_OgC%rXKqG`oi^dB$^B((I|6TXt(h_(?>)eJ~Hh zQrR65=A8S~mPcSTqv8_e`8E$dx$+Dngh%?)Q!W`DT4G+FzF-R8P`nz3AiZcEO@FLh zXgtl&j4~PM9Z_42jyZNQtX}UuImB3^L2I{!7%Sc4z+~xr@C}0YyD_VCB2yh92BZ~C zbfgzG+dLGJpPD2NRxlC@ZWg`gtn25s8|DU&;49TmdL zB0}^%m<=GPmk7Z))gd;mF-3I%O|%jN2UV2X{?h5lshKpx1?wD>EUbp-X}!Lpjf-IW zaK#O@c{}7(X;vo=vrs#McTnhBgHas>FsAQ_IO{+n58p@k+wBT|?;wzR1<2~YC6Ad) zZ2SYnKOq6jL)4G)5Ja01zY){H;RuWoUmFD5+yUt&eLAd%%jvV36Xtpz&6_Y-9cqqA zgy7)@R(E;|V2Q~NQ%q41c-~CrEup4+=ir}kse>UH&4T{aDWR2Yd9>+-N{?E7|gv7De|u)Jxh*B!l4N>EeDAIe5I`kHEVXIrseP)-Yfce zy@o;a&<3x!)ajK*$q)qe%i!c3B?_dDZ2oV;IJd%HDA)-ZkdpAQuXUMK)JZ0(1zss3 z-`vh>u7lpX=jU8YHJ)blE7qO`E9Ta_~RcyV@oOkd@{r>m82WpY&9hNgzCTZaut{BQ*pjwt7f z4gm09N_UMOARvx+hDbkvz%A}ytlU{+^s*zuIJ$uBv!91$Kh8}%pr@s)4_bfIi>P4_qsiLqcg zK2B-RlogWF6KcAplhJQI(Z)ltFw&Z1)aJ-RfE8etV_R~%iDw#8Q9+}U+-V4YRjKbP z^+lyVBUp}Co16I3pZ*-oo?gNN;#Yc@)3KUX_mMpQ|0=uo?zU|#|5Yqb#)pikhttfS zS-LDw^JtT4lXPRZO~=vBp=H`;LxC&9=&~uAA+7*)-*9TV$6Fj4NL9V+Gcpc?LfjO_pa_x(KE)5%~j;S7~*Xt6BKA zYs!IWyDJ2Co<0m7Xe*AN@2QfEZZ^SLI{Ezf#n{yS+^>5!883oZzoIMsN_+Fkn`gg~ zpjed44Or{xWe;4Mn&{}Y}xy7-Z%&0B>3LX}^6qf_A<`g>J= zrr#BO-hODr095iqcJ2L_b=^@9{;2}L6Wyg6i46f8*O;^(L{93O+uCtL&b`!t@W-sL z%iDp|$ltUDIqNTJ(z5&O3?;f8H1ajk2aoNSb=1e;W}Jd1NPcLB(@+kdg8M4TqVM&X zV6QGRGCt6U4p(mREk0C9U75G8IYVJ|r@zo?QG@@BW`0^jFNfWEzfL zy&IUCRaP*yRS=hS-D4H=2IWz+0Ofb?}|R@?q!=hSRWpG zw0Ok);*z*dX7-?qKY|G;Lka{UwSXs!wi*rLlI@%^)bVF6Q&g;6t75-~0CyBTtch@r ziT$l*_WTyipno=!ZlK2{`XO%J%D{R3I(_Zi3|YtBrz37qb`5oL z^#_Ut>>B=8>?5Z?kiE(9iE*WmmI@p#g?%j3$-FhNTnZSnpOGIFN37qZs!ra zDrn>1b_`#7x-Q~=n85}iKWKs)QHu0;!fU*}@#=&0T}1X2GD9Y+W{i$=p%3ktkD=nWIciQ^2@%5Jx7nzxV>^65=y zeV4l0@30X7Fw~{t1^HQvixoZmqiH=FiN#5$gsIq9C!MZx^()_$xJfs1)0R7><~>g5 zzSeD+vfO>A4xP73WM|ruG|>whtl*qO>5#A5{My=+2c&aBo7hQGE;_g>dx4dq^WZ@65BQ!{?XVPZg9 zPV&zg2vN53A|$sbi7DS?1=y4^kcoLg-P;W(z0>q8T{k7Q(a4t5ytbLEt8=C=Vf(~RTGXc6wQO3a2BQkP09_% zoXQ#@~P zZk(9jg3_zrmJ9RP#i+!oc`S1H=DWOc^YCas2eVEcNczu84-GM#%$4UIt0`ER_((iI z#Vm)Y@XD{PYW9g76lzSR`bvV}Zm-BUE{}mu9X2w(>;1MfV~C&ViLQgB(_>ELfs=&h zyq;ZCM66O`T z`{+m*6NQk=y>YVK*@^dJ{cQ)oh;2*kk*TaIz~?5tfN~s{f@>m=KXBl*(=Jpt#!`uD zv0W|a^wY#AcbqSklZZY7b71)&7!6imKV~B+wEX}n=+HyX#!&P-dBp$QBleoO%|Bc} zt={0kvBv+#p}ypjR`KlbkLI(hu%@f`XYUseW?_lTDtWJ{c&)S-NLiE>JaaPsVc!{Z zF(GJ4;RTcC#a8BYyDE`>N0{ra3GlY9QgxMW>nyF7*V`s5ID?Tm;;gG;dz)n+f(J8V zI&`nSJV(JS?@I5rFrzh~beK!RmM_P}hkywT`8jD$UUtqhpRTN9?3qE=LH|VQ1J5X))qBEJs}F zne87r`v546e2NMZaRgLh7w>2lCx#t0Jo-@57qm8gC4#0@?-hcH1kpfMHC$DW)`gVw z^2Ob*JK^ekRyc8AkI}%eNq-v%Z+QL`=Dwr$(C zZFjG>ZQHhO+qP}2wr$&I{mD)CzRB797wDr;;mW@T8r5bZoSR3ErPX)n_9!|Q$d`*MGiBj%H zTbGE;4GG?YT&zB)jU;FUIW%1X&J2gF$(kl}mH}f!zlKp|kT{@BW*TI^kQ5>O|1`1y%PvKxS^*rD@jA*52)RvF#)kDMViEt~`dA z)&2Tmz0`B6x|-%LK_MUCzd=<$-%g+VuT3)rpIU_%-xpst=sN$vM2)wIA3xeM;oG^n zpVv#!(<3P{y}mrjbM}E%Z5$r!s67ogIrKFAv(mT8aJ9;B8Ad}Oo%vH)u-)Rh5H($@3BU7$0T`lA?|eEtf~QjlCZEj z+Nxh|SgYq@kb;(OhZrcGD2at5V%~OMa!zwr`*ZgE4~6E<>fTXjXbrZUtN!XxH-MCR znCu7xLM)NV4fhH*UF1vb?!2AsY)|*Lmb$6o)>_xA3w=ytS69SqOCbGh?dNs=qYq<_ zs=>rkRP!n_iMn^=ERqE4f%Y>XHGVpD!NF!1S@o!MW4W&sayrYJno7_n`A7|O$o9l2 zYB>t93#>(6F9hi8#<`_ zfa&zw9tW5Gfi9%t^lOQ5h`b zG1K}s^!gvTUbh#3S%YqMzAVI!1M7!;e5R-zKWDRE<3{)4e-wbECL3HY32&7dMSc$!|D-ne3GF)d+JnE;nUQW8oG+(d9=x4GNuo>2R{%bA%Wt&@>03yhpLUl`N8(Mf%(oh9 zeR@xcJg9tRU8L3#h;GX9^V;N&cg40f+eya2o|X;wk7K{4WZ^bQ(DqME9uaq_X@{HR z3n|+-MHF!d&@`Q7SMimzx7GB7a;l^b^wKPv8XCerGpwy;0JnbReTfKKZr_dKDFtU? z{5zfK`HfOgfm&V1R0d2{RQjc>6 zc#3J=bsJo;C2hT3IPx3=kB=9euak+PxOt6qz98-Z!(3o)sRFx74vlGd!W9ul{;ieDzbRLCKphHtq`eO?lz+DcU=k9Xp7KF#upm=hG!H8S_;F2_Bu zRfrE~)e&*4c?Y1cZ-um~%TE7+xcLlteCAQreYw$Ld{0YpN}D%kyw5pZm{nVc1*Cu` z>h`z_3`7a+k$71z>~+_8DsiPB%8fgpstQdOG8$vb`bbbd_qR7ot$G%Z)^+&r&2isa zOQO+M_*VE+c(6&;7m=6fKh1$$634kVEk913b1VnbH9{nkN_kMeH1;p4y0;V z`G^FTx|>*f{f34~3uEk1iU4FGO8vcZfkG&|(liTQ0=3dmfA^4h1zdzHt@HJ0fb{V? z{Dv4c;>HzJ_bTSEP>#xZ$N~h|b(Z%0(xn5o?_d=;-rHfyg|M~P{nU zgn9v&b~I}2@ugU&Oq3C^bK(MbR$iOPk|F^zPLhXPHBBclq*cz);okUQ-6nd-dyZyu%FJ? z{G@IlWF2oQcznr)s48-H>9jvZ+nwB35ygTIAgQ!{MlLJZ;P z8vmjdJ!`y95OusxtlgqTll*3&SOpNvPp&I)*=<1f?QKNeQH^l-Nn!tq1Enl>hhFMq+0eW&JY zz9Gf-D)_WWa-SArS{Hy11$9JFfX8%dB)+?{@mBh5? zx3xb`aDxD*YxyF1)Q&wQd)@IhR_DAza)L1;mO1aw{v#PPR(p$_j*Af*NwJF<;+@*2 zG2eNwsWaqn5sgISCW#Jk6ZrF})jbs6MuPPsgSo4DKz;7pjPrtu^Nw3h9Gjq)^IS&6 zibF5+ry~8ujR2uphA(_Cu>Fs)h9$ExbCHg!%dJ8D6O_05i1jxrpFh0sa%q<>mw zmXgkPazUG|C)H(_`Y9o{#vBqMhZ5O>lB^KrVrEN>XKmS{Lu0LuVeJzcQ}hqaC&eNm zDd%{KORgr0Z}xw(=|?S=X4b$rf`>tdY)RcOK(BPW&B}~no-gIp2?jas#dej~v15p? zHA>M%X7AwfN#b%NYh&2+7rLHv_m4X@l^EyKGgBi`-`n*gM=YT{|LC|lri62v_io1~ z3$M3sV(!e+wT4>GHZ!IxL(=i+U4(3NSK5MLs7E^o^$RfLEB%)In)8vn&s#!>^qqZ# zEhIV{<-7!QWopDePZJYC-bi#yq?1?W?J~7^k&_$R()~#yBL|*)5CWy&)2E5) zxA<`@4!^)1wyCY6Zz;C{BGFGOH-LxlQ_i=fKX*g-DeDZ-il*DhWDItzZo-aX3giyt zh&{dDKACx5qUy?roLMRi?I3OBdjBkX1YR|Ul%Fv$ghr}G1jK+i4g{L7Zec4$JOf_b z@0t1VF~Ta7bU zc|jlv%=0=cGlm)@zAiqV?rQ@rbW@fpqTZA>9chfn2B*hTlbdkU7_;rzyid6B60Xu{ zIhu8sbZ-AE3NA@-L*dxt7-H8CzvmNb(@We*T!2UM05lM_@g|AbSKzmd6~0wH3^d43 zLQeLR8NWkuP$Y6~%Dwwn*TMUX9)k8FJh-9%{jw&)4)WUV1J$r90$i;osz?fdFo(hT z>QK60`CSc^-^%IP0aRniDyWP-Q+Ao38n0h2$s!%^X8#*t1hm2>{S$SIIJQ?#!vq5B zL+nxH?K06UJ zq5f!tlej1ILLx_-hbGW!j%rapZ##P@_ja%(3e!T)%^e9KFrcREm$SD*5=?+Hyd^*P zeNVKqf`;;PgBM9Ida36pwZtoClfUSsQ-$_U6i#6yPAG@(X3 zEjv(D$}ByPHi5(hJJnvp6S~&mBXHCm*7Pdg6i5EyBt9LKb>4$VXRjNp#(f7A#w^M< zbit?^AzwuW(MNhMvwaMYk;7e;w=Bo4(;J-C z-f+CZE?av2z-(fzt_pSR4h3rV1}4T*OQHx7ju>v71F24J>f$y@pbIaGZJzuO zwn<{-ZV&EE1&{}fSfnrL1uB9h&hz&M_{rWg6UOM>D&#p##&B{J$oHwIx zt=EonCu%sM>&5CYZZ*g*4S8bYk1&pFaIL(|E+V@)^ffnIt$2pz_vwxiiB8H16e#N9 z1(@Qn0zc)_e1IL<-q-GmGpQzYA;qE2qUUd67Vha6?auaHvm>60@?Ctcyn(*{+Kyok zY54ukPIz@SOB1gnl9V!0%to4qM;Y=W>b>-7jOPFAQNI}y3&;jRuE-xR z_W^#;VHZg|nq;Wx)NQ7*)DqILVLG;GRAKzH4Y1EIhiuYdkYwpU}WLmg4EK$nd)xcXwr%Zhz(8ncx!bFb+h zTOYYHp8#yS%&y_OFD-ij`w|H8lWz>9Nt}->_pRSJ%4wAK0LB>Yx-ySwftc>G4-8LE zXIN1di~sS#AMtV5PxE>RJX!uqkcZ1WnPoJiFr@=Q?8O~C&>vyAZ8|$asx>Gf-nJ+l zNcG@&=+OB|AFL|!(a3>)p44T1)l!`~rVJ(P9D;$44@pn-77X%hBp;9{v9(X=>#-LW z))9w27M=Y4Ir$ygSGlq<-97xgNz-b{#5qd&AQbLCeUv6k&cblV+{AH}JV(05)tHDn zh==e{WfQ7y=oVxYyo3Q6t!>Pt!3B3a+oID|1r4-ic3cw)K?jmNS#K^-&FD{xC+nZs4y`$Q zN!(;1~$7v1(>RD_3GTNjs7DmFKBE!B(XS!T>)FD=^jR zCpQ&{A!r4P(3*w|^H*@eJ?zmbbwqTk9^oVf?uD(44%~~pIP`s$`wgumDB*&+ec1J} zKHY4|Z_oNZQ%o{@P`LN&+=RhvSVU%z7j0@!wtMW}o^e^z_pE3Y=KK|=*S}~2|9U+> z6?Y@rn?%UCArOLb@(X|vn;OT}{>~h>CZ+g?`PQE1GDg4^L1}YlHaf)fYBcm7v(tn^ ztU6!z6|M0}D`R|KUaf5c+P+pNT0=^IG24H*Mte8VT|ZB(5)Gfo4x)aZw~Fi~J|lb{qy!!Um$-cX|Oa4gzSOLB5V%7i2! z$L758?C{dZw$4xdtOwZ2@6%iY+UizBl6*1AYe{*C!&!P%J#LF zhAl!ARXLnX)t4-9$SWcmdYsaa{jMM3Zrq1qUAc3&?%cmj18G+xJaXj6bm4TEG*58t zR%KQUr#ULZb(1>TXIo~l*b1x+&hxWJ^RiXwsr6Gm{5(o$Hv^T*!E z4r`@HJS{1-=Bd#N;@R6TJEGj{!%Kpvm3%jjHv(;@W9+466NF5Jc;*}`(4t;$OW_LjQ)@81vHXM;Ww{0e-5U4IEkgj3KgaJSAsqjKY~nViY+ z$!i2y$UJ}-9GIVN7&iLvFYEyS0?NyQ9~KwlI%SC zjRIC_eSDo;u|=tG8k-!>CVchf7u*1JbD zWyaEE>c(Hm;Z_zD=vO)0X~|jgJmt#3^p|?0kIko==_#HO(7hXXc7z70X z00062S|C%neop#dp#=b7?4J?*GkXsfOIijx4mw79V>=^yJtt=aXA2{G8w*<(Cpvpu zvrCver_u<8Br3u2C`iUFU0|K-# zf-x~oXA%IQ8q?fAIvaQOk5G?|%DXo?p(#i-4zLstvh%XCGXD(jX(wl@CuSBZJtIoF8BiIe6IJ3Jw5+@r0@Se)gFDl>hDYW(2zWQ6n?5*?eZraY~@Hf8B4KN$Qlyz|XVfXV;iPjiuj zl>Kj@{3j%C4GOpV{(p*MKm&n1`WQmQ z)CAkeoO}8q91fSg{y?FqY{Bi#ZQbwBjcsGn-3qOicJTif6gVsioMwNXMTgGM#>U3* zb*bK3`+j~S(YWYIULn+n}pne3NzQvn2fz(R@O6x7vcZr1B_A(tE`B0GlTz6SXXuP~D z|GXH9VjK*!65@1yXIdsMmyA`m2O{iVoU#YFh3f4XmH*knW15b)i^s7^H-po*FG)kv zhI|D=*MhHd{M;}qE_}mV^>9pWDrk3nG^Q#e2PwlnM^?(06-3m_MTok%&-4bp`4p%K#XyMK5btuTY zGDvz|uzs0&RH2wts7;nhPA?Pc=t`eIu)Y2KO!6Rd^&mDmqOtZ(+C8Q<&OnI@;LdfX zae`?z9pxuwcZerzjx#NB+|DkmeEOZqAO@8;vdS?ww&g02Br6A^o>C?BZ^2f`Mt!7K zqII@Kx!B}};P)OQENiJXN5{Ez;Y`T1B^$1&AbBKd8fFK3_=>XFf^mapI^{)ul|$r8 zeCNG|)-9*)Cg4nVK>eAL9VyGZdG@(L%0T4~+qyl=b^DxSc+B^ZAf^fhc3u#G%i@Cb zUoOc@RL#E|5xIwdLt?k;{__D+Uu=|Z0kY^V`gGg1U&sR&Ee70WKxnkft z=`pa?+cO{?vZfA5~BR>^|@>Zxke_<%Q{|}OBm&`oF{&yn!v7B3ttRxc=PeQOA!4Pp`3eG5MBSAeKJO~ zDU7k%7C!BCil25yw&aXwFTME8C|+Nia!xt|J)?G8?w4KQz+12l5|qyO=Q zt@NM-uu_QLC+a77XItj^lqVq>X!Fi$%6b^~2hcqT&c;;X4u8R~sNbW(JZfF8sP@yj zmVtY#IM>?nzPp^Uo|g+R4trq;Wmysu+5(1C;%(GQ*rV9ID9sLv!3E8@S-4ee!R27I zdDfu5rJZzU<;s**Pb9$rZjdvVMvUDpv8?dy_i6yRN2hT1Xa3<|=ui!4u!^V=t`l-BY6_8jXj zVG>K_T_Ohg*fWAw@Ffl>0E#wewdIY>_gohVBeKS%kYGl1c9yo2o0&w#`8cHlQ@&*J(r&vJ&!^IfGf2Edc)}2?+>#@m_>(0L z%SfGVr$?8^6T_O-Yr)K2YRVN8ShlkRVv=9z|GGORi)IQJSXoyn0RsSN!vg>i|8Ga7 znT?69v(tah%A7ws@|z4OKD%|{&FmyXAEMAafHr_JiXsc{uGyQzgAK%+lr^Q_uN~K1 z+1h2&W)*gPr@lvTTbm3<7Ngg5AO@ozcg*3CKJqCvqK;CeX%|dyo%Lahd#h=8kFLk1 zJ;3!HPO3d9N?tZ|i*aHLqRyx^#SCQ|)tuU3(+IOPPWG-yTa!HC27yUUP@oho5u~a> zsU^zQK(w6Ffptq3#&dAoic*za58IGV9hwE@0;j*>-a` zAJqV&FtHMkhnKK@X9Nv0_$;;k=Y&sPyQ z0VJ(4;gPE4wTeLSLO~BHR~L0{ysCTEETR#*G8Y%2x$uc+2YD6z)EVgMlDA$v(b?Ff zysT+*a7C`yS!oU{^McCf`hEO(e+ud%71qv>JOFq%KnAran+ew&0tN45M^-lhM?73=o&P4x8s}*G72@J>Mntfudz&5BaQFC zAIGCx)baRIJYdco)O0K`60jyM7mJ7lvGlWQLsv%w4;J%LM7QQQXs=3*S9%x5P*JxB z%JJc`u)B^sjzz47l`sVmB@+XnE#@oiUAWeAhZ<@mO5AcGS38iVmay?mGa-hiu@?FZ z;N;X$pmu=~q0wD;aA5}mWTgdrD0CrLTK3^iza$I=l%>Ln`}7W{P)c=0ZyA=T2PSp6 za2&kxRGsgFD8vkTx{t*_$9i_n2ifPao)jZ`Il1Slt7YC&*Tqk3c=Vb*xAv2z14XjU zuKQo>g%nocJ_uln*R∋rNOuEO6VHO7BMHsD|#E7n^Z^*ofDVk%Mh^J}kYBxmM1fXwTxafXpEh!GgJ$mv58-+BP8n;tY&BW7EY|96Xe_X?AL9m1 zHlV!5Orn3CXf&wrhFo(EiH*J}gdy?WpzrPe|0zk~K6Yw~3INb7`oBJBoJlVrvB+7L?7>U6UC8A9aR@h4dw%| zoBqCD-F~E<2=3{5t<0^epPzP?caxX*n{j{gj^Z^Vg^#pXG|^D{(3_ls+Wc8~6ToK6 zJC5S$G5bZ`0y8r}JVmXl;;1mxZS+v9w;WjSmP+iL-VGQjt8F_R- zN<)3}3t<}t!a*WIUV*3D!B@(%9rQECmLI~<+$#_REhJ^2y|?T3M{^$!)d(aujR2D@ zCzN1Wq(W4uUMAW()>2D*f|KV3MZB6W)Pw;ROhL5;zUyu_i?#EZoI<+asuZlBY=r$` zMj2-l1}3O7M9(m1C^v1uF>;r0q_*}K$KSz+yCPk8OC^B|uopqc0<&uvnEu6fJAps` zDzMEzUb^j%15EpBR|#hDx;=2}KdjDuflL317NeP7L_#5|Ia<@gk{13m)6lk#D}U5( zgn{0O8{@|6LnpS$A-Hd6`IqY7*b=}tQhfe#N2UNFI{=cQhw2rPXkHIqKQ8gr9_!C1 zb+6{pI)1;a1JC#K+N%#*+P}Gms`~q^o|(0~Wo5hhh8j#FYxF)j$j$xSf@=qB9TqBU zG#4m64-aVIPJ|s>{lBT&*@dmNiuTJybUxnix7Fb^Mg!el0V={3*;BxeYICReJsx>u zq8cI4OYs4K-N+EP5-kF#r~Zrb?w+c&Yt*g$Ye*uf-IY-)@WN+-`0Bs^dZc|mk4u6R zmJW@5HRdKO?_9&Yf`EDJ@Bo0cp8rm}jn|&tS6vz9J|1vTaHdb-20<)XOYR3q&aCU! zla7O(Ett*idhp&&I^x2+ckpt>`xsR{9yoF0D&&=x?fH&OL5Wx|Ii8~P$Opy?|JJwl zwDI0Z;<+n1LUR0|9j*CyI?ri3JK?z`O&7Uj>%yv-1sPYMIi5TD2g&))37_%IS3%8C z_8H;7EpL8q4fRxT+ju{I8QNLytDt;XKG>c=u{4I!K5kQ%(K(LEdMfrQ;>KeQ4Xv2~ zvFQYX|5Kje<{=m`ol*yS3AE&uUb(je&f=f9ax2X(lQB{HVZ-(QXl}FN7`>;*KIEsS zwzpbe@A_Q+8?c!#q&W;R*^Ds19gN4;-Pa*61rR z-g+i2-^l7?u&-LTh^ydAK{H>hT(dkS$d+HJmKwn|^`}d8ov^uQmzaf=;Opo04Msh5 zv@y0%^59L9g)8dKXEW?qFS2z86=XuqZYhd)gt{d)s%BO8yq^CzgO^!OLgEVhk}5cv zq}J5c>0}bl>{2B(wQ2L}Y`PQFVwhquMtO}zaPVi=RM35v&GC4#bwq7!CBgw=Kf>ar} zz)v?_ev|$(E_|&8&cTUF%^sjVu_+3~T!m+$e~^`Kqd)=*ifFwd`XfeTkp4tibz~I? zo_;o(2BR?|y|6pN%Xw;QR=2Foz3_AinzEEMi*7f`vtNHR1~oMc z@KNeWBrPxkAHBejb+apovNR1kuZUoW? z+12Gua6`{+m=OCV5=1~sb^xgMiH2gp!mj`kWeo!i06;mgSpF>#hWuL8Eqc<+rh6*uNi=N13L3z2(sKxN@dT>cT5yuFrjKbrY(zH>_DQkPy zR3lVnXYdQUmk38D7@GsM%cy5iDaaJk$;L~a`!l=9J6;w?yq0GyA*%2A=v$khQqeN8J+}V^SUls9sUDPpJImSm>NZ)}Q_{x5XdGGpfZU1DITF`QY zX*d=Y(V6W5gHW)VM%cl$tWQvN)asMSXZw&+5zCKt=K5I^+)xCZmyMu+LYnFY(*vd+ z)RV`$s-*GD2t!6nNT?w6o&Y^GbxUA4cr16k?ANfcfq+!wiKD zCQmH0(`|t)3=nO131ZfoQ!2?lF% z@7buniJIaBiAeks<*J%wy_Dly2XI$pVg(MZr}*oQJ^fVn!KB7!MjSvWi@lBrAUT>~%R7hEq7eFGprC$)qre&g@n3qkVhkzxF(^{QEmd^5ALd0T{Q z0J6T_e1uMEe7C4FaIV&hq&;GW05u$Ntaw4=p{hssiKfm6^SgMTzAh_DG-V`aY2i_< zZuTTH$Ux|Z^Y8^GV51I!NnZCZ=>Clr3siDLt^BM48%#t}{e4U)noY5qp%0eN5f}T&7B5`OKGv>gSqQdhyDt;;cdf^91 zhTEv=hcw5${i%LQK<+cBb9EQNAqXHTVT&X$Vp)43W(pByPf>QN34({L6+iR1_9$e4)rWWrs zw_%(myn`4FR_ekIjOR72rG!W~qgf)9v`=&dL0?L&hg(Y#NA z@RkGbf`&)jw8PD)@ItKVj-5o|GegO`iIHv(ND9PT&SYl}l%%99!Z}_?`ePGA2qNo! zB9MlC!}Fn|I~r^aBgAqt4EYG<6;y*ofoZAGpJ9GhC(B$ z?Vc*OFg1`P<~t!bSvLH|bIT!=Yj`0lu9*75%Z}u{V(do-Z%$o0;*GXRmLiX6syAEY zGu^pfn~7jE{KxJ9uQkO+_Ycg`Nm%+0)lGkS0Zp+4ZWp7xn^$2e_pSB5Y>JL#wU zZOd5QE(dNFiGz=Sk2sUHwR~1cJ@DFPHe;J6cj67W>Q4ZI@YJ@UNaET7SV3mfCtmfQ zY+6yc@HSAZS`rqmMP02QVwI&Q}FLZ0ZTC(N^z$h-NbEl-z#;lk;kUmS;uxo zFhJR@r~g$(Sv$X`B#f7=4S{`GQc&WbhuTh!d)hUmG{82%gcMG%AxE~4&3E*bCXX3} z&9Fu=;ZY(zC7Jp7szNreVQx~GHopX^lfgtldr3tX#W{CG{jdz58p6UEQ(7fdoh}rn zjmYtjE@e@bj+j!Trk*O4Q_qh|(~r!ES!B+s5tVM@9!HIo3Vk%RFs0U54OYoA@yJbK z47Si8jvwJ1+GC}-!vSUc0|O7D24T|uZ?zq!ynbvah{KTv94CG-aXc~_W8X2%U0?|! zsu4uk(}~=JD8nIYt{f?~mW?;b%95fUgE6z49jSM0fg1sJw&&{7bF(m8(}v;`%a??6 zm-s_+$*qN-ft)PF(Xv=Px-7=(j1i+Mnx*6oZuNn9Qv>_8>*}=Oc>-f#DOgq=qj3g9 z-N2+(XiXO^{=0rfS9=P$?g`YBI@=@(I&{x<4Q=$|oicDQ0XFPQ@~ysCvc68WFXAp2y+)_I}xWdcr*@KHhNX&&1n_OE|+2 za!&3Rv3Fw|OW3R}2rwaCY%6seXdUOJ?S^0^0>&A7t{&? zDv7neYk-SF{7+8&I6#|xzHjl=vSe=OpQGqlc))V=+M`JS_1t2l3RO2R7ZT`-|7yK1LJpYhzG$vyfrXG zFx5_zA4`<%nn%RTA3zVePl(}J z-)p2M{{+Z-22!;ijTFMn5$Nq58W%S3Ni{QdbfM)fifm^zS*gp|4pjsR_Q=2Ov>1TwbAHy1coKk_9%}&m_D9}mzKdexXL*)<106ag03u->a6uMsp@nO z0=Xqk@n0gZ6!HvLa9u&l^Vz;(I?DxtNF%@cvzER6776!*_iS``Mu{tVo@RRT=v{Dw zlDaMdO>3Q0N^Nc4PS7A7UI;huo3`4KwU@yYJR_}&%FCAgpTOSbIVj!wo(jyrER8~N z+?_c3dYcw^R@<8ke_g}2vCG{D7oCxe^}g0o3YTsrf$oFCLT4AAV=SC5in~QLd79}AQqoP!z)^Gs{1Vie==jhKVzPc<&^2;(9lyL zQ1i=uY^YeuGY04(d*RNdBDwQbX>1;r%2PJFsZ*mfE%dpghGD=~!NkB;xpj&OfOBs{aJ`^!!-7F{6K)0IQ41;>qlst3g5{PS^Vz87>X;t@0&sdD_RAC=He+@BKHT z8WUPY zfNGv_Xyb~`%s}Uszr&gq=`M&8x+*P)!}LNTdIBOZ!Se^1?*B#sch5Zf)Hmf5q{t-< zS#7XMvckJ9F0)bjV^l@~1x6*TXS9XRLGYr(F&NO;qe6HJX|Dgs-~nV&BfvhJCtjZo z+5kW{zQm<+EL*2Qe{^Q9Ju=Hd?KrvX700}QsPwn2k zjbvg}`~@IR&;ZHiRAH=;B%B}M_?<%$z!zr%q;wh+d}98!gPi7~x_AL=P8CE7A8X+Y z0d~S{IRm-=e9BJs?Ln?_o}R)Wk@<_pwJTX>G;-ev;d4>`s9~i&fwF|mzp&&@r*vF`PLp2VVtV|e%DN0giLNGh%ew#|x#jLwKKFZs zO0Rpwv62)-vF{Gs(Ph`@#<|=2k7-OaE~9%wn2wLpyMdeL2(bj;G*AOlI5mN@oU+@U z1a6?{RehyY)NAned9nnl5x>YRF80_P*P_!eYln1mxU^CoU}BBHYR$DhN+=YMP`qx} z+OU3tj|fmc@bO`&!q3g~?vMY|H&wZzfC4*D?FF#YE2BL~m;sD*%+F zZx2AeZSmP>ml7OY@36@HJr72hQYq!~F15Iv@@O_Isv#;$Np?$Z#`eBy%+}uN*gfu+ zD^vE{;mwr8>)?x0vbM!NSNRN{&=ko*A7$Q#=ucZGg5*JJe4r22BnQpo7^OiroUoyLqqJmCPLaz%F?Gbu zJ$B0Q@JkKNMRC^-R1nFo@d;Vswqh04&u64xLrHNh706W-#AQOe>dt#Bu$V<F638!8r`=$o)m@f0|#c)2Q$J2gg(kjIj?<+?XRq|Kj`BJrEJ zC@GSk#UL&Q6+DQOVex6FI=4z$4w*nR#auEGPPcc z+HHqQfR)g0ySDF!t$?DYR+!5)3gSoH34=|?9Q1ILgA40BaT#dT6*~`IluX^$JoZzp z7j>eltA_Qqn99|JgaTJ&=FohYOj0eFCg|8Ix89~{>VieuDPZOVw69naF7yeAY*7As z6Ra7ySO%WDl6LH~LR9({>-78_{0U zb*<*5YV>(BiHdK(Xz}0ipK6-*@5vN4+#ph_#d%J#*5#}?r>yl*Ja3TQ(Uj!A1#^&G zK%RJrUx(e-A*FhFa6oTj9rnOCfFyAj#m)-t)x%QrJ~QMh8;B&Wy6;MLX}b6_`$kh# z_tOphm~M9-vhiZeXk%igb}_KkNWl?8Dq=uzaL+`wzl)LM0AsL%7o~PWVfXt0p0usG z_$)A_;Ks$p&s;U*ug%%r#GF1H@$f2Cu8ph#wfT0%qcx?;zUe9$(i8A^%~Mq_SgRyO zBgJ&daq-yhfUY~x05j2Ddk|GP;9Kqm>_}9`sn4i&8&{_$?s#!#<)^l|vHL z3nU<#B*kRzOuq$+Ds+6@*zWuwHQ!NPv?Os*fn`wz>|k^IlSo-`S%J4q3*UTL6)Td= z8+N`HZ5jVWc33>bg)JiGBH{!&-fyeg`gXhDQ#zS!Iv0g^f;u1+C~<=|yHYW1*6=IM z;o|ZZrd<9=@qeDo4DAGDD}~w|i_V)(TgNHwyGUuaK@Yp&gx{;nkrunei6um~W!IOe zBMPR&Cp%7!;E<5Xc?-VnVnAmCY!;+{Ho~m5P}`mHQ-GyWQ|DQTCp#R;i3t!0p6g_J z<_`?*4|wGpCCKz6;twmaG}f9AX-oWV3&PJ{O2u2El4=SNqOdA!x5b>>W1rd03m+21 zho?emWJ-ANvkd3;LGpJ7Mu9H`2M!=5f2Gh7&RX>PC*>cZkHq2e~bMmn7#D-ag`we}UAT4|N6c9uSJkF7OJ_ zt24oGZB91;x5Q<#$t-#5b+8o?s+}q+uy9e`1AXPWy0o4qEoEb}uSasF@ojByCRs`& zgYRlcG!Ft>J~&9Nl)w=xMv6Rz72{z@>Y_B3$BAoHoTb0SpaDz1XmgTa-8Oig=ky=L zir8f=4TF38b2#5IHr1n$4yyY$?W+tJrbvjAIZb+Hn^WBDaM}5^YtL!^gC>+Rx<}d< zJjQF^-)9RGHlXYNbY>^3>BX|_({dcqz{S%VTjM2B8jRJKDPr#egFgcZwU(%9Pdy9pAHk zAC6=xxkrjk2SQ-pWMPFAm$g5caC41Vma^8Q)v9{{96NN=*l$qt; zo!iTSb2Dkeb*jNEnmCxl*`WK%RAiYTp#UEQO@lbJ}Sf)3Wo;aupSf|DkkGt30 zfA|ZKG6ZL#8+f48yhZQy$RR=6iS71`;vODc zPwr^_FeMwYpNcnHU-vJ4=G0jxeScR`nKxsWJUpKRwspW3xkdGb_cK1d95grc3m$fycXrV<0n68}P_uEr2A#Tb#x znFfMWF|NO_;TR`0XUv{5O2@g3TINsAMlv5on?TjiDjwLjMY4)FU`m>cmI^A z79f@SgTV1c^gdn<7or$QsjjFR2Cx$7uwF+oaT@}=itWfmm)6h8Z=Y4IOpzk~fCFe1 zr+Qw5e(uB2STvh5X*Bni(mLy~qzs)xnD#!JYLQxnQVBRNKet!C%eKaC2s>hQtBghDUrC5jd<9!AS_43#96h>Q5t6 zEifRR0BbEPg{#@&t_4n{;$cGkKOAl)tHz0iSmm$FL5WJm>uGdOal%q^Bz7!J4=Q3d zc5BHqO(OC?0o#IWfg^yjBoBE3wCq3taC(l!#)6!^j%~V4gaAdhz&-;$1~h@B+)#&2 z?bb)hF6u$*mef3l@|IoMBvGd*eUsE63#(F0G)d3`Kb zP|vrEP%ACX4e5_k?M*D@1r%HgfpT$>&T3f`7!FJ0ZE<*Q>!v^)eTKyd zuEcGi{V728rd++ak+z8|;*Ca;zztk>0;>~g{TN}Qt*&IUT;YP8WCp3ZfaSCR_EWVV z4988btFVtrkbi~l5p3>vSBQ2AK&MwP56ss?0{ERujCQBWgbjMwXP=Qc8pIUd>yM`2 zcY|#r4$wk9A6h4LS7-3_(}gpIXAfustvG>iQ5BnageFTCrU%<^?ja9vF)1d59o`Va znIAfD6W-J|L(OFuvzuZ91Yoe9{+V#&1$7{^7N-l4bASM69yHm+j51%tJ??Rx3iv#dkW$|7pU+6{p#IH?duyV0_ z;sBvj+jj)O8RTv+_n8?Qagj|R)ls&qgWKywvIQ*&YTLmyajNOlSvC^3Tx)?g0wT63 zIQ*Cg0wq%j_k#+*GkM@((o1z3 z8#i;dOqx$7;9%M5m~}m|Ba#eD%P6*~p`mg8g+R-&N-xz30**SHV@eqz+J!OIHHi~| zi9}iOT{FTw4`c~pBZaOOi1Xx~-UlBuq^axow*TajC)b`bq;UOMKByEKRcT`74WdA< zT;z7-+ck4Pjxd`-IF+eFcT{{dIWlbte!L6Zt%2>&TL)yyJFr3vGaC{EdWjrAOUP=8 zE{le$Tx(4SWpGe{^c4Q5;mMVyauQH`C9-Y|f*B2**_U}Jv=m)7n$7+Q+d5E=-3lN+ zv)_IIk=%HRb>T!&(XQFmX}rSJBUew%v;T(n<l4mqFLX%JE97s~QCZzB;&Tge zf7~cI(GY)zjq~(fsOXn{Gni2 zQEY9+I&~UPxITIL^18fPs*h$%KVq%~+@JrCssBI8~tbDmp`tvfv4hX*&SfM?(Ep0@4O%6WR;tInePN^L$okP~C$WKDI?LvqW~)lJ1+ z{|}r<3bxP&kyZD6!Hnd=>rwrOh=ck<#n?~vXti4z4QL;J`RQ|WL9)_ zXPG>}p!VnU^%h!QADW=V2xFd3U;r3HUxf}WF3_g-WAD)ON2t)oMh5^_J%_Tk+grDM zG`cFbowQR=jYJMx8~UNn!jRKQ5boi3-NirX7yX6W<$bEZn{`;e+?sUvZ+X7C!_E2B zb`u+)&ANHQG0?)mQpn(5Rg$F6YZ|@UEXP^Hnk>uWaK($@u;Q;i@yFYs_X2iQ&=+%8#j&x3{BltgS%!&I+^+ zDOa++2s-r4sys|cU;Zi9_J49;(&xl-h!59f$0;vU1^g7ZOmsOb|APp^9~p!X>9e(@ zmohwQl%gw2FO`YRN}WCuQU)u_0)MHgaU}0mNnPTpeoh|bFc;Hfogj{@37uOy z7`?kC57(XqLmIn?8Ze8#xPLLTm`&9HK-Z+WeRS6B>YNx(#htCABtfBk?wlQ9>ezi; z@4u*{S^Uf&9S&Jd>JVaJEVF>vVpsNyt`(gQELi z5NYCtRZ>lVgJt!*aQ}-gkNy82fntiFI$z7QaRr6ydj7uyCCc|3sOB%d6%H2A!Pa)Q zNMD;Xqn%ofrQ)Lk{pj4;2`I3#L%*NG)+0D)1_yFB^fVb)iZYgz7+Otp2?=+9v&Dq=1kWyJI-!P&`7h}Kt5Y1 zj&?em#+2Fkx8AqnIi7%<$*hkhtMP4X;~Zw1>l*FB$&5J(FGGCsEB$P@=jilMl!69_ zCY8FLb--(hj7yjNp1N+iCj~T(E_f9E{~6ooKW(_r7<^eVEC7IP!~e1k=VoH)#bAcL8SiDAj*?h<~e!dRRpD7O~%x>cw>=W1= z2akP0_H(F^M2NgN4Saa?&-9uo3Y$$w^|W@YkPA0MR>PLtt}cM7#xjEk#;DJp+9NFb zFL}{cc)%c+Z-;Pq3wGGq88b1PxQAWCk|C7H+yD%NIio)v5oJ9jh8aB7@CFC4S_#{ z;cBw#s4K9cV?vP;3iHec{#$UWi*&!Bqj=3kpt)UDN@Kv87%`b7GaHOf8lVb0#pA7Z zdru*y10XJ{4xl*>Z=#nh_Tb{QM~w&#BfOm26o-xoq-?;uxG_gv>J8M4^0bjly&pp{ z)~mrzDiPpFf)DCO{s!W;E-@U5lyH!I*TVz$&#xUWUAs6ouY97?x@t0lv_`LT;0dVQ z3{6~K?jA4$J3$Vf=FGm{)|icYv?_crfJF?*V*?C=qLorsxP(q?clwz7%k=7Hya6#0dt1`cioSan2du#`FQnvj^ z?bbK2!L8y>-}nIXgpAAvQx)D$a!kpnblU`-Ga4KUKfAnhgkd?+a=@&|xs~?oUR!Z& zpEv|J>j|bR+c?TD8c8ylI}4i-4^1F%9E<$E-FBG>$0@tUbK&?T{98vp;W2q=e8jDK z;`LslGYHqvvyuP)s^vcZz3(8O1es+5d~m9j^qAK2ol%F()$>oPN4RM9-BEe@i~34D zp>6P#K=7mP`%3aS+AZ_)QfHBq^`M$QouQIkslHsBiuG55Ii%LgT{4^MoE>0-p8*m9 z)4eTUg0?N7xS0fzO19TOIr3771m-(1_#s*(b6~8+$knkQ$R9OT zhd-7G7=HNdqpA+czw-)qc-Ny>|0rQ3Wet@}-|6)Mqzj#h_rPP|3B>=Hiod#hELWk5 z{OCF`ze9$tC)3Wp8{;P(5}(4ebC2K7d3d{_cFWNTj65eprXx-xJlh}9?*U?3XU#hU{#|M&}L&Mv(<7%aO%5m~qSA-{J* z6QJ!=mvCD+0w~Z9KlY{CEeIsd5Ne^aJd$VFO8m$UoGyh^tpHRPhDKgME+Y$dMgF3A zlzjFpC6}{CGoTt4u3e9xYT@TV1!!Cb880L=Ik7Xtc?Ghoy>X}X&{XLnbta z%ZPLVIYkNke8iqTBi6CpP%}UBg}31oalRWu7m34=mor1Xq>>!M<*MNB0q{r7Yfnie zm@)~5YhO4!s#L=TVKi7)KXsGg->z0+NPpJ)^E||uF88ACS$gtfDtdxI7Qw>jJkZAh zzItacAHr|YKI_VE6%ChCG`tW3fLt2d*YDM|^!(i(*9e9kd&kh|N17 z>G0gxhC;GX6<#X42zvEF*pM9Hyt@P{@*+rsb6RI?(JFU>1W6jx*@&ds{~mCoX-sJ`YeJU@VehnX0*!qepb00h^Zddo@*CEDS< zEW~TTvaW8wTrExdCt&4!^Xf2ELbN<>p^mcGD9|5wUqoqPDYX-(gyd1IpNou~qYGx+ z1I!QLIg|v_^b9zhx#BcTMuQPZ6Se7$)~obz&DNq0d*pWQYk>qVF9JO_)RWS!7&v&B z4UeSFCliO5g{8!O}}BzC36^Roro<^uw09ESc2JQTsBc&dGt zxR7G>I%M<`_kQJ~sOnW80p)X66=1ZYO0EKne2`hn4M{Ii2&xu}J!{7-5p|Jb?jy#- zxG798^LUtNsau4MhfYeOX6pVUvH#k(C5HP8KHqj3t>a^zehLjszBV1@2K(o_%skqt- zRK9i3A1>2IaY8W~b`6-ON+u|K;IK3GWMI_@dl=`wdugoJ)A0g-heA>b8C5q$d;}X8 z0)+nT7GZFo?pR{d?{IkfPamf+vX<4TkQ|=-l)Z}8=w<0ek(gDX>a|0sT!oI9as98AIT6N?ku>#9PRaOYyB!n<#m-WQ{#MADuVC5G??l*j{&)gfQRIBmS^Vp2W} zS~mN#Ky63SoSVs)-b!gFTni7)4NC-#v7v4|KX-v3UG%dC58dlvh*sG>`VW?d56H?|l`S@~{SK5pL@R%BeJpJ1O! zsM}V{TV7yjV~dD5+l`WtRZzfeb1k8%7O!n`a3Oe zaSYiI#&G&cJ-QD?7FZg#gOLU@r|DLEgL`au<= zKDF29Dg?zkkaSc?dXDnX#HHnAOjmCVMGDbTrENU28>2+WtVMKuW7bVhr({2yY`1 zB;8$e<)vQ3W}3k&5c}A&W>R8hR-mkT{a(tZ0C!)?*PK5~FOb^?e8hy$URC{5BANGU zIvYEL18A#L62~Mv)32&C1W@59uVO~ zoB@inPOrKVUx0w>O1blHccXdyZ48>2>kBZfEIaDG*QsR8SzE?=n;0i!N!}Z2V3U3# z;7SBCNRcH4wA;~}Q_Ao`7B*ERI6=h`kq}!;z^fRXbj|~ao1yfriM8~$qG$LofP49g zkpcpnDF6$O;6K2U*G1tE7#*=qzRsgjKesMQ99?SHMv5J^_qvSnu-3^*+ymC?GvAsz zNdQFg46p$uIw2rmzKqa}t7a{EDc<+wr5J)&PRvnC&tkTo z&D^q6kz6m97n@T$C(D67PD_8Rc4hTQwNkRJx$0`66d{E#O8Gl(=Mt?s(`-$e-Be?A z@pRu1_`>H0E_7^k>|F3KU~R*-w~8*;TWMT~t}U0Hf*Ve+@qFuZN*`gi*86eVmEI%= zX7T6EHq|u{-bS=sx3FAm3*tb!V9K3}@`wb)qt#mlo2nODrVh$Dh>QtUjN`n#sY7pW zqeZ|g`(L3@5|om_aeo9uM~nX{Q5|?y7HYnW4HB7Q>OEMf6{(lcaxKR`I=+SesSv?% z6ExUm;x0DnWCcNISL zW7`iv1N>ssH=@o#)yCuEjtLNpE-%72(=w@NwIjaq5$PuB1tEevIuQHOa0Prd0mb_U z!1HLEgdh|q#=06G{|8Z6y=bh-F?;Ch7?G#H6&5*d#CzGdM;cRWs=B&KVcoJH-aF1Q7*L8oBIX zqJc$P(dxS%B4&hiK3%qq^d(awq_DUIdTB@h_G{LO<`RI)cWMfJbmBfv@4e4|&B^~g zaJ1w`gkg^|GAL?F_8rDiB@~miMtWFny z!XQN)RNGCT2a_lS1tF1Hz<%4$G&^_~f~17TpnM6$_Y1LNksaGzjef<4zM09Op1cR^UqMm{1@l5$ zEMvG66|`87amon!_kIA8-vUZ1CN+HObqRGWW(k<}%;8=`(^`K4(IW}j>5Q;vYgjHx zvMvf45xp5IwZ2LkYE^E5&{R<2+Km;3B_db4Vok>E;Xf#Vs|qUTk}>G*p&j^Nc2=$0y;n^r)k64A?Sc5lq{)DSuq zzDf1k+7ki=6!}B7%|yR4iO8Xebw@ZG%;sqfm&z_j>&tY>(a|98_YP+M9%7Cj)ti*jZgi&GeFioh(C zAUGWG)~P&zYMb)D1Phq^0v!W0NJYD;NtR*zyNFH2*L%AD z#$6dM*q_E|Y*^V62)LC$UUuTt3Lp~PqQVqx4vdN;^aTshwRksc#h2ZNFhJrB*ld`C z2^94RnsvRgFMSe!BL7aj!qQ2{)xZQe!*Y@i9m-5HbI#qNkgbShbEyBOpff;m6TF|ks81EgoFI6(p1Z(@$6@Yj~=b$dZzQP1UzyO<~FI2cU%k+fu zg#kgmP|v>tmF@^}{hkdVfLz%hPuC&m$9_n1Qr=S?te}EBH?89dlA=&qeSyQAD&>Us zix1lgxgjVorS#c@-f*{Pbx3c@dQ2t@Qt)|rOJu>MXJnU3 z$Vt<83u|4y;Ff(YpH%9+SaBYMk^_pxH#sX%`hzx7<&Sqiz2{v(vWKLkuh#D~0Z{!% z9@S#j8GE;jdW#FS7J<_efhFJ0ng9R$KiP?0VVL$xAYcwnM9aF|b_ z;jA!_esxLG2al+KMgrzz$Ix3%FJHFx9=33sg@J99@e_fNEe0e8Gujw&1w$BT9mhZk zpD5lq{7^M`;o$auZhMKn)7$HTwS67GhlewZ7F;2bfCO{jQ=-v|wx=zf{=wN<#G_g) z^+Zq(z`l-(O(!Mi3%BtjVqXm=dkw4qmmsv(IsFR6{TTqQO*Ed2J0&r8U8S6qx&H}- zLZBjoH(MAGo?}~;h&<~Jy&WWL4^I*UplAKBe=XxWA125n%~m=tWQl;T_s8n~e%ReN z7wkahb!u-fZvk;a)L1UGKEE<25Z13Sjfu2c9nx_jE$fbi?;wL!70dgi z>V$Y|qaQ1BCO5Oo^X>kP4*f=#x5xec@@z=SKB(r>*Jk-RQ&`5kghYNK&czJGWL7NR z-C(U6&u<(q?;v?!_c-}WF#{IL@R`JpnOb^buQIosS(3howK)MJ-4A6z1EwkK#c*H* z4tU>X3sd(wi+w5tdk8&wap?ni)4MRSck8$3p2V+X+_3HK?D6d?koSHj;EZW?ZcN5% zu-_H_fOfnxCUHY^^&$P~pzHG}JU`?3sK~ItV*`m9PNO-~v9*PtyZrN@tHcYk07pp| zm{+Cy1*t_t0>Qx1Flaupy-?!WeA(&!adP!V?tN+I?_E7aYzO25mLQa)+D2)izWH#9Ak$Mr$#qEyrWY*4 ze~fwSHq2Q|%zGvDRTC!^7Kk^_%tQ#;8b4GAOfOk%SP$WF%R~x zVg;Q}_scP6ZnYGc4x0!zJ0?+H!WS*)ThX8qB98}PSYS@=AgL#Y#lj9UGfT^%NrMSU z1~fW6j9s+EuBa4O&C|b($YRoKcv=K?aODVrR$x=v#qO5u|Xv`3eN3TwtTb-al37 z%&})tL@uzb*!C}D=g(0zag1=!Ph)1o1tq&{T6CpDM=1PwTKv8^TDQ6)!nSk;F z=g=wg0_YFBhj!X(r<=8XbSS{y9^C!z3f^b-uCa41t$pT+kSj?fB6U){Kwx}iCqjC% z3dne;!Mm3gFTbzv4*YEP2lo4o9JiwxuG^$quBnf&7OvwajXF6EA7Bmn{3Q%Z!H$JU zKbfHiZjHOcudy)dzUHf3BN0vm@cem$6LFN;%qaQG$7b0USR~uiFpZQ>RDj?!+>vpu zZOcZUYxjsua~`bfsn(YYSi2fyItv164@-w{g@ zF6({b?yF<)IZhN>;QvAtMmuO{RCx%?0S9sOaMZfj_jx*uc)fPItsnG_{TzTIZh zMR?v``|F)!E51y`N2HhJw}D5y2ah&9?~aCv=@JrPWM6uu4)T&+r?7~X?6-yX(@d)G zskvB>h+8i2PigbfmO+F)J?r{+A^}m0>v}BmbUoiCsP#HDhFO zy~}66G)}zx#j`ds=Pd@RqY|5#}YK?o+OXk^^H8*R_FA-xO|9JJq(_vuY&X}n!o{rfV8{Q{&Q^CieY^iGb~T(Of9sx}>s$+}R) zCuoYmkDKOHu{&7(vb_ zwTSJq=Acgc^1a0>(?Ve{3?34WA7PAK`~C;$zm0YqezkOD3+ss0fdBwbAOHZE|JRd3 zO9NMf|EMw=SX&!e8~l!VooGxQ4Qxzk&7Ezmi;`uWGUyRPZoW~mv(|w}@dvMr>6sv1 z7d-Fx!X-F)T#Q#SNb7GQ*8ckGjgggY6%M+}!c27EG-%R5{FM6PiuQ;CzvAiW_MSdp z?u$VT<~^E`7dGNQt-y1)tObfQq&ZD7je!&eQ2~)GdHM=kD4^K>3U#s+&B4k%NxLf> zYsA>*BO@_?5|0E>QHxqf?6Q;TJMq(VTp=d15{o1&ho4X=FhoO2OSKo&yX zurY0_I1io2*THJhX;wKUkQV8vofN(B@f%rxQp?$uEHOVwhMjW43OpnexiLyOlYza@h5vSiy~gi41&M&b+g z$)r4kSx>kDnJ5}sgJ0XPMvCCp#=ck+KAw5gx=+FZbvLg9y4wy{>mOkxrCmd3TXDUG z=I=09P~{_CPE|N_sr%^z>v3HAYtenlFKTu%9baATA4BI~YeYSX=YapL(GFCW>ASmm zjMS}qg(p={zeBmimyKvm4Q)STuX7HHn{Y-pe0}8vHyszT;cySJ^Os>al7zurv3v50AT(9l&Q9M|1DRelXUFX=}|&%KB#1Unsi8xiA|89 z;@AD#RlfL1uNo`HNJ<8)E(k`Lr+RW=Eug8G+5}TixNq*l9r_RMWHLXqWp~9KT_6`a z8YXfW5Mt-`zEbBM7`HzWUbARPBE}M|bj8Uivb2#2+Lk8>i&RwJRCeeV1Ib#LS7wNE z^C=`BnX=+fDY&1xTZb*j(IB$T(5Qg}$A-dk8--)2t$LjjOH+qoj#;?t(UVary*&g} z(#=obcRr_cb`Pmi=_I-X8H#7I`AC;GOFPuLz2&S5PPn99UK&hmth1xYZ zp>h}G7 zd99JZJzcU`h?qL8bsE=BI!7%#E@%szs}R$eL|iu}JHSR?C$RfWLTc(P;vJaqskFjf z+!RLBA5Cz%=b2RZ32v|c?mVnO?IKYNd81_vGXV&X9CN;(( zDS5@MhwnI~ct^oolRlm~Qo4`Bjq@GQ9G)DfDpzLbF_zpVq^xHzvNCu<>jcOIENs!! zFw=m4huh|DTbD(<3N@=<@qBNh|>X z!1TY*OCvkmUu9ccXBt-nYZnuz|FeKzY5O_tvm$@<}B?>1cdIl7$mg)1}xu2`7+Vov)dxSpo8X+p`6%J;YIXAr1qS9A#+PLS{zuGSB=C@Fl9u? zuW~<3mwG*j<*`CDJ|UtisnX8atqg4!7!?kRZueyW#qQjBYWf}-FSpuAW!=)6KMC~8=LWBD-i#-t{@hFE$#9^aMU-tPx zrr<0**WzWQ*}+)T=cb>T&+;-qY=g%4dQ!@E4)I_d2yfH6gN9`hSc-FuVK(tM-g}q+ z{pJ=^=>f4eLT-SCuccY90n=9PV;uMxxpCKx`UyG1*u=>#6McRnSt?B0$xH)WQ%|?s zVe>===Xob(!D}RE$&*jOYpY!pyh@z7mJz`osrkhJj*fjMN|)(29Nn4unnv~b2Q7ZS zZDsAzTnvh}KKnO^q~Fq$;ZPs}T0D)9SQNR>_YSb9D+C$(wi}0qo8jRh?^LYidNAky zYqZ5cbF5{p^S|Y$Vaw0cQQV-njJVQ(EyZx@`fFKC&et-=btFA}`|XW_)eT|(M}K$k zLpc*hHa^2~q4v{Ku*er*EJmRNUMJ8p0%)LP2myy;ZJI|vw8tP?E{)%HZXG8>StNOF zA7%qCqps#eBPS`-wVdPn?n^(rgUSrSVC$`$z#Q7T`0kyySVG^qe+GTB-x%1pA( zgMLu^jktRq89226hEAjAESgyP=i(E6!ZKm?q_q)bU@M|@buOWq5g=?S7`?6XcAO0=3Aiv7|VN}DC%tbQT=beq9nI{5GXZ>GUDVv zX2$Gf`L@!9vC){}hgV8*yR#;T#5EAsI0BETHj(Zkhi}eTdya8O1WE}nK>qHBHtASnyyPD!)ZUj!Y9V0NTW&JYs)xZ7bG<`^u<7#- zYqlO?^So!9JVq|(2qUFba!KL!8NHF3J&Yb1U zIB2bht1J#)toUoosr?|45c~IoftwV9bZdXj!K8q+w{p0i>b%4XaF6BOAuM?9CM#=$ z$(F&Y|jkGi|E>Gn`vBtP&$RHTyh`HrtzexjdprX5 zYuBCSharYQA4zWX{Y4vta0RxLNrOel=;ijt=s(f_;a8vG;BJ@lQF5 z)wsU2^UtqM8zNSE>h?3kakt$C0)Y24+B}=WMRt?6M^;7+WP77`0Vl_igT0aXoJq6d zqvE1rlJA;h_n59EoV%7*@Jt|31Ci48Y#s)fs_Ef2BTFZ8Dv%f`w;vmoouGE+pbn)q zl*3R3l~i@rG$kCbDMBbB4&UmYt$;}Ce6)@RO#3IMZ7$V3_#zPpeLB8I?ihC$aV@TT zr5q+=z@!BAII6MMT&P+-?Ww4eISnttux&#AE^$u_vW*lgPtnOx)uRAK#}BR3 zHV?*yJ8BItMD)AI07WEn<|k2n3LOovCj(nxwBRXf16m65Ns03EQdIi12-u(6+ZGL^ zfN;F_(#MZ}*}W?fWVTp=>T$uv8b6LBc~ot&_PE7}=EMJNlk&TpF1SZWMiEq|3?`%V zRXYhXON=*qJ%PRJTD}4zE$Xyz^shL_3n`CCcPUi7va_Nw|KR{}%afL3Asx)0e&QsY z6m{EEzC^*;?n0eES>*s@|9O|q;>;j8jkyoSAL()egYhug2abP2>_(7)O=lL5s8GB#tq)u5#HE#{7Ro$aWJFui$~NdVBfh2U-8wC1koOrEz!< zLct$D)h7dm04OUN>5V>OaT^ID70wi83ABp*x$;l%tZ>Phcb#J4i9BVM?JUU{DQRyH zkQkab#zl}SDxlD!hwGdg=LYRQ0yUF0>Ys1*X;qHJMO%{>QI5SdsDD%`PW=Fs(XAWk zT`fZ-XgH{;vd|N65WAXdXU<+y1f|ON)54*Sa0p>oHJ_c#@B+=liH|#W;_p2*H?|6e zH9!}8c6LQu+uc1$>^fjv78Fpx66kcnen$|7LzW*LcWF=);VK4yy2CHq^j=JThcbD? z_N3>Si*kMlAJ%J9#cY3VE`#Naa#OP3`9n%y&}NKtiFfcOZlBdd?9S_8D8KZ^|6vkE z>n#povVx@ro-hr@no&SmD;OUf=u1S+Z-4Opj~CGehFgxBUs2b?UzxA}7sn`LlV7N@ zk%6;`F^#o_lk@*|j8faOTNg+0JE=)+hbM)91`~8DL2=rzSjkl658pV?&Ig6xHLa`K z0*#RU+Ob=z+f zQtgzB5SNQy$}qa_85ekuHQn}M{V#d`DI8 z7^7IT{3r&&vE2&^qEK?6`g4;n6Po^(NjF0kt(c0Y+5b5%9`e2G6`JjW*Z;%|yaN&4 zn(4asaj?V|2cSpRru-wD5c3*hsn|$;oI#lqOF|a{Q;B#WI@Wy93?UWyM&zC*0?h9iM&t{L*l zG%=Q)=}I)t&+;YQ!Il<{LLn`zZuNUWGpbB8+VRG;MA4ch&s4fkdFyhY@P4r~TRxRN zfXDPDeFk-DPruJ{HGvE^r8zCSxUxR@WmjRiyqTjnWG_Vkag~sm<}7|M%A@a}#TOn*SIf z{HFnRrD0>Y!H)Eut;bODQ46UrC6jT5>*)Yzn_%m-LMCuvTbBc+iKS(2tRN=o>J$2P z!zQF?*F6731IJEX3z#5&IBCyLHv?1la8rFvYsR;uH?Xnx;b#AH_tcRg1u?Afcs68$ zqBB*!IjHe^iDIgGD4MKzL=>5Su>la+NFAxB(70yZ|2uv^GhzNfC_4JX70Flx6C#lY zf-ZKl6Fp)dnc-C@j1g<$Z$W_k=7^*}jY1hQi2OUnw^T3$dZ6(}w2Cjqh*>FxrGYKk z%Na1snek-@^h_&4)i!{c(d%#M>E!&_TZ&Av|Dq2%nUU3N4VKg}b!X`631~@0J&E_? ztBtWOi1c9tV<*&R)dT>Odzj=Lpdwr8yruz3rY(7aAK-q$sj#3AcyNNcazQ4j4 z5e-SO)OxD;*LAU)1rpCa)kMVipHq_cSx$Hp^>{V)b|p3}?J9~GKc6GuNLmCaHt%7u zU>f=BldZ#N7qgL}kPKW}VtfCMoIYc~-5KQPM*Y1UA+rq?3AQSg@2%t#8sxVnr5{HG zkyXuTm7H9gB_48H_bb=g^Dc8T7?Q;xH!*rVa&!4S{fecLeCP=x^1n5qJm~LC(JKa^F_cKsho86P z4-}COp^bMS(f@Lrv^A}OxK`qvM>;$DPl6YV3w^HWT%2`HnqmfTWz*|54x8g|_Dhy3 zRS!ox-wb?aJO}J}s$(286ky}*9@3*J_xsU>PfLqEc@=ezDhK}Qfy7_1k84VYC{gaC~q*H$Y%Y) zY_2$ke7(E6Bw}*pU;fq!)5fqF>o=uIn^%hFcc0D@nTQi88XHPMgE|Seibaa_%+i^H z!5dGi#K7+0g}j}XJ9(j_KNAizJtAJbKU-qdfFVQ@;x$#7m%v6a(92&|z@kzE$qjE) z*=e>i*q5XbrTpv7#z1^Pajw)7|$;Ox2{-91k@(HA=#Gtp9P2A6*}M)_RE8+ z=jXi_v!n|iK`taijiU%8w=xnqear)jGR{IE8FNiT9!iV~nqU%oN8WgOTtJ_aejziu zv6>fbakk0~_w7A;L8pW?D_Knu`+P5le=6kr{p}Gpul4EOOvnLl3_+?)1 zvbx57aMq}9t<*4}S)W{|E=#&?mv~f5o{sbDomFH8=+G^9ZbOp-`0fnYo}iXM~drzY87wN0$D6c>o$AwsjQ zE7j=(&1f1gbx;`N0XrM#BwehPxMo*vO)t1+yyP6Q&1sZV_-E&sObc2fVYN*q(Nz66 z)DrK8GqUdypW`OcbmJA1s8s_M4%I`;OSflpgQ%*gO0Gv_(b}rSc5!LCec0x%Z>=op zS=2E35?)DGkyvSnH28D-(2H8BX<|`tF}$lD4m>L&Bug4UxG8eX{jp`tc$jd* zRpF36Gfo6ev+B|v84wqHjk;OCSAMuGif1mtfuA@jo(R>68j_7kFfj3un{vUyU0ykT zH#4W1AOa}3njFTZt5ByBsS@$;E$S227A<4b;gEirii*592EmL1w-hJA>OWXyRJ?AR z18tnRHLlv(8S6P&$2>nyD_j&!EiLtv+bSj(@)TDz9Jqa-{4IiFKK$DpR1+Pb?tiPX zgB|_2v^mA4fE7T(xn{4Rwqd$Y9)Sj#KzNWocuEVBr7@OvH?`+m9cxde%Ra!uf z{xu~9aQpB=Ut&rH-X)^THUW9e`JE5_lcm)i#DeCszIYah{WJ~w1Pt==+VOU8)|&6@ zfKqM4n^wXcS3@DU1-R%}(iP{<#w=A$g=y>+;yi->Sj!rm@oz3!vD!jRHjmD)d3BY} z(*u>}JVP^GI;*9-^n_50s@+pflxwGEDr<0pgUWR3);5geuHcqZOBUU(o3mT%VUVj> zjJ}Pe3kI_MTCkU8F0E!<2*R)C5F^!pJCjO-V-~I?kZJ$sMfq*2vWUSqks1GMJHZJ*L9xw0B>qymIZG6i|rV(G}+ zKw&RYP0yh~Sest%IEgw#_vVFq3;ENT;g{FR7kgRzZ>L(!0~Ex!Nnfq1nGQgh)-JkEfm2;uRV&&2Rn!HQo?-UoNS|aogRKE_Wua8TTnO2HS zlhzN_q@^9fYwkj{82jsWzr%9;D9JF^Pyq~nZe?JZx{@wYT2T^4f8&Fx1ts)q2XkSm z(5BBl*TS=>L(>eB2Qo8Wty3IbV1)ywi^boI1xM~y+~yqb;xR9x#5 z@hGZe}Nt);g(3TG8?L&iwAe-F}sXEaZY(JYiDh<1jd`>Hg%5(P{Reb0O!1@2yr@rA~cJ zxyce|r88PGCj0NwUXivn))0C6mJQAtFU3}WuD$0SZiPJW{8W-y1_G)KUVfDownBCB z+kghV=yiUsGfM!kD3j-j@v-5?T6tL=dAOC&z=32qd6+nR6g*1=n0)8gL!d)?E}n_u z>Nbn`U&ME70I_V{WbJYJOPW8bb&eBfsl-TRl%&n6aFnq|WY*WjO__7=Kvq)A0=fLh z7e{5zu=`8it@kX=tGg9y#8_S1Owu*C=iYkv`EuUcRjFC2f6dRo*`vqk z?mc6dWS3!3)tiZ^TXTPgm|0tlT(I$^s3B*;E9h=3Ue4yfMqh~eNTkzsMZ#ideJJ`L z;)l=MRxYnSzLIUO!tAQX#=#ihTmLKuo2WS678)gpPHV*!1!>ynCkm7%@4b6X+SGoPm znN-pL=S0?5j;=lF%7zH#xcx$bq!EyMmJLn=vl%w@W~zUROo~Q2qS!=4>r3yg*m#T8 zBu(9YpmhB0O}A%}^9xrZ9NQuvuFZbxxvlHn-P7Xs2bNyU>6eesp?l(a(MFpcnCYF& z0|@0(j;rUa->}AMYL043v1U$ZdY0#6tWFi$c`(|t7~2v7R_+_lsJ|yQ-Wztmy;*-B z{;>hC0Xa0G2<}Q>u6*7R=FisCJtW-+dMJD(`e>*$J%B+rdN)t^`}c=Wm@QrzeRg)P z+J+0EQudt@*@ir|yg4m0VM1|>~W zLtU;bwnX;gu)Mxu9#;v^(k`daJznOi^Xd=G5V^cNEG{2kM3|VAR6Awx{>=`F#c}3% zf7gTP+Jm?iktMM-wQD$Reh$l^>DgP>pX4hF7n7fGf`aMT8mAM;qbd9zQVdq*isF`T zeCSb|l0$rC=p#D0?u3+=V)SmqFh=wMji1}ui|F~}32t)n*kIPnW(u=}3rL%^+?K=~ zu7x>!pnW3(Dh|w-&2as`NxT!P?xEsPOMp+Td|#XI4ZaruL$q530(eyrBdFManG_pT&llp z2eRlckjBx>zb>||b@<4?R*?Zy0Q*!WgWG-=85`fP60sfj+4i0-!hox+S+vBR{&pMR z$Zy<&QK(6p?skvI+B7rGG_D?ht%&&zGgyhW+%d=>S8Op95{Op21FJ({7i~P`!W9d` zbfI5@d%+~MP!(1E=%3IZHm_WFAjjd-?BN-ZcOG`k{U$ZBql{tMa>*Rb3ET!wfPiT{ z1mtnG@94L*FGPxMd<2XymwsaAk+X3+ql%W>l`$wkY|wvf5$bEX8S<4M&|5)#1?dM* zbvXt)?02G`L0WQMD~DkduPn21K5|U29^+dq@~PH*D#V{g&Re)+{TIBdL0~}2q*&nm zJ?GbAKl8xh{+dM4C30~I6V`$OZ_Ws*yf-5&MTHS<6dJ}9doBIOoX47P8RT zc}4-$EzzL04|9|BeN(yBu}QLIm0-K`CMegh*=QSbdy&7yLRSOYguGZ1xq{RT?@a5*CCSmHdV z(r@ot>dm$OBE7ZsbE$OkI^%86*)Pt7;o(Y`*5mK(PLbQMSl6=G^NX97U~II*UKvgF z)~^r8&}k|Pu{(!6iu-VyATNu%<@%X zpO+Lkk^uY2)aAz9vTTPtRW~1ur2An|=SPPkhI-ybd)SEMoIy$m2F`002HOrE7T#+L z?M{t7SG&ExZ&{)RUx@gyFy|nliz*=ELU&l}>d zE%lZ6Vh6u>Q!smSRyFnYTru;voF8ByVgOMAY&yTxLQ=5N3fTj~$ReSj3OEwec*!B# z^AI5$8;c}(RA2!H)-b;?=_q8A=Kd)vZm1L`j8An2XIt6>G7Uc~=+;%ya@ugS0%*^o8taZb&Q%iloAN;ODS9!P;6RDPN!Q8U^w_YLgvqz*#Ld5LmYX0^A4tK zwf>9`0IM||EUBuIoBjKkZK5GkYj<|A`KXT~KqCqwloc-qBTs(9gfgglW9LANJi|s$ zIDg4iodo;kf|wIE3F?bx7EXL*pewNxz>z5~I~Hllst_j_m1K>(`j#0nnWg};;}U2) z5D?R|NUY(+u~wxL2oX)b0H%;)7){2|Gv%FOVF#uJ7@~uXcJ)YzK807*QLPrEZ0AzP zAJ!N<)YIHt0#>R>D=%-}h)2xTCsTC$jPwrlDd(6p0d2=|Ts#r@C-)tsa77ITbbTCCUHq@B18;}0SLrrT*#8i_wI z3n`WkHJW%fMz&9}DQFpPs4yg>zSXPFP8|3s@f47O{E`77zPoo?upiU zTjt*%(bvTAI>aurWZK2T7UCn1Q&FZPv7$}8&bD-cd|*6J9B&?hpm1nxtEpoR41WhZ zhuW~5rUeGe+sXHc5u_TJ2@poQdxERNlh!dt9{+5md&*4Y+S`SaF~(rBODOVZ%Y~?% z=6%hy$mh<09ik>TNkEMYgZ>uRCBcQz z&xa^IRE80iY^K}oP&F`yI`zaU*$3r*p@ z%cS3<5T6{4o95KTAM!+EM0fF`Q@MWSA$7!$&s4KS9{^KW2wY4i<9Qs~K_Q8nppjr-7N0Hy8UvnGVm#@P?S6pK zQ@EKLe5`*iis!P3P}%U;3XI#&_1H9f0)5@g8rY4+vM>YyHmR`%@!lre4KobY7BjCw z42eLa$mR?vvdcBP&s|CliB45lr|!J5gihuvtk@Y=x5EN@e60?JZ*!s~*?lV>2+FZ& zk(YONqHUrPI%{usu;((TJ#Y;Hg%u?@)A_J>5~%SY>g>ebEY>%=x=C+d3>7XYnkC!D z!6l<=jaAQuXk+_S?u&GUQkdk$#e2yF^wv)x%3|Mmwlm@vBGS~~5S+`~Fkpv^5q!>#` z)S{{1Y%;`97cJoNR&Dp2%M@V!`ybMx${^)E6;G;WI(TqMBAFcXfGIC(Z`8y(ODlQ? z5E;CQ=aphn^RLoK-Ix~_2AyA_F@<+G^hayhVr09JYFDb-OPy;~9yQ3=>aZ_t7M3y- z*L8*!nfhi?>jJiG!%)Uv1peyzTD6S$#y_QKx>`mr9v;4*mwPT)WHWdDyzwSKCiQ+= z?X2#96#L*jhcw)}>s&55YR3Ilc~n|1=%M-eb5zd}Tm}E-pwit26R=sUT(gV+u1{%< zX^W#az0kvCS@VTtSjqo4L(5mX=DTv-dHt`gvD-9}kK*H{Ou{>Bwugf1-%TlpbcQ3;1U`RwK9{SmtGTQgnp;@kYp9Zo>l|Y7l!9|XP@XhUL!rc3R5)%c0{+f z4<(IwH8NTw3QxuOAR$95gX84T*2YvdqT{Q{O^<1>vw@BI2`-`|6p!nPNlYW;NfL<6 z@n*P5ka9cV7)U-6u`!zmVLaf^0Vp@95;{p@hz`IiP=1UFwLi#?Ep&_&GLYiVyuy(t za2`N*CA}JC2~qBBLR zt5<#C`N13JA*~10dkoyeFh*&1o;KwxWnqc=0a4m+nCPi01`_?B`{aq;Rr z)qGiHYZOImxPX>jK^y*j?)TfAPc;yUv2vw;#*$PIwNIHlt5Tu7WL~e|$j(P30{lhJ zmmn-_Nd+hx)4El5I-h?GreZs0@UK=$g*wXS|;rv!{ZA8V+u<}aNQ|C=XT~u_i zw%JnOLPg!OXif?P^=|q8r(M!L1<2S?OdU*NAE=qsviRzeu8Ak_sILf&k<6uemXj(mTGicv= zK!+&ARZW4IPi~FtRX1GQi)j!F^UCR&moPP0N3!Ui zb52Mb$A}`U(qmY3$;;okZ04W=+%`Bx^|QFhlkmdU>ysm`Rzdr3Ff+7!F(7Gj`(WrS z;H9N#Twu6)Lg#sz**2{U_;9)~8 zXN!vowkG1qOTJfGR(d>oQ}9D;oiHG$%e)RHGyvM)_52$Zm_ zs32aOve_rO4hX6^{S%F=`~t#V!ma3k5z9FCjgv!KoYoc!U0%YA!xk9nF4e^JJ0t4e zGUW2C`yi-Y|2hL(ojG^Lg4^?Km0a`c>IwxYEOdrX=?szP1TVZKp2^E5UGBH%R3L-* z^zSO2p#MyNYQj!7BsH7}WKUdj7b+$=EyGG~)S-;#T9LiDH9dL5eHgtrsOXkH=*uw! z*50qDA;*@)8PIUWjiqtVp+D4{LZ#l9FJ#HCOrEAA+rE!4dVVKvI{bz7Sf)BfHs)LN zsLFmcI<7(Sn6!zm8yfZ|7@c4GIqz8{k>gTQ*fqcO#okrn)QMY8TwWPq6Ao8o*B znJcs~q$^rBg6CInXGO!3Y{^HP8-ZSsjzriQN01=iJ08?K%Od85mOM3ym-XIFUD`N%<46Htu&jxS-WRl`dHOB<9A>!TP5V3 z2ev($8xNB$RaJ1nJD0Am)+#MQOcs}aq2~_#u91jbMFf-o$uZUu6iSwuEah-mur5qL z-oM7Z09r5ZD_uZ7dGj0Ww;~Z8?a)>jFvmA_(S1yO`bce>;|DH0YyW^_h z?Zj+BuKlWe`JF&dm0m~Av5;0|DavRq(7hb=v;zh?SmRl3{5?$tqXTog0?m(-w(|@R zam8l-8e>u+-oe_`NyM7cX*g}2$z|5yr-%*uc9j-89|OvMT^HKSTlYt7Ixcn*8kBsx zth@Y@A?D^$>aJc{?Z(T}vpe08vC9L&+KPF0FYhTu2cwOq`(e+znIp5B1u1X)Zq5JY!7;Hgm zcpR>ARpt35stGYl7!1(&oUxCkOQh%McWKdfAIGQOLBM*rQJP+%HxeT(K^dPdNjAII zL~=Tl+pXWt1c+<~4r$H2CYm`$z?LZZ=BETVbG|5w43pw;Zf6=|23C1L!ZMeijF`<^3Hp z*{~rx%>`SYhAtI01hPl~G{z8kL7l`2HjH}%a*`)YRbQWJpqB9REMFNOv&XNFPN9Bn zG-ph>(`cQZN<&cSxiAERJ__*-0;oG--0C#ykRUico>{7*aC&)sobcBJ28nGLGX!uo zFp)^J1P-L~JIKA7@LY+X;WAp{wPvacrOZYxQG7E|0aO7_5Z`Qm%WizZh2?wKtb}=@ zqyVauMvQ$bS>-^*+lyr>3}_<^YsGQ3-W{k8pZbU7EQLX4zftR$OeF$o4xDlgM~pN* zGmYU3)Z>uu(;zkq!(`mxwX}teQC|u53xNZS5#2rKK$0A$XWi3Dk-1^7G!XCH#O5g4 zQmCa8L`L7(y(-SF3WN=RK#Dq9qs=$ljX3*Bx36N1SfVofFSU8bIASk>eD|}iuA?}y zfrIB$a4v|A{AP-Xf;m@k!*xcK!0Q1wgF(#Gk#V(f`WbWjkPHt~(UO?SNDt&YHm!{c z{r566R=sVI!i#$^XPguoOuRBu(;o zKZtN6o*GGxXRE`A0?jlKM%|I>2xXuT1|ry5!X&8<3i-9v5ptmqHX_~@@olQYCi4B8 zuH&{UTV2HF#joPFYPGqDU9Was#BSDea1uIOY(tNFV1C})Uw@vq!8}`zcfLA6=0TXu zpkJh?&70G%EYwWKxvMJ##yDfDR)r)E4B-3jqRD1%@;U?dn}{oUkL{?Zn%raupw)?8;ThlD4MTis48*>*oUSj_d>^DQUYsxGxs zU08M&)KDFvY%4@B)Ls-=SPOgJRQyqP*8O1mg3HvUX~c88EcX2$3UryVadzd0s8l~h zW&e+eO8;+2-Id)pUa0ntZ>zix3_3<5Lo!xns z;XL=CRERwM{5vbly396hL-ba-Dv%q{$$G)FwAl*t4^lZS^|qihE`*1W&@3oK@+o;rJSpVNO~w(nKz`iwHXD}mlnfjusoJZu8( z>}k;W`19cku=)KLWp2t$5w$gjZ-JNuifOxm}!^$#3#AHh;kx9Om)!F;{N5T7I4T0{Bxd?{=DX zed`nY=#;zRLGGC$QZDnF4QGo53{i02dvKgaX5xkCoTTI7g^ZiCH|;5H{6dbPA+$eH zJzV@fv0Doe?>cc4vLX(3Q$TbxB@X$?3%vHcsC{^%=xDHm65Le@PR&sV?$=CZE2@yA z*uo5Cf}jimIe+_ z-vmTifr<-Nf+u@WNYz18p~QH7J>yhGVf69%DB-Q26f;{eXO!@Da8W1sZFZyy`^SCS z2)92>J;;Kpv(#OZcOHp~`8^lYt_E>l^;QY3aS4KvKtrW#FET(z4w5=sqL@I(qYfie zUFrg5Odp5R1Vdr?_11U(-anywlw?uzi@KG=MnRZ)(wy9K+E_23#Jt9(m7MLWI0`vpY1*CQt`BWbIxi&b9l0YoZODeyytf+g2VDfEtnM-P2 zd>*%6S{YfiM-ZK=g#ynibvg#cGApx)=x1NSX#~TPob8KIMHx?J;4k0lVw%nJSe0t< zDoX~+)aPSsC#(54p_Vc?r!IikBB|sPz?;!^%jUQ?fnb>VIkGTsgbY)nrgY{~Q$=062?Vpe zH~15n@~wn3%mwY(^gflD!#==@=L(M4nIz&G}g~PPB6i&;zTaTbOR>78uu%+v|Le^+=~~3x0(j=yiX7IwwW# zNe_0huo0=TI=JvCLNn#8SOX>!eMVw^BOj=zJLLksaWJ6mX6u(8{%l*{47IYfgbB-0 zL64s~P65Zbp?e8`J*q!3+wJL5&^e9`ZY_een{qb`_kGvTs@u9^T`f0L zWpEC0A?ibBf~nU{ZLqBLq;<(%-D;Jci!I4NOK`WJ=~<0Xxy9vqI>bb!^5x%q9dpH% zyy*&d+F5ewdc^6Vu+qo|BH~7VKfL6KXAYRh=BZjbRYDXS%L0`ND#1oC z4KE6Z#wNK?v9!5XXdgZ}!eux1CpyZIXDC$S#9zgLlz3NvHzdGLv*(1so_%D)r}K@^ zgv#>IP`U6zIw6l67L5M@oqam*_6Fa<7jlVdHSeghCr8 zA58#GpgUVs1P8RMEWHz@t>DGjM^6t`*P2=mw4XQ{a!&^8Rr2DKO{<+gR7^mPmG83Z z5?lq{*jM|F48>@mMpQMxscnaT??giWdeH+|m_uW)yxixb2d}doJm(3cUkkTg3Ad{1 z`chqkLFvapB%r6TP$ij?2BldYL@gQwXTMWJH7CF;kFOn0cECHcMPnupo+T6Sr; zq~s1_u{!X98j_d>6>g7{QL%(J^&SS;S|1l2Zoa%zWrnq+$bHY@igmT)1{)6+1Qk z&GI!y(nXOSp4JiNfZPt;@DDS};C9b?q(Pu#qEmrX+LIOosPxxKUkUAIAUX6i7ENn} zED(7cc%daz1#ays+q`)AXlX>@!T31jVq#4Iz$tbAb;XMV;7eQK`DUF(+{2hc#W$oK-)|7P@2v?2=x$ZH;r&s~y447G^?mFyJ6Wagfg6B#Dp;df$P3zlFb(M5u#4Bu#NZ7U-rLL2h=~ zf^E>aK@p0qr2t#+ew`|4W#CH@x;C|wENEo`b_}1nS%+zHW|m&d?9NNSU#K@k(mLdLGt%D%q1Y z4zh?`Zs(^b(1g+Pr`BEBY>~eu=$A3dC762yzY8K5_TueRMBuR=^Qhh7ShTTWo2V(} z&6<~Q24OUhN^8bSreh0i!@pr}wy7F$EMxN)Wz6YkWq zBYIQHagL3}8}fV&_^*T)VRv6NSo5a$_z)6vTi*{?UfP3M$h zjKqZCdOu2<|1ESVd?5RG>ekRI_YnV3mk{kgtxy^NyF%5p#pFQtIaSS)DW-8xOaD!K zU4o&qpFoQ8i&aYF8i_FqViEc<-a4=B!gu$uewu{dx@AEkQ#lTiJLh(*=g`s0@o&Yt zb&&6~h8@aIQ@*RJtu9-iz4!N{{NQV*zG5%?3oI49Lqdew2sKIgZ7U}%I}$}B)9%ka z|IWdf;pYyV?N@jf>8Q^@+J8}3h12)AV=g}ux4m?)gFT-Rs@Z_>hQPjw0D)!A98VRtxgcW- zM78tQC3Au9toN51XRd-cK8YeGJ3@wo-jIU0A_lv}Qnk|dMY#i2_#)UIGXyq4Dg+9W zL@|&4G)eqNHUZF95;QM{ssK2Eg9%7C9*AhNL>6TK9g zY~};|t^4SML%)YLwlJwWM4r3WTWIzDzo9EdFPauil4+0b&;p<2dHN4@VeFHCpqq!M zvlTMe9aOg&I5S-OK3wfx=@`Jy zlZSLZA|h~VkTq%2r9pH&%Kwz8rjDjw$dkrc6C>~UGIP&@`8wkiEE0m)HkMmS5oROj zV#^tsdf*}@?HSt%TVbs-Zh@rY$0-IjQs&o-eEH%h2LntIwRiS{#e9-WzTG;k3A84I z+~*%+K1M`zz^w}P$PS#LtjTH!ApBVVL_~A_7wafMsFW%N7#5Cw~ zA06T$==l{jW+s#IZL1S`!)q+OiSqJAFt{daw7^jPH9~M0QlJ?Mp$yDH1N{98J})xi z4#to(!GUYwn_5^IHl|o7x`BE(MTnXP*D(B~Wi7fv_*WykL3%e$h?>R6&y%$pbp6o| ziXe4oZiH&4@#`b15ZY88)7sB|s*1{J-RokG(!>hNpsvHQZJJuao&JKuWi9G6I zhw@W>(z{P$45Or2j9h_=?O!S7k90YBL+{MR$`fDqpGnE9SV+f#)qRs-8~5B^D4ZF? ziAQ3LuxH8k%I42L>7O(?Z0d}a2Y4c@hf)m)jrq=CY}J__m#Eik`z~VM;Z`wvGHz8f zRTH&%!p+4!=^IOvmpw+P5M4uO=UA?T#hbj1=omw;-Cm9v3*9dA|D@)g-z$8+2^f-|Nzz+P`Pd z8mzN?R9>$KWSUPWtZlbGIttBWd;VCneq(g1ykq@4c?*7LFLV5C_ksIyr~k9w0Moz7 zt6}}m9PrEAGa}E6m%S#zB|GzY81zuP^HqPQ18TIn4cK>MN($CB`@#p%)`D?e!3> zMEVDge30UL>Y_A2(6Nti(Z9wB}M+D*)`EVX&u5M4D zupP5=J`@NG=8`O3kKCg8D~^3K>NTG&rYasC0#FF}cK;y)A*kqMMx#HeF^iKky;A2#`Jhe-D_@?2(O!KyVr|^^*|_F4 zf1m4pb9Yx-guwPeh$Y~Y%Wn}H4iQ$Xh>h?JbdbxQqL33sf|DqRHykC0~)f@y2p#KR;2-P#AO8tl>vt zmN}TtY}g)m`%7OM6FBn}*|Jcxa;#T0uq^~hmdhtL6^gc69{;cEqjCP(co|uw4)B8l z^g8%j4Mk8?%zl!?Rain%bAJr0nf~h$%u-(sObmpC>%vdFz;Ra;aE`x48GbJahFl>s zqxlwHrt#=vA4Ri;a10|A2F0t@CJsN&Aj7HAyD@24s}@zn8$n>O5f6Ay$<={Dd0nM| z{;9S*3zD~bFe;M1v(PW17HkozRfF-sxnJI=my_SEK)gGqA!)CnMq0d&G(izNjsld? zLo6#b;jrhB50ltQc#t0Qe2R87+GC+4tA2{j-C_Y$os`=G2lY|!7t#etMYwNREL2f^ zCbRjpP#52T4q~tNf>`HKo|52fr%0m`k#*LfLUgLeVVRPls%00|vRg4rj3kSWWptuK zG+*;bs6zBKOtphXO+tZ(N=?GR2BoUF{dH1hQQu$1ngl{motgvzPx+bz1IyIPqCFpK zRq^>RCEiU1Cq7dpr9mgI?E>ND%?z4P1Da$%cg+F>;={WU9mGVt7MZ;bY}lwz&NY36 z*evomAhpXYw(lK2-irutS`V!@g=`8f);v%&Y&2`Ik)S^YmOwqQzyqc3&T;DpwbXM; zXJH`X6y~sTo)l~#gAUu7M$1AI7yFRA&|du6jIC*md=B2F7;=a)2NxalW8n?zIr2Bc zH_qcILzI#Xl^N2^gL)AWSSVWZck}rDG_Q8-klxL;AfrV3VV}@jMA+PnCGaBH9!Vke z5_W~_vFPyFgK;YO;pK;~-bn?tujK4elx*_4+L!$LR_T3Ks+@tJD#z5+OI;&~hW-Et zTT(QtTVLUtm42q4Z67Lw4(52O0Y9Svm;B^aZcTo)!tj5D_QuE%qP#~$iOc%p*mIa` z6&&??{AQ5JM8Jui_^qB-q3AoMAJ3^6m`-;a6%kjNpI9B(Rn(OJ@H=la0J0M(Zrc#W z#MWQ$s9xMvtevGDV5}$s&U;3_Hb1dmx1u*ui&r;JWUaey1ACGcA%z{~4FMxGn&f%? zhNIYo0}HR>E%X`^aQ1y6_i=FB{8I<52ipxbL;nIrKB8DzD!~V|v^CVOE_w*i3`jmd z;ls6<4JW#bjafa58m1+wKGIU zYD7~-8}@5A!^J_Z+E^&TuOXmPJ#1!yT)!YWvQa6t({Z7>ot90N)|KY3H@sRA+NR;W zkk!3}p8GGy=-prW4&1n`u)^{~E5|WO>Gt>VW{$W@SI>2Xj9ptN6WcU5&srX6J5zxz zDM+QC?ZjOlU25G1Aj9psyy_I+h%(cZ+gP={DoQm6>x5-Dsg`(n7OI9%>~$0oD+8i4 zVcI+K5vkpuPOKxQ{CI zT{R!*qZDc>kzTRqCGF8w5{ARVqw$Vk!ri!0_W3_1@->*s(P4gs(8GV+eq#QALg=6N z(`>DtvYxY_coYuVwZf#Jmw3iNeWIFq9a&(@SsDzD`Ty-cDLg1!(_fJYLldWr-EO(h z?Au>USFS?;?JzHt?RN&--eTia{m1|0!)9Z!0ikJ>e~y_ThbK;us%Iy}K6F{-pAjlQ zf<7EJ*r$oS= z_8Ps3_5h1$0T(QzPB z%U@nKwVN!LTbg3yztu5N-n$7kUGmUO8g69lv^~X$qm7mM#l1;PoplNxeT>oI75FMB ze2IqvOmg2HSXJOvDXfsqj3U2%0UR#ruZ4w{lF#4?vex>7F29TDlgf_~V>pB%2SAjP zjLKC{aaFotujITXMwmVO#6wPHF(iIb&Q&c~Qnd_~o(}$Y6WhQr(&H{KQ36)h=9l|i zxtun;JL*H(yN(gW*|xv7Gm{!~$5v^eKmKQ{QBn9_N-Pr^B&R4&1wIjER0}X!N*>gD zqof}4N4Xt6y>aTp`=T8#6FGp2*fKbOM3^XK@o?4#)UQ#gziQoIN!#ex@Mz@`YG+(Q zk51_m{U$qB>h%Nd^4u$gRh}~(qSB3xM2kV!>Lk!_nzT6DvffFngmpF3QeX%yvW-&cQy(E03gMUt>Ks{+u~r97kg?SzlEB@Y zGQ=q6M*$h-bW?>G)m;C)?}84;O|`%Ne01tY$ZDQ!hr)!3r}ba!=_+O`J&e?PnqNeV z3zG?G*ei9DA|W1o19~_B_>d?i20z=j^A|PWwQAd#Cf6C>8+7ULK{H1W z>nPd^w(D(JrF#23RKU*j35)${40}{^M5!m3N#pZY`g+0l;F=Y=F4^ho_#d>)ELDi7 zfDqHlr^t+4D5~Wg1kTL7S7lL6-1@I-mM`_6*(|z$S3IMQuDnSe@VOp-Q*pW8s3?f4hz9fwr|>a1&a&nF;Ek6~&jsLT>{qy={M!ts>Va zPECkG!k~}dx&iA@qEJ%HphKIw^Drg(1`Ni-oy_DCqn-eM5zOdxQM@5r6S?u;JPLn)h-{r68iBH-;gw?=@zzh4xk`HAe(;NL*ss-o0T)T3wCN`E1h5 zjN$J^V%3}(db#H>VTM`;`V%%m6*IM6SWEk9^N$L~_pI|P!5mb;+nc~g3j|F!nGg(2 zZF^M0>nGlJz~3+6@1PRuU<^qU9H0fd>4ueMVUAfd3^jPDLf1F7L=vQ|+As_uerPfb zF>X@*-*%JSPrJ#7JVedqHu7hH_|*`$0Lf$y3tK~}boqIpc4c8|*<6`KK*ipun^Xa@ zb`?S`^Dw-z7q%z3*-~63j*0>2TpKCs2qSblRRa5~dQsuLk_c@Z3AY>uCL+qCh}m7V zLZ|ji1KEk3eri{{sT^cIhq9??kN%Tt$Jw1QQGTFFv3te2a@B%O1E@I`Y{cQ#o$Leg z=UP`2`@(8gJ|P;-83Q+y(=$crc2D#=wYpq$QMn>N2bLMENvDjGmgCxB9XFAyl1$>= zSe|Y4CDyOivC+Jvudi)yq0~*Ir$V=ZvuSGSHtNdd6-6+LS~?D=D)X^?somGN9v(l! zT9}jE2Q(M{-6xm8KeBHfmF=R(S)M*5-$a4RAgzRZsDz(96kq9yW>j zu9DA)9YUL|^9hMc_<+fQ(-c{u%Bj4$NX(OE+gTqyRL8Fp&)l??dlj|4@%7&(>AXr) zxTBS#Rf6oAYPqZlvRLDIRz&T-IU8G4+Rg!aj;VYK*V(22#9prUjuUP;Uwp>%dJiMS zC3&|QdA1q9W^K}0A)bJ=nbY~dugJ1I&Sn3d^sU9L z=zTwv7Yls_46#Ws~-VWkuBe_VCP_xDTKTCANWjjOlNQg1Pq}d7e1XNA$dq z1|z`tL>TVR*@t))%1eZatUY998Xy4&{0uTTmbl``qkem1j6Gx!`)Uc`&n0I>xQAiE zWsIa+e&$WL-$%FGINdwKb_+OBa~Kfw{a8(}kDG<7ws5<7CZ&1meb07ugr? zi1D(c{v6vJxQOBM0ii`F!g>>e1d_z!M$?J~%U^*P{a7qjX{2yriESqk9MR$E@Ps_X zr4-;258|Nt@VJB^;BEK0RCeQW%8 zf*)mwjqp@w-o}%q6PcAh$D))K_#*59QQ=a`ET=f{Joymsq=^FF1S$L)RFEPJ=|2_xEJ@i5Aiu6UBIBNdxcywa-iA zmle?xZa(pmEA@2P7FNBLd##ApAZslpTfM0W#~}Z$9~BHBr?xR**f$_d)qjY4;`-aw zJvd6v4<#{ti0c0LqD6=^DtKOC0Ymqy9wCa(uZ`%tg5XZxV}StO$r6_s(RwT*a4k6k zQxOfOFVeu1!vJXUQc=7>7Nn#WRv^DB2`kPgufj?Czg3n4=SoAj@-8g&VsX&Q!R(PUZ zs*!zz&s?+=aDt@+jE!MeLywY%-JwD{@mQw8xAy-8=^;Va7wh*UF(qo9MYCDx%&1?m zPgIRT9dH{oxHPxJSDysA23Yp;%NPVaq$CUv$*jGL&m*7KK;wX?>^H{k&pNU|amHxJ zQXJSftM{3|yRvDs&T6Lri zeSg0OJYBdjMg}%3P43lRqBU^X+eK*Lw6_ec9nF67ju=86RXeplhp1$&tZXW*z6A(`8I#zFE*K%Y2qZ5c`HN|J=GjDWk=2Ns0 zmQEn#6BS@zYRZZN#{j~Q_9Y@zuYAJZNx7RO$^zm5F-;*_$-mDW`y86aZ@&B^Q%s%U zuqmhrUa{KlbLi%2_cMbu;o4Q}-e!NhnM|=6(T|~4_`cgTCDh%wX5FZDXd#=pxz@jHBNAt!A-7d z+K|=*>}E817^w33=$Js0s^E&yhy{P52!`&0h?geR{wgnWDf=FQ%|A#FLf!CVEISpV z-NtBsE|_1M*zHXS(LXsHk!jzX2m=#C>=XvbMLmE!E|^_qEAh%>zuE`jl=*{SlfzBB z!d)TSQ$3?~y(QiHIu?0rq0r(X=T3Zj+*C7YNN1&^)ZEt>I!f85l_dRqB)j39J zjN@$J`naIWRHXj59dD!^R(twXLfq z6JPAfdAu~d!~|G1TL_laa&<;UNBeswry9vo!9>WI|+Mb!&bN$yXM)v<{G5)2x{MA~tGi%-H zQzIm}Dox4(k;sOA!_IPjq9m^U#@_#=4m5u8&~qJ6>Hz7^-oX7HC&vT4dgn&>HZ6+u z$PQ4ry%g?*$oJWRzdfgBXOeNy5$efj1T_Yv_$u7TP$L+#pv6Zeq&W($0|ahmNvE8I z6NmF!Iqs5dgnhW_STYF7SG$a!j}&aH-u@^`9oqqcmwz0kDC_m6PeB7flBez z3s8DHM^A!Y(uQgnlEhXaWzED&WG8v{Yr^MqPY4YwA8ql#CT`J8>K9j&ueq=5aE2NF8I5i^Nyb?cxxi^yS~(Pmy9eKGQm$ zQ2pz$0RKc7leazy{;fJq6O5@P5Gq+279ff34=)j__`dj*j|1Et-7&m^q{#$BM zlvO<&$2>Uvn)cj^|6-tBYX&teIl1iU)ip;_b%`neKILo01k{oJSK)RQ2+ozSk(coz zOsq|i-Sw-EQR&GyJe2731w?`MZzC~WkOdl%b%PiRu>@?6PGr@xi%KNz!%iEW$ol&Z z3Xv~s*i3NPc%8p<1Yye3hGAGylNL-}=M$j(UEKlP);a*MTme<@CfE>y^py{wK7KFa z*r4gqb0cWi#|sD*jy!8@*L_*i%;4tyYqjA1t8kOrf8_GFa^i7#TIsXW?nrTF)yKTT za-!cJ_dqjZKPO3)M3MJ&cpbI4h&@$QVf(9X+oR$G+N73CL?ALkPlCM0WLv=oq7Au& z3X|-LW|<<~F%IrPtynA@ z-+qZGek2v)dfjk96f9yX*a)qL~mII3`cp>pF;>voF9Pv+lTyKe;{ztD6 zIMd0v7!WI4kpJ5XlH=d8a_4Wa(D%4zwH?qa{2MEu6ene0;@NP4v>7fpk^eVVZa(<+ zB$Hx8ykxYsYzC!+X#MHlTB?3Zl=+nH*tz?I<@~$BZ6~;|bgj$E-vs&YvOrW~!|EKI z^|%vAS4bFflmLp7{MSN5ucid19&-Jfb=FicGO5!ZZ(OkU9oDV_N!q^jj%CuBoi(ka{BaN2=k;)wD>w2 z4ADYkLZ_M$mTKj$k5a)yfj5Xq!#dKVv}T5VX^jH5MkG>C!U<&vXf#M{PP8M0+;NF~ z5?gJ{KVYey24$R(UbQ}u7D&PlyE$2PcmdTQlhn^#eJ~R|*)plFLT^t03|T)_Uvvm; z3!1)!Rwj))H?!aE`fAg^uanC87-j8E;Y=Nxl+np35|Pi1pguH+NMteQu5PKK?xgwE z*}L`4g>mf(df`E`?f+^bu&a>LNan<92g{H_#kjqkRaDsDduKa5Zu0z~b^X3|W!6k( zhF+9aV`BZJECEP?3tGij(jqH033cSnnTBtzb^OT?%LUqz?1qdGHoRs#5jbK(miu^Q z$x82_8mq_Td85n(iAbi;S5?Zo_cWp;PQZkY&snyyKT7sRbWwJKAj zGpuo!U{#Is#254-<`r$FUstzcKsL5e){`NPhD$?JL(!f!_#%&T@^E~MjbiXbddIsZ9RErncSVYB_XV(R)P9>SeoIL4kcx)r$#rQi- zFr=1ns^rDrSlNFaGCOd^Np{lfdx}GzmLYBPnNK?dgz|0wAkY;kfy^mn9-vCI37C-i z@NzFn?}CUK70nm=-GO$(Pb1$XF;dP}?!^&JDXb&>*)8FUV2#!bK=eo}%GuAfrr=DDV63 z;qM3VCSr&YPI!o~o1@tjW6D1ssb|3K{($7}da}Uf$qzCOwA@99wGV8JL6cQwKD=(R zC)$gZM^FhgMUTCOal1OC$Ot9dmB0e3PX&B!4j48_Hc_#wvFGr+(B5#inia19l>00Q?5mClZ!3>~0PP<-V$#S5M#yqDjw-4^l$&5~@W-P!`P#J0=I_{n`N_o_6yoOZI1>fjYl~ z4h8mWpFA$rD~8IZ4bbTJrDEKYq;8xkw|XvBQ~+iqoe7LwhHGJ^i{Rs`ai!+%`-N`d zH-b@)6p|FLS>4=X0KLl{<68rQN==r17|lcADW;1QW)rv1M586MQd+N za%hkAzcux^vmj8&0an~ai1OO;zgFC4lq7I~6*tdJT3dy+k$y3j?wV+^OH839lq(fp zE!FJ_5dA0wh{b|_{xbFmr6E9X)};J9BH~%(36k+Lzs9Kr zI?#MA4H)EF;}BN5%M@*}Azot(!}&w5SVN3-M>Hxq^Q+xw>2Nn-R|uH((}} z-s1Ue#Z7Rhxr!<8wYO1wGxi-tNHghgi|-~CCx(195xx|FOc5pecPjrX0BR#(LzPdA zBWUG69*%e-@O%MnKJsUo14O-)+WE7XH5QrR+x}9`Iz&Q{JjIX5?WPr+=Jb~`FSs$e zkf$J>>H-ot-hTBunIZiayBv|azE=x2RWcb*I!s~MPvaSY6{3Lx<^dqe9V#KzbV3wa zF;v#kt+L%C4jmbZC``DWLz+*d9Y9$AYlVP=0)qRQ<@k3G{GE}2fc>e8FfALZxUdIWPg z6v@I883-@cjIk%0hA9Ai#2#!yzTGC^<3w!Ux66W95PqbW6IrVl*nq9m=yDOZ*5tMg zo3fgLtqcF$f~`yMasoIZ-T|*{4nkHBwmS&fI^eIiOdH-^zzeWVZx}dOluMEyf2%b# z{@|}F6{ElqU$!D5KlPOBwU#5_{)2OID`Q$1g$mc{bF$4Mi0pkPr%ZJ=D((X*>=}d> zM1EERBNb)%%Nc}UtsU=46G#MA43+?viC6E#HNxhGBj!s7B~a}7>8akRpw8_pywqMH zUro%@n-xooede~dP;>O6U*STv@Yrn5xB8D2an*Z_j-Cg|1;}@eZ5{I1N-*v3FE&c5d7PW zpmA=8C64elt;g6_3MZjOk*ZeI`5Ir}fW4=ME$vT>A}|yBBPr+vRx$97pxFAIIs-9}jz5GE3bypzq{m zVJ4G`2`|1a$3K~q9skTft5qboaNT0`JO*O>QJG0j$zvcdMj9WmKtGaKL-V#{#c4rR zM*^}wNJ%?vI9XIfDM@4hRF)#Zc(~6UHg_~ZgeGV&!O_1Ue~Y#pz3TD<`H)$ieSDOoc0$Mj z^jjrJG+1jg(nq!Al9?Qb=21R6I`dMkltwN`4a<#m-*eM|H!+?T6uUQrubCP)n?Q3ZJ6W)abr6;>nUG24X0B_NN#1I#ncvoaYT6%yw zvJXC!Mv*b1BVBT#sFczt$cU4)#sqIdoZ7r(Lad(!BQoi&@Vz=AZ5vRv8ZrXn-jgyL z$CN+j()f!+u?4r(aPzy|tPOc;%|Vi8@prao&MQSx=V^;l$0^xa5NW4w%&o%Bx)xdY zP=FU9WN&?Sk!#pbDzPon-wd;7_zF(iFTT_cnVvh64$Y|RWF*U5K12?rCBY+ZfmB;5 z95UXhp)Xh68{X*{!EW(QY0f<7sMzDv5JTQj#&4k#3zGn{r$}8*W>nWcit$$EFb4Sv z184S#63SuGULxVdfJ*G0(`hDzc3O5#B)oRof)vtua?)R>!5E%OZ*<48O|_}m~Ew>Z+Sh8eZ)$p6Y2wkQ#*qW>KQ|aw{$LFXG68 z_!4RLNwz0&X9cjUmAacXtf3}jNEd2-w$~r-usn*Omq&|z_(=OlCkLbg(qr$8xRI|F z`wC0e$3*qNP7a7HHGD=zvouB6(H%??(c8j~(Hznv-3<`c+QM>a_hjh`6r1DDy(Ny8 zT3+KOtGWPBD}sC^P8T(Rj}7dA8{j`i;%I{aKY{aEhmYj`>ixrfqVwGV-Vn)j5f69c zN6FGuam|wgZuvsF6o`()!LI8HSwPNGy`u%2-W1Rt3JTD@$fRWZnwmYmyj^@<@j%ot zXV-I}6;*bD*KgCRVr$29V8zDVy5a{Gt)C;ytTJJ70`m~2&cn6x+xdqS$D{Vhv2(l-GuQN<<@FA8TLZ?AwE;Z0SfUEY#T97 z4FoP*W}Wg+$aJ8mIa&q-Z?`^}$OT$x)|xH!l9pjbYHu$9Qj9nj{A7jbv&;frS_6l= zK?Uc`Ts1!GZBK4WAcn#JibJlMzOggFwRin_C5_U>QY>ip$`}ov} zBu~Wk)6*rE6pvD9EeVe*JABdZiVQa$?aVmv1HZw|Pv$Y9lvS7W=bDbHZ3&&0xhn3i z9+TsoIG%f^*&h9+kv#56>-EMjWVBb$eRm!TUc994hQRqh-%GifZE zYq`FB(?g!>$N&G6?r$7A!|i~iTlK$9x|#k5)$l(JhaX$%)c+*iUX({<9}*dG-{~9@ z+Q`N9zuN{~#)BlC{=4B&I<=)`x|cp!|8ZIEs`g$b`l-{g1N8^fc{cTTaU|ID)RhY~ zC*5!?Ic?O2pjJ<*#|SElJyN8En4}EGaCS~0z7%DTfw|p}XPcmVSP$so0ElAvPNbnh+usy?SFjECGvgu=(mxdq*G=|El z0SdVY0}NVhJp&6j&%Ss81Cr4&!59MQbK}0bFmh_aHnkBbn7{2=)z1$hx4t2ppKqxY zC-6V6_wqd)GJBRdfkDkD!w?XXNG{_^2Z?4>Wm1UyLm6_fsec6n065tI01ljp0gn0%ia@hmaY;|z17KR)V9mzk^B)wYiXoA`3AeyKLs83IDn`KaL5}lRq-K418 zJQVG^v>8Y}uX3!cVSyRb$UcIUv1UNly${H`!7KjBx`8qN>Cak>pQTCHMNBr6-%tz6bED?nxZzZ19gai^_kpRA@nXcoiR6o+Q zp41akBe}H}RfNzEVU)>&GK`zmemXjK7B)GuN&nzQ0OR8P-`?zEXIQa7&)mF9)@fmY zU2g|mq2;Q!UZ}}LL@WxtXyP4Ie`2LK{*fcnhRUT0$@%y1MNyS~am+xD-sA2U6{n#ECvX zytObz79uzp!%|QcCOmro;%{P50UD)L)j@zOrJjfn6M$@pU-?v{E-)OgoNE%kE8Sit z^icP5mB3r^$4vl#vg0O+y{N)X2z{D{8#BTNd})7u8OPHO|Fo4N{_Fr=0B!oh!L!I= zwp7qo;hR1-U0#mBGz{oANJwfw0G;&QF0Ng>&{A9>K5$c{q{TdzAkrLuMnIS+7yx>* zD8f3~QZ~SCCn4h+NCl&-0#ez}il3sXEA@FI`S}@7zeAAfXG$1iVc`b8_6|^4c2PC1 zUq^|{Rt0z5y1WQ@6!lDd?yqYtGrhb{^xQ)mM7j}IupPRX_SUa=G8?1-{+G|pEwt{H z9&vz%=i|4(XpiS<`cU52NS&y3ceeJn99_-LK|N)HnzK7&q_4^`IyE*fNB;qNnlOwx z1w!UdLw)WatRlbvSU_Js!<_U2NX3`P{>xPSf8A;F8vu72U|F3@kyTu+?9X{TQcl5y zW@QA&!5>@Xjdkqdkw#h_nZM@BzCKO6tVZMvM-?=ao8!NkeNIkJGt+lGIX+an%sKx# zZQkv2@&((fNNiXj!zi}#+q7DDov@G5&Qqws>1moVo8Lf(smLIU6NdpI!BL6Lk`#tWngiBdNN6JR z1OCnFMWlEzQoB@+a^D=wa+%ud*!1fV?Kip^i$9u+!Sgp)UHq#ao=xtbt~35y+3Pt> zB;JU#VhcR5yl=0gs(RGoMpO_%#xmz2p~W!vg+FG0EW%P+F1t=`0drRM!rS@J<=NUZ z!c*WYP{APFV}OEw(paZD0gFY!orRdJLB}O;N=zSjBu zMPtEAiheNE|)PX*7r8UvMup=htrZGHB z3Zao3dE}E9aWZa`^wOhl$U!tDdvrn-Ud-Q@M8k75(^xbowg8v;^@nkTZG?))l zHpEy%it%zfU@m#S`R6!!pY{%E+0(Vpa(44^cFSCsr{x3L1NU<#APwO%iPFuR2OF=G zUx1A)bOMnYBSR|2yHDGFu1*Rz2>j-t%6JS)9t+cN0-L>a@ zTclj0pJZ7Cr;EmVRK$bDd^WDh4D2j8(O5qhU_8Bp+@HY$VVR0zR48qg{Ra`&5E)sX z+Ba6uiRx04z^}r43^ZH*O>%Z6zSEeP62yt!ByNr}OTiyhVXvY8+R7W;$_%xp*rmq8 znSL(izfhfRli;}SJU=>PR_;6jTC!iJ^(UuVPH9S6qv#|Q%7na1#;_k`V_Az zORagJA-dJ8#^vW;R-GU8a-Oa^eQ+h&woDdO#9B3-LN3(7tD=$9W6rI+TT8k{61kDWLP^8hl#uTIwqLdrV7 zRAEI^dcadtm!rgZl@grfSQl03uM6E&Gp+Hf5kTKyQuf|-D}ZZ#3eBf2hAf<8Jvlhe z8K1SY3P*UZd{dfnd>^7^xivRHa8O@Hi0NT~&{k?>U-G0P8^4tp zDP(mBuAu_z_cENeSmGAbE`s9Ou1d0|uoZTTDT?9v!S-@CLmMQBe>Z|1w8RPJ+BdgT zcg|i?geHIKbZf>@qH5qZRXGph<%U*C^&Iq(a$kz(lvjB$)L=;hV}v-=&82oA$@sz2 zJSpPk6_NqR&3lD~wl`ToNX;>=M977cyIrX}HaK|IBx)v%eF4>23;C9O=7qcqzDyKJ z3*K-~^{{gM!@pfwmh5(4LBo=~AIoitc_)xtt9bw2xdjb3PV%R$4NgUN_L=tFH*9#( zW~a-_GEWRSAg!67tte#LPdp`SRo}#Yly7RT1Vo{7YhvtFQ=|c;m8q${PP$)udIdUY z;iJ(&Q!{H0HEzT^;Q3t_?I(uv!;8U`2HoD@3CP`3Q$c}Ex{LBgZcYXsUz_C3#tGT; zg0f@w!yt8s!q9rqak)zjQ9Pbs6V>#NVupD_*stwPa^pSDVCk0IjAd{%u{^`%J%~;C zEGEfh1x%@~C{2m6#&Jd2INO#K-lId8n{oynH70D3Y|T05^J^k#wC&O=&X;xuM)#Ks zZSyqUV(TeQ?$UL;kz})FFPktU)mcwg^;o=W=ZoaBL^B zgMKE@Uxwyexu`zVghRdJo?l}l-4CAG>41^eggSZ+^2oQDmpQUv+N>_1G%EOPZxQK0G*?e9rkNM@WQhoi$VJg$|{F0o?6?M@8sPsg?k% zgU}73Zaj^-pV28EU=cEql**bOyBAibBK&+o_(YaN&BZLvOaHE#B|j91B()~ug$&el zAj8a(M!7bSh+m11>!#5CZ4OLPKb3IA#QmTTtB~FENcywF$*% zwzvF&c9Blz6PIOd_&tHz zp|^S~J?CdNI?$W{Y=0M2bok&MQyv|;ofAfk&N#o5+Jv^k7*3mA1V>#g0*$W&;Ds+e`pW5fr@s!)hJ`cIHMGcJxiqA&Oqh@{h1_}9E1 zQsneeq5a%KROb*e49F)-Jz$H`A`uP{k+4~~U;YhlShh|l%DLEbo~{J#ug4A&9#_JT zLnFaE0h|SDAP>>@3r6C6$FSGYM~syF~73`aZ6{dgXv_iZ+`B zw*CVv0mBRrchD$++hXdRp~uIce~-~|BHZ_P;$vraXDJTbIQmALDLZW$WTr;#Tv^F+ zZxT%@TM3r80Xmzw&4yV_AuIBS@2$A?lPR=A0m71u2#QFPj0oX4e0mr~EmnFMb1=t@ zNT+gGOL~|Z)ZC=VM==6=de}ZOsD+6zQAKkDVY~+STAYjHc!)^ICowjXMsqG~xD5Ab zh)8o&Jz=~9J>VBEJ7!*5##`{%2$&25ZwxlV*;$qBK6SAnwrre;{s^Wxx^(rJPb*#^ z7ZZPyn>)at%zJ^kZ-l0=MdbWj9$ivv%w1DAo`K(x_DF5t@Rz%S=p|qNV!G1Q& zcA9)~)s%eVwci-53w!|yEm^eUSAPowHy8PhG6RI4@D4@0dy*i$|5J`J>CZGTwmtQF zf)D)WPlv(J60pgrIr|3(z+A8BgVD6I&%qRg{43))!YH}iYnnQh)=Ir>Y} za_K_)$DFQmSBzu9=WS|r+qsj_$SN7+spr|PL0Wy71U>SjU0{~e4`qUkP z+c_y4?JLaHlPdp6b0)(l4sC}DmBb+$B zVJy+tTqkTmKy+JcJf}`5mIMO!k}{lxED3x<|C>2I{^zSbOQCk2#V9v@cB1a?2S)8~ zbn%Z8zAW9|v5J-8`xP_v*%MED&jkL35Mv5rP1{v92Z0_GovlgK5oM&Xv9y%RZ?YP{ z8vxMi6dF>?WBJPk=tEVMVYglj<@z9P5#=a~WVF#C5Kz)jA{{*lXt69fsg(Uz>bOK0 zk|QZqH5Y1^YRHC}0MI`WxuM8y)pEfVURCu(#NaQG;PxntxfEx(CKHezhVS6 z+<7@yRp7#bg5(o?LL5Vw8T1w7yk$*+9?H$B=r1;qRhu>?`R1LgaN#x5^;tFLYrOub za-LVr-hx;x!fe|Q&|jvUBSVSxsR z%qR$J7f_Sct_^gT;Arq*tE=o`!~(@5B51WRVX9b$^rt6i)ki-nlLiW32lC#I0oAG@ z7anrAy)v8nVe^%2mV*LKd=({j#ZgsWg;?=FO*>xi&Baza)LF=q?r~P`euO9)pS2Fc zHZ;5B8?5L8@6f%?w8ixjKas+4fKrIUH*|_M`HDqx$Fr+awmt1BQF2H8E+v`-@j_^v zpoF=#7zvq(*fhF_;f3sInohUlQD*@|w}5u;u|t&yg7NclzpeRu!VTe2dqL_2=?E*? zm5XhUeqvvYbxI)u7-^)4!n~}g9kW@pq5AGc8f+4zupDVbUa`2V^%ahnZz)f*|?>V0*}6!`ouc6TNuc1(dY>z$AJvV$b z7h}|g4N#+!VFPN^$&g>2fTI;qqZ$KhRKWR~v~J5ra=1JZ^6OyRR9MlK-?*kZX)6{R zA)BE~%x))yi%ZA$%@SW0ZSF*wR7lamxQ&90ME44@m3~Cn$_?Jyi7b7lV9@`VM*UkpBa~E+Q%~O^p@4K=X@6<9WS|VM z#I$msX~Mas%Jiwz)6ZMRQAE%~0)**0rrd|#^= zo>f{kKIf$QX~zRGqvzgdOjv@>o9o85`X^NIp76{R6mqb$8wD>Q!4_mr{x_y3@8 z4SAWHU;qw|Jb)MdzpBXoZBWzzz_u~2JM8KX5&D!GeX5tsmjPVpN&pvn*#ElF75}=>$)#w~hMxbK z_0VDdWgZ3|Fo6n5;*$UrR(X&mMmx7WonRhoLS)l!V3wM71H+RBNOZf>QSM3OvaaLW zyyq-Q%u*11a~R1QOhn_4ISBw4Iso?ZL5D;a_b(UvoE!K*E_4BYCdEfdNbrBbKET8% z_mPi{z&8%?LnedT(q3_vh;>h|% z{GCa*Bi<=o*z+m`dU+T!#NE=FV8mW<%sUinM&#tNSR|xj(Xfj7hHi4_9iV^WPsIg zopMN|`bAkntYSq6j0_`d#o8g-W^m29D?6@!7$RgXV{r(8eWbLla9zX*Y9^Ck(H2}7* zG^MiY!DA_nk0OC^3Y%vfN`nxQq*u6G;g0_dZsJvKYxY-P2~?sf47lyqekLStNUwlgBqTg%v=KAhc7PtU#5 zt1W28oOw8MQR!%Q2G}J(xEXQFWZR}Z`X|bjJkGwIHQ|e~s6d0u5yQ9Xt_)Q+niHVw zC~-;(pa;0FAbkbzeX^%LpMGO!;!U1o66 zx?{a_Vl=ShuCqvF`_^&T(}Mly0^Hl zK?Scp-w$oCr)<+C(q)`#S|8?>?Al=7?6jRN##cJ30T;USN)ugsL0A+z9h@xOiyq^1{ zgOxrz^3KwR2 zA10%(FI^_GB@1uPHF9gcXSy1~e0NW@vGYF{aj%r&*x`IDmS#cIn{R*SX~IGTfP1Xs zvrrA9;|K@f4!wlG1R2QTEN150xd-4{7M=ywerp%wesp`h%=~1}gpg|=K*&bGzU&s= z@ax?1yZ61OwRCAWck?oncOBHZiXV_~QraQphokXmcVHmGA;yoDDBKfJLp4^)Mb2fX zDzLOMYiWL0g_&^rybeRGQGk4zB<_nt&=vG%3yH}3_??0ag34n2Yo~M(0J12+Xjpbr~p|4HnJ{mg$(Uz>jfikcrcyV|yJyXDjo?8*sb5YA{yHN9U3&c_; zhEbmXO+(USS0DzwfnYMrivK+0&LU)c=KLBJTDj%7W0ge3l^dRUp)YGT~Zlc)8+(!@f1(TW$sXXvCFzJ zvhaMrkXnbz1q_y8V?gkmQ2stN&vy-Tn^*VwfeWyc_3F4vV}m{t)?2EVs~9WDWpzY> z&=9gK0wgk|GuS*A4bF`L*&UPb;M%X%Y;Z#Iv4~VzZQo;VUq`%e~AUwFQ6c4kupwJTJsEdGge-^aqpv3#NDkY%w5&UvA}B+Y*yxDb`1 zM{+EV9YwUXYXZSc&tvZIBZNsK{lbs<0`exGp$~PjTKPtANo#p7zh=J!`kWeiZHymg z#72Oi_G`@7Sl_6xzYkTdGmk=Th2_TqS2Wa9XP8VtX5)-8!{nFevr@Kcn7ol&fHvQS2P11L%p zl#u@n(2ku<8 zy~$Jbx{SlY#^>l&o^Mp}>GD~y zf`g^J`@(5mQJA;_z1u=T8}}ZmI1uz=-9JKV79~}JZrjtcSLh4BTMrXePz8YCh0BM< zyv+)lJB|Oq_bq}Gd4J@lq+OO*Y#)!n1KLT!u|mERf_L6}(EcF__kcFekR%I%+pmvW zB;v4W)@#I8m1qe}Q^vGgeY%?$D2;LpP3vrQuB#=2Sw__IE;(NtmDA z5jslXhvJDFjdPS=b+2t~Y;4Xi!Uo}nLi($@x$Z2r?C!uNMW~mbJGhCMj}lgS5DKc~ zT8;Of6D{)p*30^Ow28MAPB_%+1fS zt=j}yX-JyVxx5>COG&kNO=z;GY*F9iXI{~K)Vy&(lRBj|XlFo98ZiL0+^aPLSv#SGJM@Fnhz>hh{wsdZMY4Imiu->3X}EXZnCzOG|^qU zAe2N?VQAjb*q|JP6bDY*oF4_?$vnlG`ViP@)3%qyvN$PQ&;H71P==)uYz1!~Szm06 zcmgpYk^X7uo+cCo# zzV*Dph%xm14bGe?1S>bCw{|1S(Nf6{E<`a_pWkd8Dw9g3(d&e9sSdK#RFg^tG;VKD z?hk@+dqVN@FipGvUY;Ao>1pp7E)f&fJ@@tLVG(PfN`s`T!Wwh3^0N5+sgj3OiaL-V zrQAAv^cAI|yTQ2Cbd_kKjN%p0ZP~q=YBH0G^Wm^E43jKPIu)$Mz|RE*GI_IbQsE5f zS@JU4~wrrYgi{|5WS4(0o`Ac6v&^O=?76}og))+3fW5K8K6 zWWXd~@Bpd)Cf!qH8BFtbi%9BAQV7yn?_`&Dn4)9;xtm72mumJ1>bWJ6{QgyN1ZL13z2Arml0tYX1U2ZM zk5WG?r;ZZcH5Hrj=q2ZWAzAYs;9Li|3VFTEk>(^mk%7*tYmL!W1I4EEaKyZ~S+2@IeDGla{28^|0m!SiPILdxloQ8l zt1JSs0To{KpIh^8y~$LLsMOEIg~SpZk;;|o@xay$YpQn0QI~6=h zTbFH=8m2bmC$=CZr-Gdp6=`kIfpz{LXYUlGOW18|mTj-HZQHhO+qP}nw#`+xt5(^z zUDfsPv%5RazUdPkkr$cqU4NP5ony@L?DHlWZX@kjBu0rd{Jpl8woMWoQu^x=HV{*E z;<_luV{A_1bSc7~BtRuV{^xJ^mn3!ccREdry5xP`Xynw+$3H*Z$uB76zu@32<}FWQ^`h|xK%_m zK0gHZ>b?k~RA@NIk0(fk3S^)nZe3TD>V;N08Qz3qp!glSeToET$X>0HbMWTT2es82 z-HsXEwktr4*kO*w00(ZBei~WYQ=#qXSJ+VgH^rIARv?-25p)?dql;r4`&V+@;Bj=K zQK_{Yu(YD2eq;we^+BkCWC_&jE#tM(l(eHvX&a|{uoWAbscH%m&nZ75l@1vlDN!you?yvouO|3q!WG7C_&(8pA&1Yzc z$PYL?fp`|Z-k|u(jQ)>%oL~suj@}@8w_GG2C;`|`Wj!H!(TYHL!qXc96cJHWOq!?V z>{DQ&FsvW&|2-ZH&!DN!|Lb7qv;6;*@c+GztM$L&JpXA&&6o0*n5eN>U8i+)P(qKm zTCqdDEGcS);{KEV)7c&~)NDQj4I8Y>i`nv1XUz(23NA1L zX0QYqW|)i}Dv|1dG(`Apsg*+38<%OkG6EdvjfY7LM#7Gx%#$cN6ur+07AE(xaW+@G_eFQVp;`{lg{HE^-=?U=dm>+Jws zaruOJ%|W|jhBrmXrwdgRVGvOY+=NQ>L3!MBvkLr*a?i%9zP6hiym1Q><{p7)VwJck z55d=XggpWVJRvcG-thX_jZZM7bJh_rlu$R&G~I;yOa&bjop$$gH`T_4d=LzNt6 zwKh){i4n9w_#V*>iIvgBcm7yZc?+2f&kytopHoqc_F~F^!Jmyml`4lfSNVO)YHxpx zz*0f`_l=41rix+6wyi)vJ5fU;>*e9$T~WD%dW2PVU3-U~{ybc7sip^_j!Atvz&6rK1$NPi8QMt)3hWyW%3 zx^q|D!oVJqSxYWrRJ~;geuwT-JYY!aunYYmX2@vk!;`$}}4pN?q4#enEI_nzY6=HH9KLKo{qb32|f(qTN^N zcd^xSkDgv&m&Q$&sO#1aT534u`&j%2m}Ks7z#cs0nQWE4J|T38_KuUW7O9CMeKDQp zKr1Uz-Zg8~Ql*2QsMTz)V;&bGsgNZs*Y)B%27PLUHqmeAlRSVrT}-EK3C_J$+MOCINAsdQ-(s* zYw;N0Ne?Ppy0bgSh^6?+T&ugJ&y1Z|}dwr%e zaO)MwVbP2;5H8-vG1|8gUzFP~KIpL74QF%xX)IVXnWntumDa2aP0lmc5VY?^yq@5v zVwMR(b#$ik@+;&;1HWMi1EvD`#e5i~g|nKTzW}D_(oFl;=)xwRoA`Q`54J3IK;!V0ioWJL8Ub8}?D-#=rMW(Z!Ok5X8n-tr6a&T52dBa1Hd9su;A@X7Z^N)9C^V~ds0 zqUr3DBPhz}7Rn#eTMYb7cI1-a_fGn!Zloue1k1kGYlHMy!tZ)8!WHm;L*MC zg{phEQn;~w&w9$G{kXz|cjL*o{q&8u^`uKG7g6h#C6RVM($*x2s{P*T)*D4xL=jW) zHjDoxr`%H0_djA~avW*mD8D8~A?p9nkMjS$C$8|zZu;%WP?K)^C5W_L@Gms4HfBRe ztP6v**UPxbjn#v-YPhPk1{7BQe2rLn1@u1`O1{h~sRDIBvoYoVnKjEAiHg$YdhPOq z#0%Phd*sV;tKHab5BBD@Kz2mbJ`GU1X$J!=iwHMPCS@i)Y`pMiN@Da98?$V=S0&S( zC?jqsV&aIClgKXpuSOx$lDogZjY6tEoF==S{|e|47LCujs1?Hh6cqsa>O&qVHzJ5y zy$Yr=OOMLyF#fWemb%%t?`|KT5O<~MwfaG^guY;6cY3*ezOD~pXu$!O;;hKly~b`HrwmhW(eIL3||*^Vl&oKmRY zfOZ6_e9R()O2JDq4V~ilBk8hQg1thtgLDMlTVN08e=3DsL3SJ!o{W==%y2~=7}Q{y zD3Fm)eBeHH35el;D}@?Tiba^BQ>U@A*ZJkxkq|@!V-2SeO$~Br*`wV%k)pj|3;WA| z7sXAM+u&gWS9Ab+s31IjPFt^MwcK2lPhe2{*-HiW<$r;BX&BI1HNmuE$?y$3{{iL& zQ^wB>N!=lsT%$FM|Hu3`VV(|xr2H@Qd-NBW7Y-fadzDqmjMk9zUyVZ0L|cNt?4~S_ zJ*e@OGCgt8(f_cU)Td68>;A)TYO3>k7)SVJH*HLe96u%LXQe6jBr=&MgZ=xjJ@Llq zriMxuHc|BGDUtauqK=~*jw2hi(H5@Ij{j;DiXP60`Zd45!|Svt#|c$`&2MWG9codJ z1$fZ6gqUuG%{JD+Y?^{WVl?51Sxh}HRv;bfTpvK5I=_~f@1t`^9Y4$IB!~+=wx2mx zennmraca+LT?cLB{jer3>VGl@a;?^d2qC>f2kbYdhV5F+ao+!SH;Hq|kxAKW*h5=! z*P1Gvo{7JI?QbcugbH|2!~(Z82BqnTCl_^^jJpKFa&tu;1LuX58>2jvHl1bNnO3p@$~yU;S?|o3-}KF&y?%?+5{Ft?T#hDj(a>2rF60+Me&1 zpyzxusg@OUeS#BCo4z-FI~U7Yh2GV>F8**heO`&+IFc|OI%ckDjnvLSrex-a+}+XF z6Zem@?dQJKWe3w$a|>R`zC)2NOxx@=~w!w7rpQz$C`Dqs4Nt|7qjcumu31`~T)R+j}ru(g_+knXvwMZdd2Guv{|r zx3K(lvL0YL&0YX61sWhy)MY;sbmN zLYmH*aP&pGP1nwCvYI=haLu8oSDm_fSZf-7#HC0tPgv=t{QBK@jD@Rl(ySy&&yr>3 zD9XxdN~0zeOF6Wol#Q3hDNa(l-J&WGuYDAlX`dB~b`9^i#^M95YV#gnc=3||BdYc1 z=yVHr5Z#E;jK;QM@WuLPCoXOqr_X0}9XJT*8m~JARz&vjzR8K{@Fq_7{9fzKc!DU& z<2X4l)uvh5H1$SgGfH++xyjXDoy%KJyCe1`1$2{#CTxD;XfODoe5&vW29@ZWrc%p5 z-89KRPIfX=RSw!>KfwIDnLB?&xVF}lNmIkV8324_nhpxGKMqb zvP4#fNlh(WM#4VwyQMi)a|`JBUF za*N`uxiqlkdr;sEnz--sS?LQZ>;klzG9@N;%R}g+CMyp$dL=QI$}Ee6U#9n_3;a9w zz4^jbvGRH#xo%ciH*HIm_)f4#qeV!13i+-!Dyw}*0W+umgGN7%<4k6hZWU+E;q`Im zUsEe0aJa%+cx9WMUYlC>z3XbQ3zZVOI~JvOUbnNl=FaG|;D-YG972_G)>>fSNmdEQ z6_rKjErQ@BXPkSS0l{huffNqhqD1;4ds9z|2YPHfZQhH&JO`quZ)=zrS^u<>a?C&L z0;{TBHH^WG;0_aY9T5AY2VoaB73(gVS}Lt=7SdOzFOGM15jq&deUgfswv%gX+ zyC#L)3%fu1s^ICcuWdb33CaHwP`lfT_}+>Ppva4D66t$F8b zcck3*go{nDl&a+o#2xKuGd$DxWeYnVUp_dpI4jc`MAu6eyXrTvKf9^HhkdoNL8|oY z@BoYM+>A7`{xs|Oghx$>ib;?6V;wmziTP+zL{@)1h@&9?X?M0`ViqTZ#um5gxqBha zwI_VzFXc`j3#~R73#eka8@3A1Q6$)NV~O)B7zJPLV_+3 z3j4@M&ERx8OXOM)zlUEL`MnD%nzOe=>>Ajw9azn7cur0!K!nbBw( zLD`8?A0#bPNkV0IE|kQ)-$nrQ82QRU1S*#^x3HeLI-%P$yu2)-gEB55FcvU2Y$Jl4 z`-rTQs>8MU)7Gvw|C_sHd#L)OuetnKrleo zCB`cKFo8SgOd8CI!&UBII2xRC<)N@4(s2^{econxPYg*hgO9Qyfhi5zs5@&kHXTU? zF>5%`ZGkb-6ZY(HN^?&_yRss=*={xjHL+j+M|muIs7;+x5fc#pWY@wFN(k3+Gr!c@ zV4y^hbgXU*9EeHp~Km>H3P=1nVp@qEJOb7^QOhuZN(W+|5mV4yLo`&eZhPp8Zg3e120F=x`@oLm=Npg7zKtO2UiXHrk zz6_Lmer0mXCk`Z0=8U9Q9Xv*?2;P31%MLLI%MAQHgrXBWH#wlWa7Je+A)-E7{6SZI zoe6-H^!T!DIY{*wFI1@g2ZUE6z%ik~b&>+4cd>9Ok6e#t+qi&w{Y6*;dsWg;Dl+TL0g&P{tbD{nY-P6wOJp>4 z5F&UAoY67Z1zNN$kt6g#1r7@LQ^F@BDxu+__>f5`2iTh-xVL!9EGPp>U*K7sM&Pjk zGW0Qz1qw_!00eI$N8WSmT(V{``$C|Z-`0QF4tS8%&cf66Cnv+GFR*24UE7SiCQ z!FAfgP|=7YW4$6p9^@P~Rot3XCP)LLusybAtl& zmG=I`gM8f}lgk>^V5~(zXruPR@5$zbN5io@T!fcn0RI~k3)hJE0LHaWh=LO#!T5Ja zDMaw6U|(2s4|@b*u3B~-VgHqeC(;D;OsL*bTg@E^}v|#>dNrFV@4gZr+*N zDh|#PfA7vIrl*&@iD%ndhsD_zyb2gp^#+r=SlDv9Lc&&1YC?u<+a_;t);WVFE6Yf zi$^{+fo5T4@Te z=5ARwp6f)6d5;2F{>^pp<-xRA-%I>eY>o^dkwAL)j7A+Q1+t}j^`aL=veA=&Nx+aX zPA9bO?d%u3l^NyG46I)+hhig<7x7H8d1V(C$kh!mhgxUhT7Ofy<5pThq-;DEA|w3LFrHO&O?O%->HlbsK$6 zG6?lZrtvodjCqKx@XYhC){dIkTPmlBNCSNpfuQ$D$5A_Z;)1M%`3c5}<*5FUjxmcqLtQa(cyT3^cJ=lrX&}9+=&C(0T>c@cRGv0YMR%iZ zMAkhQY@{;Ho_xp!+K-vkwVJ-gHS?EzeLbmbN=D9FEOdJDeH*vkuYWNgQH@12x^U!UUb-GLDDMFD+F zN23MvZk5@0REZ?UO^>4f>632tkJ8?n-Y0^30*G8Ydd4cU_)V@kM1GJVC7{=V$! z6Q}lkJ)2$5->s5o#@O%wB6iV<v_=ipcf-t_aY|v zxcL0X@1zbTN@CnhssEtZ{G;skfmO~Gg}`eA%hUt0X^9El_Bdymd;K?*Bb9$29tJhX zP~r@0yHU^;49=0c*=_Yrt}!ua;(F7KT|ZPayfM#KLDZL;R(o6{eoL`(-x?`L!;dOy z>oRbnd%r>3#U9c_$W>a3MTjdUU^=?SA?gU(FXQ!n(?m_^TEI~|Ia-=^EdtBnIPyg> z>h5{@)=U$pdSZ@Mw9d_)>?JvjE5R^PdUB^*_!&EQtII`JE5kV?YIDNMKX(nz@<+(* zK)_GNx+%JCa6H!e(Da|rxMzQn$8H44{26uR3)M*}MPfBu*+1g($jo_=MdyxG$qjFd zlSP~l*7w31m)@AEP)UxQJ2A($?vdKVYnn~@EcX|dXNuG}Q@Dei!K(90_S~bHVYp?s zGP~zP-Th>Qp__l&)iUsDm*PgKItQ5PK{W;wd$fN_r3bu2*G6+&Er2Bo#12(Ibu;a_ zsb?S06)K6iIK@NhLAnLm8G#!N42~z=aE7`looH$k0~qg(yWgC}M}3m96e6{3LeQL! zSpp%+51V^H@pXB6`WKT#DsHZJK%?}yyev&YKh2(%VL-O_{nMR7)HeDCpP^Ne`r6RD z)194N+XiQR)tlE0vV?wJoK}uvZQ!0bvC1Zrj35b5;<0(x_YC$^;y6R!Bt(;uBQ&o` zVpieLdZL3;8SubUd$>P7-ZSORo>Sz!UL>>k!)89hIGk^Y zbfO@bxBC9RZuC-=g#^%9y%0hN(wLbCl~Z*Jp3OzW`<%}Ng?TVx{bpUyXr@7RiQkwC_K_D+q6Wb3if*prBZaew{m zHRtmQcn=C(l>sJcHbrutEYNRid+B@R`G-~M0iRuKa14+4h7TfxLyOVd9avGMj$?h{ zEuWACHH^0o{X>p2fjE%lG*%o>h!(mwW@ zs^BHlDWH2Yyd|0h|Cq*}*6=_L7U!w0Z5FZ7Cs-w^EO=?4nws!#sgz5(7r1GGY&L^% zlID|a3#Swz1Yz}n?qfC31Cs@k_H2=qin?PFL{_8)@GJphRn1Gr_;q4Ss;B)7a@U&+ zy~Q0zX`x}LL08+D`8^%OZfD2wP@-m|u9?Db8~kyM$q1tsG39F18*--wj+mu-^d>u9 zvj^IT1Qf$9Y{#;1Jc|dq?$x{XdmM;b-TgTa$l|d-Xhk zIspw9!|U<@4OZiCe?hylv2&qwm-=|u+FDHO%zJ?;a!c`p|6JGF3iCyITl||Rk`Qh6YAfLe>^0w$A^n zL|EI#>7W(yyH+pY-W1bogmBjEWZi+<#W}IoVKkKojV{Ogo}ZYQKr8`9K(nm%?Pd$K z-%T>ry5fZSB;(Z_;UjQsxJimSW0DWn_TzchP3XSJ$4Taz*QdSr9J$CwVUnwzWT_~f_8QaKU3qmhb0buyW5F?H)3_A5Vwxui>>9WA@hZuC_8WN%<+GIG2y52hydt%@_8E z11YJPR$6rG(xEcw6$b5ia|E(mqx5-(c%hF+osTWArI~|oGbDyOqj2{l1_m=~R`G7A z0D4nB!%4)y_z&SST%i`4Z^TO5@%x-37-0!HC2Cxdo2Z{e+2LUh2{^TyF_=2d5>fr| zd5J`H_T--Z!WyJ6I?qS0o{u=_Huy$}?xo0LkRR9S&YZhm5_e?9h}BadMx@Z) zbCM}CMxXd6KP!#Y7Q9jxfNH++I-Q>lAMkVb)sM%4?b>q@;JCnf)q%u>N#yEKXc=i} zvS7TRj*kvxUj#;7Zgxo2pQ@CmaI;a7%j-8>*}6C|@w%DdGg| zW=63iGfuo#@lQMj@sFGg87I~G4mN25l6W9ohfMY&ooaAgX z)r;YUHEDE!aP;Irn5Ohpi+CfMJaZUSDq=H~l5N;`n6gq2nHUUVZ#LV&tJYb7t(|&G z{u#WY2!k!ElVf>8{cg#%pXsiLov$|gf$&2k)rtt~Q+Y%c6emXhBPc_wb&vo++ zDSfUYA<-^p4;CB^QIQwnkAG*jI}px;bcNZyqtk~*LH3}q`!S`^<*k-MG8gfd&5Mv4xuXxoPhiHx7U>c$|7YC$h)F5l{cPkc<`iLGG-UOw*nj~-a zIfx>14sh{-+oS?4gVMV7ugPv*hqh7TX*`U+fs0Vw-ci_cidVDffswE_FOO#6e2~d6 zq=~{ZO(To3DjlGuKuisPES;bAH~jFy_8R9AXd`h7&O3Q(DV}GSnfSCkLm@n+zPCJ_ z_Z0!*H@3xV2;|z0`6c&c8atsh!~G%IKcwF_$|Yv&BC>UTtP~Ha!$4wfPyG4}aVu4#lg!YGJw-pgQhE`Cqa`96qGL~^yq9k)13S4B389cJV zEha-SEl}ULzO?Rf+$iwF3GFWZ9j+=cmNpWRN;=`S6qy>up8lc-^c0^O(~-l_0MvR* z))%l{&0dSRnN6G=uuVP6C-$ z2x`)euiDif=Y!+QqBMXxVRVggz~A_(cBd`+#1aM+ORIVUAWAatQM|PQwyz3sRfXyz zM30qqN<0+6@xk(~C;v#Et*z$^U)r_!=~{>fSXRoTiXU-$9>p#h=kFXTdk>g`m%Jnz z*uB!{(7K`w!x%{zzz64mh95oog}_B87Jl69+^)Wk=qL0;(|MF!f*|n+a)1)sNW79l zHhI+NZ!bUWu2~!rTuPUkhe|~jYOIJD@L~bH+`Yy5yszxIfl7PAfk$l4wwSjj5KRt1 z6&Z%VH%dG9V?;3{tFp>1K5yNE&LU-oCf)*!WB+_{DiU@HOgXs!6(~}*QCq3(%L1hK z3MzwozXz{*^Z|tRhvJ;PHj24;i$063ik$^q&2!1R9ab*({rmg0{%ga_*}Cs3EZEE7 z9R}{m1hAy-gaSo;r&^@~3k~<1t(_)ae+rJR3Y%I{*!Pt=^W}zGTM_ujxoLV`{oZkE zRmZ%-ZI1oT{SL#fSBvM=w$?$`o;RU{qgaGP)j(4vpQG!6Gm zGJa9{D|7&`2OpW@q>&f^(C8CkDMB*45Df=R;p?jim!ltN9(n)0hPCcMUG9Zo3!$sV zxP&V}g>No!C;l2`xF7E=5Incq1pASkigKMbR@V=_VaeJx8Isyl5Akts#;R*6EXxL2 z-VKUrAKi&^i-=Jb&|^$}RBA~2s{2x{7&tz?tlrFF?-DZTgxe3~to zD?GawogY;>1wP{&<9_7M3?3ck1%#({0nZIijaETRfsqY9k1v3(8PZpFfKqz?E- z$F`vCSI7f8&%)i_Thrvn5|?07cy0!sU(r|K*^nWcpsbpmFS6m`LGt2O#UEH>M_{8l zDOU>hno<0JzpGiZIN?hI^EnwEiezI%ry_IFm1*|{@`9LN3@|~Xz$NbFc|>>A|LL=- zKH!fDNvH@UGkJ=bEjE68{(-YmJdXh;;RuRf*W{4!@65qIhWLT~U@6FDuk3UIM@0=2 zI%hw!AOC4d+EdIFItL92y-dFSbU`al@!I!#%?R-_|md3A1yQjOfd<$U6{ zb8GQGjn5nonCR*vPyPM5j^yZ$mUHhj=pslAr%s^^nGl}UO_iF;V#PUwjSXxW8g73$ zW}umg9b>(x>Y|4gY|)LL^$~Pgi(m5YyF)w1iNG-it89q_(I z`LMXO`?}7~TrBPn5sw^5Nhq^xQB+EUwYEH;EuSKtfgoSuie50i74DgpWZ~v(Qt>L? z1hZ>vRj-Y;uFa8|%C+6W#R%@dMc+{#8P@w(&0{;}^_g0AFG)?yOO;!FIiH@Gc==0p z79b%bJ`4C=c`vL=Y$8hP^S@Fxl&Huu1bsqcXDro6FU#<2QAcf86jzXW-}bs}=IefDP{iYQDt85);oiWXtuY26Xk)cW~vlxz+P3(0cAj;=7%!I3y zLX~-htVGEI=oUPy3Q^Uf3dW_`VB3Z2f{tczSgWrJQJK4FU9$2@G`ViLPt;8H^O|k6 z0KXJ4_QTZ9@I|}OW|Y82 z)aM3U{Q_5evO~ThVZ5lHOqepKFs&iT{NbEuy!nZXmSJq`dDYU{%+-+^P4(uF#S=+2 zErQn?*}}d$K#dD{p!F>stQ#UBqT9KakKK5$Ae%>xZG#MbZdo^z3kM%OCV_+M37@rXnVSQ3~D>&O%zZ{avod;mEZ#CIKkt-3ujhj`HapwNeXEr|J}i*??CQ+EW;M1p9~CXRU(t0n zs7Gz^wS7MJUspjFYM%oxaNvDKcC{@FJG@=r&udRrZ|&T zKXTk#EB~|D#XMEvga{G}>zn3FbF*zW)mmK2ck0 zFTo)MT8yT+8|CEu-xXAm9&Vr6EyVDAhj6 zuAAwJJU^D}5m@~?ct-MZu&bTF`Wk*xLDzsCeOt9yb?~&6>tk<>7m!Y00e5KO;a8 zi>rkeaTx8dza#xg79k!8dN*$#+q+M8$ppWgC;nG}nX~oB7LO;}hnxR)0JgMTjNADh z66ed&_xbMgk*E{9^YrlaRT&ydIk$gz#jdHs&Z(ybB z^^;(Pm4*-p^%{riFwybs9^N$xGn$C2JE<4*0?Q zY%-gFG_ih8s(9()3Gk9-!N?updmLd3AA_vE2`jlXX97IfQln?J9_q9n%cz2-Gc%HV zr26yeV%f8uzhilWQmpsH9XZHkfX%UWX1>B-(`;hb#e}2)xd83;&Xn)`f{hNR`CTSQ z@V4@ux35B{wBF@Z5U??5hLe0ew3j>NauOEp;5<=w5C6NRjd)VW`t0JE`vAWpha!5G zCdg>uBC_}L$H?Lm4*S)iC?wnw<~^X57V65C>`)3({Pt39W5M@7epk{QwDzg)zR$}M z13(D)u)I0a_Ka;N(dt)PJeB?I{G=ApjI{O(BnIx@V*GeI9NE(oB;{u@6l61vB{At< zx@ZZ=W%a^>;Te~I4#&i5Dpj= zj?V?ShBCsdeMcdG959cl=i5qi5WW%wsh56Q8Sz+i|lah}HHhdW$yeKkS(HoIWP;KVOVl>3{7%_q-(48ZfT zFN%&&38~gI&CG#(c6UuluKsYKSaQtUQhxnh!?XMa@WE*T>&YX=iiV~Z90wRSZKajR zTaRHK)JL)dsGC7avl|V1$1#Q(;$yiYwHi3LfwI@yp;-Q+BCdG zb&2=oM6+p&NvH$^D49j0l5Z4cW=b{P+)~xCz^sd+288q|IFmqC!br+So+zM1d4GZ! znp60hpULTlH|82{)+slCwj3M?mz%xxayU_zjv4(sl)N*V&gCkaBc~L8&EN&YbGY=a zWT>%WOOu$^1C`n8(_D$^73DHYPg(YrqABmZMSnAW2TF?7{cAZk9Mamw1m7H3^T8R! zc}8wHWt1wauu8Y3qD2C{@sA!6iV9ttvSP|i@0IChErHTs#GE0{gA3B6m;?xj^Tu&w z%#8JISo&&K>alD`Odi5EFD)T#b)v#9gGI~jtG;xkQ^@DKQ9Y(9Ok zIz$lQ>`qHT+q8dC&atBh^F+YQEwiMLH&?(V&kDIt;kUXzYPs5l20ya0X14I4&kV7S zH7cvB;U;}N?JW~)Gb5xs0zn)oU!oo1lb{Uh@4O3*O-t3yhmgc|Xf7ZY;GDM^o_#1^ z8p__8PS=(4_IdyTGORSv6cynTtP@INp!?ql{wBcpg?ooJVw-ecng8`TSKzvcJX3kI{h1E7ZKjHvzoeNMdJ5DUtwukUw zHl_@N@(Xb5I%CKKkUlD~TEB1iAn___tEn zEhcU35lIgt5mOE`xoc&yK%BphuBI$ADpn~*?861JUU8^aB3Nhxf0Bm0Cbvyqla@H< z?}cNlJ5%I*?y*I*RU*IbTkte%{U>tK_jfYwHOJir(THFmQHdFg;!1-@4LVcEDi)0$ z)9CiaePD%ujfZj?G}G)+$``pS(i~Ey%2*gqYJCPxbk0jlMP3C>=W94l_hdZZXY)TzVVdt&P)hkqvRb*t)gPf; zz?E99%VK?IDd3Fs+T1WJII=9=c%Sv6c<^zPwGlJarD7O)D-_34$f7+)Xaq zV^S+#;ubGDXPCc`MqTA`tf1691Zpo`+xcOiwnkc)^0vB)YstS-UEeJ6&|X)o z09iJxc|uv^I#IM+{C!}|J>Y(DX{qW{;e_!oHLerk$sB4ylQ*~#O;?Yyv=V;~2eTE< zpc`3sR63wM-nuB?WR|bFCVObdoH3X6GD>I_*kjyFI5Tl9l{-Z#vKjYgm-8Z9&9`j) zBkf+&?0^r`0BON~|BVzocYQH{%mM)b z5KaDnuDyg!ObuMDodxXetu2fU{$K5->Gi98U~U8f6r^KzH6Q5e3WU3m1L`y>FAr0( z@t%Wi4kZSo6SycsSsT4q)vlG#+IfFoo$W*~s2DIfaZG*UZaWR;WE3SyZ=qKrSPMar zCxh-lI%9#)v7q-rQf!sMh%dJfcjxbGh}$Tx9HBT*`*+;f9|wo;>-|d{ZCD+49$t>c zNMU>oo0rqzsf^Ph254@=vN`9+EKMg-(17?U^T~UKDo|Y%D9p%Ik`iWFS*)@f5<#?J z_;~r1#&&gviLzuU=pjl`QGq_F(SyxE3qdlP$=00gF;%+c&QzAW`Ub#VaBo@Jb^0Em?Wat zIK$sEcw%-YELhIGad2SE4ek~A$dkCt19B!#Oi11kh38Z7sAswGU*!dk{pp{?AT(QG zr#jCa;DAKKA#+(u6rnJK+4bl`_0>Ojwb?=c3UIxP_MYRX<9fZ&ww^=|z%9xy_95($ zM?FVfaUj@;kz^cvV9^|4>KIPEAnP%R7uen)n)wDYsxU{`uI2=OW)Zu<@=zK&Pb}Wv z?_pC_$#OzH91gPzh@O$?OR5L16bl_hqMPIAV7Sk-3uJNoW-XeSLbDpa+`C)^6&+-q z=q58tBGH-Lki-@Fd^)igO}UyshUFxiVu{}V+rkrnomBr9(=Fko&z{+6g#0XW? z#J1{IA&s6@G|0J#zF60`Z)s8z3WPx15{3XbiC!s80C~Ge47n5Ffby+Zhlj3Y@%2)y zc`LCxlzBjzoMD2wqwkrQG=lPKT|SN>^gdScd)6gd%K-H?RSMV^P!tH_(^NHaAN&Da9N z%x$>byIb4m{sJ{T&R7IZXJZ0HW0AI4h zyZ8t_l*LG!3iLgRN`0@cnpFbR(x0n+yS?;LzQfuZrD#LFxNo1OYM$7;CsoW0r&-CY`+RD^k|{&jk~$Wyz~LU zWt*%0Ux&~>e++x$lMC1Zow|@S+ScVZtD|SKzUJsE^1edGZVDX5kfm8%YpU8*ZC8QR z+8Mt_ zEC0f#ma`_;Mf4+*UZzpzcx~e+oHKf_U*#3ERoJf^F{3&+m`h6xsLG5zs+UzuYn3J) zyScg%GBk(t7c>P^78>-uQZZoGMlmfh?srpkz<-66tW0 zND=;LLIc#vOpF#?1d^mY$vFZNgySad%@_Z%%JhX}b5(Kkx((2e-BJUcoZ5!3WmFu- zvinIBKR1e(uh#$&wq`p(WSL`OS0{&mv9~e^fT|tRekD5@8WE@rF{4Dw%`p9dgQcGW zel60_XsjBlDl1!W;h+!&?R$)!?3A=VxivEUTPzh0E?ZBqh$F_0s zRCI5RkDi;L%lCI&(xn10IB(A-`H4-iNB%_9-s#8}`>{3f!Q-xD%`7>zv~~wPZlW-A z+J8wL?@yHGl2phZct;a0*uKmmc#{7)xpPAw>z>+f7pa!DXXGr5PRu9>9lAL%JetC8 z)+=LzioSw;F6K4UCEK|N4_+h)&i!N^#{Inv!&sj7ZS&Fz8aIuI;QNun9vk4_STp#-q?yw;i67Th4 z{B3Zx2`6!F1LDAOOFZSwgWGh@=qq^W47dnsVa}TyVCo|d;#v~OA>MT2uYo)L%aIE{ zChpD`>OQ89x}A3edrwLzTVu%u;_#%E+NztNM<4mHRDDCV=4BBw7rK^P=U{Ik)YTuC zuvvk&Uo93UKXrL1>2@u!Y+>?TvJtis-TV9v-S&#aaS_g46eJle@UZ3FWa~MAt~exV z*sykZBqwC!X~3&En<(&kZLx`?2?+DmkFM%{o|m8v1-^fA8jihtNPYk7`j*l|K7l-- zb5^qE@6$#5&-G~HU;;=(ac>brN3~vR$#tx^v&09}YW4F!Xt>~+SlG2tze+>`06^sb za1!+2YDCPy*~HDjL&?O^)x=TS!pYgh*2M9@UF{1eOb(>~REgy%Z|zeuA@e16-D9?f zJFxaP#Z1O`t;^9e5b=-GXJSDjU)%n?Sn(V~GWo+oZa4m4oP9%dW>L3oY$p}l#uwYF zIH}mSZQHhO+qRulY?~Fl{NujPy{&g&<1}{noH5Q`bFPU>6hv(2%H_|CP}0MPp9X@+ zuN&9YQ>E`?>@DvvJ8x(Is*r<^i^BnXuFn`Bdy8IPp-(8tESL$9OYWJY>fow+q`mZm zDHCQ-7lTH#!^fNK`n1>5m|0AU77Ct9W$y3td4X0Vmr!vqR_r;#e}n{(g(adO>xR;G zrd(fQASn8RJS=I246n$ZQ=(7eDE64xc=u z_YiUMP?t++^GP3Gei<@L99#n5%0>4SL&#H85X-U@_x(`H#grEqqMzc^_DL3abf`Sw zwC<0mh8H)4wsNxp4u=XckbG*Wm#n6XQ1HZT2c!+NcEZ%eVpU?AmFMsb@#y&=lf7=7efxjyzn zmpc5 zxP26u{?;V-VNvsk|5V<(meNM7G7+6%nF_23s}r4%tKsq?zF8{+U>J$~3z7ZvDzqOA z!Fq;!A@C`;}oASF0=t3o@GM{Prm1Qx&k(cfg~hD(huzogRKcD=d{D zP)dQaokBZlGT_sa&JH2WEoZRho_ui1DsS9c?DDbSx_*)`-Lfq-Q8!3kMn)rm?Qdrg z&R$MXphz+I!i5wF*8HPBO&T0{+?W9Z(|k~zK9Cfyhdt+%XDq`c7|0Y}*5@x3VLzNS z3&36T!d|1#JY5sWdO-mDnUmux(8X6KWLT^YrzV@DwIz`AqSpUNg zCH$~MlcU=f^TjP!7G%ct$+5vqnz5&Z`sX=LJio#fZ`3y~sZ4~1w<;yeNJ##qs@_?> zVWmkV7wDL`Xyr&YeK}YxzBL+^OknYVL-UiUhYC@V%<+S;!OGimG0uBf*HbIG*PH}E z9Z1xW9Qw-3HV>Quog-C?HZ?(%wG>srSfCjDL&EvcS}0Ymm{~nX$#XN;{V_u5=F)DN z8B~E^$*t>SLjN5+Ft+HimEv|n{~(bbcb=WJEVe+QMu{=It5J7vK&fXS9T=}AiIZ+s zrc#EoI4lTnenA{BE5fK&{^mHOmlWm!wvAlm2%S3Xh=#Pk&I;Bc*#Mj;C*(^twxN_4 zJ(EDK8lzpE~Ngi8h0g8Ujqjm}QbKBes|Z znSNvkXI0cKt!I8`ZrC|*e2+-y8^zso4~LVqX+))<%5bR*BIU*nbP4_mx?CP<+cfde zB(tbz{ch8~^uo>&Sfihz2&^S#RN!B{0)r$xjADDw?~oHuV6aG_1OSvLmMCpSY4Xt# zduo}w9|M}l(cbrA1u@oMd$8yLrS_kAm{MsvnK_@yU ziF_kT_bR2_l5VBy81x`@@yUPTFZfl_gU-!W9oMZXFVK@(&B~%l(y^&%GAVQuhfB9t3_%=+=b0yjzUh})AUE! zhI~ChNVdeP(v4E>AmpcZJ>h>c3|;`u-oB3YerBl&&?~6O@M4)z=!=(%HCQDEsOd)E z*^{994o$HDyx+(gV6bmD-4g+yaE~D8?CwnD2buSiWlerHu`wnHTx_&mc2pV`j7xvV1 zpKkb}W?K2`&tTrMVwhCpULR_hmR20tW48rd-qFZ-9SN;=RJvXv&lFl>0OZ`;xzp3x zp<{~&b0i>55y0Olz4<4B0{nX_v}_vaANecli6RZ4>MJy zJGDuhhgOfn>g12P?w1aisfRKh_Q}@u-?$vmi__?ry zG%qslpZ4D`hM!g=FMMMnFU$|!1YK97=W171b!%&F|Gj&dlWQkj9T@e}dS*UerS;Ah ze)T9+KKuT6Vdrf}9Yrky5Ky*15D?@4<3Rj>C*l8^eYGuZH#yKhuXX$F{nOFHs!PtC zFsv_waY`5A?uGVUm~uhX3#?^~=SdXpyE`9tUheZ-jV76vCj>!A>#6wY5pQ_Airk75 zb3;~33;Atk=1S*}z&qYP_Vz!}eqtT&ZVr?NF=ImR2QsfKNm%Y>V!5V>a?Y?v)|GLZ zO{3_xu|^!)FX>}DhjOtCH4R6OuW<&+g7#~sFI{?_@0CgeYi{pHvJ95a1VCm3wH7m< zH`Wd~P1c%}M|SE$y%Y6uqE%1j9{QWYgAwrSo+ZxS{#_N^`|z%s?p`+_H3u>D6I?AV=1~_S}ljtJN5(2 zYF2v=#=E$LogJkwe|;3D+|a5uaBXX+Z(KVvYC!p9L37!K8#|nKc2(CLtiNtP9qwX; zy5wO#>q~e{A>J}DDAa~>i8IXmiYk0HGJ2n?W5~lDfBbYy=`Z*&0KT>WaKLFqnW1v{ z;d~o2!r!h)SF$jd@EF<5{6XjhBS1c)MO9)Rnh#3z{vMoN4U&+>JM)W;az$vIINPE}3k zC5@sXKo;@~AWfAXjy_BjG`{0d(Krj%D6-ZPSJ{1EE+69Es zE7^UqVHe+FO6KzWy=>kq>03iwzQsyl^x-H+CdoHm_1;U@oMUa6cI_t;3$^TFi9IRu z*5rTu24B*PPa|YP>S?j{6FdR2WbOHoY&x%N`_Jq4`I{9}jKAp^sG3@s?fXQQPDqsB zvwUe7I0ml^K;J7Jd;@C3r^I&edlj!xI)nr3=dugxsj?Povm_fuHU`6-kC`UnV^%HB_f_P-X{HSe~sjyFl0|Efx3w zfWtCz6x{iefx`|neNsV`B#FJwLssUK@oK8xM&xO&1CZklUp7iEB8;SO7HQ++`DRw{ z#)8=MA5C^UP%;D9il6@w82gKZJ3kNrxG`Oo%sn11*dtjExxLObr_XiK7yF1Eao_%*3+ z_rGQ$UA2s9VRTC0sok&h{Wj~(Y4!SF;CKkMVjrHOk$n!t@v+Mz_l)_oVR#tb zXgjNADm9M6cF?%&tF2eVt)il_@qD!E#9i1Wb_El%4sUPrza->(O;rc?$Jw)d64wvL z-yaR^TthFN8`| zXK;WJUfmS~+=}pQL%c)NbAlVK@;_8<4z;n}v?HgAhYb&^flXxK9rwb}0=X+ko?hkj zOva^H9Nm&|mReG2BTtl+m5q>XzK2YPWBLh!1V3EkRe|30$T{k-_+FFYaA|*1y@;GPnDNr^i15CQ zo+`#h6}>vEEs7k`u$fAE za)FhU@rZAJQ13{mO_r3TX-KR3jp6To*vW1V(@FaC+o(INMXtI>voD^|PHQ)4h~!Sl zNfJ-L0kyX(R{LG7=pGEGsorsf5^+!+2+DXYt+-Jp+)RHTq7jAz&VfCN_1qU(U%(Z2 ztDL97y?weFGjJ~245^;58kLqech!6i=+`a~HkOxq*%Ko}+c!{2W9X#g1RPqyT4Dq(( z;R{zR*4FQ7$_=R=enXtyMAFv)$RLKhYg=lfPZGM@FUlQ}sw+w!P*BYL{M{%{llS=muIepHYw_{pTyDBTa zEDmUnhE3af#S_#g?bqzsZCQqB+ERPk^4yb;C3B8;uFbhV5az&Z*XB>|tvU>nK2MDX zwXHoqW)R`M`rK)|IG>espl{X$o(p=0(V5zG9NB1-PbvxaVs*@5m0&)`dFFtuOwT<^ zIV!+4cMv?G#o2_Y?7YtW)t?dmB6kCMk}fF)fK1Olacc!|X}M8Ls4hNb+=AQw61x#F zpzdY4;ZrZPJ*5S42~ItNi|^_Yr`Eg-L$u2D+{4$YNM_+zonBp2a^CQ)vKgmP*}L~Z z3-jPx+V5tE;Mtn^d~M3eEZiGcJS9v5apXXF;Mn;R z_&hW)R;>=nY%Dgp^xzci31KwvtQ#hA;Q4ci;=xm4pHGU{Qcm(AT2>iy-+4@>xX)fR zdJot+t+Pimn1BNPWrrG;0w!$H>uK%?_Cr;#qp44`oVK7UUlxl3!OQ8Z@t2)`I@i&@ z>17-4=Gd>4a;2JdNz>B2q5nG2$1<8WY=}{-7x<(@)O2>M$(PBFW5#bUvIPVipC?(G zb6Me}yM1jO0cT{fDU=A)50HRG@`NX-9X{O^0`{}7(fKnF2=Q{FAx0uU@UP6?9vvbyp)C4B$lZ09x*H>dZJASiLH+QYZUd z#-be>MgYIfQGA>|tX%P)LeK%BLx{8+*e#&&0nXo`{L(SZ77z#W&2(^C>^VDGRY;ZlL4B+c1!9-10lI zptW$^UxEL9oYAHf?cc#5kDZKWjubFSBY&CffeW)>hEjd7fexN45CEz*pU0Ae7EXju z9?N`Oa&&Ehu<^nJ_^j*xc@1UZLp2DxE)%W+yco}&S}<_9vSZ8^7x#?hv)BeI=bNYd z>VWI5>N4=B7Vd1gO!u+zQgN|y@}xpj&VcE$vY(Ui(nwu0^=oD?WQVG+!mV$@7kg*Y z+nC-G#q#&N_=#>7I)A?M3v@>e6rj;AXze!)m4Sb7Tys z?*#}Oev{tzco>~2-|?`4Po6EUGj7Xr_pVfR!DNS&+tIVw(b4=G-Z{hC^n%hQVz!3i zh+>?T>ZrH806lebX2v2_nvV2x0+R)a64}8vA(&TBzMn70rzq8r8!*ei0kBn=!XE^Q zQ0m^FaBi3yHqCDEu8%@KEBi)Vh*5x!ust)h6*2jl5|Lw@aYYo=+nl@d{_KKT9t zy*ZzupCHdQNOkKqMJb~owO6Hoz1|C-JyeEtFV_=|&O)oy2vOaq{dp(T&)#k2JWz^| zPeJxc#uu4W=4w<@j!hK}O0Ar23a}aJuVWEqKg77W(_{cfvmZ5dHF1hif_G3IDY*h{ zz{+P~uf#WT}P>caetRPkQJOb!hZ> z5hUs>H!{{R!;6G2Tp4mVbp9T=UucxmE1eCl<7UuYR>7n2H8VX~Tj;p#WiAtWzwnsn zPFGNarKYc6Q|5y`ri^oKa)*Ifr^2#7es}B7DW?g+kYKS=ssk_`2HVHFpx)%iS@9%> zsL$iLVO;-`Ixk&Alz4eD7P3bO$aEuWZaax}5b_voY5s%R9!T}I z)csiKL0m1Ie=>gO^=)r`j)pcg(s|9$cBhKfF&ZsrzpD5f>$YA)`}WbZb@Fn>*K6C_ zf3q2VrWdQzUK!-$b8-Ek>FV6;-0I{(n>H|r)n)cu#j$~j{5j(Lq-D|eli#64Z6P=-l>|!_zkZg)8h4k#dHO!Ys=J32_+`9OE@CEa^(AzEeLcm~C z!#!rfHFHCO_ag$3ITq`J2{^bT1k5tFF%kjRr2&O2(>zVB*zHKb^ifKot#B zx=wta^|OgE&1QrOqfxOtEV9s+A(&n*~Ag zox-y)!y1M_&}NARt((TA!MOo;g%;@^0zEz-Lix^p1U|2MJGj5E_CNZ)DjuPr;w}y- zLGFSQxj*0(nLJ=`AAlOgG1*%}(BcAzNsALz=P9_%O-jhA{p|%FrNtGiBq+H+E@9dv z_Ut9phxE0L6{8#1*m*$u;G^Ay4(a=At*DM*t&9Pl|4NzruPN5>rzw3r2CPgX@eG=8 zmdlJrfEj@^^X>iBLtnq{K!M$zhU@?|#Xq{sgD?&6}w?9A`b@wotboa z%O4<#ta`a!f(SClm^yyyC1E~3O%!Zhs7qjZYI$eFx108wNWRvT^9kEEE$w2U`%wwU zXeg?{6Q}a0xS5lBVPDB9%7l!v02oE7&#C|#0R>x+vLC8xY>au~967i3gn?YDOdQj# zdwG1zpCnCOHrz0sObSn8pWis08XIvxEgGs#s`;CbagA|4>xnLOpI(HF_F+wI$x?mo z<%lsH|Dh(YuE0r$87ed&h%Mc5NAKpErH3bKmR)f;Msjh=2H!KyS(eg$Q zVKj8$YGJP*O@u&JlMa)4n>%(RWqugKm;3Gx4oV#~3?{c|^9#o(oFJYIIFOMLT5Ei( zbd#LIyV(`svZy3`@%T>k&OSrOtA16@}eo1Cnst2u^V=MPuGs$YXi zJURad^f2&cdCS=Ol&znen-~5mJM%#YQ9I{lR>4xI3PT-5UuE3+C78WwV}J=OXT&xt zHVGnjLR}_7fO3~}k&x9>35&_e0E$_##DAZ<84f!Nx)2wVo~i>sn+!HqU))}=b1%_> zV95Q%FE$L9C*NTRnW}yf?*qDua#d6M0>+=v2nlxgfXF303??B&z@RxXwPhgnWjs@1mf&f)v+nlNpcuz?PBxwEBmz zSbU|Ej0X9|S9wmI`=R-THw~a(?lyQ$SRxB4ppl>jX;u7Eln&Pu$a4#V@&Cshq`h_$ zU-6Z1rXMes>}J$U=OuC^%M~jad&wf{8C)wM;Rail562=wZGbTum)}&eCKHibRRDaZysK`V_-gjkZLOAU*XfltV)Y%!sI|KQ zFM5pt8V4VY5AKs@RD#I)nLaow+J>VArgVYIm};K1m={Al1qX5|T(BDoklfV?$Lrqg z+q0|lV0;b>r2X5RWieHL^Eb`FlP;86Ud`HHm^_uwx3|SLReianMjN9xA~`oMJP<$! z3=za1eHr6%_ZilU(Hq{lucL6Rn5=PVI-xp6*8w|xRta47M=7Gpx15fWHXyyQ{k>N5#iqd3RXe-(9MWYL_>T#S zK?z66tSK627j

      Py)-jQfBSA-*!T3XQ%q79SaDktIcjL`soIB6DvTzc&J20J2s9C zIL zie&Ua6e(~eo-x!_P4?Fe8#hvsm}eRwDicyeuNSD|zc$+OGd zr%wB?i7k}Wf6Nf>BYC73BglxkQ%rsAM0?ng9At?_a~iucI&hyzJGQm&y-uy(A{rkV zt$_2&S_wks5s@kmJ*^fCf<9?mUm=y2g&JIAd4IGp_fq0*9 z+(?+AdGST*CLS9g9;9nD;~Q*@7P3+oest+84v2gQt28rgp!W zHM|su6m72QD;SWch!w^j3#*IRg_F+q64W6#Jfil(O5=LNJ}~!y$ z*8-yoQ{a;`OW|_!tJAZsv8Kr zRj6|*oCGX-qVbQ*ev1qFXt-Kqsc|HW$E`3EttC%?@kBm<6FymE&>aQvcLpQc9IzH( zCY?L*EUM(W9u9Z&06v@`Ud)D`WX@BeEdSJGL^Sx%OR#;@sqC8Av;&^uJ@(>A^cn?W z(TTuDFtEhGp8RupYdak78#XBUd61rr!P%!#-o#wfTE^fXdM~s%R((=4(~H!N)}SWQ z+h4`r&4CZ*&2i=Q;F@it=3r~}CoLNTi~IQB#`?y`U;f&Bp~mz_hW?9B_=`c5-do)O z$kEK;Q@Db^ff=wl`-!D%3SU%wZ$md5rpe9p600@Kh^RH~p(&@%n+Y^KH;vxxTRoXJ zrVe2S>|5LQH)V#oJA-L1OewM`>^J&aY2#-jofD)r1@7js>Kf(W4`qzU_$S+0Og(i{dzHsB>Q|(rsw6Am^^@?`dRCx{{7>q zC5GtolXwAXs;0U;s-tt22q@Y)I1=jl@JSMpK^j?PERzVcM{kT~ZEOABq|WIp*sTGG zj71HNnu=ubuo>7V3k0Re8DIx%m(BGLa+wNgELUp(Nm`#p?Lr4Zd(Lnc4D=IsO10Ufh_<(8OyBwU**k zP;$ryeuZm2J5h~Wlo(j#;uPvarYgUbR%riB&JdtdR9 z#zC-0bwRGt{BX)I4CNGt7U`SjuG8m#Tbm!sX~mIrkVKG>n(nd9?`G?jRZo|!HjOHA z|6r?MFf?u$nzo-{tCezZ;kV4I)3r<6Y^8oDem>s^Q!ZnpddPmU-S7#`*;Wc1STk<_@8)Wi{icVT^wyY1H$jPu z$hL=~_WS0u^JcsEGY3xG@c#JePIF%2G>HD{6KWT~YB>x7=(vak28Y6xr76sGYu;ut z1X~7h*B$4f-x?02F>0$}+arFcHTv&2$hk`?uHngD(0P(bG z)2fx*zL?Fouk(UUTrFysD`8FZ$6Z0d2Aw5Z7ARaGYTg)1YmW44Uvk*@M1c;I7wz4Q zH(RnpUkd5#s0N!c4+ORdA4vAONp$8SdGQ+ek;*=t8~h8%*n0Qq9)Vw?2yTYOcX!#> z)`=_29nT-k>;B`k@apf24+xkzh;=CPrv6Gb>^(?$)rYZw4d}Q3e4h70{{)b@Wem!7 z1pxs2rFQi2o&-x*CqD9Pmf(H$ygZHoh!Azv08Z_IfxgOfH!LCDbZbYWA3li&kEB3h zbl|rTe(l=N26Lf8Y~RlT$mwQ^Lkb!u9oT7}V|{RX30e)xskqZ3i&i_BgoVRZ4b#~H zxgWy=?Eu`PL9P2=bz>d5uD)8++WqwmjEXfb*7SBV<P%}=y}KJD%vYk z-+r0OU(niZ4ZsFd5UuCyfb^+7cd!DcJT*=1o}sLD-e5_jpRCM`>4}@9z zW*&Ee)vo!SI^|2O+Mqk8Ptw@bwu3FJ^9qmYrr>qkfxq#-lv3vd;CXin6*`mlVj<)E zy_phNys@Akp1x|g=`=r(q&@=zzBURTeaA!V<8!!{nr(~V!9M$IDZ`@S^+zC#b!CP4 z!!i-w^z)y0L8iQ`Pi8;KKtE$S%^zN?Q@+TOuQeH`#@Lby{~d;4igq^%dqrKUK21ftaaUi<|zx=#%?`O=#$xZ%iWGv2 zyUk1n{EWX8;yL>KywI!TfzP$jIicGx2tg1Z@e#>N++@vX5hwltw95Jlo7KB*;C{h{ z+OLpsmPaZw;hhALaJ5bw{kGE4CJB!dgHo|_!54N@6zIi*rY`;1ohVuhe4(r61MQd8 zs()w!?t*`c@PYNp9u>q_Po9?-r3RL2eJ+{#0Vu_<$1Z^f)50WhYpV;C11@`0fH?&t zqtY_CB+OE}U!2X=@80$kr>`n9xxoRmyJ6i$B=!K@S#M;9=k#3&NE`J+hEm{tEm<+; zC@sScp;Xm)?XF6O`d;^2pyOg^qNer{D0CJln3Ub`206y$L0p&+`HkD$oNm#47Lu_@wa zL(qcE2$oiP*i{1&Ss)`hM~pV-Ma7#<3rSx_>**u6m3?^cZfrdKWQ7d5{4ce>tT}e# z<#4xS<5C=(XP;4TntLTEXUl#|(=|{J)F5d*1jG1dd_llbP&Z!(z>TX=Bq&{-&Gfm)JQpTUJ-{m zaWr4K%&@g|wBR5QGVj(;_Ks5Rr8V~j&9SA-<+@pPv~?_l_F7M9>nn9fbLklpM23TB z)fHCc01GM}@5NBi7o^$m-W8(o&CYA>?AB@`ctHf~**_ z^n3v@b#+R;g7I*Jr>2{wUG z&qX1}y$c4HJhYT%11YwbB3c`Qp!S``n|!9!$k@jS^Ss&hGqXYrDWPlb$Uw-^GRUNbsp@)& z%hKaadj!E+gaz3AB@vu#EzVcnQ`_#Sa3=z+`#rt<8>GU&jSogpq_#k;pk~PZL0!{5 z#E)L2zJ%bfY_B1h^j3;A$$ed4;)_2XR+n#wkAS^HISvMhyJ12b?xLxAC(pqvRN%)ypp>zlzdvPowLNi7RidAa9cYrhLZho~QoZnF__bBMK~kdLC;l#? zAS*T7nx=1&x;B87fV0jfyTEt05HkM4aEOcHRE+JjvHay&u8p|P7%8C7tPse}1I36ytR zj){?fIYUL+rj2Z2-Y>XRi|MIQbuf-Y3>T>ctt17ZEUu*Z0ADviUAq3&Ll!WgP1loj zHUy~TWQd(CL9>j*2llP>Q+VFOpK;MqB} zfbd5&yr5{3f?DsKE*t8k4f#qD+{kf}B1EBG?UY6Lg#7A^g7v;qH|4nKjwMcib2^!S z$HDY1%N(J23NOD@r9uN4@RRzo>1nRk88Aykt;ag^QrzjsW?rd^b$N8!#VeeGVqx+U zIS$sm&f>(uj02B#st{Ps_k*qgK3}<+X~B8kyYH7&$S62-3!m5vGZK+p$L+A-*9KKR zlnJ*3s(xn=B%;X-?~_~Ab<2Ento=Rm9QSt)I=3v~;`%duAu(qjT&l+%#MgpZmQ{bI zshgn1rIW%lYZn1@(#tvxPtZ75KdbX1VI0tBqRl;{P2xnx?tq zT7G2wAkK#tXnqc<+iJQ7A$vJ~O?KkXF(3JdoNt$=lEd-$VND)~S(Omh`4iKv@n_FP z&c8;i(_1ovuki*z)+-3C7KN7i1D8;sWcFY{co#}@SuRDy>)9WBQu^`)g2gjtg@mLK zo&paMWJB&C?|Phc-LMVz#lGIoJvtkZ%jQV!iu0B1hG^4Xjc7biNqn0UDc7q$9wq5t*HAj#6U;C! z5IEU+Sl}Z)GA1BB6(gPiKhKLjJkX8Iuq$a)c{0UXz2-fII34{%x4@u_mYdf>fD6oK_!b;F5{Vay|{XIrl>zX?It zYJEvRZhAb1AHJQ0c5H*SD zBk>lTvt}etp0i3e;T{Rp7qAMb=#Kxghel_)LATZxB)M7g^7F21mVCD|VQ7Mn5W08! zm*)tP$nZRyz|xRRZf^WJmCHG)cAGh)fwj#6hNydF+>a#V-Q#2+q?-z*cd$my3C?;| zLNR!Db!L-;AiisFb=rCsD^#PD`Wp$7GREJ!?!uR{M&bsG-&-oYZxL&{SPu476Jbx5 zuN(aSC-d!E6jj&Z{EDf8f%ED4UvY94nxZdHBHrLUNSo4Jiu6`5(LopJ_vSfjuQq9) z0Nj-chVrnys52D|762lM%k2VMoGBz`HFHeLko;bhHDzdCaZo~B-d#^q@vkmqBit55 ze*%iAYr%=+esOb|C8fYox1*LoRw!(M>05?VK9Wk1tOVkZjymIB%{muQ=sAlhDL}E* z4yo~&sSomQtA-yulo28UOpVo1$QT&{z+LhSKA`^g@011KQ&Kg}D5b4Hak|8yQ7;WaPB%g9> zXa#PlWR*^;qoR~`=2$F>Lui4dE~W`D-(JV~Ru$8-Mcc1>b9hqHaOYWdFNu?yQ=vr6>WFPQ`#;6cEB zm93s8cJJskEiRoVj=>i7HX}aWDOfE@U-Fjc6~1!UAg@cSR9Jg}(US%Q3L%>!2G)lv zdVclek9T9BMaD;4Xt=MQil)EH!S+_ApZRKf!rFf0YuY^9@hviYf$F{zv#Ff+-(8w= zITH#|NiyOz+7>Iti`?g>VaxZl+C#1#{wtyt4GeCtql>!Ww%w@hGmC)G0s9S94CnEP zjb6&|8b-iAQI`oAM(;~rKz^pXv(WZwG{qJz?hD(MSC6gIy-MK38{QEO<)nS^xl$XK zd(>YqOF_{OW_mu#5l*~T(~8pd7vw(^ip&q9yY% zSAylyyL+#ZCm=0w#Sl`+vLEE!%CpaJQfD1Z#%Ff<&|!J764O(z!+>Ku-ssB)8x>)c zA71XRxiE_Kzaci?k*e{tyQ<-~05u%JY46@OtZ~X#ND-xwqZX(RX)@>iOP9fcF0rZ; zFadd{epCG;%nHV~n=w|a1K%JP@f^K|O<5(}Dh+cD10`v4w_R76(`OSqLt+$hMXlB# zP2gGlTK9-hv2n=Ix!v`HO-sE@6~xS{1Ol<_(@!xT%%(;3C{h!LMMH=pI569k@Ft@}Gklt~$ln1Q7-y-+p)k<6(SxEb&o=$%Vz_l7L<2w+JX^59&0Oz?5+F_@`nE`o~}z_vH5^Hr$0x# zSRGYBr=x!B7q94`P7V&ku69J{I1>|iD;o-B8(43%0FcIbj(+EuFZay~9WbU_^}Ztg z#Sul0w7^7_YS}2mep>}r&Li6w%QuMsjT_%BX`a+(XI6cFX?G+a*sFqcBS1M%2O^>n z(8weO_b`m@Iq)(&x8_k$8o~kx=bwo~Iu1>0TTAC`Xdvoo9lrZGk+Q{r+^->G7XswE zj(6^9bhh=zR+TuHFV`c^7+%zF&mdln5ox(}eCzojG-g^L&$`{8zIJ;wRU`avGxa## zz6e;jo!~c7Y2F*AYX6XcIQg6p%X%L5P6$I@tw@}2^W_~mC!ilan4CQ<_6oftFMXA*DU8w!sVyv?x@pk7Hl>U(pudfen7pRS;homd(40ZM*XUHE zdEu&*ir?t|!$N8(jTZkT_@q?N-fH>`aynzsVg74)rg*JTo|(j*PHELjZXit|$#Jm9 zTbJta+eW1#<^26Hw$a~Z18FL%vB6d;!^Io%Mq< zZu_3P_)~-)_IfoN3)wc>9WAt6)dj;|9MH#K)3*GFL{qE5u!6Gm#%{iF3bIBm!V(^j z^T(9SScwWk_N=YZ{L|2zN2?+I#qh?+wnpsU(G(;TPszJotg|RT+lR%9vCanDoO(kd zDl^!^c3+>34WraPFZ`^;43VW`^=;2 z^r_GHLna^N_5@kx09d)~I3_`B{<~_Oi;qrONSm|!93F*JCp&=!0f~ar8c`BSrs5&? zpx%WrhXi?o5t`RT+OSz3GX_eGGVGX3-?h-#Qhq$3FS}AlOnJnZoxwf+Sy^8$Zg<4N zc-YZUaS9Cu6jGwA&xrjLBh~`kVJ%4Fc`^khZs~}O4|kL?QwFe=+lspYfOw2qAc^4 z^YadaxuJTZW6u;V|kFc=# z#)p6(0Tcj%$p+3)}76}fBdGe-QG25o9m0+=C{|puf)x{ zTLI9m5HVOs*l??ne~zRfM*MKvtvIfnBLYjJFll_u|*pliVDBGYPe)EC8&xVJ+M@<(IAJ(I9Q)tyAm%CPr zCNaR*N$ds3K=;#FVB+v7-Vc@F%vCzTM0YZv&;NpLTD+4?YB_!XZ46en3fJwn$sc`t zWjXnqMBRa0h&04JE|$fqD&|fGNQdw2YtmkKRA)9y33~{4(pvBx&JR62%agpz1qq!& za$=k!>6%~-p*#_yrHy4w3J?fELP!=LE}3lkYYxw4$jzqf2a(#jVU?Gx__%tQIsHS| zr?)_h7EC6(fx9WAIU+#$Q^NJ(1SBZ*iafGdvAPc)3oXtvZRgF{21AvYIUUx2qeTL< z%zzqq+*r@nlwFJ55TRnIRE8hAVb99t3S_HVr`Vg|M4Kg!-Z&W5gdM7)r&nG3=@v8s z-UtGo^o%6jpSn#KTe93uv!S*8TU;)!tvqt$bZ4k55i*8={`X;eIAdz7= zxO*ksR`DRU75Fb-NkmsOtvZ(mu|js~uvtr&u=wCNucv$rOH+GQNRL4L)SA4Q^ zWF(zx&qMoaARipqKgC-GoA`J1L`tHmd5OT_fpnqaL<1y=d``(GkbgiGA=^MINi{Ip zVbR((65eoo zx)%uQ%m)kUuYLqC9^~)rUTeyIcfAYSEt6IPlpbY7w5!RjD&4?HZ?E8w&~6nJ!t;2F z(wgEc8rg(7M$L+s41+xuT3NF7%6{4DGPfp~X$iMMBCi(ZV+ntaNOC zTf%ayZ?%#kvhs>du=nrA9=J6bWk?d?Bh)694y1k~xjP+WJ{#Kg0N7!jB9Px)HH`!D zxfe42fJ3R3^Di2r#OIRKVcE<(spF%mAFkf7BSn*+z*TKxCDdUHE3wfrinL= znPZa6A9e&eNN|wAt%eh@vSot6vM;hWT8KtH3eXHir)XNdG6*cw;dpo%Q{i;?NUERtIX<}20I;{Ijs-6T|m(WliE=lRH7Y5r;n@1zw40_;S z`YKBq7h$QhG;OuLY)5FOx_#7GxA%S&$ZymW7e~ZKQ{wbWyhX3J&e4xM*28P-6xm`UG?g)o7%Qz?I)uwm7h+dBwWhm zW-3R;+Dh5J>9|{xl0ZnC=$)tIsQ`R?F?c-=7P#_ zWKqWQFa~%yZ6GkA)#dE4p3ZP#Y-`pg-`N;XCO_Tqh+#KeQ^#mvzFyNAh+b~t)D=Fb zO(_A6T~E);%G%0auz{JyyHEqOBJ7NgpXfD~zSS@+?_E(n?>vUro+M~LA?khyUmwi6 z-q=mTKT*Z~8>Nw4tA75%@U`AlH-UND^^g{G?(Ly?crKjYjAE)g`z=c2>n?ORjb5;? zo9gbNQ?}JxR#RV!U9!_z)syviozCh!M?KZM==i$2{0~~AT4X$TzzmQt)rL0SkF3+uNIjF6I_|`L$c*th%7i3}^#4U&{Ye=bv+hh5r@S>Vt6 z{#unwr?$Q3=4-)XiQtPgxbHHAlY7ZPxVenQ4HE;N3=0;azK5c(-PyXd695JkEJVDy zKg`^Iwh@ES2TkC(hR4Y57P9}ebNuR~FKIh3;pfk?l9*W$+&M{S=Jf4*J54m({$2?B zN^ywc^rD(5u6hdq7QhDGp_1ANL9ip}&j7?H-E1{t zGS8C`9EKWZs%=dq;~zP|7SU&da!JcOA8}K^j$HQV+>vlG>MmQcfHK1&e6}0ANcdty zuVfj~EIr}e_ptMQE}OCf5Ceu64+L?(JwdQymAg3wf71$%%)x*&FeA5f-A^f!_W(XjIHc=|p5YmYiHC#b1|aKm)c2^B{%0a4O;djN<5@}H0}SfO3Oyd~)m zQNiAaKZn~946>++RG*}!wsxbQ-s0ToYqHP0p;2{YV@Y#m+Sg0tX-QgEu#(^vuBW`A%mXd_sA@$z}hS0AU zAO9fT5C^#qKvrYRG?-(fNo0%<_@|MSs6aU4Y#}?uPd@X37|__vWU~Rf`6dVHiW52I zga%Ni%ml+Ojph~bU8_HQHed)5hY-|edZ>1ba76BBbcFrZD|q9?d?S##x41RjL zi;ePdrwm*?^kmwmj8i@2kaLL#{_^ZvwJV;pk{^O~QP?#eDq(Q6^(^4QS45euAhC4B z@))MHubAE=5)lX_=^N~F(agZ~&jcO=?(+c!v;nPsCm4ioELXPCPVBU6kUQPxI-`|e zwKh=0p7s`edXQPYRfDLb#jD7ESdH-h#ilaV@z(E;Qh{hJVLZmnzXg@oMLl=sv%oO3 z;kaU_;vR$W0m4b%f0v|Uw7v>9SRpe>0!OU}{1RZ9O>g0yt>@uTLuw*v(VF0ayt%-> z{xn@lGH4GHcj6rfX1*&t0<;Ad7J$qG@WIm<2Y@8X1cLx_l8{vB3V3K5&zTp^Wed)2O8&eIs=qJ{sH-Jjm!P zCJcb9qmMwLq*V?eHO^3F#qxw;=flH~j{$5JqHvhT=_V3HhDe}zaRG;o!nx(+ewIzw zw+EkPtl*(w=!*BiUj%#0xMh%<%m%^#x`l8hoOv&WkN~c3^;e)GnIv#I-B^Q@!9mAf z_RsG-gWmP+?6lndi+d%~+M(~qbIb>={6d&Gl>s5oUSY3qWvatRI%!2Ng>$LXYq~Hj zj==X4u}86QtLrz?4aXcusJ>)+fc9fDtZCb%uazLec|g;kmzk4ASTZOlhTWnfK>xctY2W{;6APLzqEyR+kJ8@7KP zowaH}xH&RXVKfnKL|>XpbyQE;Q@m^S3jVsc8N8<}JBN2@97=g!NZERf_r=Zu)Yx=} zPVhB#G)+J&74q5ODWS2LWPh%iJiwd1MTI`hk*(zM2g6i57AZqgh!>vPxK;{8NH}gt zw>NU(giCy`-i%>AYigiOZ=I02i-3m3q$Pc$3$`q5BD1d0MS4oLyb_)L31tc<8+zAO z3RaDvl8+%fkvgO7q#O7(P$o_4107Ug87L%N2~a}+JSA0V4}@kCGg zyOxQ`#}|G$nMNG2I6A+s&AeIX0beiLME%5#@O1GKC2pBH8Jowuwg4b<5_8hglwv(> zf=rna>~a+=)wVXdqqqGS*K}IBL#or%-p}OdBOb-GM2?I=S%a3Zb#VNp(q*)&^^x`I zSaSvMtmQ48wNUzCfQil=Deyihs|H7fGW+d@MK`@5d+&sJ8HzUfv!dNC!7!ik*Fx(6{93r1g+m1hYIBS%$1; z!3g&P>fm*9-Q2nDJ1g``l;LknE5%N3O@LnvN|6yP`-jR|U9&RTNR1 z_pZQjh!nEgx=wC=b!|X>r4oJ(sBFRt3I3l%Pq5-wmUV`9Ig2^7`gP!`n~V-}bE zf|*Ca&yF1`|Izfa%zpLZ&xi`qB+e5H!NT88Ar{>~^G8fpRqzky6&C$-@<`#Ip;O=2 zr^5eED@|KX8bgUn!$ocq|40ArY)*Jj`w)B;`zal7skkgqpz}1)=|xFU#|1-AMemWk z^r44$bOGFYazEEBM{#hw;rxlTHpzC>@90-cQI7GlV$(>{-g+J1TWrhR%9tbND6gc% z_B-} zRRrq(n5-}@D_qZxsB%YEqyhakf{u7mq;E_7(FtGC^qN=;TMk;t592OMs6BQ+wbwv# zM|7R%mHvMN{K_1bPW@j}QuUXVWce>iNjXy&cY7ymRToQ}{{i`0+P_pJn%`WV0c$@S z;HZ*aj&5-)^w{jBOv0LA^>m?pD9r*Rd2@N93(wB>_X}_LKdY$-<~ue}B!UQ$yX=R% zEy{*BVf*R3-|lv}Yli=n#LNh?gx_ev z#V$uv!Vz)b<#CMi#B&H8lD=c{fFMFId;^~Vi4+mP++6Sfdn^M6VgGiC_2Lj2!=K;h zpZyOP`Syn{l#h##hl??gBmt%ELxT>ZuP-H-jhDH3HQNv~pCf%g6*+(dJcK5dBb8P> zf&D)Tp!g^(QU~6{jJBLYbd+ey(Ir7gOn^cWeUkJjgM!MjyvAY(;X~lqT>4@EQ5OlY;^kyQ+;kNaC-X28}<&h_D1hIMa)X zKC%8h0ySt>IJo&g7D`V^AUJ9#fn=x5u7-lBj|ZRC0Pq&vj6F}G~HHv4+`}=9Tg9X8^_lv@;jaKJSFqB zvH^!tkN_qWV5B(%htH4uYvg##0QU+ps{A7qIu8Ap#5l(NAosU(;qUJj@EiX*o})(b zE2!(=Mo=e>CJTm!ywE~zSIf>SZg!iUv1{Y$Z74+0&IdTYv@gby;>`qfW~&V?P1%21VBXZY!MTE|a>Gh;7KZ z95S7;(GU$rjul3&f`yW9W9hMJ&Q|xb$%N%DdD-g8dcV(thZ--)r2mHDNEF!)pK~~@T=TE-uy3f^ zzl(e#q?YVIO1#k{WGoti1m-XiKYFL9{Cnui0eKP_ zO&I1b)SkR?iHXm$^VLDn@iwcxS<1=QA?cz(s0yx_i4n!r1ml(f>ziLF{+ zG=d##dXL^0(zaSQQA-btTRoRBVdblL+~un3S*rX&9ZiW2zA59`t&^0$spO8CJI%X= zMn4HDUc#+RfLLXREgjpfw)r0O#y0UsXykV&&O|{R!^_o%UFxWpwdrGa<7Q;{4KD*h zw`h9t2)qXsYRlNseFyU>9r~5B>ZJC}a242i@_`6dB(LWZh!;UUpwI+lOuWpm8Oh>Rz1dZ&(6tOe8Aah`IX%*qQ>NyA$sts_yRb;zrI~vaHwJ%bQdViZ> zzOpuJ`P&7DHmj;;XY5nbcSAi>>sCr%m)X3$jF`)EBWz{@()ks$I9f&|LKxN{`2>v3 z$~Y{hIgavvQ7$D2)*(qaQ6XC@`3auO??KbU4p87-GbN-s`9*@YkpUm|Pt*%VIPt6K zEB4sadvC#aw%M}=($di6vbkbE?UWoA?{t6jS#>xR|Qo)mMuBn>KJSM<-=W2v{M#8 z04pUNW!5Q*U^7dRwH~e&S}pU-AwQP$+IK@Y&LdWB^()Y_s5>3NsUs@Ba|FJHo0w|! z$By{RdTp1^>NG^ifiXjG^Q7AH)C;;`)Fx&TyJsa}aZk#Gg5ZQ+1|*H&ro2@mg#|WO z>wR;OsV?g!GJC$d91zY=IgPNdUXDWR>v_r>5D!czb9J`C6><}e6IrtiLAv;%9oL-N z-CWzkIj&&{=IQW+@eMwXUqJwM6nk%M0CI9&OyGRzZCY^X2M1so+XH(JH_k=8utH=DP4P+E+k-xf%4L3tBvwIArtd zdv<5g4spMI@e~^LSufBKV8e8cvJu)zes-iBDQD;$w+$=U?>G0PEZrikvBE9UvpxRP z=0fK4^peh%YzGe=?iL}_Sa38?Zx2jUvQAnnDsaS>{`WKIFo4DL1&gsq z9U&Kx#@f$HFTmE%+(@r8^2Xa-$y|r@UjIEu^4;xqL7nVGwv~3CvTV}Lcj


      2uWRuxjMdJnwrQ-IfY?fTc3>eGk+uDM;!X${M#htumrbN5Bh(Fsj! z>5k1iQrpSY#(7%%r|{NW{|mVKvAsLB{;zRaicQ;FVU)+~ToaS~19Qw2=6EwAqYata zp4?1tCL)99^+zbojOJgPGqe4vncn-Bj4h*%a{ZE5Pc6VE)a80n_xq8v_s#$3ib_nu z4U-E501)t7aQEL@q8xr}xed)tY0aDrZB74Q$WoH3>s-iy5_y zUbtynhA&sLy7^YU{rOV6Cz$kFnW~JtB{^Wj_C0i z)TFXTar4`(?VF2SKj9lm4!s;6@%&NY;-kXZY2&J3fI=iktxFq=vXtZXVawA=vV+gb zJf8x|1q?5?1zt>%2nFKUu&8MRw`z$wg9CX^0_eq|(h2SiMZmE5z5|WQLzP5IIl2@H zkJo0?MAZbH9f})1ZrU`6ST*jDWL9>}7R52M6c0~1!Z)o}f{ZR>>7VA@loWasW=e;x zSZRTZoB%m(F@K32kovZwK6f)KhZ(&z6Eb5rSSR=7qf8X1duj)r%iMi}SiF&c%Sv=C zo%XG+sToOC{;8K6=8QPO(~eGqN`|lYD9wlsrI-u>va>rd-Glp}PT^JLXmcIjUZA}T z0!HwJDs5|HAF2%)e-Y(l_Dr#8_7|n}x>l~MnE)sL1uitl5_tuqxAHfCoCrktH2j@I z9=k}4IzHZ6U={uYr#T|;xHWde?gjAa*gHPI;#HZSQBy0#N6w@kyes80 zD_k1HV7?<4$^aX-xsVX<;ITWvm{Qb-Q~$n$lKm-eWF{SzVO% zD=}$CIu1*6ip!j`^DWlERyNs}j!sQmZq9}24OkX;n!S6Lyt}S$n``HZQr|28X`|rU zp$cxvTt-irjV-`Jo#}&O&sUVcL z^86o|uBq$1*^cVFTG!vWRLg6ekXlomGpsb0nzLW6y{!9s+5I>SmqzRB*cy{T3 z(*YnMm3lNDxp?Ka!rUsk=eMVagQJ6B&aboc2=;TS-;f7>v(xw;7S&egJ#7s(Kws8F z(2k?C5!>p&WduB-jC#Q#?TG}UzKZuSs&z^qXuy{B)}+jz({tfLL>kesmw@~!SuEs4OT?DlH0Lu}^y`jRCPdX~ zO8rs)c~$;cCmfnG(5aV3Y)m@6e6^?FnA6MVGgof|c5~X|YKe8JMqZdXy_z!*@U_d9 z4L+ecMqaohdcD&~FVQOQJWSAQa$V%{lrxq(c+8bdaN#LAjZiQqY#w{$2`fa?cR`>! zZfHO$iaq@hq&>=lJFGq2R$|V8)`=dX$vn)1GR?jabw;VOedH}I5|Wje78joq`t%s*!d@eqf8dNZZRtwxYV1+ghC%p7JYm&IIMWxX;3D4_;HKJrW+;U#&s77*(QE7|^9vnne z(1)f8eXB4s-*3F_qJi&ASz0}5bM}TseIi)p+AuLK6>XO7zsr*)0*W1O|B`3NOd^Y;ARi5I-qP;i4Om<8LU$^jivqtQv;+Cu(DrX;$ltO_9PHNJ?MWln|M{^x=I!{q-U@$y05MLIL7HSoY z!?1h%H{m;J!#faoIl_879o>A>idp1o#l(!ag@s81HTSeUfdyv1K2)S-q#0CICDICw zTNO_cv5X3!PU{vLAJ@nEmI!l9IkyH^>`NS4#y=xM#T>auZV}A9V$1OyT{7l26dIXl z9w^F)j7GEk$jTB`ozV=lwswI`oB;u#yc~32H~mWvqhuCb=NoWo(y6`pWV&~^*oSw4 z)OX!IFakxYL35EjxVEP-202ZLATm4TMv42n(PEAkj252&wnVT^5*P&%>EbMln-XS8 zU|k%2JzhJdfH6>)-aIe?@VQ0-(?%xOUsZT=$XDjy>j;vD&o{D@evDa(MnC_UQ`)L+ zrtCk+^I|tgy?|7tpYLzS4|ZdP=zdw{bUOyVX{Z^kF$wrq4`g^Rh5Sm$vVDn((@0!_ ze8`GJ2KNFdQuq$b# zh{4`bkk{+gD^K9mC^LiktM&|_-bO>IPq1swL)>!@By{lKaYrVqpW04? z*38WYS7v<0^{|@t{-}-~c?QTR20P!~UuL`E4#4Y$shW9VJt@q<2ezk0AW#~xpT^8B zW%pBwH=&AId|02oj3=MGnF{1h*hML-Is3FWipq3{)e16(4UqGwiIk^bFp!);t1%$y z5F@;VG}wNli)3JbLu5q%GGMdSmyY$D&^2*qtH!0z@Anp*j&_rx*GSwXmb;VjhPn?< zBCKTPLMGIT*iv>*U%3wkkL)^rS7<)^2W_(6bR+5iaJN0C zF^w0qI&s;Gls{cMZ}uW|6fSM_Q`iGmoG598GF{xahi)3EVFb$neWy7r?{5{c)7xqG(+cWefH?3T;j1 zJ{#>enLMN%BZtn7Bi1IIbAuhsu0_Q6S+#i8s7u7MMYAE;J>LK;D@ZcN!5Ki6qkHo~ zCZRJ8#U&mA;G3M^UB?$PqMWwoTFCkfrFG*>N;?Y zt1oeESg?sv6+Ru6lGz!fOxRU$r0gs@=^m#^6!gcWkuYo6XcN4Tq-PgI1((s0)kt!p zrJ`+XOe%TJdp8^*X;ZpQeo#1s_oQIM+kI?^OirsaU%UNZIlVW%9avD7`;cykWqK`dR<0Zdo-XO*$2 zXf)qMJ2Tnt@c#p)wj;Mst3Uw&>~H`8SpR=QDHkWx|Jfa`esRTo(uOB+nOPJS1OFWHU2i^6oTRxY1KX>8bg^$ zhP<}~F-2sF5Q}EsoQYT%vG5Qx1xltU^rg!Eea*);aE}Z_UNJ26-z8Aa_jBjl`O_O{ zrzgbxy5->()}oB_MkM;oT=`6d(m!0_VIGAR$CcEX!R+ zudUGxZ#P2*Cp2DS$&MR}o;{Tq%4N94C=3wRuR_Q20T}e#5}^qrT@`bmwj#MHkrRl4u$Xy9bV5eY+i2Q2QQ`Lzb}qk(WN0H5AwM75 zvd3e{d*35Axq1GG^0#4;Ky$oc7cNG0%JND8&OUM_{Wu0Wp$J;E_DB&wPLrk;@BTM~ z1?G@YnFPSFOF=(0BdP^~$d1Cru3Up(w*G*+? zS@{4VV5A}v!@5jC{U#Mr<2C|E#L?!cVbus^W{I*>^ye8Pc7|5Ds2Jd)%+JXpXc;=x z^L8QXv^=rtDp;Rd?5-}H$CzRW?Z#54Vx1j++H%r53Up+k9Ax{e6^OR$LOxI;f_Wg*2De=CzPm zii$0g))VExi#R1Umf^WU(adpjC>X z^-qmiXJSSsvPA=ZN4u_`!BW^aH5-t3Wm@-8EZ^iMC=d6cijHo*Ex0�zO8VA+a%M zi1zdkzGzb&RkbTpopZpJ;cS}zcxC`NA32P3>QBYmf2n~9;TCjlqc)ZUrnr5RXcxnY zP|2Q`rJEII4N!Q@Q2;3uNNeLAGxMO$BBR6qV&W-C0ZpioR$|3Wy^E>Uv3nt+lzPlD zf2xsutZgN6@<>(9HHXRgO_*3LWWITuOuRiKN~ZkpMP!NTQINICQ1lJTK|>^`6A^7d z9PMv2LQ{3VHFT?;5@mP<_ZWcHkHUjdY8C8oBjra(Cf@)w)?l=j<0U;-COz6GVUhXl zW8CWiA4HiiW{D8$ZOv>LtdF70(r~(C*4b1p0PZ(Vrd?)cPp0lah7c4%6lL7dS_+Q1 ztcK3!5i^{8QOB}Pv2%3yt1s5jXSFrJPfhLwNNq%6(X(u=YzvOf6$j<@8C#%;u3NONT5kt*)-Ow+Wawrg?gHbk>0 zZ&J=xK28re>d^4ea#uN$G#`aWr-_YFaHG}{m@QqZq$(% zAVazCXAmVN#k@*EWvMss1I^_|xQh1e3ilo#7(IUq2>D@ID5<(Bxi-nALCwT{h}r2e^f-#pu&`Z;!VQ$OHb zdoqGQ3g+3^o}~xr%-qM-2d&9l4)059_elm9|MdU&BF$iqyA}rq05JHQsPo_6y8iDX z?dtr0e*PNT_8Sr?ey8f$t|hiapxh*u?kPZ3Ad6W@0+3f$d6ffvf3Oc5%tjHK&;56u z&7uPUEU|3ys8A^fayeh7*uyG*R<2v$e;(=fe1Fc7yz*x0`J4Nzzy@~5Y6vV|Hti0F z`Ktx>BUHx7QY$0^_X-aK*xAUqVFTA~PUd2LE5rnUxG=ILosdYBtR9JxJ!ho8+N@By zI4&X(g<7PbWq>0j#9`r;OEHSVp$6U43z5KxF{byc(jz58>)Az1Bw7D{@_!D|lh-{5 z=)E7}Y+wv7SaWl(3EL$SK{@UVJC6){C;LbOJ=$Gc*I!n(>%mtLGEM_Ezbtk z4r{kQ{2EAaZbOyU9_-!I2Z4j90nsOqCBmrYW~w+J*J?dv;-x^35n=Ud;|dOVJ}bv# zmcj1Z`#Vz_>JhN0$7O~pvGgwm3xBL^@xNQDKPQ5T=6s+m`S!yG%WVPgcQk74QwZt{ zV6tf+lPB*PBguoI!y}&9PGS;nUNPs_UJ5v7_L(9_MO5 z=6#{R?BKb~^XBGi(YxrfNB{;!EMyTAHN|Bp8KVA{7;zz-L)Xw#!Qm(qq>q;zhxEoY zokLk>plXo|E_s*`LuXsdF5*7;!s>@MEQWq=sHj$)P7I0}wMVy5rnpGG)n=!>{3}7 z5;-KsRL(G(Xbbo)U8TL<<4d&}lpCmzbwba1c8lH&bn+3&Zv;1i#Tz_FLdzKK`prNB zuRNRWcC@wLL?PFU>J`c6ikZ4^+}C*?Og&~ITS=$Qw|JHL6QFV=H^fci4Oz&9D9Sho zFm|o0Ouy#7L60!s37z@@fZ=6s(d#Lj zeC3p_Gm}IO%Xk#WWK4e=NsRV)m?UOOWGNtTC-r)Mula@+=8M)a5wL4ZPSZ}|^*T)G z?mC|7Qj)HE&p>4laFN1;L%EV!MjjbqNP!|tZH||!$dP$sb#;IZHb^L2Z7$tc0I{?S zl+VM33JFJIEG*%@6aV}%gFOdU;pqtJI6wr4@gytsRVSX#l8`IISPT|fEZ;*}<#I&@ z)aT4fx4@ic3U`_3rwzBoiR59t8UL4&zMEf2nTj1szO@|BC(_2<|HUZ!mrjUh0w-U| zSKz4FHucJmdlHe93`ap2k?19gxfX8{B8%Im$=l|@UFh|Qi{`ObtV+YH{2mHPUH|W= za)D8X^?WE>c{PQS)lSmU72xs_LWPkK5i`=!9(8Ca_3{zbNcmtCke4%a>ELuS&Vr;T zfW_b`Rpy}E;6XA?gYkFh&D6}cVtZq+GCo0@s306L3-hqc5a}v;R6nmXo4flUd}}{u zjfM>*q}3GJZ}A{=Vqg|Bhk!&$L>S`_9cJorfU;f2snpTfoO+&dc1@=`4et>`LVvQw z9vWj&q6$d94YMbCisS1|u3(YEK1SbZe^S$OR-R9HJ2YdLeuy=8R}Q75Kt%e0JKM=* z5wJETkpCoNOU^W=(^ywG)vhG_haK@w5I7+Rju+ZwDyb z{p0I;`8~dykr2L^(W&u?s^6CPnJquxZ!qM~;ioJ4?S&`i=HFXhyUjRbL$E{a zR#!<;_~)*F$LQVPiN&6{bOosW)_Y^5*5sLQm|!nIkI6 zpR_LMEY<+gQ=FW8DNq6xUsB57u8THGYPVt6Ewc*3L2(k;TLpfn52ofATc&D3F!RHUyzmN+ zXue;V{Secp7jwqjxC2jrG{YfTT@%AI4#aOJbpD1E7m)68nLOI?DX}0n)PZ*}?>`}l zDK9dtr)!O9^c z0ZWD9v3~&RxX=me<(yGH)nCHw75G7H1%U&QENI>C~4~ zjX-;8_w=oNa{)r1ZhoVsgRjG1!2u?T)>46MvF~r5lIRiSv+X-PPsbGZ@$WZQ0n(~O z>KV70Fk3~^1gj>~PVV$<;QS?C)!IbKS|~Bj_JDJuTXHwB}i-71#}LmV859jdv)h%9w&>|H7*W z{abi#yjcnhZZ^1MwvDH8MUuyiHVB#y^O~hF|E95>gVpgp-^+lsrm7#UKFpQ@Wn}Hq zaCS!lTqh}*233T2qx@H=18zH#of35B3E@*d=io&9Gf$`+HkQ7?{Qyf_ir(5R!K7nV z+V~yaCIKcvM!Vg7e(f7*_AXgEf|tdfZ&e?B+{ZAR3ryEeJlli2KqRg(KXCG;=!4 zl|+0D&}Q_h22(i=q2zg0Fw`R6RY<^bHiEkgAi6;rlmq~y8OIN!1-^g^@aOlBW{Me^trTF$x%@Y7-Q&YTM%NHN1M)Xr8c>FoxJJtS4uk`w%LVk zD4z|nPFn(|JwEGff^w*2iKr_&U8eLlwNocYx^HL8&SIqbv;aBfv0rkVZf=F-DG0xR zw;7-C137Yl*llhfgj2ydSwywWb{XrFK_tMsQwwm7WMT<8dNrybxJ{tT4+ug?QHNzxCX!)g zGV^guuwgg~o8TX!%s-!) z&X}2Q^#&V=$p0X)YBbjK9`n+hjggl&hd=vS3Slsr@<4=)W^uiu{$Uf8>r{GT2Ig^GYeqlrK5f%g{Jd2$j*@=dTy> zx<>@?Ah&fty63donX7>rGi5+;OU=`N8F>8MWEim=={-XESQvjKHBOMp*i0(PGe~UQ zyEB@aVGh!o8ysyE=ChT=`a#RPq6Qf78tAl$1&VLPE*_ywK zb_$1KcbVY9rL2TjXC-~FbF-}p``Ew$$G9IcB734BnOyQ1zpM|_^GL$JUJN>vseWLB z(J}<+{|hxt6xY95#G_A;|pUw8fXmlBhb%QDz4i+OK1{ZW{uJ+Y>s9 z*Gy4m&nAcr4V5Y2EN=$QEcISacg%)?#H>m-k?vD%^wz<12|67qCwa!v+qWL#Ga1&u z{NCa=f1B5$`TeuqX-P-HYKXq5Yrs_>d7I_g>CA!&2QD-p`mK&gTn|8e8*p0MLYgo) zH3p{tXve;X(7t)3sj+2aRm*;D0jz-Pz;eCX$=SEgk#}&YasAM2{)cv+kb@=**N)p= zCuFbOMy>tIvJbu%h_o=NQL~UAWBb{T-h19LBuw=*1%CV^?D~fI-619rGbc3`1)hCi zn1zQXD`>~H^Jo2yHk~!X+_Y(Q8{I$kh@p;7Z+ zFM2dpU{NLH3~+6L?wa8G<*lbc zE`9v&B&qWt`po!9Q?vuy8FYK=r=7Ca&tKff0e-wrhO#XW@{Td+iM0}khC0$N#mS() z>}X2sFVqr+yJ&i<5E<2;TAVcvzl+~%^_+Vg$qlZ!z~|+ojLtX7?aiYNJQ&c-8i?E- z_1)7*a>J1^G$Q;gcSpMGWJ9Kz3HDT_$IVDbB)A9TcTDTm8d(LsI^VP0*zL#e5A-#u zPBL$Fi9ssqnBJa~BlMTt8oRs+Ev~MrFi8A$=X<2ZS-$k2%e0q+#IoDXw689s-e);l zjkvWwleb(kY3VnoA}#Jt>L4|Y6Naw4-vq@G`T9+=W$G$izf11bFU^NVpYB0q=UVMIMqaJsVKLwGFh1&J^-=z4vu9Wp>NA@YgWmTIt zY>8BZW^8hdK{d3hT!x9}4pRNGcHs9bz@OfOWEoftXQS&$5hBS9*i#nD45m^Z_;RuU z%zb|GyV~q2g46v{sC#`w15?!8Z%gw+x#NyxF5go>&Wr_mB$IE3Qj+c2&&%tz-pX%f ziRC}^HN~0`9Z$B8^`DYEgRZ%<_cJ6vIP=WB_?t;aE|nqXP*Yw()Tb`m(WCaAar%#_ z#<3bv2hQUEB-s@9tV--42A4!U`JHGYmtatPx6`n_C(8-H0+SRYmZ&-=b}0mtfs@jKtM^++>JrNzrOgEyDFZ-!Lp3PcxmD*z*`YVF>1R zai}CLQ5JNbAaVw}A*?!w!izGyn26?W#rN)4RZA@+;(O(Jgp1UM=CLxb_3WHS7en%dAHFs}#u} zS7X}-3R}h~uqCJdK9}3e!8LMhJSwe-g|Snc(`{9{)ypm}TKH8lX#w`onsPdMG%@M? zK6$b_In*&}1Af(rpJUpydR5toDYH-UqU_z#OP$3u27LK?KzPb#7%(|i&5I@BF;g;P zuQzvtY%5N>qwUWKXX>LGUTXi7y7JS$G(7nOZ zvFe}ct^29(q+^>V9?&u(>=M%UeR~)>VqKy)qBu~&S@e-Eik1moF+tY z+0MK_@70GB5PcMCf@y0zJ~8O_A&izyAt|`F}3|B2`@llQ`ohx$1Gi|D_ZW~bWYG@C0MV_d3ufoMk>L*c$+h58mlSeQ81i=W zn4aXQ7#s{=@{n9vfE=75B4XeA*}nvdO*63h2v+=n{sO(hk++aYBLmX19KUVt7r9WU zsX@0V&b#Ah5ph-Ld47k=AJT(jNc0Ub2~dpT!1IegegH_wJt-1UfSyl4<&iiF!H`}# z1W;TVRRai`aXv&MNHC2E%F1*9_Pv;cL(y%;V<04sywD#Jw8stj;I>B@5EqWY3Gy3X zCl*B#y^JF+y6>=)A+(muji`j%M6BMEX4 zf7TRTZ^iX)9G*w@S0zxh>M>rKEP80haq2A4lxB9yg#I!qIa0}uPE}?}kk}1Mlgj@* zJsN}*ph;Dlnn>Ha!4bi5vo!2NZ5>#VE^p^h)Aa*FA*D_5+__!;4v$&Iy}^bJ&}!Mk z($XOJ6Yv<&QWZb!2)i6qrsTI!Q%2!?UP?%RymD7rq(Q04w@eZ-pTQ9lzxH=V>Tu-J za6P%ozN=kS<5zt3inlm3d^NqAYV|V4qt-a1`XbHygJ@zDReM^Bwsgb_jFnEwX3dH> zxsaKB)F+G&)BkV*HY^?QWBa<;Tq+0aiM`ry{vZRmIsK}Jk&26$tPpIghqL=CeT$J} zO*#wmQrk(Di@N2t1KbXjcdmq_H51e`oAn}ubEEoO^`)5Cah>|XX*4OAUo^t)+8Tys z6thm}9Df)C9WKrPFizGJ0g23|ue6Swq9tLdvLVL{j2=e~mA`{qN`uCYkcAD=v*uR& zKdFb0Vc!gZ-fMUN`2fSSf6={Oz}^wkqOk`-;Ng$evpuC8GDwP*xZ=P9l z?v9(W7wn&-QnK{0YDUbUC{mXIuw$`|fOXT#FDc6dCm}>d=JmRF(*zqdFbGy*9W@Fl zj6DN|%gi26Yt-E{r-;X~=@gQYQixjzv+E}d2;VKjM#Y;wfqHFuzUKf@1{RFd_v`%ClqgWj( zaD@MJqG6>ft`_{r+Qw*Ug+o!OJdm6MHQY^@+DZ0PL*_}zxKq$7von2QD2^z zZDp(;rfV&lrDoAok?YYBOIzS9nq*5bn@Yfqdg4PPyVV1iUV%F(cE2^4qD+W;3c-H3 zF1m{w?`S|^fE1iiBh;~}*{?ugY2@xb(F@$cNvUjPPc1?lKl0#J_7_d90fSADoq<$v}MRODcp@Q76~r1D1RGk!!Pn znyB+ra4tT5K{yW+&4D`vmYGPUy@zr$f&vJBX>p_g`&t31|| z_PShVq)=f^th{c|uJkcp%R?7re59KH)YTv*3aw|g=DbZxkE<}$p0ylaWmgRY-9$Cq z78leqXEmb)^;_e)j@iYPyF;YeJc;i$E!lN)@$mbT;Oc4pLMN|J6paQ-%c&7geWO58 zOxn2v*PP__h;IR`rF7@ja}AS-vCR~Bs9QDuwSm{@Rp4(2J6y1Rt%hQ+ho=g)UU%G=cmR zamCNgVe1I`z?bQPYQrlfo$yYT;y~V|W3_%8h7n+aG-W_h4wBi;3vwIn`elO2=k_p? zI4+l|JAzTBvQJOIT(+-w_vesgL}Ux7t8G%rl|^m75%!~rhlTd?@Bcl)6TIkC_=YPH z&=0Euit9W|t*4k)g-kv(_#U#$!Sh^@(5l9`xu^3M*<{+}Pe92J+N#?2Irqxi1>GCJ z9(v5Hlcb8~j~BKw2VefGmk{sd@4n{I?@_{joz7qW62oGl=dV&6>keQIOrQ#rEKpjY zjnE$6HXV3fQ4Eb=wla~B*yB1=MFXZNRvkvO*-wYYf2{!Zy#~&$Y{*vubhdp%4kYl& zo`7Y>EDiDvn>AVW21dD5OU2Qyu8vwbl*svndQDweK=<8nGiw4!`1K&uUMx4=wf9O5 z{TpUM<7b#RzMq8}_YM7}H!*+-YBe&Pr`a%;PFI|MNEZ*AIs>8EW@5Dn-pxkM?1AsG zdvST*vuq=B^*7P-o^c-*7a=rLmvt(iUQx*E(E?UUCF-r)$8W`C+n%P+YTRPY9kU0=T`p%J=vnbSv4V|($k)|1= zLO2fQ7DM>EY3{_47riCU<+!314WH?4Z!m#{e6VYwxiCFK^Y^pYTXHu>Bf?kX9N*+OkB&frDt_=!gizhZdKv0;;CtW35(e%?UE~OQ59HRT0;V{5OT>6C4 zP{hfhcdNrDkgSIR(HT#bP@C*BXa89d1tFRaeE9Xn5pA#eBeU(_Mj-3mpd}0cA!)!o zhSsQtsckKau=(@(Lf2~9SDqt0tX}^gv8kZ(e3R+B5e*I zr_HSzNo?aaRY*ly5Z6LzY9$Jr#|(ZN-emiI#(b6d^mqtk$shOMNA;RQQQx>Fw@pg> z^-wiYUaX)M%7epNDUW^Q$zD5^M~CGD!&2yq)}>jPoEEvLD1m(jjs%oLS;yI##4lTE>2UOGRBPy+w;*@|D)!-F|Yz4QcObXt?RG@~NrYSGE@X0(=wktf;OmYFhtT%;nTJ~h2(0z8=+pT*lTQG)e zp$`&*M_B^DuwL$qC~oRuS*!qH$HP`T9TZT&7sbYuo!NH^{nzp8y=XHgDW2^hc4Y4k z!$ZjUfrg?=8ZReZhq0s@(!Ci)8_sL^P6>q~IR#x#m$F`)-|vDXG5}L80MBSzYB3Ia zk&d?E=wspbeZKr5teUNuewx3uKAC)dT~CKGLJ5&KN4R~m|N4N7C78`$D7TsFD?t;9 z!33hXm-%JpURk)ea+--eA;>GK>@=RL3qbt{x(I7oFR zOZ|lH+%ngaZl4VSq2Znq5)s4KmW~usNFat(KDQkLn>aC)AhXom#D-uuXUUGCDl$tQ z+8Nmt{Y9=BHdZe%7KUV%W4P9Xr~&l6kB%M$XIQ1e6&8|Ti%Qi`gPkl0(qR=i8Z(D- z!&(_xtQ(K7Cx(GB0u`AG-fCBpca_-$B|UY4-{(}@hsOzg#q{O%uHNI*qb$sPRCoAjO|1b$Ljg2A_4gTsnlizLIb8@G z1RJzhMS^o3nYBDwZiyt=U*?y8E0w$WNX-Fj3+%>(4}~a$9x#FjAq$XH^iRFX zzM5xXZ+onz$(#%>mK4u{fLFt3IabY@gbCJ7eirU47n9YjOB7YF?oW}tTJBNX>#j24 zMr)UvwbbV-GEr1t`6gN%=*GFDw0E=Fip7zWBrz}6dp@T{QQV^b7Z^LKX#ZRQ0x@>I}3;$+Ft5Jrq+`mEG5js8ocv0QG6^-I-*W@GoX;IE%9Cy#(Z55)QLfim1 zoynoGN@$1grun(h6WxjWm#tEDGyw`t>1APz@Rz3N8cUTHtC7>K2}|Heamsw6jKA4R z)um<&ApGu2ny*Aw8Qs_IH1(HIsKi%DnLn=eHO(45W(&>mjlW-s2A`ihq}5-KL&yfG ztTwIw&~7`)$CpE#={S+^Je%iq=r}>A_nIN~NpnE&RRe(^CP8#@n$Gi88d|Mc} zt0Ro0*#TUl6%e;}2ew-ye?*uNUD|=V!{)m24n|eMed)<~>gY(C518Ujcb28tJCgHRjzj0+f3F z>h67i2fDd|?p%8Yy)(SU?PAj9>*-;+{aWr--^Nl^BeW=YRUf=4ucw@dcI%PrLRepW z`CsZf9Loy<_79FZ|KQkvTi*T;j!Eb_>;1P#hDs9ZYTd+~^X+K=0IB7u<=EIf-!#z$cVH|OE*lo2T0g2pq~klN3augU}v4@ z8|S@MZ$+MCSOG1exph9SH#W@(^{np$^KBP1CnQ;)5dH&Ygj4lFyMphg{64C(9iorbflIjI1y#ZH(LVe5r!iKa zTyZOX^JAnRCX0E{PD2$Jb@*X26VlB`$^)PWs_=CV++a!(g4TRZLxFWnAKX9ebfHOx zo*z@d)8ZI-n@F`IZ8+1$M@Wkg>#@isyQaiP z)H%tq3&o`%#DkK~dl#8&xe+o*B3y0m7K9kRvyJt0uZdLg7G6*j!=Ahhiz#<2p8dXo z;;{PblV3o*2|1s|s?8$4%^X@y8jATIV`L0CNyyVF+AYeR5=w{iN;S zI!t$=kqL*7!=6b@c2-0pj2^1|=GgW9TxIL55vMgc`GMc@demVgBn%zov7C?kACtO| zx9sM0!}+)cKq+puh>sU&aDpt7FI$eqWk8mSo6bV<0Wbv<%_Gao(Tvs{IN>`NPlzf zUvh3^0yAmEFoqDvPNCX(bp#cbSD?*^tlK#PvOZ|f{~n%f-dpt(0-nzC8q<3hlVrNk z&g{u%5LSm|hpuIbC&oT6<~}SnJjh-Kte&_qLqQby9#_xo_hro=N^{I5bBA4l#qJtW zHejuU?}zPDm+trn`v-$wRZyQ&i$I^I=+hi`G!r_Uuq{ll|rRU&vL={t_zTF4KU;+;5ekw+pZsdT^S^LvF1t@!*l^hi#9 zEBRy8rvH>Fbcx6NT$PUD~-r%OT2=tAV6`2)uMxr+~zYw3xW#h`q=rg{47}5rI*RbIA)O76|ZbjII|< zc{CD5oRWy3H`u=>7X5v2TY!L|B3>8nY(c&%Ly%(llEk99$ObWfJ7@~4HV6Wv0PV7j z2tfamGe#u^VG+iSYsiX9_Am?V;Mc0%qXDa|@dlj^Zd*VfP>!DcvBAOqNwZ|}L3ukw z@p|CBbjQWEH@6FCOX>ULTf5u)yJUuJklXTpVsr2it-Gz^8zl>Jnl`F>0I^$x*svEB zi8sk*Tzp)bexCN^&z`5o`>aiGF=t=5%n@uQ@h3Ld;lGh9x`^Tjxje+IWAu2^0j8e` zNW5^9XA?x%V1fT3E{Zfof{jI~V0ehuP|u3Er!X?-e~3$SkZANhrxrMhcV9b`;*YUZ z2^bw?(;bQHc_?4Hf`c9TtOS(ihClaG9nEfP%2I+Y*H}SPS$3XGf8nut=R= zR|9mqgj(5CKRqDyg(tA?J<LLC|k)~;z*M4%coHBr=&PrFNN#5}P3<$zd7|^8J|u#c>OJ42yUn^(Sdn+RTx+KqKu= zp)th<_Hk|0;wgyUVf~le@I4UjorF2=T25HUuh(xw>=j=DNxW_#x2KZ^h(2Ae5lI1Y!aL}umirjgWi;+c`f%AVc zmq=_p|%-2k{h> z7>kDnmSTkBObK|%Y6P!Q} z6~^L5GSD7sMXW{Zj5h4$DuUP=K7jvIr+vNTNvHimuCpKB;{I=S+8^ZlPd?gzMO}ia z*vhd4`)y0-jsmzHg6_7KKRFTq8H9a^=?mhB=}HS>k(s$DeV)%LEm^-mgh<~^ety2; zcsbY5%{`{Xq@bIOnzK#Yh?+z%*m1t~*LC^$Y6c5RgGLG9uuZ$I2rP?*z6{!tC4_Oa z*AqLRwEmtJiwrOz4<(Ye)Ok_1xOfN=0d^WY#M(hzH0|}x3;@E}W}!tI$0YXm`t|pQ zgaih9zn~ArYY2WdhO6 z9rqd^$Bf@U+_$m1WTeMAELG_8>mx%{zihCqpvEdII;@vs&|tpNGuDET9tspvcp=^s z3p^xh03OMJZ@^q0RsZ`l?*4w0Xhl;ll7;z;ZD4{J$I0$sv9Ty z5~tvv;I8Br{~04nI4)~ZD@L8xER7ZfpF@a~|4^^YpVnxN`C+VO+xA3s9Gg!BSdEvuoVNu>G;MxPbyow!NXP4K_6^nc ziN$?9PKF&1WS9GPhFJmwtJbt1vOxxy9rFccw5TWJ4E%BHDJsG}Cw8MlWb28Sr`cw{ zVm+f5zzy&ihX!Rx>jNNITnl|a1kkvJAzK7zUI;EZp;4pxl{*SMs4PuI7YBoTmeWX$ z#B0j0gDgA6ma{Mtde%a2aoG&15W{ab-aZcU=d0($cGOCsxqtRog?xCMn?Y1$SgV!H3_+lAEZcV z+GYPr_*!nXjQ$UKMPdW4_Rc!i&zVc0&O{Lg4Horr-G;tNL4*5Yk$pI`vQM5R%~%A^$%Zo^F2Vs>H6f*OL?=tE_wLxXC88FF)n7ud zXMqcOf3kJmx=4DMR4)${LpbLV|PsaJc@U+lvKyIQ2Q9_s%CNZBHh*FP%f! z3L__Swx}m1;bo!ldL-dS`II}n*T9PY2fv(0^Z!=OA1eRg*Hze6G{MA#yKz%B$`Em1 zORhIK7Bm)(sg4f64x_q256Xf5@SliPPe9Geq+s1<<{TH!qab$Lsi8ABLf#I`=l9YD z=Z#V|r7B(1ca%4A6v%Hlb_^hd?p{P||m3j}7bN6FJ(xIcgV&%S%tb)RGn zCblezz(VlSWXOu~mG=e`W}ouwq#*`hcYDiE;G)a0>4xj90dxl(<2ZtI z_~hj~sC|lVbK@4kFLr&u-rcAZMU*Q>#}>hLjHJqjLdRe$KY+cwsHMRG|3zO{w2v|7 z;w>cYv*7(p#HA#zB-NNJXUU~bSS`)6tmurA?5(;?M1=nGWrRfisg2O0BoFyQ>7rA9 zRge;Fus4cobu9>?W9MY6!W=D%pUyx>X<3F+t&U6r^Ha4U$hAI%2bAZ%Jk%g;!?Q}A zZ-Keay#<)ozYvLU_%TlU=qxen>X^2jk0i#jV9mY#U(>8!U!MT^Jef5n;U<|(kn|4ul>5(^T759F3* zVpO1E4#jw2l(N9*Kv<0)S$!_+>~kvg2}t02#ID8$m>s4W7mygoi|;VT)Cb-5gx{MD z@d@<|5aFJ*CUsSPl>wKK^Rw(xi6lU<#d}kxw!)KoDU{@sY0Dj=unFl!9w!4(40U}+B;PK+N!L-|ppj%| z$cBx{*^yh1lAg~0LRy+fyoPb!3GfUGyh2|zG=3G3{CBgM-(k#!q?Y?6wkNs%ii}u) zyV;|(@a7{LwmW5OkJekerz7X=j71-w+_Me9Eo-PH&(>86nX`O&67?Ys)(2TDt0TuF zD+uYMldAHjolDO5Q7W=+{T;^6$@-29NbD&s7&hLrQP*lKa|JsB+rR^bFyrCOi|6m>!wSp0T^>Z5n53wAc)8sUHqQ{(h~&`ZtH2!AfTRzZ{0#`VWVN zjsJ_oL=?F&-Tk6(8D%%L5rUfHQ;1xRce%quKjLT&P4& zxg;WuZqF+5UN`h=6-kYmCJ4RwH-|-n;7VWoo5SGz-~Pp6WU1jC{||>f|BJ(@|KYHt z9rS3YhXL{rw-Jtp6OJs07B9Qk`4NsY1tyv7^V1W1NkU0>C0r7&C*Kn$Wei-EP2oW3 zAp?aihQtD~7{-Wbca-OktLv7$k&RBLFV72QT(ZuLkMEO4{Ud}jT!fUDDX^gnjJGiP z$JP6A(iwa@o|;tN>X*8c_O6OGavbz*6&xV}p@aDp_=h5+9bSe86<1Jxh3=l9+b=XA zZy)Hr4i9L;8Kj-L2neu;_YR1g8=}f1t-$elHTms@b8Y2nsV?V9jihs{#pUI1fd9#1 zrIb@?m#_c;g5LjqYJypV z4mo4Fd|+veN%6W7Ak||i6hg_%JGy-C*!NE!#3zoV6zLr`F)_RYgpSGn+2)@o0{3} z-sj!ZEi6aJMM1#Idx*d6p$OjDcD$*?=S821zUWAD>oNyk0t7LBjQEJHZ?@lMZ%in@ z6U=f?m(MJZ>p-nr^>--6Uk(urS}vq$h%guwTC!P2A|0X$I6>ej3QT91g~oXivYx&% zr$|ocG*}aHZE(K^Umt;t_5=z}oC7sYmUc-ZS`eRWgezCSsI(6$kqJky5YINe>yOJ| zt5Xq*(<0riA zZ(v%FJY5%C!!AKjx&VeAY%u&xeTPspd<1plo?O_v?(i!H(1Q61L2AT#(JFfdoMtBJ zWb3{c46~D7yGX*4BZQH394C~U@LE_Ry()2v7p~_+7_6#FF98Y`8IuTB(-W7+ctONda}a*cTajf8iF8_ubxGKo0Ht>By_(p*r0>u#l)L z48(U0O5w7dBLIaH5!PDaLrAOqU_&G+>ed{sJbF7W9nv*vZ^o`#xE=NCM-RN#-ZQh3 zTyPL==E43XSMm0m71yymu6_qy8NpCw!nFdPp(y8PWem;XtB|CFKm5InkRsE+X3Vb( zeq|edM6`fEMwYJ4K>!K(i_e#lI2o&m+_6J@5~VUkx2TtG z5rF(1MNFlR5YFI5uuD1g;US~fA4uv+d*y|i+Yex*8+YLdbLsZM1bL%GAA3M-wel1| zNK7bIeBve`ftB8GYwl)*p%ZQ2Cm;k)dX3u3c@M$q*}~@HEP-&t6mtRhGa9?q-9GDW zS~aN<%-||c-8Co2(*__k0QiY{^SJEUP?4`etQTKYb+>2?WIalRqB?F;^6U3`rr@I+yIUDUzcp7PPzZw)mnLwMSu z4e08yiX)wilAdUbV{rGk3}f67oW(v{E9?Bg7>B)KYJtteey*cKu%ycy-f0|&c0wA0 z1!}rsrZaZV2##==nIqH-`8Ru^A+q=`+GjW|0V2n&yLno$-ztB6cW=D;;-hF_JK~d7 ze9j#cuFsA5aUUeWmsiFlX+&PJVPsCr$2ug;1KVXj#1BEHYp36tGl@JX&w!~10c4tB*C*S zB7()Rx$VNBKjCXHQf(9m!4 zi%T6O1MU<$pYDdWzdKH(v30n%javxMjCyyiYFh| zl=&0y%Yw3rYOuyI?;9hd#e_^$N0U`MnlIz-6=irfg*U|JOE zE5v?1+|69JW8wPWQ0v<-E5sXg19gT>kl@k~);3V9XFtY&kaO$=4yQDF zK1C@EQVpo4(`r>G_EHtJAzfWLxe}2*xK{QfuVzkuM@&je*9Pw|7sF63qgxX0(G$+0 zrIS-xV=S>_Lj+~`wPA3L#1TR8Ujim{X%l2%SRg3=o+GZXdRWQv$d_z%8)FGH5-l$A2X#6p5slDbz>@YE8Ng91pnu zoU>iSN7k)^N)w5rbv|3#!SP-Bl3v&X#@F=P>3PdX7sFTYu7-=edZV|_O4W{XWEJx( zKQ)0EVa51zi^+BRmXOCO^b(i&X1Yw2sH}0w!1mHCkFg`OyY*)hia)DPChym6hrHe^)Et6*kPanLAiIb?yTZ`=&r=}$BDspr3& zSICTdR_u`j7d7LP(r~pv+TzuLR~5BSy|HL5;13jhiv4POy3<^6K4>bF+ENgE4SG;5$_=0pjN$w; zc>v`RS~+HUm$H()2IJpLylRvE`}d(tCI;`FtyQ`M}l9G*5Sg`>Rblcjf z_Rt*|WKoNS-bnau<`3C!?0RBQ5YNFBMd?GYzR<`s*mD!BO`{#JS(?lKa2G;WOGw=k zwwAt1q4~VjB(m4hG*p{9SmUHZVyJ+(Mv@VOO;hZ|Q3uqZA^quigBX@oFxYw$k-f^; z$wQ2Q_l;bpuS>q0v+x-t*2y83PJ<>M%mxs7#%z6}vMQu zN1f33>8GUP=kU0s!dLb`f7Go4+UgE<_UdXOM(x0#TlfP!a}+L_L)xpRovt>jqX@gl z<(QQ;uMQQ5uoP>J|Jny$|4vd1e6Ht&=(dL$97a$C-F3{3 zvNNw&HT04x)>+fuZHNk5-*p8KwzA=V45}H`QMe3BS5?H$$M`v$4xz&dFTrk-8}eEX zi|D$Vv90u{zW$k+65~Wm_lBzUp+5f{TJwf(?&+f{=8wEFKE6~Rh1a?h(QBdCQy{Q^ zQ1$NfTnw^|ST{nEEqhH;^wIfWBk1 zSy)ghcqJ9fWqJ818Kr5c;k>=o;HwB=>$E7B^|TvlPo2u>P(BS(E!KAiWtTehtNd!G zUJm50+lKnE&Yey^_M|TIbo$epE@ztubs6@UapDZU)DiFOv?MZRTCZCBFj!7g-zyblSMm$~)CaYRf#?I8lWn)J?zXsBMcchy}~s z@d*E)d+^`==N@f4rRE|q;}u!#kcX;2Z|?-OzFIZW#US>$%*%zI1`%1G`_@O%*{hQ4 zcbQsQ6}ibQq@mhJ+>tx%5c*fC26g$M4Vh3ytkL)c7GJVrsaf2a_D@C;odVl3M0ep+ zC0u`?E6Z>pMI~b3-O@G33(20j5=8eTc7_w5F@Pd*5{p8V?xBt5GXF@J z@(HbZ3q%@^afy=gi}2^d3hk-U21rIBPC)@a;JikuRVj(qC5|$Qv15zGbVMU18pXiU*qIbQC7_i=e3k$&@l?g#6Mj>uOscd|RCJaBK4Ao< zUX!pBi4-dS3M0WOT{`$0s6!aDCbBXlTQzRkU$fN9kxmE2Vp8!Ex>rxCRGQ+g$E-GWj>Tx3(3-&Z!v2+f(?FxBCn$dOZhkH2 z8^-B2NvJ(m$=>UiTTVSbUS-~}Rkv0R{0PE*s_(dRb`=z}gGCfMNJ&*a`xg26Clla5 z-jfawML7?W+iUKr%UiDLIb8E;Mn2NRgo&^Y54C!v($HveShwBnlCT1%H0|q_{a4nr z^~Lx9{C%9cOj(@$$xmKj008*)Uykt=9PQ1lP5!@z6Ai0>%9g*fb^56IE&wbjZMHc7 zX*hXub`aYqPd<(&{b@N_;8)(V`|_Hz%dFm5f~UJ$_;onUcIXG88d`Sna7}9{6%<;H zW74L%J^`IEggoNqSgbEXk2{V*-c0XV;k5Wttu6X&`07gzvUI^fa6%A1AihA9m&r)F z`WxAYkT9%{fCsB;7QM5V=U&!|Umq@iF6|Tk-O=^+0MZ}Zg+o+pKkq)~^dUzjg;!h4 zQ>e=?Vp?W~u8{x{LP8xF5q$4Z9HepYG_GJc5+YK7i&$y`+zjo<^%kQ@PcE#^$y%A= zl`v$6=*dr=Egl*QN8%wBQWdKtB0xFJu;rSjWsVwaHT&^brRtY}Nvq9Z4}bjC)THRt z9)I4=hyby}wK3EmRf6=gRjoJaYa`C;py zE~UuJ$^WfXct_sPSUCwMJZ*AE1e#Pj^d>&_c`$01xQ-JAP%V(G{nnTNNYD3sHWfJf z!}kCY^p78{vL#-D++=d*pz}anf}c#Z=6ZA5N3P$7x1-!Qvi&>!Y4_)a(>tSEIW0I$ ztSIA2E;`!Q0ca^PqiQHRF%G;Q*I&BAFdQ0kGXtFa8h*u@pMp-ngI-fa$wZ#GMh5Wf z^i~=@HaLAcU9F4!Xe@_xT@e2b-w4G~uW=ntxnSr_5_K(aJEBXkAXvGR`!$#mYK?2i zipSi4%Mw=7!z|I9f%6TbKyRm`dydLv{8BYD>~>7h*iSN@qa4H%yP>HFzt(~#U>o!h zdBXJf$D-6rYV8FVq(px!Q0-L30IyOUzfH>SjCM=~DhL#CxYnh8Vp=;SWU(jpG@Tr6 zhRleLdzDW%^|8hXO&-+pz6o$O)DBjMrwElWWC_6-5HYtmo2mNH5rQEliHT<*=S~owy|qj~>s5YpnP@i461@ zs%iPxEY~diRgwPusH>S?8IIrmsD@c6FKOT;WHs+INW`zZXspdJ zJo&$gM;YQXbdyNavp_$+Fg}sdYB-9^-Bn+(VfD`XX0b zE*1A?@pAVI4IdkAR`6h@8$Uz&HcG&W^q|3IbyX)L$wqZ16w=h<6Y)YtxEOmDk@RU6 zrC0Vuk&~pq8z{!P&a?WZKhq$DKm{alpdVKi99toK97FHk?H^ec7|f1`0|B-5U$SaF znv?zCub+)FB?$nUU>ty$kyTgH&dzxTyfGM9?5Tuq->QYO$mNs4w@D$ zxr@;k?DPR7pgJatDE(##PxfgdwoHtXv^o&>o?7Jz$#>EE6`ppTip<-wE6rO-<%p}?bF3~)W;tTj}YlD(TNmqERN z@1F}&H_Z`XGA=GxBRfzsiX^KcZdgMSYldCU+1jUUk&^5FpFV^#mUlvN492#X8X$&d z#j2)*`k_Kn7by9Vi+zYsMQiEHLEYe=?&o;f*mF`91fY~(_^~u{& z<}sokZCr~3S~$%YKIj}s;p!F zyd-Bk@S7tg(HlOR_owKs4n|Q5XNy^u$9-BI0?KQB4!tgp_c#kH_D?*CEucA5z&?2= z65H?Ydd@nWl*Ub&=*!aDyCJ)lF5J%!-%SJNZOYY-{8}b>oA2TJn2_`n{$5{D2e;R%GM~?*^Hi^G%aT?ZS2|JzCl#}C6|)w-4*=y05+O^1e|hRw)-mrk#ubO9Wa-~CgN*Gi}Iht|0x*Ec$kPu-x0isOE~sGYLo>_z8PFzNDB z!m6~2Z05Hr6|9b>UuXOgw2eb}@JjRaAF0dE1Re+2q^7O=JLhp_rF1;LBDCN5Ue5EU z&Q@e5CFit;)08%UX&#rQUE35s+b2kiH=U!simu{tL%v1Gq#o~vZH-2re&Rrgnru9L zwEyyWetx<=y?wObw10N&V?Q@NpAeIQWGHzL6ysmLVP>R#M$)e_AH9!Q-rzTM`Aj}l z#;nyxz5Ra6p7QnnF1-3oX#RaKzA+rB5|@-vsF`FWmF_f|@^Jo*+Zg@zzczi0)#A^0 zfdK%h;Q#>G|4TqJu+(#Kp#4e3q!-Y0Fk)r?_l#Xd!(m?KmPJ`SZfK4GLlK2*TKuWfUmi2p5}Gir+1*B1uqjLDo+sTqeoq-+wdkKL?S3(H zC7c*u*V|i9z4w^!_xU5LyyqR@RlA*laff9Rj`2mB49>r9dowJubQqh}tNfRQW5+Nf zuGci_g(AXe6fg!ko;aGGuy;ZLt3^D6ci$WJ4Eb&y(cDw9!r1RxQ9gVrC8`;N|6+CD zlEySZE~ZeRrUi@KI@o@#y1RfmB2Ph|9)P5$*nUkYU2H84k#2Gc&2rgjvI%m`0deM_ zQjHeRB5P8Pu2j+Ur%-oVHl-n#HldfY7kc{Tp(ohElc|+6F6EgdzD5=AbTUVWlXwAz z5j`YoLn}H6UHNIBfGoZ!HH69IO_E9r(nDZ$HQW1n8Pf)D2duGTsbRjeMXWp&kui-u zR*NJ6w_G?$2AlJ#Fa@=bl5H*$GaUPqME$+6RrkrD$kC-Jlg>*9?dA?(0O_#<^U)}6 zv!|nM^_AD4nap=QH_Y$Up?%PZV$4uE#>rPow@S9Fzx9)w)H{+7V^&grBno3Ao^GUwK|qP*Il$Kt3QZqMQKt6kjWo zNWD8soe*8GS%=37wIe4sYbVb%2gp5TcT@!%Z)5 zf|h#~^X(8ii;C+L>9MG&(y6wRyi=PdmvJ=pRH8xCtOj5I>z$iA&y%nE%XUw9oq-^kk}GCm^TtsyHSew z48N*JpdrUQRXi+_c56Y1(}BZyG5sjgJE;9qp ztP;B3M}t+_*;T@yJDwiHV(ASf)(jt`8Qq|@Q@EieDBGS?+ee> z?8X}lp+s*V6@`703S3N-rA)TrLeixAyYJ9X-j%XX;Q#qwb-PLe!}}Ab%Z?2I!2kdI zvmvNwX{rCy%Oq%NW@P=JBf6h#-3?Zhe|jX9avgldCaczFRjH83yI{g_NzQ}0K@){G z%uVG8C_9E{zqpit^8}`#pPnzA4UOR0 zYes%^o$7g_Wv@jG+PJ_6`5Q7RQO-aNyma0Wm{cl$fgSvW>rTOO(TT8i4jZuOg{0ci zm8FypLN>Smg+KsJ_ZQHhOJ007$?R0EA>DadIbkwoaG57sEYc9BqfpT;$6)LB*D-p^h*{W6^Nb6BzSpWikKOFPzJ>|7~SzojHXKjeS~IVeuCfFt)5#ddhVU414Z zq?fGYA*Z~=sXB`vrYj#3vds!9%`?D^Dsx-Vp3J%PA`b2d&#?H86kTtfnAv_yF;G4Y!bS^7s}mLSUl;F* z-|Du#Z&g~(%Z9?thNPk=e=UGJ>x6cWi;b+)!OY?{#=+z@?edddu)N^`^ED1nl1?d3 z5{1hLzd7ncqD1x~uej;ZgbmIibJc_Mdl2H-8+VO>xK`wB6aGjRkXNX!(uZZk6#N;7 zu`v9HKSMZ`nv#V%Hli)`u@W~?Z9o)OD_wu0lhr*<)H)&`4^D0|QHo_@O{U)NhX&sk z0^k5pjP>@idaI#YOuj>+v>MOp>Z7eE-G0Mikx(Dt2TLCNF>E3{w6*wW47%2)DiSjI zNc@33Ouh5Yen8mLY|afN*k^EGdACgTp3>NR6*UR2dEu%VMjcvuQc}d@-7Fn?8a(#l z!X3+`OqHifr?}O5@%nuxKf5Z=d32Sn8 zvG|1{_r7Aw(z_O}+ZD(bBv%&xC4FfidCbO(#r>552ZkW9w~`;Qa%=pR83+&_Gv$7~ zIsUTTI6=Qwc+(JgSOf!ReF!$FUu`lU;kvyA9k@uG9e#@nPP&It^bVvkOI=r_-^Q$X zS-)$e8x8YRWjD*)mk1}z&Aypm>TV2_dg&DoR4=@p{DKvo9@g%+r0?IB1gojK@#uBU z%H+ z($GM41P*wap$`CJ%}FMedQd;_>ycqJ{Z-K0WCkQA?hR?VAL<8)!M=? zhMGMfqb2JKKN5Z)XPWyK{lWo&tfNSKxXX-zMW2SKVIc0b62fbouoEuW;oq+=BvQ&6!cWDK0Edv z7FDsnwSKQHqh89kP@0KwPd4YmQa;myQq7jis56;!$$SE%i{h2B%AHp)FB6%bxLQ+n-pM1HKoM~2YfjgInLPmOY_}++yY1^1fq=g za984+j8;5iRr6g*(bM-_{yp#D6PDOz`Au@8&e{vJ+hzR;SO{ZT^bCwYk=ut#J|RXl z6inkPi$To{HoohzcW1Zm;3Q)zFcjWQq)f{J%fLK=&4nTj5rW0v{f32n%D6edZaV67 z_seCck})p=`{)*Jiz>iHbAn3+&#}Yslx23gyA^X)^;4y8f~J^bmnBZ;_G6_sbN^CRJgb7lo6sq)F_H2d3jiV) zv|l)157!`Zwdp|*5T@p*>ik`L@p-J`=G$+xMP0qaNSm{a{-j?P%!sSY9&c@J-W)+Q zCU}0BcG+?Lp+WT#LBm+}@(2Cmm~L%Wr~W8(;X4F>U35M3mANZMOX8(=?O^3*J$o1( z6}kpotq^p}=fA+afsRw9mjF{+2Qal_|NGy`-p&p{ko?y)yT%S+`#X}~ab5Z=zBK*? ze8`bJ)e)-SeWnTwrQ@}W!h8^xG`? z*cPIAHD3z`SX^&RKf9eB*S{?8Tjii8$5{tP>}~_Ze_7m>bMM{hrWt!oz^9bPu&4z{P4+3xX4$_-X(vq^F+0&8Yr z&`bcE9&1s5SB<3Q4~Ami-a1prsNiPlv1PE-vL63sb>j*!R0#278dZ%eMP`xSnmY`7 z*@dD>eIfp~y3ndDe_7qbzpd_1fYl|H7Y10}51UJnw;3KZEh{)g2y`^)Nf2LPPDh9M^q@Ak{?<1JM}&_4QY%?iz;O> zIm^gUBPfU5w^{~|7&`)DOrHP?#*oiTgUikE2OSM3;((u#8Oo`LzCHP}HIo^4!x57d z+6Jv;oJa>FV?WL?9v9mwraA;XQjJDkKBE?i;s@nkD*oJQCn5 zloAg4R~*09@0)G9uo=Sj*-?Vw;~K+0Gl$fS$K%%U+m*55-|k4?>*L~#@Pgstla|!C zt=AFOjJKhCrcEJ3%lQb&GQE!4j$Qk-iBsprUEaQzGaVt1ibMMdVNn}xPoG5_dV{=6 z36Iuv)%^f-KROk=fD^k~bfg3(ICUJ&MH_f`_3c=+sl;42<5AZ=HluX>>AF!C+%Dk> z<7Q*d-n3)OL^r~K+M}pWUnY5Xi2W5@EU!kv>_9Fd5Dc2B(iuRCz*pEjS)TidxVHz< zTI!c67n9b2)GEx$#=XDtMtD1v0Zp*15TI%pZzIXZ(<0Gpurp$tkU7gfqU_ zBA!UMg&rB^7#-g*CH@1JR zlFG=3idh8kDcHP_X+jGf0@68idr_f_+RUZsHvHetBs40gM*Ew zvC;p~6F|Yt`FFwWMUct6217c1Rh*6oa>~vFI3?7{^Oo}M8OSQ+GSP&xaq+81BB>p1 ztoUsu!=r8Nj%%m=jAzbg@zOoXAA;kC>BR*ycouL+2oFkl#+c+M_+fs#KeXUz!ny0>j<}-VidMzxoo2I#IX&t((G%4?F^^7@dhR772W&_-e4e-SO&irfJBBdjO|FU} zs@o+^sQfPD6@J0sC1F!nYT2kp+A9#n5mo06^vW;mgKMDbidxc}yD4!!6c&SD<4v_< z8`JMh@kmtB8ZZF%rF}@n2|PQ5gsEEar~IWQ%nWF7YCUu1h2)I`A&n(=S;YE!HwjJG zC@1~_#R38PZ*}L7A7>Q|T)=%}ud-B=(fD~PrRs-M&pu>SJkV7$2lj z4l*u!|FbC6?N(|pnSk$ zdT13SYS42lKzXL>OA5*`baGe^=}ZfQWQprQwatDux-HU1uq)0wb@#Jb8Lg16#JmX9 z%$G}FG(E(iU`q=m-e}qfbvIVDvrGS1DTPEfe{(j)2nB4VXitRkhR*ycz02OPjK%}Hwgfj$c#~dfE501d-4xCkuY*GbvN=zh7HMY!bRHt&sy5@?^>FKYTWGtO#WvY z2-^27;_FVHUw8ug+;v{TcJH${$o#_`tBmhHj=sy_vlPFtlj(`N+m6xYgNxXQlF+GF z+bJn2em7hPa?E%5NQ`~|358A(m|%>&lN6bIr!o+#NIT|oBvvy+vlGs*19zGpczqC( z2rM7%c=B^VQWkTXx<9_!Fv%)as3C;GMV=*bpg&tGw6p;49UXT&uZIr|0O7El{6vkL z-9347FSFmt$?iYYgBz8&L@}2<&TKqs{0=h&)samNet35;GiOY~eeg$mu#`i zt+9c5jRa}!S}LnTd6E*UAC`n)TqyVr&ub)PhyO8tHs+0&0bh_A-%FZ-O9f@53DQf8 zSh2A#vsl`bG0_IqOGxm>o(yA+&6?@{YN}~ovND2{SZ6<8+Eo9{HI|&&ZY5fFOFKY; zKM0gw*{T5*(^*L0e;|G6B2;^3?OX&(g{3^( z%jmi5Nqw&soroQBs6Dbsn01ggPr`p6{c3B(30$kzVu0x~eXS&2O0rm!k~|oJX`tL3 z-VTPWnxHLu*4`N4SUdoZ<;4Kwxb>lu22>TO;ub%cVuC-%c+C4|i6OO6fg7Zs&7NP1Oo^kxW~p`#R-LbC0JNEL@y?0+ zaLe5!TCx;0mE~&yOP{C4$2IN_btvon@!hn0PGasoyRa@UypaM)+F zt*+v_X?CkURv7NNMJr_?)Kh6J>^-Zf1IK zgQd&g-!r~@lci5Ei8wij?*$K?sVNNY4{yAJw4m}gvy3&)6=iQ)mR@nhpFEao6#5Z~ zk4XQ>cAmkrzGOUCZo>Fk_$fWvO!SI`9X157?_?H+FFzBvN3m=Zt*6LZ~j*NBb*r*Waj<4_*{MzBsyV&IrR|~TUFa;-W z^1Y%R&|O7ymz{=A3l~@ZWar1hA|xd3Ux(jQbzSp&*K9K0aFB_Z<_p|IPKD1HEK?6l z=Xo5jVbs>$b?cheg@K~|K*OdS@m7=0NV|cxO`d@t(V{youKH6_%`jaRO}}lk-3iFf zRqw34mDXE2X&FwwTPK#^`O*oU<$RAQdC6oorFn*2R5GMgF~WqS=sFQMd7(647@-q4 z8w_*1!*rJ;7~G;_O~-c@4P>XGbPTJqQ=QcQcGyAFRzE6cu45;bqiRCiu=tRB0Y^QB zFH$XmR%qN&{JpDDWqoY0F>*7Z`n&$AtDTKQK+#5d=$E=7&zb7a0o9VAJ!OUZPMQ+T zgNH&v*nG!+U=F8*#EJDaDqGiUM37wUdriXT_WU2R2&H9WucbafM% zD%yXMl39H~{&SEyx(;HkfYQcfMnn4@A)tqDW{a>dc=U6Gpt2LY{7g2qs2U2c8 zO;wS^M3yr0>r6z2@$)AlV4y1j5oT7axAU_ZPCwv55i=G6kMVO?*Y=1zU3{Fv$E&tY zy`AocXcdyfOJB4VqFvEvOo_;3$%A#jAmMEqK9O5P9Cd^eo7Nae0owhr4CDk)50s&D| z`7UW-U!)D+?!9?pKUir1{f&_e(T4~3Bzmd*FxM##{yLs6Cf59CGgK|+EW@D1?=RpL z{Y@@Wb8i_I zgwhDNnP;uqPQ`_((^tDHYt9&FY{0^zB`bBMNYd;b*_tQ!#Q}|3(q{KZ?4#*-WsS~uqT?;^o`2U(kdvM z4($s)3FJW#-1FO*TS*N$heb3Q@GIMPV39xcuS-DqPwi#u8xUjB?JYS@kLp!ts+gGg1 ze5n=#A9v)DwjNzrf0V$T17eSwss|{aM2$lC&A6b+W}(`B+g zd~JSYtF1Pq{V*pQ_unJAbp;izX6T=BqU?nS7#F%10zc@5QeF_AR4mCY#vHB*_2l>X zgHlUpMcRX*VE1BA=<*W|Y2*cweNQ!n@c%H--I}cwITpWmlmQ8d92n_VI6o1+L!TC0 zvZx>3kmNW+)Ev^RB#t|Z2R32_5`=!997i17dUuNl_kz^Qz`${Sk|7q^iH6=P;7o)pC^4-#IbYh5!PhSD?#6XR-ZV!u@PEvAk-qr;0gyObAeq6k+w^5uK`P$ z%UL*sbeNJ#+r6npXtKIG7o;(bLYdxV?UkKtN{8xKiRxTYD3E zi*l>Tr_Hwp<|*^btoLik_p4oX2W!TLc?=yIbT!IZ{3;8ii^)k-mG(ICu8#Tug1@{6 zwi+tB(O2LJ9j)%B*tnBI)t zEfeGHNj18NUcMd4Fk~;d%7S_^YhypeTTmuMVozS2pzdbnuk$)v>x#Ax1cTn`t*c#? z^T0<}k}NP332oVAm2@mNCP?Ibl~-a+i9%GYby8X$S!V}j#}Pp6_CVUa@lr?j|<@~hbl9mR?5;3 zJ@{OGKxAL;`iXsT>Vhqc2towO*)t--_dPrvy1*jgY$@7J8_>ydJ-Ls9qrCIgWMqx_ z#t+3e0Y99O)I{hb!3DVEjQU`9f|8M-N?OiTpd4tm1m_3;Xop~MN!%70ZoE_qy&)j% z-P_IY^YFS0mE&sM)Vs>-5am*e70XXunQ&61BAz1ra&e<_gTN{~exd3hkJ4Cy?$*Rp z^KBW-k-4xS4dh2SazEQ4_kE0RZ~OH%0aljQh<#!)?R`ovQ)O6+72>HDLQJOL3a8S9 zh=|l%0R|Z6=aCBlUtkNgBiSz!AvJoreiMOi{~i#?y%O zOGEI_2X?oXel(0_(@eQSfxfHSO}SFVOsWNABtxh@*%-1jDJC^l2s)bxp&5`)smS{G zd^;Z>yrRig%APLaoC5)cxVp(Q`dvaGvJsP;^f4Y2kT~k1@$q7b-|*FGQjxeQ) zZtZZ)j(uEGb_FRuZ1}Me+2Ey_T_+PY+TM%~b|wa}MJS4yhIArWtxQkEIA3H1uq}B= zp3+#>`YJapCbHf83;J|a06(&KCoiLtDU*qbd4t7SZfLDh<<7kM{`7{0s^hp#3+BV< z?tX_rHR($k7PQB=ML&1I#Ii##hv6g_>&?D;_WgWGSBg7>=2B3LK5}bFy_Lm!$fzmKd=B*X^PSw_-&yAQHq(*AJ2{NUmKmxOIQgxeFWB2v(sF)wax2Oba zkjWS!(rT&0&;~q}us6ZUu8&6)?W`YS7LzTFq7l*3IC#Dj;-JWi}KD3XucJlvA0jaG|Fy#47?0&2YU{^LK9?;+N0(`x4#~)$~?!7uY9Gi zlB=k^Dg2TDHFL0Vz;y6{j$ooDH-IPAI z#LvZ8uyIH{^aISY4myL;EhrYBkrDHT`ilyPwA`I-NlV(%7om?G%!iVEvY#j7Bg5;Z z;!k=#Uhis9QW%#tw@525@Epu_eGYh2`>QDLI+^b(SoVaevBNyBWIyk=ief?qfirX< zS+2~MH~B>YXXa1h*52hLv$3BuO)izzF6o6^dHh_|y^qjNO@dl13xd{x;MQxTu_7Zb zQKAnG%MtnWU^vs{)i>p`jMO=A%8`X0iAb%aM@`X!=yK(jggdQQBinCg3i+w|{%D&vG-!!3U5~8xvCUVZ+!!)1^0}lgR+0*&1iXGKT1sFN_L->uv zJq*w*+{;%^|DB=NUk`l2zUvjZ%{PL9g;=>K;Wm@p`E1r} zE)ek3BI@z%{A~7|GV(Y%dln$PmpbxPutBF)jq^C$1KR@Uo1;7+tEkAi!A3tk1r(hj4Vd$vOGLs|dpqM`4gp|5_F zz^YAm{aYD!oqb4mCbTz~VO{VBh_xg1AvxvzOz%IJgC7R5(l;Gpms zd?1ZUev0Cw2N5uekF$j_ZVzDe92qeOzhajVFV5ZgJAT+;{G1(`zlW()nnruXl( z?Cxyu2bKSH>*ZR~rAkB1mcW`(H&fz#DDvUzJ;v@?#++=Nsy1kf zO5SFUS;-L?KaPwgRYnqzQUFIwm2Q<|hE~b2L6xqXj$u6I3dmBwnS>?pzZ3x~dP0gB zVeI;-@_tj~Zw@5~XJoljZb@<(TTLl$NMbTjX!j}pXu#?Z!sHNck?t)oqjW_Sbw?}o zRc?wy_e||i(NA=P{VU?!~ zm{y^s>C9#mm$9^TwhP20nD0rDHSI3=T%n0N>RXp3uYO4G1@7q4dsB)Y08@=FQ7wzh z7&MF0;=)_JRHqS@BAXwbbOt4<^u2a&oA-O}bpDAz(E4eI3Uf8J-MB^;wNxUa2*kf| z8OS>s^3p`-HD^wpb>yqAreSSH>)6e~U3}nCe1}_2-LTaxaL0Xw$)TdDr$mn->0`S2 z(OvwhY992(-^)>xv8%)UTQ)K& z&nf9$+#VIi04gRK)4dJFiB7woH0K)nJY^H6kLd?O*oPHyO};Wqa>UhifZc#h_+VUk zY?m{y&!)%y0@frLa)?6jzc5MAU$~|8DaOx9xd;zGZr|F&r1qW^*c4`z^@b&>D5TN4(cdSc z{Mlae2IVjR#jje9Xd)8=LZxR4ARx*Ae=73N$-b7Y!X`KB=S9T0itln;W$R}* zzgV6QfRcNi93S??CuQ8{c@xe#cRe~8CF3@0+teLcm)H_Li{hyo@e*!`=IYNx?BS+*M6-W1<~ulOVt?K|R~bK6*rl`VI5Ird@P^0P6rX^O9jnTztA zhbs#F!Wi?nc1M#!6;Eoy!o-#3-meg}?GOJeZhA1;Wf7{?D^B&Fvps(<4XvF~JU_!` zi~r*1>KT`)0R;j@Wh5rEh@+5W1`0h?j~&mK8^XIkmXD(SM(&l=yk_l5Kz22>G@w0) z7z3fSYJqFXklm1+41D%topD!ep zX3;M%BC)J!^0v;lE)f#+Q24IBv`N;9sc_&~$>!qv*_F;A@?{8LKB8P@pc>;VV`tmWzCc~4C(!uuA8zH$|;Ain2%_23@d?~~Ph&R`w^KF}m2 znq!urE;~3tAHW6^qrM@)1lW&s?|{Wj=N%t6dkx$HQ9gxnCO&KhCkKzz%u zi{uazLF(9QSiC(nHU|vc{GJ|YXA$^Jx`B&BXr&2e%csFHTaGS*LV>nWki!yZb`dTb!PJkiMp*1d zZpIX^3E@{6;~MJMg-8+hqE9$(ez$QRfFzxi8i4!#%=7#-TOe5`Nie)e!D&+{FpJ=& zYx&l5W4yp0ON8i$(rS7~lo$9XHpnaV^As&^Vm(71T`(4S)dCzZ&VPKc97V}!X$1r| zShSf#&RwWjYxHIN2V-;GGDcYnOt58~_AaFf@SUco|aIYcZnIy?@9uvF_(p%zTz=W zTl`UCgx`{y7wq+>ywB`Dm>VyoSX_hk*fEPkB`}giBqNtH@p7mn4r92}86Q2Bc~Wh2 z2$WW5q_IGctMjy&?ln3aM8SNP)2IPLNyl zTM%=`5BVMUq_xsF<;EZSl`V*0 z25bfXIr4pvO}ElH(ZqD~mLg9^$YA{wZ{Qc$%OuvgqBQE_-#O+%`ltq#s*R_58m9@<8WvzyLW&@`iS z)C|}oObgSZv{9ME=E+M4EfJUK9J7}J;Rib8#!My4n}d=sZO|v!K__1P?AJIPX`+A9 z%jv9?_=%^D>E}9p>EaeZ^kTUf=4!Gode-^ZHoKMX$|EeG%`S=jziI7AtIMfdn!5jo zi&wW*+2lk5{JU!%EUSzvI$)IUBE~y{Aar<8aBW*#6ylL+7loSmQc`1E_q{vP4LKp4 zxQ9B&XXDYdmyMkpdVCu*S-(sYGB6~+v_^@P0~XX!2=d2A!d zrU6k=qKf>O)D9@aat9}=340cJW4GX#H?l#pYWxfTo-idkDfew*%?x0=g+EJg-W#GP z9z`62bNX5&{!pCQl$8ox;Agf}r4~pqmrs#OW&}}FsVp@s{N?VG;j!Jf(^s}W{iJ$-HPNm1>{6rvP4x597H$;%mxzErvZou8y(`yeR(okxI4Hi# zb9dhR4oe)P zs%TyJup~9+2mJ`A&q{9yS`z%U&=Xf!Nyx+5x_N)g%7;=IWEF0!47fR|sTgk|VwFYB7QO@E_w(ero6ffZ*c1W(dI8$xyzX7KmBcOC)Nng=r zJ|n zEs6Ayrac2s6|UHbWp@+8!or)IM^*zbZ;vBADPDs$qB{j>p;Jfcs^MwE0ImSQ$RAsb z;SfIXYf8Vz8H}KL6ER|NRXJL#YNzUnF)5r{=8Q}CQ9I_|drvp53uYf(87=7dci!xz z2~M|)FX(h@@_=I1{Hi&%J5Xxyhq~N{mucL}Fi6RYz1$CAmH&!x!IPJ8KS#~ETE_9D z3t!PGP0aA&O3ornle!0aLvz*|5R{aaKDjjPO7wxvB+WSe@wn9~toDevDr3uTX*hq~ zKSxyy+QM2Px#XSi8DwA>uNms;djMx?6s83Ea5x`|;+ON3uTuGwW&vd#?yDbjKar7N zYxXRuC^#8+bA0D6keXi|xp)J7L_RaGsg}84VY)kp2^&U)gmX^}gLqsVE18{~X#v(P z;8bOcOD}l+L{l-o{UAzQhSo84@>6b78s+b9`e98nTW{fvBI(8Q>Z5d9HhE!kS?Ptk zI;f@gkN02E{CjjfMSt>PN*4Fjak?vkMygr?+v(T~QEFu*v^Vve*tN(PPi<_|;?${{ zt~yQE$5!*GcybE?4_i||rq z|Gv!e8UHpiZ1mQyJ-Tq6SLwYaX#)>$vMj)!XgOY5Ol+N;VsUQp&v8kpt1!rKKwR=2 z5SQ@(+l9_Qb|wG!rUeZh`~CkQBFFh<-xAv9x6gcR0fWGZ-PgAAFj6-jm1g!&H3qPk@WGl=GM2%CFE!l|9VV1j)%UpGHKxE^kt zpJl=wP9K-=h(OV+m+jle$roPL35HGapkhA;Me@_NPDzA6Ebh0q$WQJ!?1wp8=LZk_ z5F2I&HS3uEqk}Ck8nH(x7*kw}#-#GH2y&oCYAB1F zEF+Q{&~$3tQ6W?&FTQcHtBgiWNO!kQD-%pU)@Vb42}$NzH;pmTR<{2A@XR;lc+BPoLuhBy8n^7e!^I3Y#jQTEp_Y5^*%P3^Z~+9fwVb z+t;AZ-*@bRpA?d>peK5t%ROCs0uN$6>uvR&}tw z)*q5rLMOrtK)%TU;E?OO8wxx5I(ntYBCDoqY7b@*Fc=Er-Y%%_s`x=_Vn@1f+iLSh z1MMS~UQTR6->G-b@=za2`H^1Z`?_%-Z^GXp5=Vg37Ho_B*7)q-i1;GIwD?=pCzcGP zPq6@52}56%Wa53AwjHh1XuS~!%#C!RUW>-+={#^*JY9F>?R=6`9>YQU<4Q_<6}M9g z{fshHMuuzfhGXre0;F*?*q?uRaD@%>v_CK04m?f01K zGD)Kved z!l~^-4imvr)=&L+NXCzl36oEC%+L3TCy#mDeIYaLJ3i#^W_Ka@$@8uZZ`ys`Hvhn^xwzfmrkayu&Jn?om}?7Z+^MDi zrI%wcFf9TS+xXhfVZuyO|DaU_!4(#_3lNWvXJq&kRwq)iWY0ux=ddu58FNfXstiO! zQs?0INYErJJ< zM8#$vY^p>UL!Fo10Als3-Kd~w%Jx4QdMWbV36;Cup#D`%M^2MnGEEbQUKwW5D|x@K zf)R-q6U;IZ!G9#fv?Mu$i;$&Zt1X$Rr{CN3+ky@)gCmYV_=0>OCfq~Lb$(X)`Wo;( z9FhT)BUic5Wc~nlLP2dPn;}e>Z^3UlXjdpkhsYq`;9f;NM5pk4e~zC@+XN-euwHGo zcsn>x#{<{=Ka~UHM?6^DWRn)oY0bmi$j0*_&S+^Kn$Nu`B*oGn07WOfB;g~RNgGux6@ANs^NQ|6YUE{sqT!x zf#DI0c!a&U3FCT^M6OjocQTiBhg!$FbX~aws0of2NHM*N_@NKt1_~oYpDE+91g}49i|hEh z)S%7Iucf?l$Q)-`S2(V>oLF=laZ-)sBNi4!G=H6JJZyh2UU2)PPxq*8&20TTb2sJzDg zFFqRf4#2*!+J$)GKWr7eavQ?l%CU7X0_6 zQmVWY1Gh}~|e{c$wJq@`U4WS!* z&GxxE3QY>iM|&`l`-u8UQIimHOTD_*t3Nyqu6G#mNca{@0S=Sy@9@WrY1|7{G;&8!}HVc8M9HD zpj@I_w~WIyN?53D${qQ>KQJ953T?u@Nxs$lEVpOw^GNF!DCn?c z`ZF{|d(PoVD5l>(2Fijs;-%OTe-+KPK9XjHkS4QEF#SyLAj^Y}05gyz@4}z|3zx2y z>8}%Tz|r3darnR#frZJ=ey;{ZI0C8>gne=Rcv>2ikHu zS(SVum-r1j8H;4tPQ=B(Aw`n%Zqp8v@Stct`mwhV+SSQ4lV{kCE}eA1*NPYREav2k z=~OGPNy(?cePP1&5#SN6JacQlSz1`#S|^*T`0Zx_|!Hy#H>!Y&g}=Mc(-)q8F2Q1|{>YxdNwy2{+My6BD5t^O`;^|~Sj&F%j$3nCU^ z!j{hg&aN2X?EZaZ>;L6=A)r3oko@n7ITR?lDq1r%dRvu}0AzQKH&pm%AUlZ7Of15c z$}iv9sdSydi0xMCpKURt$3Dm0JAorv^1XN;d?y3PC+B!@JxV$Ez`PX_JPYSSC&bh2HGTEK26Ylc*3s@>IU)F^;(&oUdT;@ws3hwcBh5bdo?S=wphj?MvC;zUE<}3#^Xr*F zyTVgkZqA?$G+m#z+TPBduAnOsu;%Eq5|&~Vy80Knf*>0>+x{OxUFH6W5Mm#b7$DdD z@r)coZ0X50tY&|W--ZD;%@E+nH2gh&yZgGFY0$~c4;a7wd(#X(SS3~AqaX$sm6gwm zO>qjOO5_0`h7+%g)iuo1iY|CEY=I}HS!A5r59(QxByayE&U)(5slP~C1>!XTtWfQ(GYh)0ok`HBhpXvZ{mJH^EO@$%^yjcjexy*KxXj{iF zQAc}x{@1s{ParQ2{}#I|4F$b^Xdv+s(+SPgr{^H#=1Q{(3=w()OaJAR-4sAOy;ixm zqKx#ZRdiLY&=*9T6eY`&@lYe=D#gF$hEu+H0d=*fLfV*K9WXJcS9rf6X&-ydZF7)z zPN)Kz`ez;mkIZ1&oP|ERFPprM%b|0HKs-}o`Dx+=NS$7DmQc!ey1TUI3*jX7nvQ;x zQuG0zYIcb!9KP;-8TqFRd*NYp?nt@}E*AC;I9a*sd9}NNYRkuUJPNnXVTU%zR%Ry# zU}DZG4?(2jA0=YC!fL5bcj=MHrgga0`4k)}Jv*R8q;)5ZufU6%>Ua#VDOtWfhKbks zF$sS)s6;2$l^oKWT_~s0sI9yJp-KcKtkX0J~;JMx?dEaENeG@EFgd9Fet< z{UV*r$9O{6&Xof60?pQ`?G74A_1}0ks4o_*T35MbDa2x4UOeI1$az*v7Bv5^PmK;qfZSXK!sV5&HEvM>E zPWTHv#+x+WG5m9x7`&AGlMUeYo&ZU{#Q)0c{ol`WssESdQwv=FC(Gv;13*9(faw61 zPy4?upI_t;TDlfQa4XUl(O-PLzb2hmeOG!fhW%q9Zisp~!{?71|HSj%_y~XUUVU4L zf3rIv<-8E2wJdPT`F|)o=NQ|ccFVVI+c=HWwr$(CZQHhO+qP}HPun`(J^g#{+D7D z%m>|(Cg&b~;cOA0_)mdfk6X3X{oFRZ1~~WvqG`;wspSSeoF;k1^9q0Y z&y2e@F=f`NL@hjK&7F3F^`(QjkpRC$#+Cv9n*~p%d^?nswEO(nj{_9Kf~RZ;MLbkq zQ$ISet1kY9C=-;kP4gsIe%&z3wFaXD>!IZBa>M91X60q{-JQIl;!jKtgXfn0{U?lD ztBSAOqP!>TNZtOYT@lcW$u{NS4(gd=Cu?{w= z=^Z=2hd(84ggynNrYV%^WY}G+mh7sa#}mVl@2HX-1&;`;FcE0k_NKUUs(DJTz@vER zrcz}$C8I*rLI`V?n>wIz)i&Vc?j>YY-QrmJJ9?U@-cY|WRWHLMf`6@P*v8sd234h> z@kCxB>-@xVYXkLr)3vc*mmjIftl@bBY^`uCV^}P9Y%;H9j2^{GXJW7>XTJz!;6IqB zp-tDw5UW>Ng_qm@{$8*Y<&4-meZ6$^uKW%P%FN~`Ccb+l({IswoaezwkiriMkHZ+7 z;~c6ZNKR#vc4W-M?Q{*6d&YgCV!}47+o0`_W4s2Akx3U@mdq25`0l%=bBuImlq_pz z|3jpMsLi&2(}?FJQa`|GRr_7*Sl<&>zjEX}u`r(3xBr}KP%zQj3{qYjDwBLdrAc10 zcQZxpHzOIePoi~O03Cm_m?eHuh0zWrI(ON#!@jN9XwxX+ePXd2T`Q8>mRJd>Fo_lJ zu#K#>7BP38qb<*hND(PP!6Z5L*p0x>qS_gjcJBS#a=Au!sz`CmZQk>8QnCY!vAymv z>mgdufgu`sye0#3$uYQFB({Dl%?96Xe#H_MJ#>Yq3}$;DNVHxadPGE8zl6?i56*zm zY-5wXH-5u@=%m?w?fT`T+c%P()$x4Y7)Mve0CP=Mw&tp$ky_*^v$CYfS_VEypUYLxxz8S_gSRn}kHIjO2+a zI(O#2y+sugbR7=3BG?;$?%?B{hj)0qc-fVoP8(e2-|HNLxf`kIWlwLf2Y*4>m7~7b zIQw6lye-I|x36%Q&`46G=zp5w95#joX33QwagPM{_u*c)Yi%7oqrxfEZj2(HCUp2i z;n)^FD>g@9In5EZq9|9=<(bO~9c*?88cHK7AQ=VWn@fSpGx22;zDvB5?D14QmdDjG zZP@$Tx7t4MS`)p_!6`aPcsF_awfK7XeqVp?0CIHN6mDN-b9crWP!{Z%{y3bposbYI z9>Q!;^Zpg`PMnuB{4u zVJdw{f2>Vfbaoqy@aYQhX_M?sKs~m4R#s8f8n-A92lRU0H?DGT8#Y1WD0Jc+h;67i ztViWPNjmTZsMe65Bps&Vs=wTP&iqGj0f?JngTBIZybHA`L7h!Y)Wf9-VEhJCO{2|lA4U53&fK&-CvWc-x z^JEjqRVD7UkpECI;oPa%My{I%*xTsQN88`MD5WLtgGbW zBc^1hlim>#>l|#oQ54x@4UA8 zQ;I)!fH4)En|eNeCqW>9i^0n|pVU85I+%xarb98Kw$?Z>%`{QD_aLq~&9EWBzeMk_ zM2s|Wq`4&T^OgQxYn@F)vWs}-alguPnn-n4GUDhoI13FCBa0l-1X14|x8le3W&9sL z{ZNB_pd){*{NUbUfz4hu{9s@>HwoZXlVx-(=D@Ff(-<-j+HsgHhU9}z64nfZ1(>ug zRHmr8NbMv@<;ZyU$oG_K^OKctTzNT4d)VK_SEfRNCjE~aCbo$w%fT{UoD{>;uvVbz z3B!NlS^cI(1>*K!rGyJRD{Y~!n~@MoHS@YRoVpkes6j3>k`3lBQw&JPnOwYg8biT` z!7*gO!o>%sAXcFDJL9+=ODoSPBpvdX^9v-sDyB8eF5FZiRK`VeMpCN_?(uy6K_rRR zcbym?`0;8+kkyM&5P?8XDnuZ(UescR91Vux)^300d*uUQDxPBkGAN9{XJ~Gt-+e<7 z52Yh$Y0c9FXI9GI1VdqiJz%Eb(X6P|$x%lI_XSO!x_e|B#4=y}YTO_k%A~`%`Fq%J0Bl1q}zk6L%(?2sSl2=kllZNNbr_tOGvV>eQjCeZT_86hp>uc*JpZz2@cJZOC8F@*UOi_lUVehN|7u88 zq0$2N+Nb_SQC$-1yQZomsW*K}NE!L@HzwsM2~!+3Qp@|t9iuT(EK~kdv(1s0otL^n z(>=OkkzJ6CwB!0cXogxoc+Bo#vz*V2jM&OB>mSYoyjinvo#%LH86IZy25ef|bMV~T z{fxiN#79|FD84*=8Ms4Spr=wjoY$TQ}pJ z?Pj;HB=-c>^%y1%%2|{Sgk5dEiQ}B6(>ZiT()sM~s@PqVV?Jry0bj#wznt9>sU@Sb zI22l7U1Z>d6ms5gJ@T*<957l$f0hZMhy2JhD+5_3kXaV+*k+9wtd-czE0+g{hE0c( z<%YrJgYJbIXU=A`Ll2S9pdIJQb{k#Y*4o<5Ra+B>8)xS_Q;*&XvgdX-)rHZV*7{U^ zORyTYQJX6i+rxho>!A<=5DtGzGO9l%ng3}^F#k~#{ zBGW}dirn9dHJh3)d{HKs0SKQDTi=reaMJ0y_eG4Mb+{QksffRG z#RENLZt&Po7GWTcr3ywYNBRJOk&0e8CnccQFeCz7O1~|D)~kQf3xph6DMLz-`<YE&?q&66AXfhB}{M3@25RR;L|8@EX1zqmyz z#8ij)C+89W;uc+6G`Zh)@^6}ygE@uMXN`FcSxXaH(jciv=zbg~z6kuIw>AHxw|)F? zy)E`X^|tQ+(A(0g_Ef>YRSc(&Q{79=l8-ZuDbcQY$M(VTL+0Y3rhhn{62h9tKYCj- zEptWLa24J8y8k9w*eW^~iRugFBd@>$KHtrs*^A1LCq-QY1pwj;s$xo7k002WMk@Vb zB%!KoQ?HHSEx|~29PPa9yE@8#%vlLnlWYW=7`HT$PM7$i@*9oTu{3!v&5?{*^)ox zqT{?P%AQi|(ewW#7xga@FQcu-Uy!ynk((GjQW~kX2GvurkWEtzliw^KU7-!>FL9 zJ6^G>3WY+RVo^9oqr$o-l_o!L=pXU_UK>>2t!EQDK?9 zt!Ypo`;r!$Wr4d<%T3P4mKgwRTzoSF;m`cy;nJ%+rl869R8=3pk=} znTQLYgh$>Ra8HEuf4D`;G0;Evc;v#QFMPp4;@y*x%`LY~?9S5bQ7qhl^tMN~2|H9^ zBW~UbwdNval`mWS@6V0qe{YD6n&GMo_tOv^@@KIAzfbxO#*Vhm4u-~#wEuv9O&x5V zZH#D4%}oFyA)lW`#YNcI^%(%{08V~B2>{^x`22Tg!n<(~d{mkz322Q{7s?z@1;;l~>1!N85o*-G)oUhEvT-#LQ3F#7EfF zN66Sq(9lD`z>QDWnODb=Tho?H-CD%VSJ=c`$kb~h}-f3z+X&QbR+JQNmewmv7nHs+7 z+JV{nVTF1j`MSY*y1}_RfjK%sIobhP24RH;;f4C41^QtHdck>mA$bO&1qNXS`XTxH zq4|bkg@)mUT$;ARX1-jSHk|5KBIbTv8rEW#0RjfD{Q559R)J!c{#@!-GLDhl8rHH- zQR0^VVitai?g?_v(Mlda-y%lc_h&NcS^=5bfmuHb{;!4q=Y{YM1P}ng3xN8chkt$( z0Pt@=6T$$K-~hs~z`&r$;P9}ZkhsX`(D0N1^bniGl!%PrtW?6h*u*>&8#Gjes_N>R zI)ujBTGaaHs@C?#jvBMRKD%GP%zh0H_7nCG4v&sb4UP2A&kPzH6KoI|Zmn;wuWfGa z9BytK?w*|OLPNve+ycSE-@-maKis`NK7ajvdfojlKNQEK zO+N(3C5$M7A|uTY4w8gGFp6=cEg*{OtZpEhVFAMsi0k=RQLMlUL=hByA3)$YlmtyV zP@;sudQpAETh^4yx`j}U zGNW5IY(~9ZHmnr&U$!hI>xXw0rS@NTwe09Y(S7jY+V*^H%vI$XZN7|Xi$-Wc<;v)xZ?uCkxs58IL7-Osav`JgD5 zhW^258eOwrs8_YSUmth1w!A3UgSOvapOd*CU}AkhShs!Pa2~gT1Tgs!qGAI`8Mi^dr1GIu#|F?kZiDHf@?lKH2C+77L)fPB z;atZC@!kElR~?;g5#s6i4W1IF*v5}Yn&Ufv$3ULM%IF39G(Aeq3fiIE{o4NM0?^KP>u$V zfiBpESZN(2owLz|S=+ifYHkjJwKi-=+r&1J>>hxLOA$?fTk1__ER}&t)EHMMsXc70 zN|l4B9mAN36sAuHurR63RHbktZ&10KG$p8C8e$Nyk8nIoCIn9zkUtm?M}#wwlUV6O z3#uUjvDA6-Kj_bLVGKSW)A@exY2Fi|cg+>kxWr3lokE>dVlYerqyXMs zSs07MQeFKPABuU54?-bMlXOW1P}xkt#oN;FelX`_D3tdRuuAtW!vFa)ls6dH%4zBW z_lhm0eibs-6^#n-E=v@+pMSgz+$9HC>&|-{A2(J+LM}cov2~H~y6{5w*bM0keW>aBpU3!V7}54=%|@aKI+Y-Uk=}V;P8h@_HMx zTCZ=BmybEZCqnO652G)3h-p_g!Xy=SU|JP`-gyQ}qqPsMv8RUK4dD=jY9Ar(bby#n zcmGR5KY}DQ05Rk?hzJJ}Qk0+_DZ8wJ90(AA^S@=TrpOrqZ0Wy&|MXz_XE6N?J9j3F z-*hx=H1xDawuZF2j!r-HSVLM1eHZ|pNn&-u#n#i_AQ>d17@&`eM7 z_4}I_7&s6PfzeM0BaKAkogMvJn|xn^iR$7o8FJ+X8wj`Lx8;+eaUIydwOwa!0HZki znF8_8(cq_b-v4|mPWlG_Jq;t1IK#FH0HB}aMlbL3#(mmXnK$nf4!+F5cnC z8KK0?&kb*YC^#wmPh#!lw`qeFoU)`u#c*?KRY$%6+X?7U)_C4dNC zbqq`lCJZM&oVZ+!H5@o88oD^7I)ij4ir2*`_j~juK{f?-63@vbThzo;wqiP0Q@>PxAac_`gZck!#mt?o zj1BZ1{#mwZr(;#*m_z0}d2=f))IhLA5xFG1U%zm{C}9JfTp2U!iwv=~p)t0%H_xuG zw;nz}(PzD#Z`fa^xnFWB3reU-Ke<0T0G5%Kzx@UBFN-8?35!Y+cQ1{`tV$(kizO~i zCTxg|#_DGwcLbVs1e$jQ++_K)O7)T}`I0mH*lt1DpFuHS3pj$~Dc) z7R~E|T+jsBqhm#R%%k6VA(GJNQqpHXnhBxp$_YJrH) z6pKlbbrFtslMQ#{g?l$=5_TribV8=C`%P8}7O(0rSjwFn*Dd*!Y#B8S82#xq5BuZL zz9~_BdVVtXwzKg~r1<&mFIxI7uPA>m7ydW*wO6LhM(%GpuO;v`1Kk#6H#R039wshY zKLTndhATNc1ruIcS{7PrR$LNZ4su2^Je8m6ab7&dZC4tSnJ_a(qMsO$az3_R@$~Ot z>9weesppuMhK7#k5&!%%eJxe=9@GTMG6I|AwRAHJ_WpA(xu7($}2nEI%aZH;#H9q z88%UTMAxdY6*+d%UethV!hdUC0(Cc}{~1Z4eJQ3Az5}vDtrUFg_|=G~b^NY2K3ij& ziAD){cw;*w)gQH8hQGYU(QflR?QIHoBiass%445(^Oyk_JZW&Xx{>&OJ1QfAYxPtV zQjB)jhHt~dRMha4KHrY(w1U}a^P%N#LoLepg-~dJU^0|v3h4?yP+dc@*RvkVzhvN7$YtV;>~qp0@hg|=s{n@gI=)92$KoM$ zH#Lke&L{rvIQ(8DacdRZ@LF;%JqG6i&Ty(DRqm{K&{WD*)q@lPmCo{b0_`2)mLB0z z@UsA4TUntnm&Pv6M#?rpw7e$c#8$mfLcyomHdgYFiQXBfIp%&``wPDnn(T+Wd)?^|=CF|f8PMEm6Y+$2>e;sM* z{OS52BJjeM162N}W-8eP&<#}oJ4S_l7F!?qfi=}YrPXyw_MJC?qcO=PXIvPp&w z>d%rrnIp{(vAnnZ4l!Hj0+#@}f^$?n<6^=eYk~X6GhMD7AhJPVsn^UND&Ts64V? z_6&1DyA+=^fBtUmj9_Xl{4~dRR9yUE>h*MUUnm)C5JhjeO*?MM`?J}o`8e-@CM&~# zWg*&n@R^S$`zr+(01H=w>(apgrm(3aK}JBqYB*6emz#rWGAqGR*x}VlmXil$M}`mI zh*#l2j`?a?JiPvs^!~F+GFXHa@IzmIX|QneQQh65W%%ZlJ|rW9`q+N#Bsdj9q4o1= z$ZcRa!Ok6vZ&wN(CqyUdk2UByH0gj)au#Aa{;TjRd(-wVxxZ`Ty)LXCueYOgK&%lhm4Xbf@M+HkhM7+V zi_ToGl2ZJ*L2E&W)Hj8ECDEkoVJtQHpd8L1ubC4g?;cVm^w8&|DUX)&JMmH`f_vQT zo&B+3L!3>A2kBOY!rTsJ=3J65^8`R{^-J9ghI=aBfWx{kLEF#$c&`{ANymRt9} z#NbfYrH$a}XU-=`&E(;pRhFsR^|Mv{_9T#~yq?`~B9{A@6tIuH-*|;#xFVv)c>SQQL_|bYxwux{TUo4{wE-7d{ zg2ThiO*XYbY^gSqgVL;M*RB0NCOO!GmItciR%om6p>a(|D`%Xjw@9>;;>KrTZfM}) zM=OFwJ8gQ)0L2Vlhw1?K$>51fRDjP-8P# zf&QrRy>iX+J>*q9heUrlN4X|6*%`mqpaqtTCv(z2cga=zT41nCoG_>mjixb>3ACNC z_$qNkX?L(zzuA$tWjj2qoOqaJofO>urChZ~6Oz?yxe6%0m)b`3aK2^hZ;EFuf!UJ{f?6VeM5%F9~-5(O6C zV*`(b0QE71MhpnKXbo>@1n=(-Lz*qA7zeM~G{GyfXBZioTp*2@GP4>Gia{P0kQux@ z0a%8=z)mB0_fXJSTPP)BAak*RRd{|$p4@06!bkF+EqVS8%z)LJAhk?D(UxEj7RXy} zka{P$`vPbOR!~U5fDaUi`Dv(HOQ;uVNT=aIHmHCPav)O?(U<=~qSBx~c|JnN7@xnG znh(et1CbWnS!@|ZECg|gaHlAWWH2~{ zkbAIb=%VoIg0O2_C?IITrEEdhATg&Z5z%&`xWYd7Tl~&-eAsQ=or{E2qmojhl98m3 z9k!<#zULzvy$MC`pC_2B3)wzz$-QvETL2KQ0RY-ff75F|%Ko-R)IDX++b!-G4j{KU z{82Y3qk`B@d4$mdG*@YkYaUT9QGTvWy}!~iubvinTnmsmIFL;tcU47hfL2`)qFaqI zZvBg|rImNJu*ktg2RBY6xAWvZLu5SXVoF0Jkhx@YkYo-uWIjrgN(~Z6F=BbT(cgXX zuk1wP>vZa}bTY)h!+Qzj`8m}6iRE*X@}i|#5@hC(q*~6z-bHfr%2TM@GVY0q=81l< z|7E(4btQOJXN$3#bwiBpMM*|Bc6G%d`{Ucl@{!cF} zXOpc2K^qeUAapsE-;xX8pXEFdbF)f?Z;bx#WNCat6LP6La>WHKU=3hmg+qcYOi8|4rKt{Ge6p#g&KPTmnA)KwY6gK zOEm<0fmpQ?+1&`SUL>jAYrZ*;5`2d~n}fa9D*dXlNA~8&waX4(DIaZfVbACPR?{M@H-G#~LEE1nK1h=Oo^5H+qTHhpp9urqHoNpIu~ zf{l#ebtx3?DaMxB!qPL$V5VOjlRm^rgV>PB$>WV0lZatcETE_E(8ijKM;%0sy$naa zO*1~f6R9e5PQ_=2#T9!$M4%f=yie8Qz-auO(E8#)5gho@$Qc$P((P5^EQ6+0MNQ)4 z(=Er4WKgmI)$2p5qJ&V@3hUU6*vTv4!iO}8sW9ZiWR#dOwDH2Jhw?Z3&n6h1>gDun zIXGyMwhg1P!l?CBiCyee+=-;Ca&+;7Nwg>l({a=f0b?BbVveY>oEZn=5h&2; z3D}DXvO=i&ngG|?@?qPB+hir~jr@B3Dwn;2H*&3?KoPIP6=b4$qCP2EKbUNH&71^g1+5reI zD|e)aLF|SCUvASyR@?GFz)|9TKD6Fd<E6kyhK_tHd-5X%cYvs>6$mko4@O<8N1D(E6pS4tQhxgVCSvi8O$S-FYL<9U0Kh?7|dI@&BXm$ zikF;*(wa8toqICPpqH#>GtU4boI*BROedS)befXwT2n<`RRlNzwyn$0RHj5MTUp)T#-8%eG1uYI~flPiq~4YUkaedmliBHnTHwX?+51!jJ*PL zPY1_;hJjdtySIkp+Q9LG!QKmkg-C-7M~9QxfDI3W8}AQ6aSPc61~WDc(MgA!01m-R zhofSIFCT^p?+>}2hVf3rNnpbD0mGng!}$WjV2H+C=)#Fd!F|(bz0Y)iHQo<%nZ13< zkuRP`)XweKEg<&w3W4<&vGNePgc`*$2>Hz_3h2jQpGGfu zhi1xJGAtg9wD}H81ECP8diVF4vjQh361( z4nY)hW-K8E92e;^hIlWew2!c5S{Gy5HkG7gC^~URxGx+IdVi$HsX#UBrnc{PLz3>A zkmN2-$&g)EN33isbFv}>(@*wJ27XF5ihfBZKBbuDH&3%zKAtmPF;E^oDbAg9hub$a zn>Ow+A1<+WMqyRZ9ml1`sSKRgekC^e9DcG+j`j79SyGN^qKlhiB)iuz17W^j4C>hptvup_17Lxk|dg2qF`ZLq8=bVZk=goY!QL8NXZTGBL{lR6rJ zaOC~1`)9Q<>4$pYjWF(8ND)MD2aZ}2)d#b^d%9VlY=ankAnu-FXB;u_w z`v>!ErEzD$a#=O%X+iVY8meL=^I~2>_jIXyzcC7hWk&5sIxhOsOoS? zWvOW;(F488J5@t%rQ3j&!7~-2+mgF}GOhZlg=zxJ$g)t)avxovF_*4$tdqsPStGCc zm+TtG)3$|2XGd}K@>GNJbf>y%Q@cxZyJpkyeToJ_j0iz?h={%@UG@p@6|R2*uE)>} zb0}^{Pd4@@XVgKblMiRrA}8yR>y70NgY=C~=1%))ha_+Jhm>b*q;qPon`)RvEMOb? zOvksGXRM&UYMt#3qQQ5}K#tgU4)w*?=aqNMd3Wl@$BW+w{1p6<+6&4bxSTM2uHQS0 zCfsu1c)zGF&v@6`k6j7!z57b-e$o1veeH?O?%z@!%o)7PZQRU7HGFUW9dme}L#wYX zs`GVx--{fG?|!|D?Jy(_Hcagq1lY^xI9Th5aVS&vvWjABsN}q3;cYeu zxMXj@8_vuXvb_o7%pxO{A!&`}ZOj3C!{w(SN7$o?c#^AmrGD;rj~P%;e|9(6Hs?7m z8@ky_d5ZIXZ&Ubag0oL~-&@oATLt6BkI?%Q1}R3({Yc0EQe?Yg*u}SFlPuBFE0N9o z<$1kWstpg<=aX92F)45r$+FU@z5*>jLE3X&@wY#i_rM|j$)WY;Zwzm99d9!PYIYce zr=FOZotc`Dn~}}h@dX9PbA!?QA!@7BqH@o~kbS=P*usUFiAS_*ekBoUy&XZc_P;Yx0=gUE>8I`9 zi?rD?t1#fXSFjVJ*5#VK^$t?jQ66kp3s*IRsi%_*&&}b6VOyW4G62%nntLB|CdqD= z2HwGY1c_jm#_-Ew%d0?722VIMh!?cM9%X&CmNY~~TejRQes`WOBg&8`TYWw+F7z6?ey{>OqJ2_M)uta|`=2>>T{VMkCf&8ypRdK@iq0gsXObEqB0_{P$khnL9 zBFhPQwHl_jN)STA7&>v3;Ws8RLbyB59mlZ|&-?V%VtX69=Dw@}sZN5=2=t;Fcld&v ztcIA$QOXx;p-=Z9QU8<}2dB8!Ix&?x`O2Yd#@)?wE`p(I{}x1PL%^0$rc4JfA8~hp z?p;rHsqFr>;u^1A;%3qqPFti5KM$K8tuIexZl8WK>fHx5Jes;oMmG7YjtD>&Qlw@u zo2Dq#vARgYYFvFgjOBhlZQqDKiwLU(dP?Is&T6pPRDnrSQnU|7O-%wPT(V2=5B zr>()mjV4^2EKqR*h?{AQB!dhs;b6a2+CLQP{hw<#V|ag8_6zY*ZOT4IvY@%1?P7bu z&26Ir^4gZV_=KJ4l$yyYO;H z3^vJ1Pgb2_HTR`Ue<79W-_vyMlAhgj=j>a!j=!9?`9lXn&wS&OaA&ufxYOWl2wSNN z6n=~2Nq6F{IaaN|$Jc{Yd#)$#!vw~g<$~6KBBIeZ7qD3p@uy$UF2`8*hrzJUb-{J^`2l4Dn5^N3g-$4k(ppjhr?D?f9B8T7|l_Xv24%3 zp^UL-^_x-8c|APu5{$#|rYD{!<%}l}Ml}kf*;0zA(;U0E(X--@n#vWzbTLin z8|J*kT^ACyF>NG_AAAwVx`Y>Rua)gpaYkNJ-e}kc`?E4&hO8|w<-(GH`MML?**+C+ zEL0ObUR;iLEL2Qr7>lE#uXFKoFFCy;Aray`fHbi$F-JTnEM_dOdxLa3v%e&0kmvL; z@mPb0U6m+!%t3p(76Cqce3Va-wGRZ~WcyxDDw6ZwHh$fiwRYDcbGXJ+UFa3-_iN^` zEoVQ7@W;QGd<$l1DlL7>XFt8^_DTwVkbQ46+xg}RDZEFgd62e@$?j*NV!G12C71OB zBU0Bk+NhpXhUGj35BrR}y)XS4G@X^ThY?p7NCd(viO&T#)-mnvP|Z8OWC!T{<`5kk z{gYTXg^n=UZ55-zkBfhyKeV;&H9icqjgVfO?vV`2O8(&7;{kUv8D(T?qu~wa00#>H zMfI9)SiL$}j&69QU=~NLrxdl=-~Ac75U$S3Z};bvm8{Tr0{Y_H2%}<~llpEO5!%W#lIAp0{ddCc7<$RmgzPgE7kuUoB9LL!In03l%Haw6mwP8xq zQqTkJFfO(5IG={41%=RH%yt#(kvS=wW}?k^w)tHYZbGDY@mIBC!CcAQb!Fq7THhAo zik0FTm)MFMo)eoCH}GCBr5Za zM+W^v(f$gZ-mQj&%PQ7GtorpA@3A#YTB8d3rglx>jP;GCwB@u~HbA&I)ja0qp~c`b z8sD*o21nU=yo=xSYdNj2d@^ZjZ5`08db&xC)Dl^n^()%3+E^ENnF!rF-mUAYbZBCK zp*2(|VQO(hztFAiEJYE*yHKIUZVY&bJqvr#OfRC5AjtQ-y_vb8}ltUZkOqOwLtV8?0nk9rTpdv@%UYsSMT_? zOG|lww`O3wVjZUo?wQ=&OW4;d;f!qy9I@^VW#zJLXuZ2z?nTn(&J9Ce>`iI85w=j9 ztzQ@hPvt!KZ5e$ya9x(O7F%2G zzgz%wPhIBntdELfG4LOV5jy^O9aL7%0&0b&4W_R=fq58pN z39n`F@z-wHRFJ4nn3r#f;SBWdrjd@OR)KwR)F-e(8#*vM@~f@z zvHK^*CzYC%77}CtVyA5S#s5 z=IdP3fqcJ0q7{^%w=1k>5+)6QoQ(4zj|91=KUV923lvBrHrzgdYLKgbb6T z3@h3p)^0HtW)^-LCbj`s*mN#99;Cp7y3iU_fcjeWj&d|eW8{o-l&pUA3mL){0uI_T z8p9rbXExTMukEZ@qiPOtRJ9Y7%6=O5*!GQI?d@@YYP?xCP~ZAZHmhaZ!KIB z62uAl`(!SeOqT2o3ur89 z=MG`?zc$S2D$4+Tw#1G|LbG^Q6u4dKSZa6F$q(ieF%lDgIu$}0H>T#6XmQI=i6k|N zeZ(zFr0+?-A-V!TdX^yn#rt?KSAw+aXO-C33xBxdyFX|3SA2lE&f~c2Q(#_z_zfKe zWo0<^YG3C&1som)g-SW`Z~0qfbQ0f#2cTrKz&6iP$lJecQQ)Nal}djNtp*mEAcR2O z8i_cjhrVs6Tixa?1D_G!TKgTI#)&eyfPOvta0Cab zU5TyQOS`@zb4=}b5a6~Jp)seUjRZq$(hJr7qu&D4e#0ebNJ}cO=qCf6mxE%efnt=C zwmX8%loc6k$$GBFD=nr3Xo2$D&nEHvcoU;I6~Me(L#W(O8QIFFoCFD#fi^b#a}y^d zs46xtIqWPL^J+qm5gLQ*Ei%PZ-0Rg@7_Atq!9|k{6IKv}(sFgS5qu;{sT^6jG%34K zQ!pVjvnp@Q%5%uFFwE^RUbLIWpevB;B2dRbM~s4^IQR+vAX69s zkIf@!?-wf15VLXSM^NLo6)~hh`>lfYPNMcTgPRuy*D(yPMM6;Q2Exh3bS{2*XkFrX zzlVr&5et^AH?~R0fXrG*af~o00zUO|1o{JRRN*ZkV7{y&OG}36az+xC20IcF&lW>Z z8WXK>>%oZ!1EvZ1&kaB>KhiH3cw7|VhQRRT!`z${ptMM#Fhm4uE#YWwbzN(oCa0_ z8OEE_2=K(-me9u+^V!ABs-r<O)BtS&lLJqT1y+PWw6@KTtDS9rzTw?p4K_WApcm2^J< zi9@FPCd!5!E%=?oHLWz%HcRE$io;5vt%DEk0N!l zGuUm}KVgFwuS}!B$?+1Qpj1~KphH8d@;w<70sXG@-=}9=Pax0H!MccX8hL|>Qt>ZB z(o`CD#-;<%Eip~}=x>`48{8Oa+j6L$cxhtEZ`8xd{a=i|Q>>_Am#w*M?q%DyZSQ5< zwr$(CZQHhO+dliRI!SkRQb}KYm*3@hGc(6C$M~Ke(E&;)3#vtCAl>4V%jS~vxl|^B zuV_H1Sxc2+QLp{7QhsG=h>?t;=gaWyCOw5!%^@ygep<@^GB3ui;2*Cv08{vM%i6P( zZHuhwbeESH)gJGXAMKE@Fo)@hlhM~N&}lS(%l|i(y|0HOx*q}Ae<8K|5>R_ zjNXqwWG_Pa*#>J^B!#ooLJAQSaT0N@Wn+W*6|&cdXjp4_E@5<#@pW^K1Yc2VBmqO2 zm5#tp%x8AQWC1SF^k~0p#=H>{4!aSpQF>TjqKjhl$YSN~2H=@v zc1O%Eom>VWcBL)kw2ge@iy-Ac!gQMDY#YU5B%|znc38EBaKY48+oUCJAF*u7?i?^q zDxwEl}uZC5o}%=!dPZ{f-@rN{XY?LIVVvqWdIRvPz~9EwgW$J~W&4~8)4CCVZaUhYCv?-Y;-6^7?A$dYDaySa0Q4?yH3155{G0klHeXB+MYIA z;W38MTqr+vlrec9C*go<#Z4NrWW>1p~AzV!?=#d|8J%F`~Lwo+v1fMFMt35 zv|4rFC{a;X1*Z99rnf@EKMK+n!a~t_D#`U+Z?_t|3@>5ZUkGjdDx9?%z zsWCRPYIml!W7)K>R@BnAd1VK)dRsZ8vx2!b+1!e1wQbwht#bBGrFTDFrTZTVOP1jb z&vu4gwz#8P+1)GasT=)rLs^Y;oWVu;?+;6z3&yr3Yr7tyU+K)BanwA6ZX_Cy(F~_p z)(c$g1s`1gm znw)8RU)II1bC^f5#i<)v#_oUo>
    1. User: {app.desc.user}
    2. Cores: { - if (app.desc.maxCores == Integer.MAX_VALUE) { + if (app.desc.maxCores == None) { "Unlimited (%s granted)".format(app.coresGranted) } else { "%s (%s granted, %s left)".format( - app.desc.maxCores, app.coresGranted, app.coresLeft) + app.desc.maxCores.get, app.coresGranted, app.coresLeft) } }
    3. From 77ca9e1ba845c8cbb1566f803b591f6a826b0f1d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 18:41:00 -0800 Subject: [PATCH 1290/1571] Small fix suggested by josh --- bin/pyspark | 1 + 1 file changed, 1 insertion(+) diff --git a/bin/pyspark b/bin/pyspark index a70da5674e1c7..ed6f8da73035a 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -51,6 +51,7 @@ export PYSPARK_PYTHON export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: +export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py if [ -n "$IPYTHON_OPTS" ]; then From 4a5558ca9921ce89b3996e9ead13b07123fc7a2d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 10 Jan 2014 03:28:39 +0000 Subject: [PATCH 1291/1571] Fixed bugs in reading of checkpoints. --- .../apache/spark/streaming/Checkpoint.scala | 20 +++++++++++++++---- .../spark/streaming/StreamingContext.scala | 17 ++++------------ 2 files changed, 20 insertions(+), 17 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 476ae70bc9447..d268b68f904e7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -165,16 +165,28 @@ object CheckpointReader extends Logging { def read(checkpointDir: String, conf: SparkConf, hadoopConf: Configuration): Option[Checkpoint] = { val checkpointPath = new Path(checkpointDir) def fs = checkpointPath.getFileSystem(hadoopConf) - val existingFiles = graphFileNames.map(new Path(checkpointPath, _)).filter(fs.exists) + + // See if the checkpoint directory exists + if (!fs.exists(checkpointPath)) { + logInfo("Could not load checkpoint as path '" + checkpointPath + "' does not exist") + return None + } - // Log the file listing if graph checkpoint file was not found + // Try to find the checkpoint data + val existingFiles = graphFileNames.map(new Path(checkpointPath, _)).filter(fs.exists) if (existingFiles.isEmpty) { - logInfo("Could not find graph file in " + checkpointDir + ", which contains the files:\n" + - fs.listStatus(checkpointPath).mkString("\n")) + logInfo("Could not load checkpoint as checkpoint data was not " + + "found in directory " + checkpointDir + "") + val statuses = fs.listStatus(checkpointPath) + if (statuses!=null) { + logInfo("Checkpoint directory " + checkpointDir + " contains the files:\n" + + statuses.mkString("\n")) + } return None } logInfo("Checkpoint files found: " + existingFiles.mkString(",")) + // Try to read the checkpoint data val compressionCodec = CompressionCodec.createCodec(conf) existingFiles.foreach(file => { logInfo("Attempting to load checkpoint from file '" + file + "'") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 76be81603c76c..dd34f6f4f2b46 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -496,26 +496,17 @@ object StreamingContext extends Logging { hadoopConf: Configuration = new Configuration(), createOnError: Boolean = false ): StreamingContext = { - - try { - CheckpointReader.read(checkpointPath, new SparkConf(), hadoopConf) match { - case Some(checkpoint) => - return new StreamingContext(null, checkpoint, null) - case None => - logInfo("Creating new StreamingContext") - return creatingFunc() - } + val checkpointOption = try { + CheckpointReader.read(checkpointPath, new SparkConf(), hadoopConf) } catch { case e: Exception => if (createOnError) { - logWarning("Error reading checkpoint", e) - logInfo("Creating new StreamingContext") - return creatingFunc() + None } else { - logError("Error reading checkpoint", e) throw e } } + checkpointOption.map(new StreamingContext(null, _, null)).getOrElse(creatingFunc()) } /** From 645f5e83ee38a329332850bf44ea67b0711a705c Mon Sep 17 00:00:00 2001 From: Prabeesh K Date: Fri, 10 Jan 2014 09:33:31 +0530 Subject: [PATCH 1292/1571] Change clientId to random clientId Returns a randomly generated client identifier based on the current user's login name and the system time. --- .../org/apache/spark/streaming/mqtt/MQTTInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index c8987a3ee06bc..41e813d48c7b8 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -80,7 +80,7 @@ class MQTTReceiver(brokerUrl: String, var peristance: MqttClientPersistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance) + var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) // Connect to MqttBroker client.connect() From 7b748b83a124a2b9c692da4a0c9285f4efa431b2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 20:42:48 -0800 Subject: [PATCH 1293/1571] Minor clean-up --- .../apache/spark/streaming/examples/JavaNetworkWordCount.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index c37b0cacc9a51..2e616b1ab2846 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -38,7 +38,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` + * `$ ./run org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); From b7c92dded33e61976dea10beef88ab52e2009b42 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 20:44:28 -0800 Subject: [PATCH 1294/1571] Add implicit algorithm methods for Graph; remove standalone PageRank --- .../org/apache/spark/graphx/Analytics.scala | 2 +- .../spark/graphx/algorithms/Algorithms.scala | 56 +++++++++++++++++++ .../algorithms/ConnectedComponents.scala | 6 +- .../spark/graphx/algorithms/PageRank.scala | 55 ++---------------- .../StronglyConnectedComponents.scala | 8 ++- .../spark/graphx/algorithms/package.scala | 8 +++ .../algorithms/ConnectedComponentsSuite.scala | 8 +-- .../graphx/algorithms/PageRankSuite.scala | 27 ++++----- .../StronglyConnectedComponentsSuite.scala | 6 +- .../algorithms/TriangleCountSuite.scala | 8 +-- 10 files changed, 99 insertions(+), 85 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 0cafc3fdf94ce..def6d69190960 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = PageRank.runStandalone(graph, tol) + val pr = graph.pageRank(tol).vertices println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala new file mode 100644 index 0000000000000..4af7af545c256 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala @@ -0,0 +1,56 @@ +package org.apache.spark.graphx.algorithms + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + +class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `runUntilConvergence`. + */ + def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergence(self, tol, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `run`. + */ + def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.run(self, numIter, resetProb) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see [[org.apache.spark.graphx.algorithms.ConnectedComponents]] + */ + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(self) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see [[org.apache.spark.graphx.algorithms.TriangleCount]] + */ + def triangleCount(): Graph[Int, ED] = { + TriangleCount.run(self) + } + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @see [[org.apache.spark.graphx.algorithms.StronglyConnectedComponents]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { + StronglyConnectedComponents.run(self, numIter) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala index a0dd36da60f32..137a81f4d5384 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -1,11 +1,13 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object ConnectedComponents { /** - * Compute the connected component membership of each vertex and return an RDD with the vertex + * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * * @tparam VD the vertex attribute type (discarded in the computation) @@ -16,7 +18,7 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 0292b7316d392..b423104eda358 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -1,5 +1,7 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.Logging import org.apache.spark.graphx._ @@ -42,7 +44,7 @@ object PageRank extends Logging { * containing the normalized weight. * */ - def run[VD: Manifest, ED: Manifest]( + def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { @@ -109,7 +111,7 @@ object PageRank extends Logging { * @return the graph containing with each vertex containing the PageRank and each edge * containing the normalized weight. */ - def runUntillConvergence[VD: Manifest, ED: Manifest]( + def runUntilConvergence[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { // Initialize the pagerankGraph with each edge attribute @@ -153,53 +155,4 @@ object PageRank extends Logging { .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank - def runStandalone[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): VertexRDD[Double] = { - - // Initialize the ranks - var ranks: VertexRDD[Double] = graph.vertices.mapValues((vid, attr) => resetProb).cache() - - // Initialize the delta graph where each vertex stores its delta and each edge knows its weight - var deltaGraph: Graph[Double, Double] = - graph.outerJoinVertices(graph.outDegrees)((vid, vdata, deg) => deg.getOrElse(0)) - .mapTriplets(e => 1.0 / e.srcAttr) - .mapVertices((vid, degree) => resetProb).cache() - var numDeltas: Long = ranks.count() - - var prevDeltas: Option[VertexRDD[Double]] = None - - var i = 0 - val weight = (1.0 - resetProb) - while (numDeltas > 0) { - // Compute new deltas. Only deltas that existed in the last round (i.e., were greater than - // `tol`) get to send messages; those that were less than `tol` would send messages less than - // `tol` as well. - val deltas = deltaGraph - .mapReduceTriplets[Double]( - et => Iterator((et.dstId, et.srcAttr * et.attr * weight)), - _ + _, - prevDeltas.map((_, EdgeDirection.Out))) - .filter { case (vid, delta) => delta > tol } - .cache() - prevDeltas = Some(deltas) - numDeltas = deltas.count() - logInfo("Standalone PageRank: iter %d has %d deltas".format(i, numDeltas)) - - // Update deltaGraph with the deltas - deltaGraph = deltaGraph.outerJoinVertices(deltas) { (vid, old, newOpt) => - newOpt.getOrElse(old) - }.cache() - - // Update ranks - ranks = ranks.leftZipJoin(deltas) { (vid, oldRank, deltaOpt) => - oldRank + deltaOpt.getOrElse(0.0) - } - ranks.foreach(x => {}) // force the iteration for ease of debugging - - i += 1 - } - - ranks - } - } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index f64fc3ef0f52a..49ec91aeddb32 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -1,12 +1,14 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object StronglyConnectedComponents { /** - * Compute the strongly connected component (SCC) of each vertex and return an RDD with the vertex - * value containing the lowest vertex id in the SCC containing that vertex. + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. * * @tparam VD the vertex attribute type (discarded in the computation) * @tparam ED the edge attribute type (preserved in the computation) @@ -15,7 +17,7 @@ object StronglyConnectedComponents { * * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala new file mode 100644 index 0000000000000..fbabf1257cc99 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala @@ -0,0 +1,8 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +package object algorithms { + implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 5e2ecfcde9b0c..209191ef07867 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -14,7 +14,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -24,7 +24,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -37,7 +37,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -60,7 +60,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index e365b1e2305b4..cd857bd3a18b2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -2,13 +2,12 @@ package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite -import org.apache.spark.graphx._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ - +import org.apache.spark.graphx._ +import org.apache.spark.graphx.algorithms._ import org.apache.spark.graphx.util.GraphGenerators - +import org.apache.spark.rdd._ object GridPageRank { def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { @@ -58,8 +57,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() - val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => @@ -74,10 +73,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } assert(staticErrors.sum === 0) - val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) } } // end of test Star PageRank @@ -93,14 +90,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val errorTol = 1.0e-5 val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) } } // end of Grid PageRank @@ -115,12 +110,10 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala index 696b80944ed02..fee7d20161cf3 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -16,7 +16,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) - val sccGraph = StronglyConnectedComponents.run(graph, 5) + val sccGraph = graph.stronglyConnectedComponents(5) for ((id, scc) <- sccGraph.vertices.collect) { assert(id == scc) } @@ -27,7 +27,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { assert(0L == scc) } @@ -42,7 +42,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { Array(6L -> 0L, 5L -> 7L) val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { if (id < 3) assert(0L == scc) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala index 0e59912754c05..b85b289da638b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -15,7 +15,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } @@ -27,7 +27,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -47,7 +47,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val revTriangles = triangles.map { case (a,b) => (b,a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -64,7 +64,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } From aa5002bb9682ae6271db1fedb2c2658ed04fd4a1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 9 Jan 2014 21:43:58 -0800 Subject: [PATCH 1295/1571] Defensively allocate memory from global pool This is an alternative to the existing approach, which evenly distributes the collective shuffle memory among all running tasks. In the new approach, each thread requests a chunk of memory whenever its map is about to multiplicatively grow. If there is sufficient memory in the global pool, the thread allocates it and grows its map. Otherwise, it spills. A danger with the previous approach is that a new task may quickly fill up its map before old tasks finish spilling, potentially causing an OOM. This approach prevents this scenario as it favors existing tasks over new tasks; any thread that may step over the boundary of other threads defensively backs off and starts spilling. Testing through spark-perf reveals: (1) When no spills have occured, the performance of external sorting using this memory management approach is essentially the same as without external sorting. (2) When one or more spills have occured, the performance of external sorting is a small multiple (3x) worse --- .../scala/org/apache/spark/SparkEnv.scala | 18 ++-- .../org/apache/spark/executor/Executor.scala | 7 +- .../spark/storage/BlockObjectWriter.scala | 4 + .../spark/util/collection/AppendOnlyMap.scala | 12 ++- .../collection/ExternalAppendOnlyMap.scala | 86 ++++++++++++------- 5 files changed, 80 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 584261df04967..08b592df715b3 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicInteger - import scala.collection.mutable import scala.concurrent.Await @@ -56,12 +54,13 @@ class SparkEnv private[spark] ( val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, - val conf: SparkConf) { + val conf: SparkConf) extends Logging { - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() + // A mapping of thread ID to amount of memory used for shuffle in bytes + // All accesses should be manually synchronized + val shuffleMemoryMap = mutable.HashMap[Long, Long]() - // Number of tasks currently running across all threads - private val _numRunningTasks = new AtomicInteger(0) + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). @@ -90,13 +89,6 @@ class SparkEnv private[spark] ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } - - /** - * Return the number of tasks currently running across all threads - */ - def numRunningTasks: Int = _numRunningTasks.intValue() - def incrementNumRunningTasks(): Int = _numRunningTasks.incrementAndGet() - def decrementNumRunningTasks(): Int = _numRunningTasks.decrementAndGet() } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bd202affa2e19..a7b2328a02676 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -186,7 +186,6 @@ private[spark] class Executor( var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime - env.incrementNumRunningTasks() try { SparkEnv.set(env) @@ -280,7 +279,11 @@ private[spark] class Executor( //System.exit(1) } } finally { - env.decrementNumRunningTasks() + // TODO: Unregister shuffle memory only for ShuffleMapTask + val shuffleMemoryMap = env.shuffleMemoryMap + shuffleMemoryMap.synchronized { + shuffleMemoryMap.remove(Thread.currentThread().getId) + } runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 61e63c60d56e3..369a277232b19 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -181,4 +181,8 @@ class DiskBlockObjectWriter( // Only valid if called after close() override def timeWriting() = _timeWriting + + def bytesWritten: Long = { + lastValidPosition - initialPosition + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 6faaa3197f1bc..d98c7aa3d7d3f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -30,14 +30,15 @@ import java.util.{Arrays, Comparator} * TODO: Cache the hash values of each key? java.util.HashMap does that. */ private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { +class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, + V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 - private var growThreshold = LOAD_FACTOR * capacity + private var growThreshold = (LOAD_FACTOR * capacity).toInt // Holds keys and values in the same array for memory locality; specifically, the order of // elements is key0, value0, key1, value1, key2, value2, etc. @@ -239,7 +240,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi data = newData capacity = newCapacity mask = newMask - growThreshold = LOAD_FACTOR * newCapacity + growThreshold = (LOAD_FACTOR * newCapacity).toInt } private def nextPowerOf2(n: Int): Int = { @@ -288,4 +289,9 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } } } + + /** + * Return whether the next insert will cause the map to grow + */ + def atGrowThreshold: Boolean = curSize == growThreshold } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index a5897e8066c4c..50f05351eba3a 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -22,14 +22,16 @@ import java.util.Comparator import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import scala.collection.mutable.{ArrayBuffer, PriorityQueue} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} /** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. + * An append-only map that spills sorted content to disk when there is insufficient space for it + * to grow. * * This map takes two passes over the data: * @@ -42,7 +44,7 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * writes. This may lead to a performance regression compared to the normal case of using the * non-spilling AppendOnlyMap. * - * A few parameters control the memory threshold: + * Two parameters control the memory threshold: * * `spark.shuffle.memoryFraction` specifies the collective amount of memory used for storing * these maps as a fraction of the executor's total memory. Since each concurrently running @@ -51,9 +53,6 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. - * - * `spark.shuffle.updateThresholdInterval` controls how frequently each thread checks on - * shared executor state to update its local memory threshold. */ private[spark] class ExternalAppendOnlyMap[K, V, C]( @@ -77,12 +76,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - // Maximum size for this map before a spill is triggered - private var spillThreshold = maxMemoryThreshold - - // How often to update spillThreshold - private val updateThresholdInterval = - sparkConf.getInt("spark.shuffle.updateThresholdInterval", 100) + // How many inserts into this map before tracking its shuffle memory usage + private val initialInsertThreshold = + sparkConf.getLong("spark.shuffle.initialInsertThreshold", 1000) private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean @@ -91,30 +87,54 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var insertCount = 0 private var spillCount = 0 + /** + * Insert the given key and value into the map. + * + * If the underlying map is about to grow, check if the global pool of shuffle memory has + * enough room for this to happen. If so, allocate the memory required to grow the map; + * otherwise, spill the in-memory map to disk. + * + * The shuffle memory usage of the first initialInsertThreshold entries is not tracked. + */ def insert(key: K, value: V) { insertCount += 1 val update: (Boolean, C) => C = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } - currentMap.changeValue(key, update) - if (insertCount % updateThresholdInterval == 1) { - updateSpillThreshold() - } - if (currentMap.estimateSize() > spillThreshold) { - spill() + if (insertCount > initialInsertThreshold && currentMap.atGrowThreshold) { + val mapSize = currentMap.estimateSize() + var shouldSpill = false + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + + // Atomically check whether there is sufficient memory in the global pool for + // this map to grow and, if possible, allocate the required amount + shuffleMemoryMap.synchronized { + val threadId = Thread.currentThread().getId + val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) + val availableMemory = maxMemoryThreshold - + (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) + + // Assume map grow factor is 2x + shouldSpill = availableMemory < mapSize * 2 + if (!shouldSpill) { + shuffleMemoryMap(threadId) = mapSize * 2 + } + } + // Do not synchronize spills + if (shouldSpill) { + spill(mapSize) + } } + currentMap.changeValue(key, update) } - // TODO: differentiate ShuffleMapTask's from ResultTask's - private def updateSpillThreshold() { - val numRunningTasks = math.max(SparkEnv.get.numRunningTasks, 1) - spillThreshold = maxMemoryThreshold / numRunningTasks - } - - private def spill() { + /** + * Sort the existing contents of the in-memory map and spill them to a temporary file on disk + */ + private def spill(mapSize: Long) { spillCount += 1 - logWarning("In-memory map exceeded %s MB! Spilling to disk (%d time%s so far)" - .format(spillThreshold / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + logWarning("* Spilling in-memory map of %d MB to disk (%d time%s so far)" + .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) @@ -131,6 +151,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskMapIterator(file)) + + // Reset the amount of shuffle memory used by this map in the global pool + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + shuffleMemoryMap.synchronized { + shuffleMemoryMap(Thread.currentThread().getId) = 0 + } + insertCount = 0 } override def iterator: Iterator[(K, C)] = { @@ -145,11 +172,12 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging - val mergeHeap = new PriorityQueue[StreamBuffer] + val mergeHeap = new mutable.PriorityQueue[StreamBuffer] // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order - val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps + val sortedMap = currentMap.destructiveSortedIterator(comparator) + val inputStreams = Seq(sortedMap) ++ spilledMaps inputStreams.foreach{ it => val kcPairs = getMorePairs(it) From 210f2dd84fb2de623745a162377b989712f7ef0f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 21:47:40 -0800 Subject: [PATCH 1296/1571] graph -> graphx in bin/compute-classpath.sh --- bin/compute-classpath.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index c10725e708661..e01cfa1eb88ae 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -40,7 +40,7 @@ if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-dep CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/graph/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` @@ -61,7 +61,7 @@ if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/graph/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes" fi From 372a533a6c091361115f0f0712e93ef3af376b30 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 9 Jan 2014 21:47:49 -0800 Subject: [PATCH 1297/1571] Fix wonky imports from merge --- .../java/org/apache/spark/streaming/JavaAPISuite.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index be93799a2ae20..8b7d7709bf2c5 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,14 +17,6 @@ package org.apache.spark.streaming; -import com.google.common.base.Optional; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.common.io.Files; - -import kafka.serializer.StringDecoder; - import scala.Tuple2; import org.junit.After; @@ -36,6 +28,7 @@ import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.io.Files; +import com.google.common.collect.Sets; import org.apache.spark.SparkConf; import org.apache.spark.HashPartitioner; From 460f655cc60b1a1759b6f85ae8860595fc7586b4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 22:42:50 -0800 Subject: [PATCH 1298/1571] Enable shuffle consolidation by default. Bump this to being enabled for 0.9.0. --- .../scala/org/apache/spark/storage/ShuffleBlockManager.scala | 2 +- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index e2b24298a55e8..6e0ff143b7179 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -64,7 +64,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.getBoolean("spark.shuffle.consolidateFiles", false) + conf.getBoolean("spark.shuffle.consolidateFiles", true) private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 diff --git a/docs/configuration.md b/docs/configuration.md index 6717757781974..b1a0e19167d1f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -371,7 +371,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.consolidateFiles - false + true If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. From 8ae108f6c48528f3bb7498d586eb51a70c043764 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:25:35 -0800 Subject: [PATCH 1299/1571] Unpersist previous iterations in Pregel --- .../org/apache/spark/graphx/Analytics.scala | 2 +- .../org/apache/spark/graphx/EdgeRDD.scala | 5 +++++ .../org/apache/spark/graphx/Pregel.scala | 19 +++++++++++++++---- .../org/apache/spark/graphx/VertexRDD.scala | 5 +++++ .../spark/graphx/algorithms/PageRank.scala | 1 + .../graphx/impl/ReplicatedVertexView.scala | 16 ++++++++++++++-- 6 files changed, 41 insertions(+), 7 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index def6d69190960..2c4c885a04c78 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = graph.pageRank(tol).vertices + val pr = graph.pageRank(tol).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index e4ef460e6f88a..7fd65806268df 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -44,6 +44,11 @@ class EdgeRDD[@specialized ED: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() + override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = { + partitionsRDD.unpersist(blocking) + this + } + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 8ddb78813557f..ed8733a806389 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -93,25 +93,36 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + var messages = g.mapReduceTriplets(sendMsg, mergeMsg) var activeMessages = messages.count() // Loop + var prevG: Graph[VD, ED] = null var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() // Update the graph with the new vertices. + prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } + g.vertices.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't - // get to send messages. + // get to send messages. We must cache messages so it can be materialized on the next line, + // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides + // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices + // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and + // g.rvv). activeMessages = messages.count() - // after counting we can unpersist the old messages + // Unpersist hidden RDDs oldMessages.unpersist(blocking=false) + newVerts.unpersist(blocking=false) + prevG.vertices.unpersist(blocking=false) + prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) // count the iteration i += 1 } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index cfee9b089f9e4..971e2615d4afd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -98,6 +98,11 @@ class VertexRDD[@specialized VD: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexRDD[VD] = persist() + override def unpersist(blocking: Boolean = true): VertexRDD[VD] = { + partitionsRDD.unpersist(blocking) + this + } + /** Return the number of vertices in this set. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index b423104eda358..179d310554ec1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -125,6 +125,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to (initalPR, delta = 0) .mapVertices( (id, attr) => (0.0, 0.0) ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 63180bc3af473..0e2f5a9dd93cd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -14,9 +14,11 @@ import org.apache.spark.graphx._ * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a * fresh view is created. * - * The view is always cached (i.e., once it is created, it remains materialized). This avoids + * The view is always cached (i.e., once it is evaluated, it remains materialized). This avoids * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * example. + * example. However, it means iterative algorithms must manually call `Graph.unpersist` on previous + * iterations' graphs for best GC performance. See the implementation of + * [[org.apache.spark.graphx.Pregel]] for an example. */ private[impl] class ReplicatedVertexView[VD: ClassTag]( @@ -51,6 +53,16 @@ class ReplicatedVertexView[VD: ClassTag]( private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) + def unpersist(blocking: Boolean = true): ReplicatedVertexView[VD] = { + bothAttrs.unpersist(blocking) + srcAttrOnly.unpersist(blocking) + dstAttrOnly.unpersist(blocking) + noAttrs.unpersist(blocking) + // Don't unpersist localVertexIDMap because a future ReplicatedVertexView may be using it + // without modification + this + } + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs From 2578332f97d489bf0e238b2dbe1a3e1d0258a910 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:34:35 -0800 Subject: [PATCH 1300/1571] Add Graph.unpersistVertices() --- .../main/scala/org/apache/spark/graphx/Graph.scala | 6 ++++++ .../scala/org/apache/spark/graphx/Pregel.scala | 14 ++++++-------- .../org/apache/spark/graphx/impl/GraphImpl.scala | 6 ++++++ 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 2b7c0a2583f21..6f2d19d0dab8f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -93,6 +93,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { */ def cache(): Graph[VD, ED] + /** + * Uncache only the vertices of this graph, leaving the edges alone. This is useful because most + * graph operations modify the vertices but reuse the edges. + */ + def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] + /** * Repartition the edges in the graph according to partitionStrategy. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index ed8733a806389..0af230ed29bb4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -106,23 +106,21 @@ object Pregel { // Update the graph with the new vertices. prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } - g.vertices.cache() + g.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. We must cache messages so it can be materialized on the next line, // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() - // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides - // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices - // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and - // g.rvv). + // The call to count() materializes `messages`, `newVerts`, and the vertices of `g`. This + // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the + // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). activeMessages = messages.count() - // Unpersist hidden RDDs + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) - prevG.vertices.unpersist(blocking=false) - prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) + prevG.unpersistVertices(blocking=false) // count the iteration i += 1 } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index be9f1881500f5..2dd1324d4fdda 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -65,6 +65,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { + vertices.unpersist(blocking) + replicatedVertexView.unpersist(blocking) + this + } + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size val edTag = classTag[ED] From 8d021b42bc53a81172d98b556a340f7c2c4de0f3 Mon Sep 17 00:00:00 2001 From: Pillis Date: Fri, 10 Jan 2014 00:07:36 -0800 Subject: [PATCH 1301/1571] SPARK-961. Add a Vector.random() method - update 1 --- .../scala/org/apache/spark/util/Vector.scala | 6 +++++- .../org/apache/spark/util/VectorSuite.scala | 16 +++++++--------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index f9c6cdf2be8a5..62fd6d8da56b1 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -126,7 +126,11 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) - def random(length: Int, random: Random = new Random()) = Vector(length, _ => random.nextDouble()); + /** + * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers + * between 0.0 and 1.0. Optional [[scala.util.Random]] number generator can be provided. + */ + def random(length: Int, random: Random = new XORShiftRandom()) = Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { def * (vec: Vector) = vec * num diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala index 23d1bdb193372..7006571ef0ef6 100644 --- a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -27,20 +27,18 @@ import org.scalatest.FunSuite class VectorSuite extends FunSuite { def verifyVector(vector: Vector, expectedLength: Int) = { - assert(vector.length == expectedLength); // Array must be of expected length - assert(vector.length == vector.elements.distinct.length); // Values should not repeat - assert(vector.sum > 0); // All values must not be 0 - assert(vector.sum < vector.length); // All values must not be 1 - assert(vector.elements.product > 0); // No value is 0 + assert(vector.length == expectedLength) + assert(vector.elements.min > 0.0) + assert(vector.elements.max < 1.0) } test("random with default random number generator") { - val vector100 = Vector.random(100); - verifyVector(vector100, 100); + val vector100 = Vector.random(100) + verifyVector(vector100, 100) } test("random with given random number generator") { - val vector100 = Vector.random(100, new Random(100)); - verifyVector(vector100, 100); + val vector100 = Vector.random(100, new Random(100)) + verifyVector(vector100, 100) } } From 669ba4caa95014f4511f842206c3e506f1a41a7a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 10 Jan 2014 00:21:36 -0800 Subject: [PATCH 1302/1571] Fix default TTL for metadata cleaner It seems to have been set to 3500 in a previous commit for debugging, but it should be off by default --- core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 3d1e90a3522a4..ac07a55cb9101 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -74,7 +74,7 @@ object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { def getDelaySeconds(conf: SparkConf) = { - conf.getInt("spark.cleaner.ttl", 3500) + conf.getInt("spark.cleaner.ttl", -1) } def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = From 8b6b8ac87f6ffb92b3395344bf2696d5c7fb3798 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:34:08 -0800 Subject: [PATCH 1303/1571] Unpersist previous iterations in GraphLab --- .../scala/org/apache/spark/graphx/GraphLab.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 437288405f307..94cfa7e126e0d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -102,6 +102,7 @@ object GraphLab extends Logging { // Main Loop --------------------------------------------------------------------- var i = 0 var numActive = activeGraph.numVertices + var prevActiveGraph: Graph[(Boolean, VD), ED] = null while (i < numIter && numActive > 0) { // Gather @@ -109,22 +110,25 @@ object GraphLab extends Logging { activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply - activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() - - + val applied = activeGraph.outerJoinVertices(gathered)(apply) // Scatter is basically a gather in the opposite direction so we reverse the edge direction - // activeGraph: Graph[(Boolean, VD), ED] val scattered: RDD[(VertexID, Boolean)] = - activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + applied.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + prevActiveGraph = activeGraph activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() - // Calculate the number of active vertices + // Calculate the number of active vertices. The call to reduce() materializes the vertices of + // `activeGraph`, hiding the vertices of `prevActiveGraph`. numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) logInfo("Number active vertices: " + numActive) + + // Unpersist the RDDs hidden by newly-materialized RDDs + prevActiveGraph.unpersistVertices(blocking=false) + i += 1 } From ba511f890ee0d7f85746126c4be734538ede21ea Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:35:02 -0800 Subject: [PATCH 1304/1571] Avoid recomputation by caching all multiply-used RDDs --- .../spark/graphx/algorithms/PageRank.scala | 1 + .../spark/graphx/algorithms/SVDPlusPlus.scala | 4 ++ .../StronglyConnectedComponents.scala | 9 ++-- .../apache/spark/graphx/impl/GraphImpl.scala | 47 +++++++++++-------- .../apache/spark/graphx/GraphOpsSuite.scala | 8 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 8 ++-- .../org/apache/spark/graphx/PregelSuite.scala | 6 +-- .../apache/spark/graphx/VertexRDDSuite.scala | 12 ++--- .../algorithms/ConnectedComponentsSuite.scala | 16 +++---- .../graphx/algorithms/PageRankSuite.scala | 8 ++-- .../graphx/algorithms/SVDPlusPlusSuite.scala | 1 + 11 files changed, 67 insertions(+), 53 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 179d310554ec1..ab447d5422903 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -61,6 +61,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => 1.0 ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala index 8fdfa3d9077a4..2a13553d7921c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -42,6 +42,7 @@ object SVDPlusPlus { } // calculate global rating mean + edges.cache() val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc @@ -72,11 +73,13 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + g.cache() var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd } // phase 2, update p for user nodes and q, y for item nodes + g.cache() val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => @@ -94,6 +97,7 @@ object SVDPlusPlus { val err = (et.attr - pred) * (et.attr - pred) Iterator((et.dstId, err)) } + g.cache() val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index 49ec91aeddb32..864f0ec57caff 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -22,7 +22,7 @@ object StronglyConnectedComponents { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } // graph we are going to work with in our iterations - var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) } + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() var numVertices = sccWorkGraph.numVertices var iter = 0 @@ -32,10 +32,9 @@ object StronglyConnectedComponents { numVertices = sccWorkGraph.numVertices sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } - sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.inDegrees) { + }.outerJoinVertices(sccWorkGraph.inDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } + }.cache() // get all vertices to be removed val finalVertices = sccWorkGraph.vertices @@ -47,7 +46,7 @@ object StronglyConnectedComponents { (vid, scc, opt) => opt.getOrElse(scc) } // only keep vertices that are not final - sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2) + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() } while (sccWorkGraph.numVertices < numVertices) sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 2dd1324d4fdda..987a646c0c18a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -32,19 +32,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] with Serializable { - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - routingTable: RoutingTable) = { - this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) - } - - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED]) = { - this(vertices, edges, new RoutingTable(edges, vertices)) - } - /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdTag = classTag[VD] @@ -90,7 +77,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) }, preservesPartitioning = true).cache()) - new GraphImpl(vertices, newEdges) + GraphImpl(vertices, newEdges) } override def statistics: Map[String, Any] = { @@ -166,7 +153,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication - val newVerts = vertices.mapVertexPartitions(_.map(f)) + val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( changedVerts, edges, routingTable, @@ -174,7 +161,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) } } @@ -336,7 +323,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, edges, routingTable) } } @@ -382,7 +369,29 @@ object GraphImpl { val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - new GraphImpl(vertexRDD, edgeRDD) + GraphImpl(vertexRDD, edgeRDD) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times + edges.cache() + + GraphImpl(vertices, edges, new RoutingTable(edges, vertices)) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times. `routingTable` is cached by default, so we + // don't cache it explicitly. + vertices.cache() + edges.cache() + + new GraphImpl( + vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) } /** @@ -413,7 +422,7 @@ object GraphImpl { val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - new GraphImpl(vertices, edges) + GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index cc281fce997bf..cd3c0bbd302b1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -53,8 +53,8 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val chain = (0 until 100).map(x => (x, (x+1)%100) ) val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } - val graph = Graph.fromEdgeTuples(rawEdges, 1.0) - val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val nbrs = graph.collectNeighborIds(EdgeDirection.Both).cache() assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } @@ -71,14 +71,14 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val filteredGraph = graph.filter( graph => { val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, vpred = (vid: VertexID, deg:Int) => deg > 0 - ) + ).cache() val v = filteredGraph.vertices.collect().toSet assert(v === Set((0,0))) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 094fa722a0030..c32a6cbb81128 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -175,7 +175,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val subgraph = graph.subgraph( e => e.dstId != 4L, @@ -211,7 +211,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark { sc => val n = 5 - val star = starGraph(sc, n).mapVertices { (_, _) => 0 } + val star = starGraph(sc, n).mapVertices { (_, _) => 0 }.cache() val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), @@ -235,7 +235,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() - val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set @@ -252,7 +252,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("outerJoinVertices") { withSpark { sc => val n = 5 - val reverseStar = starGraph(sc, n).reverse + val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index 429622357f0bd..1ff3d75633f22 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -10,8 +10,8 @@ class PregelSuite extends FunSuite with LocalSparkContext { test("1 iteration") { withSpark { sc => val n = 5 - val star = - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") + val starEdges = (1 to n).map(x => (0: VertexID, x: VertexID)) + val star = Graph.fromEdgeTuples(sc.parallelize(starEdges, 3), "v").cache() val result = Pregel(star, 0)( (vid, attr, msg) => attr, et => Iterator.empty, @@ -27,7 +27,7 @@ class PregelSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), 0).cache() assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) - val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } + val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 }.cache() assert(chainWithSeed.vertices.collect.toSet === Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) val result = Pregel(chainWithSeed, 0)( diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 573b708e89dd9..d94a3aa67c925 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -33,8 +33,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("diff") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x) + val verts = vertices(sc, n).cache() + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x).cache() // diff should keep only the changed vertices assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) // diff should keep the vertex values from `other` @@ -45,8 +45,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("leftJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) @@ -60,8 +60,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("innerJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 209191ef07867..16fc3fe5a21c7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -13,8 +13,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -23,8 +23,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -36,8 +36,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -59,8 +59,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index cd857bd3a18b2..de2c2d110773c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -57,7 +57,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge @@ -92,7 +92,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) @@ -110,8 +110,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() - val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = chain.pageRank(tol, resetProb).vertices assert(compareRanks(staticRanks, dynamicRanks) < errorTol) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala index 06604198d73a0..7bd93e0e6c000 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -20,6 +20,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { } val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations var (graph, u) = SVDPlusPlus.run(edges, conf) + graph.cache() val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size From b1eeefb4016d69aa0beadd302496c8250766d9b7 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 10 Jan 2014 00:39:08 -0800 Subject: [PATCH 1305/1571] WIP. Updating figures and cleaning up initial skeleton for GraphX Programming guide. --- docs/_layouts/global.html | 10 +- docs/graphx-programming-guide.md | 277 +++++++++---------- docs/img/data_parallel_vs_graph_parallel.png | Bin 199060 -> 432725 bytes docs/img/edge_cut_vs_vertex_cut.png | Bin 0 -> 79745 bytes docs/img/graph_analytics_pipeline.png | Bin 0 -> 427220 bytes docs/img/graphx_figures.pptx | Bin 0 -> 1118035 bytes docs/img/graphx_logo.png | Bin 0 -> 40324 bytes docs/img/graphx_performance_comparison.png | Bin 0 -> 166343 bytes docs/img/property_graph.png | Bin 0 -> 79056 bytes docs/img/tables_and_graphs.png | Bin 68905 -> 166265 bytes docs/img/vertex_routing_edge_tables.png | Bin 0 -> 570007 bytes docs/index.md | 6 +- 12 files changed, 134 insertions(+), 159 deletions(-) create mode 100644 docs/img/edge_cut_vs_vertex_cut.png create mode 100644 docs/img/graph_analytics_pipeline.png create mode 100644 docs/img/graphx_figures.pptx create mode 100644 docs/img/graphx_logo.png create mode 100644 docs/img/graphx_performance_comparison.png create mode 100644 docs/img/property_graph.png create mode 100644 docs/img/vertex_routing_edge_tables.png diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index ad7969d012283..7721854685fb2 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -21,7 +21,7 @@ - + @@ -67,10 +67,10 @@
    4. Spark Streaming
    5. MLlib (Machine Learning)
    6. -
    7. Bagel (Pregel on Spark)
    8. +
    9. GraphX (Graph-Parallel Spark)
    10. - + @@ -161,7 +161,7 @@

      Heading

      - +
    11. rS}LPnuf2M-nUE6a;CdV?WtN%j^(>W%<-c- zYA)LPR&eC|C$S7!CqBG8xR(FP3mn zyM}O^@J5D)_ct&`H>q?M^ZNP(nO|~p;9$7(O;@_3NrT2`ef>1Esb;eW$0a4Hiixf2 zjCBo{i_P&?{dDl;z~J`w^ubwupjeE%TbP@MhfbH%`GCkI^uP*htF?{#Zl~kqoaoTX z*h9y0opROBoS%C0-==+D;bd37XR63(1;o9 zF3?L{+luXP`qb!1)d_t4L7>sM?_|E*V_Z8Y3)9oY`ues-v$r$yUWL0@@L0IkNSRd` zoz+K2MXu7k1R>zS1jUo%;GAo#CB2+g6F4!(}Aj(evqVmAj9!O0RdFP2KleR6H$eR^OWGR3_I# zw6fbsrr95#i>miuZ@ac5=Hcqs54z_m24o>FAj@8QY|H<|6d97t#)tkUj;EyJ_QaDCl&#fu$nY{dIW&z84(9qn;0~ZX(_Cz zG{=Tv!GU$TVjH>LV$)v6Gq?QQMwoLv#2tV6S&f0i=h9@`G*M&_=2x~gv;ELM%qlnE z-4s$Z+{dK~r2|fDX`YpN<;U7wh6RV^H8b6^%~|TCGlMm1CF8}?9CXCShO$R-V>EKM zg}36ASWsWJk_OOZ0;mN=73!Kn*N>S> zxq;2)V^?R?w)T7is`A@WH#MtRt}kI}D2OzJUR2uFYrRRIy)R}kTg!xf1s%Y=NzLIZ z$n-Eo=e1veE*C#u{h*};(eI{w&`b{tIGf_={`|>B6Kv=3ee- z)fJ9RxC;<1Bi}-LudfVIOim|Gt6C0%eHTf-s8Hr-gj#A$*ogISVW^m5DS52$ui~0} zuc_~YGw#7sBZ5$8&z3p0e91ERUgILFQJY9xxUy`Hk#?~xPg%XnVOUQ~i%>aB)PW)a z0VXbasCtaKTqLRFSB>iNSF_l0MADd9Za4|HnY00+{-l7n)_@!$#+=`R#9$oyULY^8 zObwj>7%{J-dy1D8c=7)ldFd%e-8Q=8SX}iLU+6aCKDh zNA`l2`?4en(9s+snJI!e7}Lt5f&^+T=xh^tghR1$PPULg*I3{O5eS*iDfc6&P5?nrLyA@hA02&WO05U`U8&?lM!}H^bH}CX{h<{1>qM3$d0Hvh2Vty8#h}Z zUIV%hSR$;UCHfTqY|AI)@0HDN6KNJtGyu%2xczbJ?M`Sa8o?fiqruH6xW#mTQ`70) z2U6_C-)#OqVT{p~85g(g7N;hn6T;K=R^`!@4WjPhyz|i+B+MUncvBy)C&-!-GGepL z8JPb33vd&9yv&x7va9Kl^&j+rg%(UrF?91odKK$P_t-Eym)RTw?%F#TyE7c-G_YQ} zC_Lazt|`8*7W5)I;9IV+MCP>Q&v|#f-u}7c$UnTNtAk&gaaSk#ZvEm_L^^T5O8bjP zT=MBFjj{jFVa$JCoMSd_no=kL02)dF0NnrW#W8R;w=$x2v$i_c(6rrTNA7jN0EUa&qUDjo)ciZ8i z@ipV?;p7?%#kW^^Jbm&l1&nzsKHO7^XjLv85kz=O7P64S_|>VJ07Jk+V%29N-8a#G zPiD(ZrXlGMAE9{`Lu>>co6k`L&iW9wwDQB=BTabv86q&n4wLg z=)BNB-Ab4VmhCa!)J2GBYS0#SlHi_0^b_J|4ijE%;#6C~{X+{GP|nBL=#g7-1M%O(F&J;)Q9$qXCrRSj?ZR%bUH!ddU|_{5=3V>nW}92u;u(M8 zy+Khw&O~6ki4%SjN?^ycq$WvUzP8@2sAYg=BagolSs2KUt}wx57<1@qQdfqHg=kL~ z!`qv}88Z7;(;J&+RA#Vw$wOeNPz2z?qt{s&dZ|-NtP|f8)KnwOK|%Zn7t%OV%qxd7 zqDw2oSvvjPG&qb{UB{L>8nDw&rj(7S%bZga-Z|PCQxB!wSZ1oqkF}t4PE(kKd`|o3 z@SzRQ)ee`J)Dng*+S4`@uq)W>ukM%XYNo=$6aY*Od>BpP>gl4-n^X;#^7C)gi&&eZ z8p|m-0!EN5&0}zssy#4)6Yx)!NL~i8HQ)4n7z8K`-K&AL(=ASzHosP(NC!X|8ET@S z=N?Z@@qxIaRiz%GjB|6L1z#?vT&PWt(~b;yb0O}m9pA?>Ghg_|oNS<!z6ii zGwyJt`KXwkQosZafFle`c~Rbg%4B798JXxxR)RUOGTAxBBy(vty|Opw8tfBD)_qz1 zUZDsOT7b`w45urV_4N(1kqjTk1@DiI1mL^|TO9ZTW?wA4(lY@JjF>;{oScB>Uz0r$ z3TcJQW@S-i{PDNi)cfR;GpFEJA|S36N!GhRE4&b#>`{iWb_KY1KdEErbF(P#3eNCi zbd(3N`23P>hMK<;TLgg@{;O~w8Uhv=FJjgfum|d?@KeVSX${jKvo3~@{+R5A!7G+;C|0^o4 zHSgFxG>7-+4#;^TrQz_+r5?;% z=?So)NU^FNLIs`5hWX%ZSnK{2=f%v0L0m&1rbV{)&Hg;+g6;5Q9*%W~kuuj>zZQ5) zz)!hOtt+{+#;)Gd9bH|hhQ27fm^6cihD?T9KoJpT>n3b9!WZ%9+o$a;<_2F3RVG3N z0_dwuyn{=&rh<{?>1N{xbg?zEbxEcCL*2?om*jl-7qjMG9IZ&6qX znPVWqxE+Z0CUFYViL0i7Md}H}I4^JTAAx5ii0!1V7!Cj^c|krxMk1ipJT!pwMt*YC zj){FXq_Jxp+guS@XY0&~oN)J;=_rYgx%e{Q2#@;lEzsH2q<=DAO1xiUHX-~N79Wh| zwwo%u5x2Bcmi^=@9|zwJA&-eLVis?gw%HZnvgGiMe2DYCCFm`Iyy9N~iUsSWv}b;2 zW*XfwqoLR5QJ7SL0MrLRPAMs~0Jdoci_?&(Rr$IEY8UwXOw_$(FFIP~y3v>x zng3WB4B8pd5}UFR=9DT$m^q8-cwue0wqIKANfkG>DQTrkQg9po750~{5l8X@ab91L zy)lZi$bv06ezKJeThg74o|BBSx-A%6ypDU^YAxoXAq9n{Mw$I`A>z;N_v{cI<{F>y z56}C>p@j*@V9(r|7euJl@qTI&7ej;?jce%y+zYuA zsU)^@h2WH6s^bL2F3DNJR4EoHJe!fzq>q?SkMl+;L<)RpyG{2ueBZ>7Uh4O=A`8_-b;)N%FB=HGD#!%HhcJplL z;ZE#3lZyq=Oqi^^3khdbL)ra@imaW8o60p{HseuPRi?6SWchP6>}?%|qi( z!m)~Gmtu>4VPIX+zP%smT2|b;vPKd_y8-?kzFt3@>ZK+~)#G|8^#S(gk&`J_mY*?( zC}=Axn`$J|?jC+yMa_(nqYKa>i$|`eA_4|B9N~(Lo%|)vMXoTdSL`~8#(keq1S{vo zS{(v`9fxxPB1>$Z+LM#Po484j&uKXb_7y-Q_!R!mZk<5Z!5Tf8~Q$=uDs940Tn9Y z6Ew%DSEduRLkqx7Zo_)NDHA>I2Jjh@nTF|*2I#UE_?wz-&#|6@3yH~%YL-+)tHZ-E zMwMn_<;veetaMzVT&H&Qxj%RQB-O#tM! z)QR{z`_al5k6_g2w8+zZg~OzUpm3Zk+PT?bJ+rQgbmna~KI6<+?L;fj z`=VcLb9@2dP*Oed<{-YHk(--9DK$JuuXSKnKO=mkPL z>c3?G@I|*(jja7dV=|Adt-G0gSUBd;r*6I_qxX2DZ=oQzP9YY)nB&bDR)o3-yn!Bm zproo1vos zrrR7_TH4;7ZS5UBGkPl@cV;doZoviK^oEvXd7U^#qeyN{7{68@Qi4V2*T`MnyWp3i zW}~<-iGW-KM=w20=O;Tc)xN>ikAq45k+89QULP)>{9vn_)z=#kaRQgH+!%ao2-5I! zJ3O1X@cZ?!Z~6#Et&VgqUVbyt-O)0N7MRj+vSXlLO3eP7Jrd_yv9m9V{J{Al;fS@} zuCw(|DMM!g*U}*@T=6)BB7Xn6*M4Qd7o0m+;6`+PMq;_ies33Q>&v~|)6P8>EdD`dQI9dY3|0JyNz|k0ViKg4x z9<<5F05iSJ9G=xjv;{(g!I2IMdk_zK`_rH*G_WUFyM-Ha*#Jt*l|)+l$*|If#zNEM zs_19HIXop${WYjrRbuROq#356Qv+;-JtO(&R$z^zNx@3Vqe@feDf_o3r+BTH`J?HU z4b0FHqyBT7TvVk>kRc$wlvA<$5wsU19ZMMZlyz7^Yf;}y&6t%0g~nC24w+~g#3g^M zfEhwj65s#$E@05L?+_C>)>MwMh%rL3Bxjt;NRTW=R|OjoZ&0>PNKL3#wF$czfw)Xk z!dgg7J}9Qje}cy{iAu_vl5o-IPX$aE{agLc;MTh<rc!H(RWo)L$djJhMArIZt!iAER|6T`z0RaPrl|#%+jfXS&$N>OSe4!yjmEGW# zNx?}ct?kH;wvqAV=BN)RG)V)zDe1&DK&`FHMO_!%WA;h-Wo>k}2= z9wr!n?`!&cb$9lr>-xT13;QF-?$)5r@Z&3l@B_=x(%Q_@#AN^T8_4oZ-{f4!@ZGDg z{~O)h%*N3C-U|@_1M9@d(#XK#Qs2~S_?7GXdb`kX@xW(lxd#YH@QX!bb!IiuKcZ)4 zcy_gw0C-u@k;U-QNOgDMCUg;ZgmB}Y%Nc>2<6ho@l;@9%Lw$8G;3Vw7fRlwQ%K8t? z2+*xJFs3SmV4^#14AMhJv8 zBAV<>KKT0j{&IVMcC_~8+CF(@0EAs)B%O$2QV&82kUAJ7wQwW&z#nJ_LYSxEYtghMh0kIbBQiEHt2yAQ!tS^2DlA35D_5YDT;7;h zI;MQ}lHMbVMqj1o(}!5-<{(o-)aXYA}kVyju)Ue}yW!DgM6SRarN zQ|CrUVtSm%hTV>_vV4sU-8?K&Xv)&aO>#t1^1FqNs8rZDb8cP+_|-puupDRK%KDnj zf2EH(?w%hqNjGY--+uF_Q=ViWB_@?G%fbFE6SLvIkV*)Xan7SVgEImls<}Yp| zfYR-ciIF~@ro*W3;WW!kW*)@1-P zNR6K)0m?vJk_B2E^<`O*%nJOd9t5XV8AEX9`2_U3i)lZ9>0VR?D1jt3e40p~t6x7% zQ>vN`UW!I;h`ewAit%~6ULvRjljkFGQ@GUV{-cT)R6xtIiO>vgeCCV0n2 z7Tq5}rKApm*>Qvb0hWqFc@U^$VdBWzR|BAWEN@a9w?o$jhI?{>Z#xhS$<)4wcue?K z@5Wn#O?!Igj!=B((gQmHFobjMLNF%@M-w5Vd~D2@EASlpF&o61M9#5x@wy|?yl&E z`Ee#oDhMoEV_sUT8)m{S z{#p#D?C_dU;Ew(z*w3#~fUyuGxxkE#PsuAKr3RCjTd5_6Voez9N;1EKr&iDw{p5l9 zdl(5YhOP>O6iKz~Hvp%1)e$)U%Pv1exUmPBS@rM?8XaNdaj|GvginAP`=65D)11P`94B&y+E0|LB?o^l$sku-^GFXMFhiTGTQ`n?V&6HJ=HIPreD!c?u zJ!_Nj_3Uxp zo-%>OC54AWp;>(!u8rUG_c>s>?|Ia%sX%_-dRuO=@*;>Q4F<)Ip#3|QbdCu*xrX0^ zykR)U4Z5vQf`IZTb0+x{3VWIYXq;_Eyr=j?6>3$rKq174%i3BPoq2xMZ(c%EQeP8y zw16R(BE)PsDFDCXPxyif#?Yf?b0u#umcI%>0A`|mc>Tr0vcAGcI{8)T-Hi<4AD=eE zM%@-8PNBq+4&XAv<1xsfd76WIt&I}X;hM1@Iu>vut)r!+J9|uXzPAMi8gt#bL<#$j zAnH-d7DE2ZoSiq^#$)`t*S6v6(#QL;;9tU^7<&uT+OQ0Gz~ zDdcK2Vt_F*-A!gvq`Q&VZCPLOYD7PjDnKTRnT}6s-Vsk#wpb4mqhYr;NxZZvcPbyI z$q4gF*X*<)b}3u_Aj0-;eY|oFa=KG>@I^UwObU8o zWhF1Yc{Z+gI8?oR9Xl>OKJ8}(!Bq9Mibx?HLg7kVKg%KySa4#J69T3&kdQe0lJ?;yG9ejzz(EYUBB>FgC$s6VK++$z)f`OMQ#@#Qi6CDFe~08maMlgI%;5&SfQ-ko3eLef2)$&Buajqbms4w=UY zpzQtp=+PI@!uKT3oG(zP{=R5QmwiRko84+OZ9p=xUxn|71yOQXehJUuuVq8?Gb5hJ zN_=pX@hDu0H=FI%_hJ#o@bZbbBN6QXWErJIld7L z@iKWR;jsO*HP0pABrdGx$f_n(QEi?RdCu3dco?CV8{lEAv2&R%bU9m>2@Aaq8A+56 z^T#(fFUQZIwOuC1xB_kQS^eRK3Qsrk;pJpf<7U;;TI)9MI}jJ04=|q_!(iK)61#uf zuy3SJ>8L&nlu&i_f_SBtO-fl}fNf(%D`VgZ+eU_(Lq@Uzp=rJ(bh(Fx-ZLG6Mpt!$jk&ktDff zg@3#?H!`A{Op>KE(LT{PO(UqZYj3Wp^hCE5~t(TZcVX4y3P}|AJw{^48f9QaL6o zIFrPkdm3h({w3dBtr5&8-)x#caorsqv&tY7rtpOuyNC_9`n;_Ea}a(BVnl5|a?Z*U z1jdCTXG8`jdPGY4r~i*8RhvmVgzN2e1-Z0P$I26CX?~&o4BmH8M-n~M+%P9>z>-QJ zcr=9XrY!N!^GXkb1@n_cZMvqelVvv#S3Y0Pj0)NZx^~7m5oP`e6k0J-ujx^iWW`@i z)Bb-MaU=^D4#@eP-~lHCO-H)G`0YwZ$5)#$BzoEToz|6L*l$LQ~u<1dM&QH zMUKmk<5Ie+a2%Q8TOX7riU`dsT6eJ(Q-)wdd)9)kH_g5IVXz5>Q;oXXluIf~HcfKi z(eBCi{lE*|=*LU{df-a2g6ZpM66Df(^vuCCtXFut)f=wUzDs*>%ioa z_EF-bHCBtF!h$)!`j^lo#74)YPzoK7f%H4jV!Vdiz2CVLOuBzACWSpN9>Aw{=v$Mc zz`w5g=_X~gjN8jd+Rc~H_7qS=WI1#qs*jxwSJBRUzg-Na>hm6?~rD>w(fd9k@CHn6W*Czu_Ca#Q2E|fNa-wP#P zhR4PecJe{plZVYxrl4^_=E4xl+LTwlFFB}J8l%badH-4VjwnsVe7JSU6z!7|{mOP| zRYzY;sJaa|-8{?nZaP(ky&^%Q9C`}+b#Zm?V0~H6?|om6+GjKdz4yKd6mT%Vt8yW$ zVSqo6Y_Vj&>rR~84x*D$V96nWkk2Wrbjn!>UnI{T^_$6Fh|U{OiPih)16I=ssnT^8 ztra6+WzX_s3(=Ahuf~?2j-C2JhwpI2%sQCfikP{HY*j;2_~Bt|Ia|9|xBLQp&qaM9 zuzMRrL5C2>0fB&c-9=S7%JpJqWNtyNuV-#{BryC*O>b7-NCGn!J+Lj?sH)Z|U+y@5 zq7bX4^`h~zE(%aUp={?lRnw3xeHHyFx(xN+4)jW1Ev%#UThPYhCi$i&9q=IlV zUIPZXj6#_3%EW_>!5WUY5E{jlSs7_)G)dFBlzkhR%n|GKTFO;aCXL`Kmj%sPfGesQtRo7$AA^$$qO*jX7HF)1gw)OBvb0qw;uP@GMAUR zZFp|0OP{3{-_RhN>Ac}YjU1s2Szr^sA~y-><)&UKFIZp0+qBg}*h%}~#^a8VrIace zq$cl!!cGImA5oq5hO+_)mAl1zi!o5IkU6iHjuf+tOcyOny>AI1mH3Frw}E~vtadU3 z4c5TXIDQKG3DrGV%N71nTKiO|+SN*}|JRt<6Ql;LkW3k=IvfhT_8#@ow}IsNAgdDq z8bmk(6eQ@|*W^-HE7^?Svy8gXcQ*sZX#9^~@J|4go&qF8e9#|3;Mfp3AYdGDA@`9s zHXaGTKtq-x0P}xkH1ex8>t|Lg4aKz#f~KKxVz&hy_KH^>+jSKRl@crpUbBbNlE5Wx zo|AXyZP(8^$G-1B^Ii8(+;0KI0QP4??Kt;Vs#s3VhYoe1EAG`W)tBO7o!q=3ey3MX z!q?px`%Y#NJA~;cTQ*|0i5<&2ck3|gTRtLonQ2a@ukm<8rfZyNrY>Bh-wGynE@Hc% zW~(nA=DNKG=ojXxcwY@tvtf6Vd(9lWrqx?56^enbnhH!g6Is$ zqdy$Tg!Rh2f0f+MeNPB3QOF#jttUah zw5HYYpNJ{g+8CX5l5h#GE*IpTSznGb2SH$-NfJb%K~YXsyjB5)-($v zpQN29SQ$JVO{kpn1Uyq(1CY8vzSH>Xf!^#lkt=#huN zxwlx@@`Yb0a5_iRv-G5^mONTEA8J*Zf5@)itF2Zb_*GRfh550de^{Puh*9%N*{v{T z{IAxL=e3bWR&`qW$)0kd3zRomN?wZ2NYCmqktceZGY;E?PeC&pux)tgaEwSkxlrmg z6DZeuVwZ2u-U7&Q9V@yn`%4Xwi5k+N3hg38*c)01uR5|%AXF}>*x21HM1M18rMVz= zeCB|j*Iwu7N`^Kh$WL>>G)d~5vgXRNndm8k{5}Z>ueDIo>wPxE*X^+9>&=C?(^oF9 z!0H7ew%4Vn$y*0B{9A_I`xhyUc-Z;|P>2H3q{z4s^$(Wee^>k&(%RlY!?a>lInp;6l*BqfMa^3M)73JoyaE zCWp`;l{-Ug+z?*ex7=h;I9$~#&};~=tn7DBL2eYvt9WH3)%E#ZgW3D~o7VsYUE->g z7Sj*Y#d5ECI;-udel|x){>>fU9yms3w^9Xr@{02zg$m033(7H!8zrf2!6Ig@Cr*=n z{(7jP)+U#)#h|bxL}~D=&{xcLJ;#Y{@X}yKSCicElZg13l9_b_mzEJKB+L_Nu=r`J zgGDY)v+%r8$BK4YwAyemZ98}9if(q$CI*OMPU;v?PZr;7f6IjBu4sn^IDM!2{E1OR za&Rb=FEDPrtaMNSWieOJiIq`R7l0>dbLWMaBrdKc&99@=1%1_*v5kQ-66mmk4McK% z;UMTri;QGr+Or$?`1I=J7!Zi9-d4EfVw@*=GAHK`t~YzRV(0P4Lb5jY(^^hpx=D?u z!#l(cg9Yvy1UsBF`6gKt=efWRvU#ZHH`PWrpf)lXJ){{Eitjp3dJrtEZp*n#s{zIl z*nMDDE}o!k2qmT(K$0OR&30t(EU*61sO}h?{zF%>GA+)6^X`|j5VPz3Y zs}7iYQ&i4NOD6b5K0)l1!_QADCLFt} z4||yBgi+={Ml;ws7k{MK{N{ZP)NxGxQLo9oj6O%suWE?X19D5u`4)O+gkKKR9I=L& z4$Rq|Jy-v(oI|zA`VnBM^Jx(o$Ko{L?tk!Ufn&%4^K{*&+x^7Y zgAsbq`eI?I#Xujqx>2NIEpPE|LrLBZbCz@?OQab;3u3=-?gnWTJuB=X8uoxPJA4M{ zC`G9)TuyKKnN<*)jlUA-C0os)tQ=OMUlb?6cX>Ghsiu+HAE}y2 zJ-~xth(*N89Iu!vacJEvw1njqWa58P9^QaW+4`DN`Q za!?ngF`kV*1k~+%F-b>>EdQ&QKuk))qQ#t^J6Q{mFIbLif3C+>j66SDbsg_o&l5@1 zZ!GsO>Vp5g5)s}EZ^XL-Yys_*E0Vjd&K;EjCfo#&eO(JF4gdVo+chJ(sB3SROg!l1FK`_l2L& z(e!xP&~w{0EQ88B&ghduft}E5rq$*;e}=x0pSPSLDmuKNgjCOkt}>Y5toBpW#kZ!U z?Fm>E4NN%^iME_N9Aq{+)MLAlgPmtJZ|9TFwTv#}vWwBxhk6nr-!!bgo_c8kS5j3M zYWe)=AgZ{UHfpWrt&umn6v!?FfU4U<5=vhN&7SXXYRPxNp}u$d9jpnVgjvD?M`@!6zuc?JACV& zAGdK!gik}YkvF7d!)%q}d~)rR9x|2%QVyWUA1v3+8~Xx-a*K@Q6@Fut_hp34@%mQn z!gW@oR}K8zqpb7Y-LX*APG_j-1*v#02EP_5}hk0A(+02hPYS(5B+!FR~l z=hW>A6Wc9R`5LC$Q`p-D_zD464@3q#nV2{y8F%`Y8NSU;{v0WOiPaE`iDp>84}aO) z@tt71SrHb6$4n6Ky%6oT4@=uY2t!>ES}K7n`{aPTfttP}DPJoUB*h{uSq|%&Y1fRc z@Suu$EfKy|QND99D`u4NvYbi0e0T3(c{6cw+x5c5ETxSYc-XTxSx9u7)HAdol*+>* z1>q`9=Dmi(4z;R@_~i&?4sL&|@7Oq!PPV1sBn@_8D6<;o1=?A|JnBX_|`gqL3_XsZM zcM58SQlMJYm`8Ah<1f0XK7kVl-O=DR&*D%_gVRL%_(>n;I9DOKVX;rd7095ygq)BD z^n7#~DLPZ=nx~Av`K)IhWo>X7W9)8v1yE6F#$Ma&`h020ws&C~53a&@BCf%=$u?5@ zhs104e3lyTgkTewXy8jC?jQdo4rmeb;RtSIzi|b_=st`3f+a0;ipa{XVvNckJ+86- z6RKAxTgK#t@_M6oxef8I+P?g+KbFJSLm#kuZY#|&Km*DiK6L5@6e@;SDm&{Ry5~tQ z4h_=MO1W2?hnE}@-}mVD_9>7k_ZSJLW0}%rq&67~4+Vr|zL|=T z`&T#aQt*6* zFY7sy1>01F#K*;(_`@*oy(ow-4TMrPlkJHIC^l2U!=x%3Y(<~mMPgm!ZIP09MQ{<4 zC_AN@^QWDAN2HX^c&8>;l&b@Nz>6aa1-eS!UwG;uN;mPKOE>XuN}R{;%B=&u(%w+; zN!~;H^rEed>~K(rc01*&BDaEWUR-xIxl}s$Pm{XxvdZ3q7fP)KHRd_67lhY1K_Q!U z$T#6z6*`ErK{Ii>gX_n!VpLObXW&L#KR6ZHSr4__?p$ToPg; zEDRmw@C5Uq@K8hqmxV{5#Vac&Gz^WQ3dePEh5)t$Qw-5(bdQiCw;1%Lk*qPh27}Jm zy#^gKj953u(k(qeD6rh&4J1<5WD^#EUFY$gu3a?Vp1n(K7t^W>#cIW78F52+3 zR&=cHri+^gBoC=zI%QejgG!XaZPA^5qW&u7ZN%I0$ry$_Y0XQ%1N=#wSK&bzk>_fy z1ehqpuI+H1L?c1y?`IBQbk8*JE@lfXp{Of%pA4l0g?qUOvfn?_!lh0*9A?U*S*+s`1g3u7Pp zXkzor#Ap+CP9jl}ifHVgYQ8#(d(nI0Zwh&C^2mFazs{jEqdC3KwGHqJH$IeX$>(gE zUiPs}Rv%D3eFWvgoc@T~z?{x+QD2W0#MdQTm_OreT0X@3-YU`J%QKF8%)fZ7_0Ae{ zjOv)94)GN|0du~^cq%ZV<5#12KqLKTMEBjrINs5iE9aw6evFi6K*sb^KO*v@aa0Ic z?H*MdSTJjTLJp&)f5+YkLPpL3+kh}aNb5Cv_zDmR;=ueNbYjNLW#2+F!uYomXm1}n z2dtKcGw=LFNly+&-l7oLx%k^)c?`uuU64ef6^c#@w)8n|I@a1dbnxB)UF{hxn&4SW z6-CpwSn=~b-fIAIDWH1EM7p4>=wxU$Zo=86eFdTQM2i9<63tES1ZAc%H7|w*_W*hN z=c5ZfB!p`yGMW-x1Pe8+i--}HeX0=W4-C?NLE|{-1Z3e4srZ@gSG>|y*vjC`Hh+8bSS_#U z;k*vWc+^D~vT>^7N-H&9KPM?sJw>c@IwUNr#$3MMBH1z)!(ookU?tUumf6J3s&n_0 zJ!>$6xSJn*pY|bUo`Q|VD=9#jO&G7WEKc%3Gp#in;P*)0DhQW+5}zW{ic8C!Xk9kR ze#9nv%W=#wI;Cz%ENeuN{-Vfvqy8P4&nuj1{5QuCCRwR$syvWx?b%=Xxe_=$F?})k z{m77Vk#E57+=2qi5~FfR7!YC5hh1TjrKPF4Z0L!r5vYifhl$AzNS=mlQ;63Cd>PDQnWM=w@x0D6p|D|Eqify0D)I0 z^3-=+0Q$OXoE#9`BaQYTOT7|^*Aae&gJy5O7I)%dzn3PpQ1O91o?A*mNM_dhq&AW< zus{qh%rmCzC3UPWY0TNUp27@gkoeY$p^~$Iaq)rit0AEfG^rlPV7zJA7n(&9;^7Y_KcGXMhv6@0D6=bS4%(NQ?Wj0r^k@;6gd-_zyWq490TKARrFxlK&R#TL&7M(e&d~;axJD1l zMIn~rfokrACKgXFNN=~p|4jKn__LreDBVGun%mxW2coI<$G#mk-<|{x;sC7-#o5lb z9?D>b#DF2BS#41~D=O*i;Hei+UP{h~=(B!UF0&frD3e$0FJnYa=vtQCnnf+Hzd|lL zl`y_t9!89*6ryy_9P`Q@hfZ+&vQN%Ux_T6XI|(v`>h%E1KHrQ|yG;-#uWrnh4l4*_ zS7Qzd%^KYpS=X#_k~B(`JkRRS{i!as|Fbhgpd}jS4b$KI(ZD=|bOoey^fR&ol$8Rg$i4qPLu_o`2Fn(A2zg4#;s}+tA-7XA*beq6Bd$#R$ z(ifoBr_Swf|CmrdMb1{cg$fx>M8hrf9iOyBCf*hs0QOZ%g*99Ve|9}3X)isg?x1Oa z;2~#sQ;>avo~FmyljzfplNbn3yNgVQmuzrF_qM%x8YyL&0F}j~vuLnbtB^`|S}asZ z*Oj*mKDVSg7pmMxhgLA}O*8v;cH23tmm8U)OayNleOArg^s3H4gz1KF1o-wPaiZMq@_$TMTTZcT6@6PS$kmKT6*B# zTBHwlhu*H=Q|u_+1wr7OUP#;P#d1J3mnDISG&#SR-OSmiw&7WLfO3M>>a#J}uVxVl zrF%`^)TyW}>Y`40*|YN}?%6R~K_o=*uE^_2@nM(Hout*&MNSkiXWB`TaQ_%8PF1#6B*xZZU~ z==i(MiC~{Be)0e08U1CcbZ%PDj0ezW?kJD)uisbG^eE=C(JxjBOuhx>c19;*^p3H( z(0vVbnR%s&dq?U;|3V|q{p|&FpDqnBW@O)ns5rNZS0j4ujL$Az=2-_nzn8@&NuvAN z7N$yy{&_TS4!N^up%Y;VWg5>4SD0fmbBt2UBn*FX%wlUnSVy3;eimc>L;6<+)8rS< zEkt0^_8u?T1tjC*4btt5EOMF*Ol32J!juFG@$u~i(H-jQY{dCNQd|QJel-|!#1>CV zFi9EK8NCg3P`f-~@G+26VN^EJ3?b4@*k3$i-?(qsp@G>3>REzxGMkX^ePuX?9Ak^P zyg_B#@;}c@cm9?7&?cEzCJW)yqm7{krD7K;dFSTqyce1$c!S=gIC*xsbqiZ|dBfPaLqpmeE5f~(CVNAl~zX)f^VnlE}y}Yn27QDjr;+K@% z{b6$S%Gy6!bxu*ARj=TZdblH1gd--l@;f}y#u6?N27Iv_J_-M0d`F1wGV@Q}IDrBJ z!u;=b<6oli@6la|nzb^nDwZFHFvqqXY_MK_3l1{7j)-+jikb?Q$_A2_q@|_xrYU!y z@EU-#OQE?%twu{skSn(68(3gf@mIz?_8+c8L8p-l0rRvm`snc^{?qQ4O;7o6pNGjh zAiV+1E(fq}oZJy?bbg#2(YjEl?36uvbYGeoORhKVwEou9+bwlGQ2{og|vN+mX%of_(}*DT8vXWe2R%NM>~5;r76CPD^y04Rn>V z$H&n!CH|9QRamwvOwQ)+ocgh@B}+^S*mhb?bfkZ)}n!FoFZI`@kbmM5&*S7NY89*$Yu^J47iZ&wAwMegk|^ElkGq;TXE*hRJ387)U@(Qb;1Pp zv#8aWGrhO)`#FrUY&(pVNab;H=jXlSeq=+xzc4`v!PPF&mh&i@=d6>>!qPyzu^g;? ze}DWSh8~RzIa$2Kqi8>EqpH0DEz1@51Dtc`?il2LV>t~)ps*avH4R-QmXpsaT2Jbf z__0^Jty0F&n%N9BQ?cJZfQN2AIo7iIOma^X|H;A-jC^}BVKDlr-j!>NmP>&L9_Au4 zSb+D|TQ;A`Oc>xJQvwM(EmxC_Ok75L$lNd{4 zAn{^DrMv2o;pW@}@nyWVMMbwa1$GN6P`-TtZKWFPQu54$5YffQ|AR+hNP>po;uR~e z{8eR?)Q6(~^ePEud9i`$QK_dU2u-D?2{Bp@I>|Mvt_Z?+6XB!LNrs+NDm2!=U~e8OrdEsw%2azmS0AsTbqp1WrK;7GjG&bzST;|3PaJ5 zo$S3&KWcR8)_u><4OIu@T)|Vgm2^I~tX{4Y9@>WtxQGZdH|~*t;H-EH`-q`v= zr?WsSCo{NluCh&>Q9|4VNgkErkwVO^W=Hb*TIp@)z_XGr_ev@Z!}|c{lE%-E&Xlsi z*rK@k;5Py)Zu$(A{fuK7qay0)we2p7o?UO&BDv&X>rP@|SR(I~fK>HVnC5BlWM?Qf zS7PklwbC72awQFS{%d8*y?Wefxb5Bz*DOjAJNpk|YK=_!P551N0qf<>)xQ9qwM zje(sf=y+P9BqHSUHF;G$G?8JQk-G$>1Gly_-Glv|zwB=hgz6dgCJpEjNqPfTTsL>stDJ zLZ1{qNNiuVe^qjBEX0w^(qXckeeXl!ZX{4a-xkKyp%8xAJ!?STrNuC4r6*NDnCee> z<`e8PU*#b)Uu`BOSng5-tUUQY-2asYF?YG@Yl8k;{>{D2$9N|bpby=hPfQ@Y`L?}{ z)B18Mcbg|g*jUqY%9WyzQ4pRb*6D_cs;#X&U@|c)d`ZVKyYka3A=(1j_LS*RK)Ktn_z&D%j zY_`F?UGdjpZ}SEX(Dudx<=fU~g!WRxP3#L#Pj0;_^Aentw2+1>k|2J}al0wxnl&#e zf*Vt1r-yns7nC{$-g>G;`7oI4ssw>kORw3^BAJ$|uQWGOkEh)!H0C;{HF*l$Hi{*~DoAE9n0Q^%=#ECx=ApsoJIv4JKFrs|<#xkG-IVdY*XU-A=9sF&f;m?i z?G8g-V*QqZ1k2ql#&!0ZIrjEOyBR&qjOCnDh#6J!PD@ReyxpFb$z#?TTXk%7D?)R$EHZ%_buaIyX3k~fAqr! zrHEu+3JA-o8Cq15P~>s5J848rFTX}l(4-|@-O-_ewnhKlZ`h+aBta7SGhdWU@# zqn?yyD1(`+K|JREe!!aFZ*in)Hr>m+uPG#XDqIoW18$inC$ zTLLd+uyVzJlModwY|cFiDqw4_=s$E zQB`=UtgiL8Ad-Gxf8rl&0g0y2x_VO7Kf%`8hv3c4BdPN|YD04D@8XA?>u7g1Aidi- zw0oT)v}wF)bFJpkyPChWTdVvp)knvp%i*+!(=d?_Z_%?)X=R_HWKa2aV&27apQDg% zU=pQz_kq%YJe&DKC|)gua{XtD5E1vG2u7Is0=@64^yA{@4;=ZzDDOyei+5SMqRbs* z6CXWbFz?umv-=^FeI?gP{s`{BnDP+{35aUM?Rj(g^D0Q#a~=@=If!PD<)yAO9TM;% z6x@?$xcdXygWNG`nrP**{)rSOsSGEEH=>)o1@MGCQ0Hop%a`Ai0C|D>3kwINU!+I$ zVZSA}zn9;K9K`D3+Ndf@3ulxqDD;Wzj{3v$4r=liN*8kMco;)gSS2hjv%1?*r3^2@ zOByNu=BsR6>Ho}$-g6&9`};DQ&Rz-6HCrSn_4?fd{2p$1g2)fLJD4nwV%o?3YS>z5 zUeoN2$I%1yH60zv|S8G_y|4b*VlinddU|< zRZW5d0SQ66b@vq^bov8+<01s5W>72f&&QV{C+K zWn-OgwVv(jK!E3&wxYjx9jb6_s=)P9qLd9=S+M8GMnPprBm0JJl8lHTx#|xl2{W?YhpUF6>>;pt@ND=rj|yS)+Rx?TgIv z6$ZH=dSU7K-J((Wyh{%6jBta7N%{!2r_vP;1MFZGap)=THElXNFlkk)`Q>LS zF^<3v_#`Y9_+z@N19TetRKz9-26khCqD3*{tDQbw=wAa|%S=#BMmpHFP@-L$S2-{& z7R`bs2ospphH?Y+C=@p|pSUi#jM{gU9-7uk_s6&HU_!L5!8qS_8;+Nbo6xqbP=GoO+Pi2NXp&3wHs4)-cT`K26>^{dh8jtv4=*4$cA_{pLnI2^U=DunyQ50IiA;jf?wl(;L z2N}i%MwjmeZhlujK-_14nc(!b;_+k5J)3ikhossWle)nUr;$;YG|!dLkfQ#AD10+} z7fpPB5XCSkLI)90jFk(8D_bloz&R^vvLjDM^4uKM0$3TKaR#wG1&hGH@O{8?ay{o( z(mi9wfp@^Q{0z2i_S$2`e`N!%nfsmSug?#whkRFL-hYcaRtNU8`7=OyPb_fJw|nVg z^bBU)OmMhdVqAQl;So7LUU8a!6Piv8w1@c4`g0>6jftJ~SjJXicsh3I%&m_Y!`DnpJX8ELRhD-c_>(FsN%)DJ+ESJ%V2Icf z73To~uzO1ZNMef<{^owUS-yEnfqQ$ue%bb($W6Bl^}|`zgYsz0ZUArVR%7AP*;;_>gE^|2=s&`tVA)|0abb z2yck!M?v#l$LB{dz<4*wVs%f8{gpSTmo$HgM?EHjU0IA*Q<-dXj2gMC06d_HCPLL8 zPoqqnkvh4=wv_E8N*4-KpfX`m%U5}%Xj`6%v{7DNc?_+RW1}E%TrP9OEH8=!$1WJm z;93Di$FyV`s)aRR)*^|43Aubc&=EYUs6KWmouMbzR8`Vk)VZM$6RKraUSE(2w`^06 zAwtioQYBf3bX}J)+9K4lps-2}28mM9FKR_qrO@F?=0&x^lPg1}?OkYGg+B3)B2r;t z$A_xk-s|BS!wwKv7OC;=fJ0WpRA_~Z9y0D&wvIrSMvQhT9#!t}5HF|zB1@GU9%0S! zpjpM4E|EZOSmF4(Ah}fFBQ2btiYL|TDe38IGXD79Y0aJa(r$CLJKGRD2+^XNI+rYK z?69C&B?QDOKO(h_;jL%N6*~>1w5Gy{m$I@aOD4FFpEQuJ#x$E9Kql%)7c19nW)}>L zJR8{LlV*g&>f{XXA_FKO3mbwkGU(8Q?m_48O=`RTlnEjW$_5Tw%#I^VmWG8M89c(; zkgyRHx;E*dB9b-wc34ng5~?V8Q?*beNFS{@9ge^vHdgERuO<*gM=@4d-M*gv;%nmP ziI1M_Uu>{w6wH(w+oE`P8jPqe+8VK7@Z<6d-VEiRaaz4MegY~(~|I!?BCKj^5G+=x6=wcEgrJ)|+h zB-GnSDfIl%Z(r>aHn7X}=7Bou%K@L)%u=u85}+Ml!TrP9Hr$}XeCTqPMpq?Az2)TC%z4Yv#1wumHYwKgmkhQbAP)P29;WS!0=G8;6g$F3n*x=eMFhk=v`*G!b<{Xw z-eO^%E#cazp83?2%&)1jBv3Hn_{+Fd%B0N*J+OUfPrLBO2u3h(r_RJLHI#)5WSZP@cw=IY4vs)3L&MOgv(CU8#5Y$Bz?0Ig)FaKbXzI+q+N zqSo@fchO~E26%*3tW;)cdT#Al5XzH5k{?$!SFBWUD7RE0VApV}Qr@sGLm<;~<1jCa zw~P-F9tKhL`k!W?0hU7$Gb*!qDz%r)Q1b>KOD~Aw86AQd_sLWJ58^ph>-y{!wE^AX zr<%x-m6a8CDJiF5zq2adm)w-EktQ&8yWy0TpC%_xO&4>l(kk30jpH_CX=Pb2w2j65 za#XaL0kZGiQy$Tnq>XDT-orc}OSYw*y4Rfd;a*h{VRyu4+_&9UmE|~0s!%{&MH=~fALN;P1kL^4LmBbC>_0nT5Ln@G}N;VF;722dR;?9FL*Ac`Ye$K1s zkN*!_?-ZR2)NF~y-mz`lw)4fdZ9CbqZQHhO+qUiOp#L7X``&ZTxNqyVYLsTJs#&so z_mvmATT7PEprv+gT~U&1d1t#Lu~&u0E^tjQBQDkojcq%_1X3o^)S1L|?lLEl4o_)) zmw|v%hmC627|UvdB@@=EagO2$!&cLRtcrsc?9?xhkLf>A?oB=~IMTbFGIGW6#TfvW zhlS^Y_Yvnq7sI&Nm=Wcm#mw1VS+pJLt_e$-1%9U6iYy88Gx`wl7~Q4_aOaEeA(=4J zk}#Lf>07{YsJTT&#=}{{3Kh(VAI;ushwoVH?)SHL=c=}^wm&zM#{3&a#})+Eda zNQRsz3=VJeV|;f#wEJ6rm)X~Lmg2ZG0YXX8XOkr_PFrtkDOW3k`C?ef2I-RNBUbW6 z_zE8#N8N(5f?eaVVBC(4H~!J@m~cQbp1JA9LqN4J$yN7l?N&C+X~CW(f~_1)-YRN_ zGD^vj`O>M?8h!Jy4gAxXh7)r;t}`MiqHv3gk^*PHCUhvLdC{zm zR6d5gql$gQfx81ozC>fqqJe$WKyFZ;Lx1nRWpeObq05`)JNf<@V}Ait!2WZ6^blg$ z2@2y7TRKZbB$bMsfnB=`i0VNpv`SbOg@Q=y=FW=i%1fE!1lXypbvy2fk6lPySottF zcIh8=Lq0VF6gy##O-CAbBOLZ31n&lsJfnt^Jm*N?S1J=yGn#{U{B7&JGYGyixX z{%({$bCOo8uAop~7*lyp1z_1|jPfJ*WlxMuSb`VCSE)7CeHhIwe{@((uQXML>|tUMaEb;G`X=32@CoI^fvA;nuTu?`f}Hy(p;b`4eK zmz1RTOU|#$2!|!oR2Y+r(CYbxU;feSf6$Klz%D7Ex*?FP&yI8@lUzQC{)diQOEs6E zxn(C90c1MJaAcn97?n)InbfS zXAN4QY7d;0+b>w{xalV(8RQM=DqLr;&1R#is;?|_*&QKZ-+j^{zm+o&=MO?C_h{bpH!pQNv!8(6I=ruq;gMh za4+0pEMG3(*k_LD`lI^$RmI^O6d7k#KYg5hP#anfWb~X5GL>QGh${h}dLxY`M zeZmz&NmQBFBx-|71f@)W8u6_CkWS1OMx(ZNi@eAv&gvW?>zH~P!!bONud?Rym5C0M z2yLzIWsO8cA*-$Kr)-Q#5$&J=M#^{4!4qW7cAcz;_V@A!=t|Zky%2jH;div`cXszV z;&&aX6dt)S=*c(di+4z=Xr5)7#J-zPJI_GpCLVJTU?0P;6c3yd?)A++iAZGXx9LX##?f`fy@g}x)^ZEOXv4_*$;^k3 zYwD}~w%+E@U#C~J#r&Z1Sm}(66?i85&+wqiSAVjp+GK{8Hf_!6+ESLC&zKkGTLE1) z>I=IwN$Z7phwb*(3IDWc-@sLdE$;jNYB}l#^-1ZtSr2mtcnVvjlD63*`)2m?8ENSJ z2?cP72OdD2KFcBz*fYIZ-EYyuvx1!)5v@gb&53<&j`=+}2FYjZObqSGQcV5xHSd&RBxsoU_VY4?Pi+jcdH-7A(tSY$9p`boe*PgmKQ zuL7Pk1a@O{nB+fiQ;cIv>c3>(u=Gb+ALz*4;EO%Fn1LZ(*Dn2|Y4H76pBEurjbnN+ zs=xajx@qxG(=jolzJtMSv!NQ2*;|msf%ROF#^jnFIv3M=5Y$7^|Wl_#sL3wwvIB5OmUMkK|G=H0K{k+rIU?J4OWlW!iX zOuzcIZRd?`^=zN!-Tq~!iaD^I;o5Jyue@we(EbluRAflks@)67fy_t6@n@=n?s9uq ze=IO0TfP8r3Vr{FeKva+S#2|aSNS=XD4JZybbU;|Z@I!2EUos1#;Y~_b~9LaF}X5<)A1q&yP?W)!m@L}BX+8Ex&V@*3L;H~o6mgnVd1>UFi|+T}=@t3}^E zbs0sk-CzHoX;Ktw;@`gCTi?4~@7s>Io`VZ_0zbHZp9U|mM~A$?e?8*^{0sv4_eOlL zRO@$QSfAaTe<4d|MIIOG3AFa+A>W=Eh(-6XFQ=#XjB(0@EIu zU@6eLc|q#-RT1=(tOS(qjW15FV0OnPz;~15+8!TJzLtaXsNOp;o*L$Y^vB;APL9;| zzOefq_S>93?uzaE(&7X>?j^7W^LXEzbbc^87W7Lh`aJL5EB+xM{9KIm-P<$lj`|nB zmGQpU`Q)kH$GW`ff}5iPi$RGUQxdBq(TlaCC6qylA>oTfD+H?x?o^MwbXusVp<715 zMq6zc+OAQfh*RopU`FDsfRJd3;33UO4!ZG(xnJ_ zk4bx9U6Or~#AC!kv|WbwTUZ;4a!_l}P*Gh+guwBUGY%k!7N)dByt?s~w$*TuDN&SK z+itjO0ia!Zg($KIS^cw`?O>{y%@;u}*o+D!C0^=*=e;;L7l>-)0b63Tj=H8rR7rnB zBBJpZj9v1LpbSoLX;?+>7QD1y*4_A4EvmA}VR&4M*h$F5el|ps4lAQtFPM83MD68V zY*&q~n`aDXY4HkuXSph&#e_$Z1F(MJ%&WqRZC<&()-eBA9k^=IlHP2|wMWk?=$ume z`s%VGF9eBGVizEl3}vEXW>|{$Yxj|qyUX;n6F-7m{pR5WEnt1o?cibohGQ($VN?wz z_ms!0FOsJVKO8fFU)(5ys4du<_tl(6zsy9Gf-A#8#G+B#PjG#C+p@E9!c5wYw?vru zz&)yvZf;k%z?{C7s4-{OSrKsiO&>@nXc#|i+`ycK&tRjao@xTfDzO$cT8Lpbh&FIF zVT+xsbe~3~tjtt;s5eq4r=F`N&B$3e)<-l(5rQx;FcvzWr=K@l==R}5TXJ}EAd)A6 zOu;95E@^WXoyU>2`-sJ%S@U*&on^WwwrwL2nU+axUm7%$Ei;Y?YWd6-xi(2Wt!@<& zmp1xHURS3hbEiGXk(8>SoM?qAkO8$5Z4!+~={)B`;%FCyIqZ+!%EK0k|ABL@Sz&E# znc?6sCq$auN1@GsVJuZfAZ$n(lo08nMw;~^k|HVXlH>4na(bf53nfu$jH0PWBOR$X z7YYukb!5MhaYh5*&R~x8Dqo9w+Hrb}8FlY)z(NIQ6y$H#t#!%6) z+08e+dwk7g0WfN0aOTc3<&hOencKq@1WT$qL2pTaE6UZa9^_up(1e;R+W)jwohSE4 zV{hjke_~Z~B5)+`P?7q$`BokJFeQCUqBRYJ(+aD;kJzY!iBf&z^rosRl20aCj@v** zmMe&rq>EfI>o>Bz2QqP2w<)r!PahGnhA&p8GtkA(Oj@hbBEqURNS59TtJ0!lOnAu# z57C|voM>f1h0F}gRz#5x+V)B=-%ELgG&@+7-8QwoRuoFBR27Qbc{Wg|hL`Yqtqyy2 z1kHti(%$}a(d1KA=@p$;8Dah?PvK16NPl0vPu6(TvJUdH=(@dHJaViNF}H9nOkZ!) zc(RouIPOi0^irk8piya10%5nZHL)h$M3HiKuC*{|u`$k57%@F`5@W3_lwz%F7vfg3 zT5V&%h9fs3O|2_SYB{=E5N@3Gxk0s@=%H5e0oDhQuLP`D<3paQ9y%%cDIhmHtY3ii z4nRi^$7m3b!{3&}G1v1Jhi%~=Bm)kl`L=s93ArAHTFG{BH;Mt8NxNtrH`gpnX4hs_!7Kq%GdFGijq}Bs#$IGD9nUxD~aKv z)i}ZpOZsXnW!Tl3dY%(jQg{K^_ZbqRPEXM+Y(vmh+|1Og7u%0kB{(yuz?|fs`)2q7 zD4|_lT4SzGItK)u9JtQ5RHBF8ANv|y0wMW9iPXlsv=oFZ)t68^67pecu9SNtT zZ@K64R<`eDjyE*2wM>2k4$n+njJtXG|oI|LQ;zzCO{-kC^9jbOMGA8V&J-JI91w2>nI0kXaRbW;NI*=#?;&Cd{fCgsaV8-Y6j&b- zVGxX#g`#gIkiqc6qK2d3vk zavvxH4j~Prki)n5o>TB?_Z=JZZu?V zimz1C_7rCBh;j19ShuVV(DOkZ!>=U6bZMC=;XAU=&6sGScPQl2E$mLrs}~MA;thdW zbMd_$X2Tpb%_OMXPB7qQ?JD!fc09w=A9iYZuu3SBYbGC=E)M-rTS$aTE`8a%Prh zdLiurjYkEXOB*Y~ra)gdHO(W@uZxN|LE9L8eT0^AVc2j1D3!ItYSCD8l5;X!gEVBk zL&oXcK#awE`Tdc(Bi5e+ca#~!39Dp|Si#mQs`HtSo{k5;rMlb?XzDF{gKokfiUtH! zqXWZL_I0ojqB2cxGsvie;RW<@relivIjTy>a*wRG$TylM?8cM>IQgz9+L7aP`b|-o zY7hPVhVtME+KMm2hj)b8AKUl=ZuCeK{o7e1@@!%oV|*6HrjQ}gkioHUZ3Af6K&o#- zSH2NfpBzVz>`cEXX3a6YQG$|Z@!_fduh~qanyQg^s3r1&)G*CawUZ~f}?@E z&WNNpv*sCFC82f7Q`h1kUd!qXPkj zvj72M{ttljUq=C2w4i-dR#pVgOcOaWx=Bqyg+YSFpiqQ?kdkFZBncCP)4>?4;U`Gp z0A_=R#B9*vE48hwx0^d9s$B-!T@|X$kKc2?_Ii5cD9Jy) zqj_&T?st4=yT4p#xhH@2yw46KBF=4U#%GfFbVik#J*nlBc3m!V$Y$UWOdkOVJN+lz zbC30DG|WuQ+Go-TSUU6!&czjrp0#14_q?7N3^N;xwDG67mgv$=l4s6*th>cL@|h!x zvgjRGcVlwo zhb`N*Gh&+CgEjbC{kD&ta(Zi5-9qB@=q5LIyObIn6VIo#yMx*#8>LTVn$5|+-1hFy zVOcz;3HqU(yv^BNJKaM=z`InuZXH~dEPQtG6OHxV&z+uayd-w9lPssF?YwVLxjpzW zPyCjjyvqZ3D57c7&KazC7|m%icQBT<$*u(PyQ+&m3Yk?U9pahRd|yPpa%TurPb7hd zMZ$hP;|E@v4neFO(#}b<9Nrcf{(wwvlj9?9f8qAKv{T>3?aC)xyB`zsy;*mUcV(ac zpj{!3Jt~X4$NL?hIK*9=7$e6#+VAGsyoY%-o!e89M_$?)^OBG3*}Sz6Kh5XAVYMd= z9_QFUg{=CI_h}zOhK{0Vb?0A*%D0o7otLhEY_}@aYi%lQZA~Zb*Dtn?s$f2acT{T{ zBng*$f~mU5aU(~43f8^!8E?&*kxgc-wKvn-`C}~pSp&ddJ-oLrTV$=YO7+aK)6y$7 zx0Y43ms)FTdK=tsxHHI&r~!bsWwkoMG}B2Wf9?jAQoN3Qp{BI&Y6}1Kxo`lh*{hekIDVR~y6W-qn@yq~-aIc72mo z$_M;ocB}*C@m+WexYySKr$rO7pMKeQDiWN#`#%x@%Jlb9gJ8Y%C}DLa1REr|CVCQ( z=oss0Lo*R#jG>U3bw1B=)CdXUd2{haiO9It5MI|;1KE_&!R@*p3f#L0r$~b5DYiai z`_5X!6t0>?oP_Z~Zl;5L9K&)%Oj^ll^|Ln0vsjS~4L}VKT`EN^3eR+InlSu}$RDix zGz)M7tm*PNq_=jD=sp9bHD8b6c~>F*q(< zOOMVT42{0aR@(szTsI)xcxzLV6jzG*jz}5RCeuHU^G#(b9S(p&z*4;bnJ#b`6+$BD ziRy`>M2U#7hdhcG@xhZ2Fn~9iTmf`NY8)n>=|~rf>zH5+g6GX^Z?S!t&1z;jzhuh; zHo#-uESVGLIyCdAq0ncQazVC6E^@}<{biof3{VtEzyaDDOVmV>G&5A>-^yn~KQZ7P z#WHUGCO(L#kqKoGaza7}_Bk2x?mg)JC8AD3ha8r^t*C8up_qUoEczjSl9VVcxCK8! zM{0fnBZ2tZxcoR=n@R>YsQO|ZMhlEZvikN8LKK0UC2nTZi1c~`SDuw%&_KMqauWB> z176nYMXz719nA_3i78{=-^boejJ$h-zy|4cAiZ>?sNbe}dB(Hc4|tTYt9@#N`(YQbn=iXy znID{YtRg|u&&!_47jaq7CwWFIeOxLQq4S$yjL(zN3&!J`$qy^v%MW^X47yD|)w8@! zItAvAX3R{~mThmdx3v%L69`+ZUfc3re7w=cYK;Ow8))z=$xUiwi63aW87gNFa0X{*| zifII=)+*A2nEN}=S$}HwR+ilQ{@xP-+z-k3AyoW#gK0+eG4I6uu< zeTy8@i+lgnpk(qHDGf@QcmJSY;;k>$W0k~i#zXgNZ?Onw%Lw1~zLH$khN1mF5t?UH zulardaIjzKmxsQ0)u7yxGq_I{#zolY!hyO8?v{7U&y0aOk+^39ud)0-qtqWFlrl!4 zby@R!kD$F^ZJEbIObF;aR2c&zPcbiSJH#ueNv4TfEw}@^>UdL9nn)!z_Qq@5px+@& z=o5JNLx6G$kFHK@MI&-;D>H88c5Sw;DMR;7D)Umxp+y<%A{#sFpTf1pchz$Sw2*=h zgB7jL0#l1loicmuX9&@<4z}gBqeQ;1NDsnws@*I&I^`$HBvXsf_1X1hU{H>kQKZY5 zvHr2UE2c9zI_;@}QmtB6V>r4vg0&VO5t@0JKVBNctYSL>DL?f3yqGecXa*Z1Wi}CE z!e(NJ^M5l=1|#6yZ-#0Sno)PHl}_WPlS3O=$N~_#w6GAuNDW6W=cwe7Eal&z4Rj*8 ze)0VgHHBS8K#wybHP9tlBaMum9W6h~uSn<1YHF%P1S{<;wX+!S3d$`@g=Bm2`-&Nx zl)yisPl}S=2e-2d$1!^(`_*Vbh>GzN)sFwb4wTmwuVSfk{PbF^-JnENVd)YD1?8!DCv9w$mgrW$!cqc|F1A0i3ycg3xRq zM6rLoWqfq{979*69gtgEv$c_Qm^gt8>mHB}@FeaSiWRG-NhEF_UBx<0>e9QSLJCgwxbMT!mMuF+BL z?MrV7LqfBzr$JEf_?$1O5+;n5Nj$iU&j(&Bf$N`iS``}1-@!w8Hy-H4j(BoPllYh| zr4EeE-@{1woFs}0&Qqn4o|O2vOw!b$Sdnn3*ylTw-P?>H*A033B0)UcS%SsgC_I8J zLFxhrYE{<_P9&1%uP@kJFU-}O{~llP=Td~J&uQpZPoMO%PF-I}IRmtEfc1%|1|bl` zQ$6+;SX_LV?9ky#p5rDq1SLTv31&*M_|J$~7=u(Mq9Qt+tmaC3r(~i&D`gc8cBdHI zfq}z0j}z}Q$}OBr#hjbjb(R+YVJQW|JnCdIPCUf6q?ihTV`1jpKRTQ>GvZDA{OI)` z#eMJIgCEVVy>que@?9H6&)&g4N6ur3nxKGwCBjShlGmVCPJHElft;bMMP}{_=-`4c_vI=VRON0#)F72dMM|c{`JJdLX@&A{SQV zQ`!k_-!09}LW{B*BBF0cAK;1lNMo5Y7fhd-o77Rga(h4|hVQab2iT0sd78{UXeCq| ze-d$ELv%Ab=FQS98>9ZX#0+&lp1s%wkuRhzeP*JkC=EKJBv_sYDSAakh5P514*j2JL4&rR>H^)gzv`)o~*7`XvHU$xD6 zIg;MrYhc}g#y&^}l|8lf_ccn0g0vKGNB4DK%_kyX4A!4TzsR%m*Dmv8UL7Y z+U&v7M=<#KAjWi z%Hw?4T<(LTo;a7sVJxor(C_rioSX5-*&U3y{7u`8>>Md54{9xh-;Ct8pt;QV#e&yL z6Y?s|by-Q0JXTp2Dk*msaH{plwGF@cm)R zjOsimn(PUBydST?%=a9#;CEv+pEDzzwQbfn6ESRDBYiAT{fBc^Y$u)7EM8{bNO*025SD_b%rZtZjM(F z@)1QC zQOcnu#-0uC^DPyuEa}P?yKE^>ZU91e@=I=Cwk&PrRW7@DJ1O67B8>(1MF#yc#p&k@ z7NY!97paaD?+`1z+&KyHp8?XAC4~DSjQh~;SNo{03-Db(8vJ_r8R!EF%mW;bV*u!b zG|q!`*bL<`_z>bR8QNhV`SUav`k`ET8@c}p(8?~ne_8$VNgZ+yK`9foP1SDFpKAUb z(aUlq_U!m@4ES(PNr*QEsxV9RyHf60C6c=Wvsgvgu?yr~*s#{QR6vw1tZS>utea%% zQYxTq2;EFUyyAh~w6}|`+o-HU{zw*Ax1}-k6S#AJbXUR9Fi$g~pZ-vTM<2}Ty%6|%6zH_rCb8gRzi~E5^#Ce0 znnB0-3c5i6m+3f=G^K2S3*j_7(vK}_@Si(i!=ntMZgeOCuwVj+!K;9K62W&x!fs$F z`>#EsuYZ(KS=k(3sKM5Aa0g?@d3nGG6o2p83AYh~+dvK|0`6ab@do{)1M5Hd2)oax z5%GZrg9JS=5O-mN-9Zod1U(24{XYFe;!nb;U=33$x55`44-=0W-8FLhr@0qv_|o-C z)|3$C<;Z&l$R4hq?@))u5J-{ysmLC1*dN$Dr0;X`>A*dwfLeBAb(CKKR6wi0HdFD* z)r}`WH42FOaLi?V0+WTl7%jh`vJcfO;``)|-kFNo_v(i}Jb1+xGqvn*s|=s>KR*c{ zX9}C!ggQYElSz6qkmZG#b19S$xiXagluRUkDM}x*UV^D>& zee_I4Iv|F25@;tPcQ>g7nx7^qamV20yXr}8BOR}Dc$p)>$$XZUCq(5~8YF!yMX)Ng zrX%pttIwG=Pa$QVYydDS$|?>wu%)wmaq^VrzA1FjtSCantmTm z#hr_jjelqwa_z}hV-qv<3uu3`NL1Ob`QT_O%sv|C4&T#992+!$o4p||iZEu#l@Gy7 zAMEY!)!-IWhsu7O(wP?s@@q7jWYP5=^OcSO`GD>4kb9cpy2RPK1KK(fv)|Dek0A1= zEY=)(;*uYP#j&>Xz`anrJs}v+$k`zxjXN2h0Vi;1 zQ%WA$W{mSksLdD|II#n6$&HcV(6^1@;G7%lEWdP4n0yZ7+Iz}^5UK8A2t+v!uY0te zOyk3dq&{iLLsc17)JZbz%YL^Ox~Vn9iS>p`^~t<2T^hQzalG%DVos`jj23W;vt#Zyd)-&1_ce^AGfiNMXO2$Y7$8; zs;tUg@!gwNmGhUD0j;CGGHCT(6zetTbB>aF6t-ov85wT6ry*7M4ymRo?i%_`@Wpgf zacrIhVg+8ohvEC)Iue!)U&(I^&s`P6S?mJ-dq6J*Hg15fRL`EcPVO^;k~ca&xr`gQ zM%>1gzMac6YP!~4Lv)V4kw5gC zf-O4vy+RSflT_b$qUx(1^)waZ%SWQ zDm|+-F8c5-rM^$p!I6CyBCO5p8=L%obsqwEMzgqd9Wj~7fy%1dr>Q1R& z$p7B3u6~fxzXk#ViU$S)BK-etSpRpbBGdS(|4mgSR=2hIEOcQUa*h!EN3%bc%xXgv zgi7g+9IqvUnkTcJ?wbwW2Z$h~UYQ2?ASBN0HYe=+<@*78AHNAG2}n7>u`;D_*+h#j zoefqe%#j{}c~hz-0{xB(P0puV7aD-$Y($Eq1v@>|&dGPY(8TJJTvoRE&?k%$QiYj1 z3~3}_=N7ByYMFSmTm7O4-8@veE(3^R=03Z8aJ|k#K<+t9PThzFv-?t#I*Djnj3@4O zo@+Y1Yx(5{BcH+O?ml@1)&&kg-@b6Y4m<}+#W+`k8_PeOe_C`tw&2m-=n8p2 z`2RHZzmLr|Lk4kX{6|A;K>y`@oRp-gjg9^PqZ*zU`!#+D2nb^cJ2wb9Hwb!hh}G!j z>}(%#2&6xqTkuW-7eqziG|bMF4p1@?Dfq^BtabzrOgzg<;|ol5_=R)?Zenr zNIC-_C1Mj5b{Sk1(QrP%VjpNJL=_`sUW6slaNFQ1X{fB znHib>0-?Z?fUd$2nz0-<4*)fJXgS~>=1a((tES^jy!N0Rn579YN)uwFKK6>R& zON|UFK3%CnAJXX9$hAgI!<8S$=q8trrPxyODj(SrvQGLCJQ{-y5y}bOu?iyk$cs08 zM{JxV+bgx27!s9|R%ecWF0h0V?=j`sSb5!cx^OTVB%{Hv>(M!o^#{`;6X`y@9naKg zUX~oHTHLX%i%8`t(qfGtC=Wm?cAds{k#&Yh#4K&0hJL}@tG|{FRrJrrn`?}ED>=St zTzQ1s9<&9eBAdgt4~74ufivUWoJp|~TJGsG!w1SNdWn}cgP>#FjIZd*@7 zE1egsbTWMbApL}0F#k7?z*vmLFaE9}@Pmivy%#j+`p`lDgF6f95yxuScp*21qmQJ` z#O4od1<1*FM2`Mwhx+!<-!`B0Ek^S%H1&Isk&P!8nxkO6z>sy1Dh(>c@X@l+924n4veZ$DHqW} z$n>`LCGMVeMI*QKDs z6}z{9(rLC#`zSG>KU;O!pjIEd?3gU42bM->&{(Lo*ifc-2~M`-C9T19n2b`RxLr4! zCx#fK$GNGYRM1ooKWU9LR++%nRNrCJt}4vGN^$sciwX5_E6sCp zq~vT$%F|8Tn5dPeawt8?L`~7aWOkq}Qfp|^>6?b!w`(%iNP%-eGBnl{&^DAsX@{=I zX)>Z${>u{g6x*7XVHeY#GH+?5T8(WQ*kqDwC)4$E^5)=5%bCj@Ju}&;w6!(WAt5m( zfB=!N$%cwjoBmeX)rkLNfv$R~i;kchr6INS)~rL9?D&a)MiC))pPus4eUPc(pwJ)U z6q;fL$paDg)wwk>W|(g`kh_KU%)P#m<<8OiPTZ--P#~quaL8Y3Nw^fsHSfDq*j}K<9 zo8xfudZO<;=N|#G4@VnP78w@UWE06;KyR*%kf)m$dxq;}Y0@4JVf0^222`B`>c43p zMZ%X?RbMY2tEjonbrb*4o}Xsdnx9AJ0p37so;^Zyb5M^6nDV?~iJ1JNgiKE128Z&j z)qE1?hzG_j!c7TgOGJecdq}ZkvH1}9_ZihbNo#(7zkd-vbi3oor}M#^hUJm$;iY)< zF_$n+MDt=Ih)97fU!-FPk*h(Q!;Zp*@6%gH zSFh-obGfY$^peVnS`S?#J|TKy6n+L)pnjVa9B|xm;j0o+=4seHgJpc&5kH|Ew>Wnk zvxeEba*BFH7?l$hZkXYr!R-%fK@*O3d0`t2?%TWy>RnECCknhC`MPH)?VX0RbRMNz zRX^Z;jeFrhc8OCNf*b+wjNx8MeVj=Hq`r5bb|lwyk0-x&M|=RTt@V>^v8S#;6s2_7 zzy&&(Ccf~9TYWx~&&aAet{40H0DCgw!P(lYV%R&G9576#7z7Xd;!25mK8KCXVi-O{?*w5(lufMT0O`C^W4fkmMcijmZ{iE z^*>WZG}t^x`ZW7BFC^CAF5)$MWvkWGQs4ihM8giY17Lvy0kJ>;0ipljSSV-e;%@I` zt?FWF^M4pVleLwX1pkw*Q-n9;g#(;P{7Zt zc_amQ?qSAkdIIIBOLxBvTXa=G0}8jEq0$JzG9BH*sXGVn0(N`)CPTuk6mwlpe8a|_ zA_>-4IXbgdmnR*`Lh_G2jA-L-E2_oJM(s3)fSr{8(6*C><(y~gg-vE}&8n#}>3R!7 z%H6(+4MS4eK*QOczX6_|jSuxEvx8h?to!(UtxCk;D|3OEEn^8a3^$Jq{}}1Up}7n` zSnHWnT)Ha~^}Hg5KA|2Iz}kO?7b?QTkk_swktPTr^eM5(;IY~ZH)yr9_70ZVV!0CB zN~0EV-h7g5FU#N|wwEZ=DhycO?k|^SnGYxqU4t--KSsLDUKGUHWShqJ2;vIpojmxY zTXOBm_m5&R-NIjCv?q)L{}TnmN0#ZOC|xKhb`gJBBt)3i$=)SS6w+VTv)b5j z1Q*Df|bhQ}^_joE%0mkBk z7MW?IT}$HmbGCMIrS zW`PnsAF{`1tX)x`>jMmF6x8=ZAiTu?FtlHt)X6+NyP%=R@)&p1GQB#fFhHQ?Mb-{J zn&}06`Bw(_$YKEOCdsga-fm;Cu+4#I`eq1k`=l#R9`(;>cPKE)Fea@rZW@zUL_x_M zQcXa(Dp8?DFBsF%u;ymd^K`>8?_9e=0HYP57qnv?3{s8oHiL){l?WVSVG1I5o*BQm zZ#tN_12D@t-GDt!py{ldhFi*yt9W&TyeGP^et`6<;2%V*a+nlKA+d-2#hn61_;7bZ zzt~g)i|^2JOUf{-v=lJ>&>Z>y=}cN>NB>y=Q@*>C0|BA@zh|wT>|N}Q?f;J_#b`il zXs@O4Lk8y}#*M81WvZp)iH!jcYFmfrl+y@~At8bT!Qp}S4Tpq+jgSP%;*?z^wOu5a zZDO^&cG2QlT9(Zku`Nv>mEz5qHo0P6YR<7N%b{n8dv{ho80<2#&*n(Gr4)qZEOyEGq}p`UeE2JITD>6g{kM6(Y$6iFByv2${a$YOsY z=16-q;;=m=gGuYCKumI^!DxeV)V@5F=D^!GJya{Xcf#=Rmo@Dc-mgIo!Uyh??yf-( zPm5?{19@W!;cC12c<96I4w0ohqb9fTpCAG`a?H-xY{(yNf&5CNH z=@vPlL9G_VJ^q`%eRAk))UI84_HCzJ-scEnvt`JM|)iP0<6AK?rUyhy4cW-Oc zzKm#KRPJS%@7Th=7pfpby!XuFsbT8JXa@Jv_R+JpfU>gPU4{2=n!~mZ(vq+58VXG{ zpuT)<_3rA)skxYNY_O1!pT`bo?C|Qg^hAASb$91@(18dVTq>;n+0PX`6I6HK7sR&1eD2EBuCv|EF=Go5)ApU=<6D)4g81l3-!=Oa z6Ga9B>Q~n{vPB1B1c>bwl*sbPGn#yHn^ea$@W6w5hUoGNG8&#qKD)VvAJ>$~NY33| zRkN3mLo73l`C^~PzWYh@-?XlcjdMOAIT+8>cK7vEPv617K5y>_+L87enjMly>ww7Kdkh#O zsMFb;6I<>!kZa}YpMin34OG!cuY<~6HZH;Pk&rE8A#7qpJo?4bp&?OLQ0%bT1KM+1_5QQlhrLniiOA)#&7OV^o3(8|$s{j;D(^aZxz#_CU{ z?-xj5+{lxBY(I{HUZfK*L_p)gFVEJDZ_Td|%jIi0bEj!Z0jbFc(Jw|)wP#W-nk zG1_Mn?m5;%iD2PKGnpIFpYp?KA%Pa#djKEXPq&?iF(W%2(2A`V30;H5cZQ#~IdB0T zuM9yklt46)|A@SUeuPq5U1c|8^`&#aeSm`$NbPVL{Q?LT-x~5VvA}O2Tfr)WQRbl_ zad_Fhu1ysQ7+~T{X5F!Y0%)@Um=mqrQkUZ_mNJM$iR&7D8JXMo&?OXlBt$+%nQ^0} z-5_A3S9DYw*b=qQ>-DqZD`Jv@D?|~qtY122*n5nC9vzkT;=t>v#DXII-jKkV;K*GxGqxka5vVh(=NR`n0?z3n+FM zkzr#)6Zgq>r2E;;a-QIj$`7K_3@6cx(FY{xh>kjakaOl}$j!RaO%VU5fImwz+s-xbK0a*+7A6H(9PFmX8et3yP;Dfw9a`4scW z;pk7CqX=x+VGTCOn%q{etrByF>xBgr&Lu365GgU&Y7nj*lf|YdcYdBB$d7I(cM`Vi7|J5wYi})yT2I2#Q)y!bOy5FmagL z%qIYKX0v-1>0x}MTSjPqt``-LaY{?n)I zX#CjGSWz547=MkDD4(uL_jx$2&?38#JCzR-qaU8<(EA}&?7GAcI}xmU+XY3s+7OdN zv=WJ!?D`qNiK?SlIUn*;{BhJGiJF&F(j>u2*J=5b*5B(<@+D%!bdscLlBR0CldcbA zYX59|i9zrMVNYSPrSpRdi*b8xRM&BDrA zbx?z`Oi_S}q)-+df*wJlJw69Ix9<_t= z=4h){HOf^YHd4QAZzi}_R{EUSzJ6}Sz8psbXN+>F{i}bA)Op1W61zXkit8V4R*H#a zPG!)+0Xk<~i_QY6^mVfz!?lq6;brgZ!+~J(bG?spOJ;uB&SK!?elCmb@0w==)?wx4 z{GBW~lCW$q;^GK$8U`k0XbJ#g=6;&|2)+v^#_iCvS*H)MK?!y)$YFhQO8=`T0Hbku zIb$hG8KyVg?DM-_b^4XGn=rXW(iF+p&6>123a$KiZ?ne@q{Z#nyNqG(MRy*gue1D& zsx70!OJcd7Xs_GYPZ|^sWK|PVZ!^JDkHIt(tjWnP1;QNx$5%yxA5N(G&9rXPyy_69 zhR$@tIwtgEB;beCzZUF}Sze}!gZD8(fn0eR6i-s#M1jAQKxc#NQV z9Q#Np3pbG0G|6>rklhpPp%(J1mMKCN0n#(y&S$9Hv zuTebVxRv8npC-Roiw*Pi%NJtS=bhxen#+@go?uPa_JA>ncf76D-H&*^Yi|Sp( z`PpqEWf#U8myIQvMrt0=!=OKna~1WE}o7&}vsWA>mKo^TgL zkiZgzwV(OkcAm@Wua4}D^0!a^sf}P1Eg@^itx4?1x+mc&OJ-2*tFIu;$;VL^me8DO zA7c3X6K_0$DVS$yE}M{x#pzzh1upt?UJ0>*?f4TM`fZ&a`7N2eRvk>d}bF zxFa>M1mLUW*u3Jv2fncNX)-Z%Eg8Ih!@(Cg4N(JRm9Q46mPj|u5&jT`WuQHFAA8W1 zp_Z~1(qY(<&uoS-#2RLmM6=x~H8rQd$tpE9>J-w zqepjZMcQ*4tA96I&yuY$9ct2FtjVh}*`?1u{ z-hiqo@bvCyvzxG~QYljZ{q+JFO+HWdsfj#gDeN$7P>NO|U zmA3fA#CXv4(Dz&wPF7EUQui34q%%V=nN!po86nv2?SY56Ijq4$-W2@ZGCEXLuwURH z4Qs&&vp2g_|MIba+uygJZ<8C=xI=UJkRIT%)-Q!LFdtu=sVu7wUW;}T?2dEl2*Yq>P;(%t+52h^MI*a9)i1`2@dU(nk4`r_m#me<+RiYWXB;|WS5474bBD4g zb8*Zno5T^><`$HqHm~$fx)b{eL&q&tEp&ac>aCQo{jHV1%%FDXetYspBsp1~QgjCF za9N%2#&=;kSzl{eWs%tzH7$5ToLmV~%PrEY%I=R~PzQJ(9-UCq#wlSb{E6Whh;ghu zm1@6-YI#(;*tg6)*tE#BFEH_;!5eLxy~le3*(KZd!rO9%>w3cV#jM>3rheml@8uPr z(HTDM*5cjb#QnxL;iA-nY-4KfW z$CRmKmA8exkrRVEb`i!q{AGsjwUJ=3!l9;*ep=cM>QhV8JMrNSt?(JF@L3F}DTZ?d z-V8_1tnl;;(_?wWCMIw|pXwXggxP!L3OCd0Q;otxYJ2i%&=H2+uMd=`5e5{n*687a zUk1YEm-%opv(yth@*<1(&$TJj6h2$%lU0&mB;=De<-ns((B6kjKmzC0dg)J7b4Z`O z#vK>VJ$VyKM4Q2)J?TBoyuZ&NX|3`Oc%|amE(|&b@OO<`r390~)jgm@@1T8rnn#SR zb;dTY&dpELKmy$}20A-`t>?y>I+=>y?=xCCyEq4eQG?Rs{l`7F z6L}c_hYRL;HK$}}W&@sODJ*+Kpsw%mHa=;0py5l8^VqvNyN>~S>6c+CYBpuhPXNc~ zUD@zc-jjh~`EQ+{tZ&Fk^@Sg80g!I9AAMq;+{FH*Vov!a5)M7i7_Tp99SJq#OseR~ zqU=Iz(d8dK_OS+=_&zPDhnIE2%|@YbJEAf7CLshsz|^=7)I%V6$Lf_t+EO=l(r!6& zhS8&Ky^Kx0`~=-ZO1po(!x#B-2{8p8MZ+{1fDh-G8RvPSBYBeSNetF24_Xg=LI$jR ztrg|Vg8fO2{fW9sJAG;r?&KEW?8J#<_5~t8;(;M)z&HI%bX_C@wFq~|-^%x1XM+;{-rXJGY7_o&pweVnnhYgoYeFN#SfhG!oAD|# zqt#+SF8&pBy-29T46`)?M-EEeO32USrRU?{^Zf*(dJ?&FQ>^as5R@PW^BmYvN&U=& zaz9t{D`J7Fg>i=+YSdKJ?B|<;*^YR&(D~25z9-DSK6zYQT{KbFVTo;6BZ+S^euIl@ zcIpe@qp4;99dn1S#>#L!e=KD^ukUO$2AfNDK%1a-^5XIzB)R7+IE?B}J&X@!gX`Tr zno5dbqycI1-l zWy2Qf^j3hopMb@!94mnBd1yuCaIjM?HnbTBLn^F5l$>a6syD-X-7d&u+LxV&%JG?M zy~+Lm!`o<}e+oSFA5f7%|Mhmqe}O7(4h6iZ{`>{H0osi z&fkrq`EUH|`81t0-W8u{Jx^j@C|+lvPX&c6hr~t`1O7AFUwOOPERSE0Z{YdigOtQ; z35_%Wg^Au!P4MXQ>A7V~zo%D0Qf1 zdm3E-l~NLs3xE$gwB0WKVCGT)rmLU8J6Y;NJ*qx?ulXg3sLg~E4(Hua^8({0tVBCT z;4y18^;f%#R|Zkaa#u3-Q~NsolHMlcCcrax1v7SV6XtXFPkffrGsWWB+Dep-P{RP- z4VS3&w2PwmdPm88=kYq7qxG2O!Vd&zf-BQdJt;sKF_qIYSv_CyM1_f;X^}U)p@CjI(-k(?40ozSn>=ee0u~p766ESI?n|+L~ zA*)FnPEKWV$rvCXl{mbFH<5X5@ZT{dPZf$O_y^PfDlZWH8`J+?Uicrd{=Y6_p}X_T zbPddt3%!>U&CHUNik;`6FcE~0ep?WFi+byNEI$eydTwNBPoSNbh+^g=Vgtz`?lalG zvn;1mK40DbAe;m8U`EwaT4{|$0F}|ySliHa8r&ofngK_-&CWhTKpZb}CecbqFJ_e#=`5 zwmQ2`xT+F2hTXC4=Z95}fy<4V!{H{P4SY65L&;(p`0pWSIlE;i_goY0mOx%iCK7C{zNGC- z5>6PN(n3`5P3*N$#_h0i3oI!o85=9zs13>%K{^bIW=bvD?Sg8pFYy1~>2X%|BAg2j z1XK(2FRukfPZtY&yMG&1rKoGD;Ao(IV}Q}DSOODfJPIZlI07k|brTowkRc zxlC{Ioo*)o5!4DuU))B~#pS5nq?)9KXia}trf-`fq6oSl%eIrz%o}8O-6UA5f$dCj zMl!UGGs1SO3$}`TNd_Ag!-vO2&#n zmijXDu0XrV;^<6hc1BJ@Bc;>ihWzqUR$A&J z6Bi4EI7+J>s>fxf>(SXk>B~YthW$L@z6pewH=L85x8hJo72eX_}~-ZO7=@Z3;8R&AFe}MYJEI|`q`KOx;K)U zeUQiez$7nHAw6US=Ahyg?Zeck^njVajGxY$1f|yG5>!?gglVbkdW;CCa|zw0Djcc_&hQt>tE#NA z436J_W$of<=Xg_CSOxgynWjozhV%?bYE9|~0?ozonX4(3V(ICHb{Bura8 z>6o+~|FEU*gs?n|5~t&SS#le4`f(D>D)ZUhe$h%@_5Cm|!MDP(DM%CL3FqXT8>NWW zclX^IGrmD`M&E^GT~)-@qh)nlU=Wx_(ecHS)DT+_QVoI1=Y}7phUha8imEFJ(QsTMmRA6lc#oa49&=kv6O$W5|K@ z04QNj2t(T{n8-m`L0E@1X-brOv9p=(P5i;T>%h^NOBQpc3`%`wT`6nm-=KZnz+eTO1Q5|i^pU?`v+j>T|4FOuiS8wmY0FjCYW9^}&Zci$V1FiPYVI_#b$ zc25JBtokt64UU5`T;M)^uQd(Lx*vwxXmF(er3i&Ks?uK;E?@dekDwK13C zk~C(Y_!xc&T18)M-5dNkx=Ty(CjSg~ernLvZ}-zhIGek)8$s>oPC#Dh;xMpt{^vhL zShZ0j02T@eNF4KDl&%bLadom&Ft!4i{BKH+(a>?g(ZKp%O(oOOAFaH#O=g`QP9vc6GfyglkG6-hvlBe_VvFA9dP6kx5qO*|ZZkQJ_-v+avM$ZkEa~M^whe*S_C|kxjSf^PQE_ROd^^0R z$diha5X;3})>7C)=8|rubY`2VzRBTqJ%ssM)5}2oicW1MgUZ%GE1~(s;32b#b9c`* z@WGr(c%3F@pl3S~P9^mpT=sD4%lugEEd53p7V-;!Z+O!+Z`Pv}oslZI#SG=7Qkp@9 zne~#1b;NgV5zw~e4Vk3`cmk{n&lxqr5HD(~u^BpCxT6hGB13zz|UC&A8O}W1-^0R8cT!s7svCAo0z*+X( zm+w-WFeX!eXRq~UTeWphQnfYJos6<4!nzPEcgTMFF4DR%s!JI&3_Y4LC3M%KkWO_J zVz7*xUyJFGb$E;7G&%d@I6Ku@d`624am-V3hKp;k%eYr!=)8=)yZ%bzQ(=FTt#VYL zu!kVY?VF^MCqu4I$=B@?TdXmG&1%d~R;hXvox>=lRgFPV(>2r_NCJ zNJF1r(FVbx3=3)=-lSe#`hQ~(&rv!@7U6+jJRtV*MDyE-87*`z}84h-HGY|I^J-YTU+2?>y5$;sksoi`&wJiPup1U6m+nTjBxCPaZ z^4`KWgTRHff^Y6d&k(OOHPbt8yEKkDOZ$W9-OBPkYP+KC0gcutH5~1Iv>b81k%k@F zzdxm*OkR?gm%wisa8dPu=e;?O=}d$w1a_0DS`!_)`UTs}z_7T(4dShDhDN8Mv2@lC2cj2H8I?eMe#{ zd53&;f)HKmc*zR(u54WbL8s)HBSr@5e}HLfZmsu(JRfY;u~v0*BwWy1s->mnqEde>O|G$5pGvbv{s*9#yZ*nuiYln^-Z*v z*ZX)U)F5bACen2u-_Wtr(-W$NYi}gd{Zd}#-4Q_V>eTugYIZfpfYB zUHwL?ndX!y@@kAx&>vAw0AY|B8C$%Qj-9JQO1QXw99#fX}9`ZFRn(1Pl4jv|5?qr9{ z`IgIh_NvVwuMgzcKy|E{?!1HAV0^5R?qdA1gWh0xC=Ru_tEj^;&#%YE?6xgcrdng+ ze9KPFvE3|v`l+K^*ziEXYQ=^&UCS<+g-R?T>SVCwY|E^hc0tDFHc}*NRdx7)6Rt&e z3@Zk8ICG_1O;Hr*l*Z=NViT_qKE|WkEFBO!=jH4DnEt}c^ydxG)#i~gYnv263R_P_ z2mG5dg7$*`CJojl4;-`w`lv38t)-{2M9%6JL>Z-u4!3-q;2sqPtXAl>?v#@~v(jZq zmCXYS%H30h;1Ik1GBo%GVub99to5=(IXX>eWnYo38Q)Wvt{n{&5UjsTaUp~I|hPtaDzN%tBTJ?_ALh}!? zI}?V*bR{52M8zGGcG0RZ7y}^NLjbY|SP8Bx8|n8F+55k?KTl`oJwkjYePZb!+777h z@q;;UNOTP`o=3#$?8ShxRHf(P@W|ie$(cz$BJK}qX!3SigddtTQDG^cs^0Btj$qd&^cGGR|M=sWP`~9Uo5c|O> zP#|aVo&!+a{yk9cZ*>Sn2y{`#x>=P$HmvvME;}v^AqT&}nuzeWUP;d&LxbUq^4E)84zAfWTGGmdpPo5dp z1DEa728p<`VM!D{JQbEDf$T;qMhlNFXcgtV5?^K`)&dsw(1kV}OCt#zZ?(?5VqYyh zvR;61A7HInCO=@NNXd7LeF}_m(UDdPkb&o z9t`Ud%?7f}h1#X1)g%|jGC59C?@b5EVuq<@z_WjD_2TaZl^d^d_z(bvq(n$3r#3 zGA618`YHTWX+KWcgGaL+ro@bLkMcSQtngcyM5P293hD`Rk4Brs6j8tpmzb7C#`>@q zhkc*?4$ApoSSb2-ath8Wm3I&md7-ovm@JfeTaNlHN>v()q(1}1=j#P%-2hti=#j=6L6SnPVc6F2GZFAMR*8)P|{Cs5om5;Ph1-DOMaES=u+~ zWU>r3Ti&&O+K*gfXLi-V z4XR!tFu~ZLE|s>^=fVQLMzFGRiF}3H-o}X=4Ima($Axzfcy7ufZ|_BRuvxIn$4}X( zsp4Wyc3GCYX<$h~<3>xo-D6vb4S=KRt}J;;!BGt3GW!>iR}-b)gcoczx}#3LO6t=b zl+C((!zC~6|8i5aHO53EKnGzpxbF_XMfv>XHci_mvoDb*;Hw2H2r0aL0~?X^K5;1 z*on(JEe9xQCQE8P&rCN-7du|Tq$A%ny2<3)y^hUDO6pU250=O1;pzI|2ET~BF3Bd+ zDiq2mVLeUOd9LdrN#&w4P(t`)Wv5*R$WZSU9xOOO7Q@bTM;Dk0MA8gF@c=yTh2?3q z!tGR&~esz_hzvcdDSE zvWKm>GAZtOmM{!Vau_~gC+hMKsHyqZ{P$1tVlZ=tB?V=&gL^2&2&n7B^6`nA{cHyH zAq{c9V$v1HtoKFeFuSl`m{SRL=Vs3Mn~~h8Zfl~S85hNEj?07Z4U;LZH&g%Gmeu9- zg5PZk)&%0s)EyE$bo2vLa9;a`waLf27t%~S1-p-D=q`OmX&HaT-U}jB^s@5T`N$PR zci-5M&!rH>f6|>Kj1-c8%bDD>USMbGLgeJba@3v9 zwFhbg^_|NwMLR_sQo&*19Sg@H`40r3yUCbaUQR5u(#h#?<#kCl%VcH#l>)TzV9fSW zx~Ffnbo?%Z#Xj@s3Qpsp{W6sSHzx2Pbqyi{)xwoJM50)2}Y-VzqSkYMn&(n=qJiV(PqB>r<=4gtIf^BoEZs1)U2W4!#&N^i1J6%K>sFR-_~MBg$e){MwJW^ZX^{&uHQDafr5pfF0v6O4q>-p zPXZV<;kcQjE@Nw4P%yI7Ze%$ak@WA@Rb6*F+~gfJmi*XV zFdT(iHq0Kk?CZUhr=N7_A~P>l)a>iMrQ>M@Q0|Bp)=b2{ImjyE`U@rl&wJ z(jUxwvWR^xbOZH082}12UzH(Xb$?B?n#`V5g^pOJ0AWC$zZ8{D@1#u~eEE9b;-j}9 zcIW2iITufa?|K^RzourlFYGdW{dhwuvtgzy&3F z8`N0!yr@)*BebOrY`q*jWR09;lM*5-1Wj1}((lR>Xm0ru<#@EBR%RP1m-5_4M!{-L6B_APea zhEF?LE*4HyiG&l~^}3Bs2G_99Acl+T4=ie&$rO&1_ukg_bXA<*0aim^jCu z6w)8=bMqBd<@y7wNvwgh`R2esl1{P;r-`!3ft7{L?qtpoExa8^dQUx0>NPC3l$4vZ zdY_iQ3(rlQUp<3if7`2<8#@zbOKUm;$<+7eiHTBDQ-s;r_(4}UxMpm!`oyx9H1m+7 z2+m_;vL@FOse1?fRPNQ#l}0eRFjhz59d(CS(aPVM_;NWc1Qr+x`l|}q6!_ESuoHuU zrP5Qna@T&kzIxqt7C*2HUN*s}CJ<1?6*O;LpcVX$pp5 zWeAj%*#Za*P)CR+oSGn*OZELr6*x>!Rhju!bf{|_#dUK}(|Tq3aaP^Q!QY36**cUC z<{Vfy)>Sna^w14f@8ES69hF3Y(^#ALuB*lAV@yz!3XQNaS#?KXsr8rQ2V`j5JFw>N zSeLkX;3q5Ud} zB6L_yO*yFt)8V=tq5_Yd-M4cUN~hekdN_PZb24OCvl5nHxI8PGt_ZzT%!uy+Z z$jgDHL&5+2L5KetLVsAqtq*e+XUE{kp3pkcuZF=$Wk6wn?R&K!c6b5{09rrf(r-Z5NUcc5`xu5tJ$-uJrF3%rg>Z7)z6O1J$DUZU^ z>8}H(Lr|*!G-8pU&fZb3U9FsxS!3&+PudKM<_?)Np3SwCx2Lmv^2ZZSqAs}fhstYw zd>1=2S`F_o8t7s!OQ26zH5=ELy*{xnZz%f!1$tj))gijIj-N z84c>@Q#PuCOCLr&{94UAJbFv5q2CU|jMv?6PKkW*vPVZoU76;6{uXBJF0iU@&fTl$ zeTk2tFe%0MCHzBJuG#u8{@0ZJaVR_48BS^Fp!9S+7I_%@k869*tSPg2ozJNVBVxK{ zW!=N2?rppsr)xG+A7@aVfV7fodC8LJSmw|Lr&cO0D}IXQoxRqYsA=HK>?SOL6m(p= zaM$njkbBuoI!X;t-C3GD@tvv9^3=x=do~Y9cA@K49uyPYkCmc_~ByOdgGJT7Gb9e)aDMOAq=1oXTgt;P>v1 z${-UqUR-yFt-#@MLoPQb2#XajX}8Qy-U7~rRwq=~ujCjz%0C31zrh!wx+5cRkWRG~ z9s#P1A9)5zip$^oS3Sh10V$8(7S@L*1*>A3+ZYhH+Q(!MUv+1_%7ci$HA*4NgP&H8F~ ziudF>I zf*+iUIND?^}bTkftAw|SKV;}OSuTGt`%*A2?e#QM=tU0wt=`*hfcpr=EkfxnCyH%bqe zCZ@BaQJ)Areqnfh9NdJf8-(_Tqj6b;m><4NMy>KUWK--)SMF3b)Aee=p8$pAq?6|q zNWLCn{U(>T+T@GYN@0IFA(RcT5p$GX95W+NGKF63JlMq?LpX)_JOz{>6<#9w#ZiiJ z3G=frOSeIl=wr*U^TI2r2xUM<2<}XlgF}Xc1DDOA@^tP*W_DX52JK8&f*6hLX^Ab( zD1X>F?H5X^bq=62d)S35=^R@oeTFe9==_$_W$}FSC$bjwMM|O}am;#ADO=k`Ku5eBW@^(`%Zd5l z?8w~b?C;;-!~&k&?rFbRWds=A-EO9wPx)D#-(IE-et=$4@X%q~EB36>clS~wLF{G? zVJ;ngXjb9T-5oT@YAAlGQBQu|`{UZ>8>Y;I@?<2@s3t&{cmKw=SMLr0o~_ zS@5zv%Y9%u}q~!cJ>-ns!g^{AaOKi*^#m|_ z6&kJKQ@pxXnMc5fp3AoOBT;eDHO^e9$7%G~C3VF`mrj$Vn_#;QI?%oGF^dlcuo<=& zzr?K|JeDqAWB05O8Zg^*q4~^3!1&L|Da3noIdd2NG3I)ctJ^-Iw_om($IxDW6PvLO*f(l(CrG0qYjzf6F+Goy24c*)?T;V+`U9)5=*P}NUBk4Ou*@2iXzQjn|J9)13lfG(wi&xJ-vI%7ybhq^2 zob(bRhOVZlrbx0_d)Ps~+e$jMrAeet78n}UJ`|s;Ne_<5<`dT$s0SZ+r#a|J%gm@U zs8)=|%6e5$yAR1be@Eh{e5X1%`x>%8@R73pM7zl#!)zEJD^o-QD}()fUj-pz%w|Mw zVY!#DvccRpd3&N^gAi<;en4jr?f~1D3RW@ z(Fip$lhteq49&Z`E|k^z`A6J^shbDupfTY1j}p#2@Nkm&B!Ildg}oTOW{2gJL;pT2 z)yVvyKBqPNkLe%|2jP}$un9LiTWU6{!c$LDrtWO8Xy^kD%eybVh&nloM`s)*BYpNC z(O*9!JRRPE#pTi zB`Wm(awTlA`3S#B+hVOk^N7Dp%-IDi-V)1bC)e8(g(+Cx!l*k0`N4QHn54KJj>Tz0 zb@P+d&5jdidb&9QUC9V2gLQu4+tWK{se5O zse&!JQ(}6@XK}z|!y}VKU_nND7$Wdw*qijoYsf20)5+uq&--& z-_hjRs(41}0)3snEtgx$9fj{ot#2;mzZ$)_KE*A}@oTzD2xZ4|l%TUxjlxswS8of; zn_n|>m!>7s%<5vPS(L*fsmnLrh&?n;!cC1tUS^(2B3R1gs=1q-3?YVYoK-Pvku zmAFQiy1(F$aNpms=52TK0HCB!&C4C0)7;ga(=Czj&(k&mpzhxU#GJuW;2I?Dp%qZ2 z36_H@W1)_EHyp%cM&tvF)w*`#jA1%3`mut;roY;bzaw?^Q%dW^?zng2C-E#l&A_lv z(2q9jf%u}uDpbM^p+f@!;O^kUVoi#3bOAvslN*((PBum9iOfZ+l5!pHm|&fY>r+!U zH)dY!SzSpp@(OXLBR8hvrK8}5xq3=P-_f@7(eKxmO{xtoiF{?~csWX;F^NV7Xclr$ zQ_x@wCmMIWzW04vP3NanQ zt*LK{3XSpXE82e>f!kMkK;AMRxzQHID!Jg)Oim^jde9vy>JEE*7z{%$tcH2y?~$Mr z90o)xILHo{y~_5VQuqF0oc0Y@zblEXyKDKAk)695qvlIgyUK8AEkjCl(?f!#*4sNW zx4WmaA~Nj$s!#Y`%t7VgQ8h8f%huf8<4E9oTUYmDuq|1i%Wg1Z+A7p(&LacT)s~t) z-C5L)%bMkOUZ}ortpd||-E-Cxf!_zcvTcLi2y3Y$koSB(ko%KW{P8s!DyS)s!G-j8 zgJo#y%uI?pZ6-H6lQDnOV42aH!4Xv)-P;(e=f(~#ka6b;(n}iL?4*@O?mKOCbESnH zYTKlrKS#_oK5+H;$aFe5RS%b$D(q<2B;%(oorG!No5MWBMtDYjdpQkBYix$ zFGL~W4(_Lb;6Nc1L-bg4pDMHrIczT`IdpDqGxlbC$6h+ zEYdjT2Ga;HdH5sq%pnZD5sIa*>*IuHgSF?lO5J1-Z;mj5b8^|Dyfg1)A1HHX^J)kW<56+$wickWx}K})^CZ_K{HEl9D@*XL)HkKU;` z$wY{OFL20<@Gj`qRZv`5Rpcme{6oXiTMBVYqiTe!GKz=~sKcPXDed@@PjdU(CIpW3 z7uv^6XQmexGl~`iQhcqNe)y*#dj9=f(q4@39OwHMyZnb62oLA+>#S6NEYkcvGH?{1qPBmXQ(Vr-M7;SjgaM?zoan45JIl%e? z^;0B5AnRj8WzXGq71+QD)zXjg*6bonCM74_Pss-l&|JRIY|uQ#8%%gIhls91mRGh9 zD`JY^tByhYBVC@UiW+yI*eGu3yu&Quj-=MN{evlkChn2lxw2%kJZs|1vMNf2l&E>3X1Y!MFOZE+Wq+B ze46Z0y`P~kkzYLjdK^mKCx4%{Tj->)Y;>q-f{R?Ma7f$QdCN7#%f7`3eij-PIwf6u z^xZ6$SGXe-`kyidi#bO|pzNt;5mb!AHR*fvKFk#U?JlNUWM^BvtB>}!9JIHXS_I>NVIWH)dxJf{}Yfwbkyfag^6~$ zE$RS8hAAB!+Mb<+vVhdo0kjSC9noQ4=SapZAV`W#$Rz0UJ0yB-lo^yZcR*T%yhFC2 zmPp#@5bT0HK#PnJD>1G?T#8&tWFkc8kuYX2a=TI{Zmnj%hsr&TxNq^P>cGScn{f8U zG3pb1`~_k~3X`}CuJIKNg`W&Z=*s*_tn|&H!S`O^zf*#g?ltKo1Q3ul!v8I;FK!92 zG5v3JkfQPSkMR1{11d!h2ih2(u1=$!MjGkT5Y0ogj)Cq6H#tysFduRqpk?ZBDTAHI8lYMxCmbCYXXXb44h>FUlmbM^)H)( zTMNzz6=OH;LEM|ahx|mYeW8$d`D>n%P}#_nVPd3l;m1YBZ|MYaumKc92U6$?RCi=G z6;v!mdzEL+jtgfFSE7v~S7gd6cuJnFB0?&+xw&UI4LAMY-!lQ*CZIWAd|NJTi4bHa zZlN&A)t9ZxTAPyVgW?NJmMhCwmt2e(U0v1*A)iLo0|f*MG!|p&Nm49!6xFHa9F55%3; z#z=>_`_3EUC>mkZJ7N!3rlJ!v$K0OCr;bG1x(Z>Jd2L7VvL5*QCmkK!QPd^UShy_B zrXq+I(+LyPuH#Hm*k8nwTRb&DN8oDSnUzKQCrWITPLh{uNt4=S_?b(uO_ksNpj&8} zcej^Et&4w(_9diNRC$D+@|7K$Y3FCC-B(ARqP>>M>X_9T$~aV52&Y6AxG9&z$0{V{ zJ$O03fW>AEI5xq)#z(h#=-cC9y}Pe^{}PUA&VU@Bxb&jfegz?;e;_+RX$!J1iab^8 z&OZ=yqP-9iGMW3kZoYOxWg@?jFW&*%NiRT(v`&n)lVs<)I$=%(PG`oI(Y~IAoy%UwJ8xB9&VR%Td+TYp3gz3$8dz3zN1Tn;8#3U3qH7=&AYRLf2_ffmnxPY#do zY}paf3%!}XG=EZx6oAauRqj+pcgwL*dPvt@bBLF{d`~}9;h`}4!C7_)s#|CBi-ekg z=^mMXr2!p}njg8ApN!VG)D~6-#X0j$mNIYkzN7vLym#pyy?5mvzIWLX$o=vaVHZ|J zK#`+W@FD;k-}~Kre&@Vn)EHI!&)RdYHRrBcdu{p! zI<&LwzJx9GWSN#pn}D%pvfp8PrhA1vmZlB)#;&$%i%i5l7d)3>3pYy#n1m4FP{NA@>JBNtIyRRZi`;~BsZ_ItbbT&XwnMW zPx1=wPU~8WH$8!Pi+ezofbCNJFNRN*8Ls@)b}6N?kd5BnAhp63yE%rt8UL zze${mB%ib3g8dPOgci(I6Bv6wMRtx;-I7$f@jA0jNk^IA=G+-j@$HJnry9;65XDYh z4Poj2MmLya^RNTz5wgseWQ5#<)6J<9I{H+YroAqjr@dlkE8|tw0oJYvSo;B4j*VHX z5$@4^xEt#Mwq2Z#*Vv+nx$OkqggoPEVO%W1y9ayo8Q30_s<_ig8w&gxDuNKr#o83< z>FkX#A;!AIoE|-i)S6hqm$?;UCR&%iRyo3dG3fX7MU*{g8tmSxbV!2MaEXG=6ln-q zfZnj-z*D7G5PL^trZu4J?sPmvp9WaU(X7f(^yfE2T(2Ys?clo8c4kp`Q!*H2>qt9$ ziZl|0Bay8RfvSwNY=+>S1ONtg(w%;;HDfY&ONYOJZtp1a#8!}W-)o*&fp>CMMwIqz z3Qifh4ietX9N>)q>1F_RW)T?bR$Ml}Ovif_TN}k8qSF zib0cwUZe~nB%Aa7u6`6Q4fWj5fI=O}K|^HHA(T)X`m0rZjM9AR!5Q&S*?C?I zDrTfS$nQkPYN4f_O?9;kmAah}m7%x@EQ($>6fMN5>tg}?L!acUBHTV@UypmTN@&!D z=12>@(kU-)$l|lB?+Xg!rs}OYL%ruip`e#);S6!S24D&1y;LVO`|*|F-*xEjHEGJ6 zpdX~DeG&wll6qYup&lR?c>_N0EgYLb%X4qmJu5L6=^QYO9DWYnEQmT|z>nsGBdotz zQ%0%C?y^pJ#T@xC1oa?}ykuBFAl_*#rxDxB=Km^JIi%D(JQd9@fN=6v#PGn~;}dP} zd~i%XoRav+O5i{VbWPysIkXMhy~MH__jE3!sbA3?-ZmPWKK&0NJekwp%lRvWv;S*7 zmL$N&LD9$@@Xue|ssGYKw2>Y#DSc)*dRnVWb)Bm3F!N#^g;ua=E%j7r)Ov;}5GYv8 zj?BnIMQ$})&%Qes-C68T>&{+(oWBA;S@gQ?$-|T2*1=Bbp1Ztvx^moi_%~T4C_R855|NjfIN_aqqq^U1`fEcUf-NP@D+}!q&fT=9Bl~|QN}r(F{2t3*W!lPa zG&6sk6s`8{F*1XEpemR^owYW?<4EufrQWI=FXfe5x4NEGkjH6ea$KFI2rCliH!i^+ zFDhUgTKMyzr47GU4GeUhSliF$(%7(9n4>>i2PU?#h>f6^w)y1qmf^cR9Lm;YyB3%$ zJ9NEy%#U(gZin|4fW}hI7Q;z^o-#k;tu+p(+?TFhdj(0U%3pAv zrXK*)UF;YEYcZHD%;i5asrSjvIs}+(N%6rSvpKHc_&QJ7*71eBeJnKf z?IJ(0rJaigUWUIDm?xl$G7ZifEvXy-={eKpXR>K4082Q^O>RjX-fo=C1d?&p>;1N} z5X%URPMstVpkU<O|3~YEv^pzI@ z**a;Kwg=uo2`yfb2rXZ*a-e+5_gFG9UvvjBUz7(lUeZh6Hp3ckqFKMHdfjolM-b5s zxly=AmJ{WCLQ|7f$aTrO^6y)B{eJV4C(0;kS6I$}puuT6TUuu_Ca4d#;%&Lq=b8D0i&`27 zYWP1Tde*M9*rZWCDKgyKZ8z#CsEEkvHac3Z>A{Lr4RPLK5Dn~w_o?NzjnN|trY6*A zldD~|Q!G!@0JV2l?6k`Cd@sJ@!?ms>#GC2HL$~9bSHHfrl8YsL6V-Gk6VK!iB4LXH z$$Q}Lt4b@zHz!6jjQM$dKP>cpCKbn4li(TA$^c(BrWoRpMTv}Pj9vfK zgL;fzUsWUdiU0t~FSbbi2d+U_$TM9G7Sht*Aa*WfY zkK}F6paYDp=}y=Hr2$`^*$^J=HF=vKeO*Y4WhC+B!DUpnIkF^+sF_W_`0iTu zrThBZBBDw1aU6SQGQY`w#?a*ppv&nISJ-t6AR^CF19VjmXQ=n;>2%svs#+6FH=r1xodtx{WW!a@u3tC(oRC3N?ZrF5gqCs2A~No59~4asy8<0qMw zBbf+}(x>!1xjm!SSS4Cw6Bl`}4@q5T6yMPN4>QT{8|WWBkC)s&;iaJ#Z(d-3d%lyo z#`*`d^M2l$&x?Js0RD9)Zl2!~Qr!!v9{1Q&ebs#p8SFJpC^OzM{8Qh|Yi|3UAwfV! z{x0;0|4ZNgr}6oJR&W{MEunuxxValL^MaBJibj=@!v@$?qUqF@{j??pU3!ZEBu~euVAqT@iLgZUDka$_O(t6fGzm&n^ z1&(0;hLHfdea6ZIEf)IL1E0TS51N;1Q33k4QBi=zWyXs4)srY2bstvPM``FDr@dA? zlp*{&^{)l~d8-D`uxl;sqbKO2wMbN0t;f`)h2A`jt(^Wm#E=-TDbMoZE67d{V^%jrj;@>UlPs7#tLxEI zTJW%C#I-h?Sl#ZAVMUn?yJ8_6C}w%Y#HYa^$Q?4kRSx4hQ<)U&PD~FK2j#VdZjiWZ zb4~E&<+87{RQzByEAec!46GP4cAImN$Bo2Rd8tXqx0lkjV7NKL#ZcS$FmeJellK1p|F|3t-1KTbvQRuA0O(wjG<1)l(_ zrW!D^0OKM<&1?A3M$X4J*(&~9w|`VdUDYSHCGcwU-mLr-MTNJWrF08)EicEJR^z6B z^Wqj-eq55|+^GnM>JuBNN=g4m>xZnw@cE2d=QxPy75YWAS8%3j%yKmBd|La394PTb zq-h^h!&ABzxx{1fLvAKuM)B4+kmeUKQJW|byW_NQ(wkO^zG?&Z;8$bu|w;w=ff!167s?&4gmiAziQfgjvlUQABJQqI{6m!Htj{frIb}MP2Se%}Yj%(Pr7f zHRy=I4Tf{$k?CSHOL07EKv%^w>(e6^*=tFB(o1{-SvD$ z7ddv2`zaiUbKNf%SSpg4p@9zEZG?x>>K3<+xizP@EZUt^a-h0wF_)pwdYi*j|=>ZYoz$I#y&W{q+KDxQ==lh_Ii>8V3 z8y%k|b_n6Q2I3C<2euOJ--vp1s6h!xmb?JI;I?^_uETcLu>$OtfWh~L{FLfyXfL4 z6dA@A+?ASVK^+qA5H5zgEJirROXs@ov4(zaP83U1+4Kx{8TrK$(&gEqK0@DSb%5G- zE3EZqn_g$kzR6f%!92PHr7dXl%2*o1k5|BQAj5Ef3F4zz7?#+Lpry4VcKwR)hU9(g zwr6S&h0;O_L)k;BYD^4u%oL`ZVcR=sM9-&Fq-K5^1G!Z53u!V6VBQmvmmc*6JtKeW z28-6z(U!LG1@|%!+ufkg*GG7=HSPh?rwJe={%#lpaZPdKneYs~84L+^E<|n`fJU$W zoe}Rl;{_y=nQ)0I2s|zDjkEmdm0p%52`LCNsLT_wrHqsAYdV-FUzt&6g_#-72-v6m+VeB%X_)9jpCfC{z zjHB5l)j`GlSAK91+2$t0aH!-EBt1AU9Ew0~e@W-i99QG5f1I7EBhAhi)2i-_bI!mo zJ!a*@X|^lP)XmJBpOBlTLvMl@I_mYRBogzU~Qe1*RgX}`^ z*f=Ce#gMbT6%z&x%)d}F^W`iMCprOVwIb*sz8wPYnAq+SWQq7V|UvtNDp;2-SvOa1wH-4F}1 z@0j+zGF!6^llg|IJ=2$-;bQi+>fi9Re7XatOn-7GH|ubbb4 zSbfSAAfx+WF@QIfgcU1$mc7K?rQs(<#Yq6pl_g9O3`z~_$OY973Y0aWjmwha9`q!yR|J)v=3GIV+g#P8fW|9E-K}6ImET|eoN?J%Q4vzuHw7YIv z5b!sad~Cx(a>6OUKe0u`&I-C11I^bp>DNW!@A2C4iw~_A4U| zpL{I%(+}O_zKi49cgOq7-KT-S?duZ~1g`)?ycVW|brY$tCpz&i)Gdt_kXD z$QPN_gfvhoDtt?y=oxm6;_zI!o9OTxJfgS*r(5zqktaXxVaEOz2{LcQNU)C#f~W2% zYW-IW;L((-o8-htV^n2-oL-BM&f8A`ty=(IpcgTOw_s1}_LvJ-T(6>0+im6q8LDpS$X)zalH@M*8YQKBGU7<;!05wz8-5w4|MQ1F@HNw8$m$6B{BSWK`}FXQ%2Ufje#?b#^+viAmPUKuOCZBH7BZ{w;ZYXUjgYIhChZ&JZtEG%3nI z#`^@H_DYxL7c#8Pvoa>Tu-@PMGmY)7L#`mFXioh^2V9HAV6-@^D$Sx@@lwfVCz~fk57-&cS+juE3^IGeypdK_vnO$( z7ZP?ZBtH=ADM*|skM+v$-24dBzlHduL$=Ca@oN- zuH1kxUSiLwDA5a*j2!$y#7L6#yy|xNp-}XZ_+b_&IwD71G4on}YCL?JFuPH;%s2*m_OcGqGPjrDOMwb| zDNw_uGh&7HU3O&V78ajkKTDLg=-@+hpnv&RrnC4!bNWK(SHfu0S-@RAP{du0 zcBe#?dY}y7vvLdBISl$}Ih((@p3j!S=C=Cwup7&*Ka%6}j_p5ptMM);?;|p>TGU2zJifJWlgHOHYGeq44IH3#=YIY z!Tlst3Ay@B`m8*dEf@53Nlh`t1zkP8T3RiwnmEwXaHKh`x|m8&VX%xb!PS1e4ATbNX?nib7x~+)~3XIUh`7#AO0q~NI8&j57 zID1-shI%O?0o~CS^MO|Sl)Ilp3$i?9kurFjYgqGcLRqqx3yh}6g35L{^VCJm&cshNwER~xE zpx4{c@k1(yY+`UH%jjEZ6Uw8&*U-E%r>o&cF3EkmgfUvmo2q2uC(3XFaC``N0mX^} zY9B9Tl)W-hcAV}Q8~%4>qq{C%b6z*{D%TMUiT zAQv!L{PbJ-Q^~g&{(#`4>KcFc)fJKFs#lV7iB^kzc2n&!FbST-Vr(1fiq+v*=Q^OQ zkU=Hd4{4{`lla$%+EesQ57bcjM})@43oH%vMEZn=UXv0&-{>7G+~1TWUO(O7j@m!G ztVqrz>#h`LP=>1%zhvqNz^EC7Bp`nuM(RY&`!jyyJA2h!IsKzbG>%n#Ju2;U-;d_^K8%0qkC=Dxa{dl zn?r^X{cb+6(7AJNuMJUuh1?cv3*><|(fU|>I45#c|To^{@m28cX^RS8LKt{6|D^nEF{sS30$B2yUJQwyO9MXzDdA1U3w6`~Iwjth}c;rU3W+O{Pw< zrpE%d*HWiQ-5vOqtlbN=d?l*jrgy72fVV{CpUGClS9gTOdCHeJV)zYan5P3^1|Cg4 zAypu3LuKK>!*nnStm2iMN!txfLX=?BLzitq>etw89t5f?xCh{B!E+cf*_k2lL=-sk z#0=nnKar1F*uvXzSAM3)3}aV47^qB$<2(~~YSz)fm)D~^rcdRpsskmg`z`&mRM=2J z-gxG|#tWA0x7_jY2Gu+I(IzOg6Z2L@N~}@H(p_cCZ;yjkgkRrY{dv4kXSQ^AOkTC@ zGupP*v<@P3uohdRn)Lh4N6T+OR#M*xR$WfPB-;A0R=mGr!D?g7 zc`2Mef$?f(+?y~mzX);#otWw7CEaIM$P1KLr3=|oG8Z`rALIS5zQ;L7OeMErCK%8S z{Yq7vXR>zaJp4vHKG34V8Ke^P@B`;ej+uLU z|A3a0NSlIGkCtv5@L9PoRIFjO$|y$X#a=F#rV!75cf#%a1D#Q-$X2VoT5X=f4)Hf zX~T_X+>7qPKtRkPK|qN9%ZAGVZ0(&q|D$B;O=UwI>2n%!3AXHQEzp}Q8W)mAbw zJs4zIO6nsQOEwRYcp?-zo=$_GZBOCMe{PbMJaPKr`Pg5*DHyR6woS4Y>@wOh{y zU)SsN;|_>j-yCC92X*&cTwf>9yLBfN3B%aq$A}objAlwhB@q3Pm3OcJt9z+7Vg#qG zM0p0-)IY~w6zG=xKs z9N6OYP9|Yo{7+WVLKA!(o-&%Efe~^L$pRNka!1B^YbHiDaZ8Xnk^9N$j5F>jJV+7i=cI=bF9J|{HJ|tQ#3nzsrCtRYgL5`pS8_l{<6s_ z$R%1~2+6Qp3nZTncv>)w43ZQE#n6ID9uFBQJw3MNSk{ zbvd(G#g3BC!ZpI5u&DPQ1}=Vq7ns;T|ogmc}2ys4pgXlocaDn4sTGY2O8XA zvEq_0ai_b#qcROL&J0gOW-6j?AdE@x8>w><$w+I(X=@JH#|K9t2n-iuJ%~7;N%zHG zRbSp||I~%PYvC9mDgSzTOK>9z_JA~)qm`Zn{V=p=DmISs^9zJ&+j>%YUoW|s_Wx(b z@ZpFU?}q{bK|}!oA^We4p$ssy0hs((p=Fk4q>hd`z8@kf7b}mVG%T5pM(#?S@Y)3+ zec3GzC@muli`Lv09_j#^CxyVn6`-6oz129a)1r2{smsIDVWpwNG2ePt*&=`P!F}!i zf$QT_eE%^mH4kF6whZfVc%-b$h+vCYeG4T3eyzo4j-~1A_T20C=3u(D3)cjL@2#&RF`j^N2x}2$ule^ zS>=h9)s>Re+{V4QK!Etu;nnZly1a@rq-Hg5j-wiuJ}S;AwZ#fu(N0m!5!-^T%~~xs z`77JzoKE(Vr}Ju^KxTLnA7#923JeABn{RMoWQQk?JOz-gzyLqSS~!J3=>V)88TEX# zcmOZ5h?I3Lkl2ejk6c1o%oy}3MNx*QP;8z&JaGHfYJ-QVtBM3a+6u@%Ur|Q#-EI3b z7n!AvLA8sVf}xUpbP1q|hVmP8e~3o~!?lcus~csr?7>0ax_xqXvN681{)%O3v@ed^ zXpP|jdmE7;!UDB_IVsASxtr8zuG9?38jux?aV`9tHPuPKjh~l)qlRzbAb9?+f-kvt z5!kUZU*$kg218_pe7w1hF+a(q=OGv&sj2KjgruQDb;U#C_;_QKEkWiX&3GDR=@^+! zxq^{}35V$=aV?9oBRc)NeXWm~FCOt1!1>7j`EqsGis+y-3YNK>_CSlgB-rY&4tX+` zDUjVOE{iVQzC3(3;>3!XXir~O95ch6o0VkC$DmoC;N{4=%=yNQ{bDD}8y~ix5w!Se z9pN8QHlq>`6+CK9c&#pHE1lIWEJx6U3BLYL^s!S?5*@aTje5K^75N@5%m?mwV>O^w z9}Gq5fxyDd3`0*`VMwwb){*z)X9iS!U`^P9IT~*)CR&tHP9VXBN%${ZY5X(I&2eCn zTarTs6UFLI>2AeKdLSlK!R~o0=>aZGceo5pH+bN!E%;6!0Nj3P!?d$-x8sH2BYzj* zV=#<}_KqZ=d}H!18t@JHu7LipE!4NL;5~yiXq|xu3!yN6K|9i`%l5LcbH&@G;>6n2 zOQr1OkES#ex!94c>wEK9Y!BZNoidR5EUiK(qhb6)cW7R~J}J#uSc^nYsa{Y&X@#iV zN@u?qgadk{EjqPbptk*07z23F-|@fV96oF)nh;B5MWTO&peXWTT0sXs%)sYr)2@A-D$hYw3=rCe&yeYIpR#i;+U&3ts`mk- zBlGCuxPh3mp+DXcFgsP*SIc|YDpI$ZlB;9eKCb9F%MSoKK*qn{fvWN;aLX5eP@Cp$ z{l+2Oufxf3&=Lum5xK0>x*d@LV~vTIPoa%RIycK@cS1eM5JR-k2SrY8#5+{7)2Y!?Udtp14o6j30 zrh}ibqjg>?pCedQb40x9cVgK(bb-^e@QjzqAlUXuOl@7xta;(+G@8pDL{??2zmVRi zoG0H_G|f$0zvzCCpO5lZ2iK(;?pHbV1HBDAs}45%NzOUzyifQLDT!4`DDfG5U%=O!-KkBbI;ua zk6{l8`uga%Mq(fy)Y>plTvI7;I=AbNG?P9|n{E@a1RkNrE(UNVb&s~u&{_GAw?1~v zq#$PF3~JC6IR_tBf{4e;!YcYO(Tu{f$17|&=u9;1B?~W=zaS6uix|xa`!SV~s4KTe z?+3mU+Tj>%FgfCYD4=jXM8zwJji@}1oO#f4+wdNU&d)t5*q*leoQNuPTob!p`J;y% zuNP!xOQabtS<4-VA@n2HHv}F#Y}mW3&dk$rNrPOtIsg(7=mrSM9}A^b3Ba!eV#3XZJLY z$tpBFs9!8D>L^Hby}yK3_*pux5ys!V?g?f26~8UavDpJptJ0kNkKPh;EDdbUXo^=g zc`z)OhA0wX@1BQm`u=dem%0{xYPlh3b~Suo0)3=_^@s@Pi}C&mxiYs5o)dvkuO6C0 z>Jh8rZ?3;2rONqXL2(*LK{v8F9rtvLNuo->ozTWIy+&{C7MiTR2fF+B(gW{xXEOc1 zJHa)K&5kf`M)P|T83Qf!m7JwZW0lD~t1F#eYb zQn7G#F|~KMlQ41txEpz@0G!+a|J>pitzxTyDum>R0ke`4Bpq2wNgbh$P)Pj(`l3D& zj$;8|MdoMKyi$Z`c|GbnF)_o8EctHuei6$&y`8lRgX27Qb}*Xw z%NWwRdLdym9pClA79l-ZhLWpXj~Q*rP}@uPXd*&Ov%VEi^KRx2 zH59d5qXVYp%1+p_x2aeIB6MuEoS!qR+nxL9l5@j-jhe`YGR_vrAi^%q^qV+1LMW`x z16N8fRbGJRGW}{xU`D@r?aNbm1p@c>cUniyXJYv$C=LN(8E+}8@$AsWM*0IHR+`~M zjG0$BQ3gR6(=kS+ZwX-$b{h?Y);gB#w+yakYE0a~cvC|;)Gf(2EKs6pFW zm?D+*rg`mjv?zB^*ocPZU@-3~jNcw7K=>Zw@l5slkx&ddG4s1j6Kxucp*+Q*3VeGx zAy^XlLQg*qd17X{MW=s^5o`Wf7t1V3Bxp53!?Y+d7lQWkDM!g7Zj^|g3B${~=1|CV z+XZLseBg@eWjIt^{$-F~Pk#_`jX%U$t^8?i`i$z7rEOwWZL|m8uHr{!@(1f<*hQH~ z_yN~AH0*t?m-&yO@(7LJ-Q150Qcsw~RzW+_ zKTwYdTSQmh%7ggOh{sqyPR*8Ll@XxeWm}Bvr*Q82B|Z#iO+q|ME9$9c7f%*K^6QUK zWR|KQ&;S?8=;jmVvqK7Q)K$3odvjC+-}nVqlEWtcak%)g&ymPQ*bQg)VUCW3h@%*qbgIjrApeBE z+b^sI<}d7JVE}qBPaFPJn0i29n?4ADS)6f5HY+7Tx5+Y26UKR!(k|L~8 zW{*X=hzN69Mz7YL4f>7i6iu=kVwuvYVy4PB}36EF(m5n z`vtDT#PT-$dB&@s!vK*Rp9nFSh7_^0R%RJPh$Jq?I+h7|6`Qa7Y0EJH>VgZdIU=y$9GD#+y8+%4f=(Hl>jv z!u_H{I|=Zi`{B*O>mUi6EE-X~c5o1+E ze_bpS?QELz5XzkkjtMA+lA`c)9U7lsBc(2cBtb4hMN%n#R$|6>Lh@F1{1GVP5Z~g6KQae9 zU3|X^D;Ez@*u^2lA;=++3{8zp4b2R{)nRD{kfMU&{zo+U7N??r`x^~ef1?5Wzg|Ji z&efLr|Hi)tz`-b7MFyFwhmc`YGL9HoLZGDpAGSc*y}FY;3( zhHTn`^{rFD>UKK@E6h+}^Ju%{r0t~I-SPE+59`pOXtx-BuWr2A-3$m)t!mmEAOf1G~+{{ z{8Xk$YRjAL6^G|3bP7{4-c2@*)7Na=a66{Q0A0GVcq&+f)af2YteGT_KE>_;B`NtB z5_Hq{)5z{4bY+}!4wZf6A4_=wn2oD$ieSrIZ!tqx+K8Qo!UE99JVvmZg0K>9J7ZTH zjJTzEkrX1Ht!$#Tr?+pet!BSa=W(j4HwY3UrpAoiefhB#%`QFCo0|2CVCT*%3op^< z@P6c=`qhyhs)&o@c%lhrQMD`R`>aD<;G&$wbt{Xeim{`gkd`y-54ZZEhQF|>{M7x7 zIpi|^CMo+ZW$d>GKvRfndVzZsCW2Z`@-X>s_&2sP#a94LD_P@#o31k?^;JTb@?8Wu zvcVC>;Xo&GmJpv9Hv%gc0`r&wniL6ll!#|=lrxOmojZ3l))2e&A`{b} zzLe3l-;GxAce5(c2ed=QKf2W(_f_Y=&@DdIATs&lbVMP1Ii z+FP279|R#Wc&zxIHf5`3r5_SWre-P^aKpw}^OW>D&0BfMCFi&P6V4&EA~MPr22@FP zKjE+zg8}d)c0e0(3dF2h!AdE6s`jSgPgu(uXu&R0!C$w(Qj;+iPgX_0wK8xV9 z1O3FykH;+n%y;$Z++xX!?est85>feyQ&7m7uZo?d{MPo^K4+}y&jZWvI@l#yYJWN` z``y$2X4>dii|&pm_9Kt{O@uoyw~}$uu{W@!ed$1A^PzQzO~+{W&sI58q}w*m>ZbL5 zFM5KB-r9+;#Ny{X=`-7k`+y#>NJg6g%houQP`6#L_{hzdK>Qn(R6Av%bu$OKy@EqX z;`)-lds*8$j%dK;MMpot4z!W%kaaThD{J8fN(V+3*i@m)M!R1J1XMw?V%)xXz~dp+3Gr_>T21pQ_NXmS{U|?+S@H@YS1odsUxe*PjnlzsDL}=> z)|DWDgH1xW#FXI!e~TIfFH}g1SBWTVO(A_JP8oi|C*5DT06o0JBxU#kD{LI$keJUN zxQ7Abn57?@A6F`vkEl4FQE?)f^&VPnsOB+piRBDwxmm>NbQfhuygbe^YJV5u67zt~ zeCIx1sFI$|Qky8Cp%hM!qf6pxN!tSSzSS>O3CJBA@U?jCxBn-h7J5@T;r|AE&9{Fm zS^M7z^*?>!73E|G{<`FiV!i0OZ7|87@J3*VoF>d;x@i|R=OO97LV1~u2o{Lv-Z=7e)8Tuo^sE^Sm z2~p#O-zsJ4{s4*0zv=R(EL9S|Gn%`)|0{g>pv4~mRxb!hDG0w zI7CA(g7zbeek42rEMcgq=1ZjhIz#pbBj&kd^T(mEK!ILDH7|1wC|2R;y}$Nz+xv>I z;r`*|6S)`JO>fdxIex@^IAI=NNotHqKe z>yG+^pVxCMFDgkD*1>D#OlSKJcRZPl$7}-`bn7^S+2i4l6rQ&uiD(`qMB1@As+py1 zn)P(V_le{VJXVJ>uc#iiW{s!p1_+!c!;d2+|^JxhB#3R*1tRvaW zhc@PA`3<8^)Q@0-l8?i`BsGr6piAWV+gc>#AwBu1f+7@x{hsWZv?>(dymQ&Ynglon zv@4nI2+X*Y8bL+yT6DS8_43;3W@$qq;;}13)PRJ_i*a#8r5$K!!Fz{Ics4$_< zp3hxBQK;|`4+0iY$5=z{Ss=~S-*P>9X8X!7?t^z?4a!jKFOLS%8bI$a{m_Tqf7I>_ z;0W@FjHY1A?BBS!kBM+s%Unk4^TtWP?FJsb{0L*QIgCVHdy=5$PeHYtR+T+hkc=3KMGn=7eLiMQ{W z%6`!H;#V-0X6?&eqAFmFH`+s06RzYTWfhwszBDEXsLWm{*^lbH$b27(7Vb%7kZg^C zvWYbAYg}I==OG+JTL34Kuw{h57a zM?EcFip@N-;Fb4_B&q4BPwkr%%Dfbj7%HR=_14-nRFNz<PQ&kF4~tH$K~3B5t3ye1Jy9>*ivs$+9YUM`vRRMdGg z37wMzs8+9TO+&dnI7#`jZe{-z9-To}=`!$S?vcfdD?D>mm3u#33Pu}4F1?>I4~u%q zcl<(cGMG6}M;A_}Ma^Z4W?Sf7%K|Dl<`cV0nQ*$)wZWgQRrPTM&sYUX$#7AA- zybQ%^C;z2v8OK*nSch~f#!M7Wyx2gk9XkA{_pttzQ8NynNsojtdbX8H(S}={gtpe| zLqRzDgMnz$C+@9?8EA3ZAnT+=0u!NBtuo zx}76wT(p9>Lu_MC$h#H1Q+Z=H^bQk#q&m>FbFH~u zg=VZDBjrB^I&8uHPMyj%f`G00L3_~anB(%9Bd|`Nkl0T}V)*6v^dn4vypO)!hymS| z1CdCa9Y@`SV#=peu*h)#u#Xw}@Tr3`b0f23ou|X{#I2j;3tVo9;7p-XIMARmw(?4_ zCov?8mD$GNr9Cp6_u=+O`lX4`WdZNtOB*8Cstf@NgkuRv6fp7H3L_qkdfAf02*udicr-%BM=@cbswnoA27TRl6 zD=2iBAn1O`rgL;cY4RPLcpFxUkKFrp0x1=4kpsG4@^EiW7^&}`n1~da&xhhXVYks} z1@(mMqqyUvNb5@9`jsJDS&xNOjz44BcgRFP03)`=EJT-~7N%w*NJs3Qb&xAph^kJF zVcqu{@4jSFwN14luthN~6xVN!TW8KqRq)y#W9YPx$pQF|0ZTBo+TT+0(>)K>mKz9a zh-(N+%BiA0OGIPU`WQ2xlJk|l8A0`<4%8Tjr3l!8XZ?oC@oEgi;nY}(i$~bQBBbEB zCN#;JMwAn5AAyiZ0L%ztMYI6skVcu{Nonh>K4!TuXbvx$hjv&QR81Ts0H-dM z+AUkd9VZ=iY}>YN+qUhblTPl~wr$(CZ6_Vu$=7?IbN0KR=j`*1ajpNg)~H#t)~vc} zDHh>P0)NF$<(*tVu-7g(GWK1UX;ftOV%ohI>)b3uMR7)%4g0P1uJ4Yp9E5F#yAB1R zBAOHAHj0VQ9?r7dov2RfSE5~6C!Q4DkPmxBtn+=+{gq17Mi?BC{^UnPK>yZb@pmc} zv2}Lzkhin=pND9J{+VwgXCxQNL{wHj?C%KB=&s?yl&}p-l@`l`74A47CASgCNY<&m zRo^Si|1;s-WDa@u_ey!%>PTHR+1|?51Hv}!7r-b-O@o8cS*ET)q9h|ky-kp}bOV6a zCvVsbZXz+@Ne}bNRj1%l6rGX)g#)NjJZZ(1w!$K2Zb|F&XAa@5xkKHL9?@j@l1|SF z5o)J+dUD+8BOeI<1H|+nQ|A?`3RqcT!+g_dI+sQaD-Q0j%@+~ zQI15Y(Hl;D2mZp)MXyvoFfm<1xs{}9DvFI8nZL4bAL9BXWiR^_)O8?hDmY8J0EvCx zF~LBO&=F@GlF{4mUV873#obxun0S9Z)&5f6+c! z5dqRG|7b#Ni?-%E9UPWWh8NxP`%c)h3vtq!@E8Wx@6j32b1zoTOmd>LoCJNuc`i{9 z%nYVcbshc%)%HOao?yw$^bMCC}AT3o>L>hEJ|-bZpk zo4L|sx@4Z)VaYM^3Y2fjUA?bLit`?Zd1g-HWaINUIWXd7FE;$o@eco4nE&gTCB#jv zt?mA<1Jb&J!?}XXih@V&4M;{WWMp`Yf+N7nWLFH-3I9{h``C%go9J-15(RgGm6JB8 zpll7elM0Vh+GKQ6#K36b=^gs*zW_8zTf#g#Cs!#xK{Fyjv-A|GN%50@kdK0jr8O`z z)<4!i6bLj$9UUVDLmeHxhnY$~FxDTIc&Z2PZ*pJ>?$EvQAJd)wEZqOS1zTqmM^ghM z6M?^P!_^gB)fJreZ@Q)LbVuAmUcun~~4VCWObP4w%V zc#yBGp>A83pQ5UzBh(D@KV(ZtePey_IB6K_zu70LtGDC#$FAJ}fBOXgPq{VeegRQT z6ddEv5&zYc|7WVz-*?fH5dYt@D=1v0Yy=E2KFTXcsK@E+r-Vl7D2a6hOazQH41Ui@ z-$>6y53nIa4X^|$IP2d`2Nfrg=KOQ*69fHEApDo%JfU^b2$3;=uYycU)iWav@whb+t z^krE_Yp~ryDO|9ghxdWQ$N;JgrCoaC0MzPMGTuAgMX}RXO&hn&Hg}V;-THkaTPZPE zXmRqE{9RgC~BYKe1 zTS%1>oEXX&iz7$h&Z%sUYQ(LWzd87;+O)_K$Rz4wujW1nt`L;1?zv4cbbJ72RPlCs z34MgcOrf%o^S^X52FmQx-~)w}TE}+@S1AW{`$!arY9q$`MS~%cfQ_eeu(JtVx%s`n zH7cGP`F2bLQy?Cs-M<|GYh({o~h3c}I(XZ05gp zDCIL%L+9cmc}5rRhQ#&FHFpd-x&YmII=_C0jCiqv1H`S+k7m3(ar zjh;WZ@{`RgY2UD!QDX4CuIlZ!yI}b~v3(%u0n$GZ3S2gz`+}Vz^U*xz&he zw2Bgr2URZDZhi6=28u2ioXy6(_O&RilI@bilaj|)veX@H#MCd54=L3JEloQr@J;-4 zrzfm$aI(rE+eKbS@Yg3s!m60P8wPtF&eN`2Im*EE5q?E7Uzk#8eMn;Tp#_ipsQFsoGtIlfI7#2oYH|%~J9}al5aTg4&6^bf6F$9XrSr^y zO-&w8s-KEs#;Q2E&Hqh*c5@*5k+oCRWAjQ#6w^&g-wKsyb1#WtHt_KaLVsKQIyck2 zUh$DobfJVZ2gG3yMr~F?m+?8ICHwh=!P^SLToaQsaHxjc8UcTJ$re497S|?mi5IquEzd8Xj-pvX_fYmUh zg4__0CX>(`WcDO!1tof}r{cYAx#a#LdznFqd*E+bMKs3-B1+l^_`d?dlEQF5=nnw0 z{w(bO1%&^r=J>B%<1)efD_|jSAa3HFTNGRsR!&YvP86IKAJ)_!>^K>fq`1xGsDP32 zT2T22?bL)MiG-0c1VuT`__SD+_z3l&0QIcj7y|>8|Mox9;J`m=u&?J^k%m7Z{~7-t z%=zDNoQvh3vIp|*8yw_+D#8Dc`F~-;Ocfh7ED>a1Qah|T8oD?_#dXETv~Ly_4JuGz zKtTCGmPi&#lx;nj$w;gm>9DN0z6riS6u8mZJfAVXw`A_#rmV!1QXIdNTe+U5e2!YD zH&Q?Lua3KcQifzf8772dhbjPX2+s`p2DAbArU%Bn0pz@)hCtzt>T}nn_tavw0V*8c zMFjb~caY#Fec!rbgjD;}e)`$$*8UuU)Yhb2S6Qyvl*w0S`uO=3#BaV-N8=WCwihbn z-=pX;jf_5`+0?4bwZf3uQLB`9+-U8wOm2f2MRE*l*7LDd_Fa2g^|#xj(h<{sjv`}u zX=Kz2!T~hH+gIk9i9R>0dCK7fYqn|g45WzJF!|7NBdD2$yt=k z{U{Ao$v$j^h5B->Kg2?oO;!BgDcqu39u%71Rthfd zrs>n#a|I(e(gAF$TnTJ0`&=H_slw5}kWcLRY_^Py?4paYrd{JC$+GA*nRPI1gscX@ z`c(HFgYIJtvI&>-dfHAOLEU~hXm9Lrq3rjQ(F4ZObA2+I z@I+sBpjN+OSI@-fp)~Y{B86lx#<6h#67F=g4@j5 z@bw@;?}GHn^&l-?0#K)_Wn?q^j{D_gHXv!}B`_s|crFjwY7UpR+|y6;A@^}M4lD&w zuV%+SK8eG>T|8pE2O)tjP5mfG>YdqojZVoDKD4mmWv&uZK8{?7f(^rcHDB#sEXhQIz6A~c+J=I8%VeH7S#>Pt{^HZZdK zZ<3GSltmIi9{!rQZJ(h9`>s4FM=sdzt^6X7TtOIOp?D1}l;htpBUHpg9dDxP1LH%QITV}^SJ)j@3-f{8~#A=)Ue z7Ak-T3bdhagb~!!fIXim&^RS-iTg-SYSXaJdQF(=rV%^Ij98V5{F-vjOi*R$UTvmkKh2GdQjB37GnpU@F@vy8 zqtUsLE-F1N)P1@7;x(iPZ1hc_u5?Vb0jyjp2JG3&%r#6pW|e+|aLbM$*R9?;&UKu8 zrxpT0__+0iGSV@1U=%q_Ab?wM;c#&c0gbLO<^ql&XcCoc(IMZaIp-4V!N&J|H2FOZ zH=716^J}i70lq#c(D{}eSSW$`EJ7d-^jai^1LG_ z2$+L>;6u6ik;odFOt{JR8n+U%*^L6!2<4}Exq-#gKcI+4QiZXV;^HlbUa<{y23ZIh zw1;7{TcjECGzAf-f57f?Ej-bfDqsG#@gdRl*&Lt=Lrlmh07ejiBAsp|5qm!!&fl zGR{EhXuFOwD!-^%A|p!tvtKP!C$A}w&h0U){b_^A?D7duuN!DZWRV|J1g9BhKwg?4 zw2^p8r7pLBPJt1U5_3jDH8-x$W<$bo{%0f&&N@tJ*1=>m%w6N9ezi^bQ^&lE$aED| z%NYIL$_@5QAc1XS6naDPXQ;mTGPYdDxf>>-`yduAL2BCe!6gDbcA^}=Oco~(AkkuD zni~TK&SyBDnPe5Q4i`$JM)E5r&!uIb;h=K4k(F=1X}l8^1!UMtMaYKRe74~{f$`VY zo?^#72tZr-i*W3WkR1u=Lx17JcLsSHhlJZrV6VuKIaVO)xwwRhA(#s0h}YxLzPW}@ zWOxA1bu^{Bn1T9h`3H;n^ni}B%LbDJ9{kA?5{IGWQo-KkBW`?e@4tgnN6!UDdfZOw z#8WXxTV!E_QAxHcqUZ2RDo1fd&IReDU&{?l6^+1!OYjM%`NGs7Z5WaotsG&HWeAQr z$P#uY-0ELm6GJu-0a?Qzg4qyah2)qnt0G0-1m7KEDp+$X=43dR(dR-fleRsRgPtj6 zbTVv2ETL`*ny-qnH%FF|x=KK+hin#KL({g-dUGE5{t^N*-XdUzS+6cYkkE+vxb+V8 zS9dGU$JVy~<7%Pc|4Cs{Q5OAcsb6{07Rv;ghbilZeO`j8bf5VkIPg<`-D2-E@SDsp zWi{kR$#h9qA{C9u%~Gb`#{3&%LM;7Ws~Uq}G3v-r-vtY?T)71_ZoY+}>CZY|vh^?2 zt_K}&UHLv`d0kE)^Xb07!t|I_tdHax?gqf(po!`4#>w1;d2iL%Snm437NG&Qasf9N zkvdmB=_Z5~YJBypPbRhvCLA8gCPeH1+loSo%e)o>2 zF5x9UxWndcXgt!g@q=9zjpQ(#m5p;v_@M8h=Mo?Q@0HG`Z3_`DWjN3<+qQwoCBo{L zQyV+EhtWQbv=oKJ)Po7s=5os5)Pw>lfzsthcSG08c0AS(EPrROa}~YhpzvQt_A5L2 zI8N6wPE?#j$QRfO5>DEoNfZ6O;i{qh&i$Olp;xBcW8uMrLjlw;C+!($@W; zWx~_1$vKyfev=6Y35~RxTvH`76JIfp@A`@$OJjUORPC$VIos35z5+$A+w=+aI$EO@ zZ$WWMRl-vriU)G^zWOau``*+72f7%gO=W6}GJ);LqfpX+?HMBtx2c0ALikgX$mVV<0pwzof+FRGi}mY!BArGzz|S+ z@%V)mdNq;fA=MSG-Qkiwj|-D^J@z>|fwS#QSvBHi9x{$Hysw_Rw%@**?`DUuw1H}H z{FXdcIGYS4f8`SaYvtwWE9@n}p^@O=jv*GKa1%MGA{3KMNn|9a#J1!>!k@aM&vSMK z(w#kj$BlJY-eY*URl$j-`XocXEf`Red|4*$sv?#|l%sl%7+@pLN_;L-yvZ17cJ{{F zj&;}FYybTkFc6No9sk@l08i{oeiJt^j;T9)P2~T&q3&BfY~7DB4dYRD z7;%NDKC59bnYHU44AX8c*&*q_^;vcKX%LEc$MJ&~AqI3$2L$5Dq+QO~>#aQ)6cfYCB%r(GEs9NfQj3ml>p>q_48%IPrPA5ONBKMbuQ9L3IH8 z7PG7Q9X?dSu8$JRdJMJLP1>}G+|ukZ-Ic*CBK;{X>J(GA*W?FVM7nMet-YEPi=VCM zu)pj&d7>)~?3cJyWdd_Tg~Ram7_G~qT~#Vy z%C&Is!)qi3ICWG^Kg;lBJ@6<#m&tS;6E8(Pv~Y@Q5X?*+*VR1?}4x8kT?^!7EC9uz|3EE!8UchXgejl9a_ z?H={RikV9d1(j=wY3!H1S?og$y%DrXq&$j!r2+9_`)1D^>7><)nT9P6@4$BE?n=FS zj-HB1@sx~2%)&?E&nU-vLd*Ltfz2z}rFYU@yz!E!1xS$0;HBh+qLU0!6XuqE&XZB> zzLDGJ?&+lw`^_nf_8d_h@3Q*vg=8cLv}BbPOiHe{i}u+dYQ?{G!Tl$TPEH<#m?k1CcbK9UD> zOel348n{+jL~G4C^Ow(mrwJ7~;0NsL%xgOtF1>8E9;Hm?R0hOjWYMNUoCfq%vRE(T za%HMI3Y>l4xb`1W6w=_*A4Niq)!cLTP3KPu&e!hclh(BiaSzXhtuHghLj{g{>P9eolwVN z{rTOd`~Wkj#}Ej(b8b3)DDDOpq}b5&X#vPyW2IBcnIBp#1Rc>Df{Z9qt;PZ$e+gKP z3o0XLFd(LkPX5?lNIW8VAhKQ^Ax;%wuL7wSe66(TY$@S95LZjZbwjwg&0Uf)zE4mS!=V>l+rD3dVKaee^&=IKQ z3*QxuW#I(P(}4qfk1qoc&wo4b%f~axm^3`PEGBn$gKUJgBa?q9AK+VX`tgzQoJ_|c z9_lwOrJb34T@xD%D-^bm-TsEIBYnV;!kdlWZv&M5+e4H`dV1jk$ggMNf*$t(bZ}sN zVo>48izJ_YNL$z;qk@ZoFV{=4A1YE}R^J?bHU;tfwg*aW^lXuVa>z?N#ehmwNm^^I zy*{0yRUkc9l$IW6tpp0kq;BhIH?bnh`_Z>;?4OH39w20d_})Vh26muM+hs=>CP$~t zYNHqx{TdsA7R>wDAMDeRrYYsn=SjiuZ3V!3@0LB?r7-sKb=ArnF87()j=5LU4Dc?2 zot1t9LWbkmyj~O$gbvU?xx>w*X-G8n<}%hY)(ZHU=1Ms{u?jm)1ILgf64~`yiO=3j zWF?r^!Q{9GkkPa6@m;@Um{$f3wFE*>N+?m^9fk4#q#Yk8a-V;DO^e!UB}fnQ+qWph ze=Fkt7q7~DFiZY>8=snuo3a{;udSoml(D0K2kZf$S-wOXAfcpd1tKa)&YUu{B2V7C zB+Hm=KR+tX$rK73)+o7XSxK{~SYm@xnUklu9w<-UB_`6Kr!B?zEW!5zudUPYeC}li zFg}u9Z;ZQZd*XdN|2RM1XnoyESNmo?ur9#7Q~y01IuCd_;F%jY{s>xtJBm(qUy7Y} zo-E+Gt|I6fCDoTDgv?|uq}Dhf zv!YxQ?qM9YyuysUtw9plaL@CCMmKNg8Of_~SkjF2Vt`@jM{NVm-9}z?6*f;w9(1PCzr) zfddGS0ZD7Ii&aSt$8|f@!VI$*yA6)4y3lHqAb2vaXSYsvydG>(usm?2FaX7^Y{ltI@+^5Jo9T-U~gGuph?m=UiPWIV4s?d1c526Y?3p1kVhPzV9n%LxZ^-R6N~T+MPI;OR9`KI=|&2i-Rk4ntsgm31-+Lk$`B; zNX&JGZcjOLvQpr){^*Z}5(eSx^#I`;oJRJw-2v7mVupPOuEuf)QTjd)PHQ;SL()-d zzuH50ljKkJj1qodr1u09RTBsMj!6XW`|Iv|H_J8GyY4PL_@iNx#H}Rh{vHM&h~GfA zq)SFb=>D&{_f>x6N^o1DbpQ3*tsZb|<{GH<7w-h9RR8k5U6uF29+vk+0z21K;`xaL z7BpS1R8`@0*mqbWUD@xGw-gkVYD0d!N)zF+XCZ*GnglVNo7(w3qWGpgum!e>U>)4+ zfWj^14bt8srYN9W*LVn&V`FB}bLs78{XSET)7?u{MoSZFTr~7YWHOub^QniTpI3)> zofHJ`LylhFq1g9FW?21)lL<9iz1|nt*|#no6YGWE6m%5ziC>l@GD}`B`)lM5WOkb3 z#d^9q1Z?vUcj9E(rI@=l>WqmEFaA5Ib>yzo9tD`0AdbarrN-&Pwh> z9@OtC1B&c0SuT8Smzt=QCi9k*)pSK0OHJk7P}`Pp)hX3d1bnEzWB@InjF@nWxuQ%T zra6tbG=a1R4q(~QBt=siMz*G+92O%oX|#r=1)^?>OaY`#ZumX?%l3oDn5;}J>#1qF zQE#b%$VgtuI`1!wp~W>=$M$TxVr(`SL)3*;Hx`~)Pdj=i{eC4kdguMH*fW$4-9`5D zEy%R~0+mC1 zl)n@Jo=XfzD687wSbGEYBzNnpU6yK%2lFb1euT`yJ~wVi#zlJ9By)L%Y{0yb4d{rD zVQE~4y5ki+!Hn9CU%0;U6?34I>YHBY8Q{7s=)}4gA0yi7dA6-;VLRIi^Gf=*#$%wj z8~FB}?3^xRrWZ-O9m4$3*Yd%j7ZyN>f1`vx@`W>}PSyMg-}O|(zpIqiM9SyOKX3e@ zL!BSi_iK4p{O00&c2^WB?{{3lviX`Vd-sT zRZ;0LC!+Iy3+gC`bK zC-h<{Q{v)9a~KlXE@s5S`%a>QEWT#M#`z^Y5kwr*>nq_bdIzu-ZSwY$CTXJ_();7B zIlZJSQi;iERCssFzUWZ!hew8o!#s&%vG?-23Kj2x#g1XIv?{t974HSbw?xv>`4vIv zx=?2WCL4lN+M8G;4iyAgTbz;6*fIPQdAyp%C@sYuZ}#v z;N!x!BI4USCku_@ZrWu1AM2|*5$#NoBa_WJIMA!SGVa_$lJ8;L*tyr-06IX$ zzw17oaA7-kF%C1k^)hjHna$a!AwBu#*qd))l;QT%?`PB;Ww{Jk;|1mB3x5m9p zWgEG9d1PPO^%GUfnj6j=laMi0aFo>VFbWuSz*r1G4A+&`CJ70yr<_Dzl1Lb$h<-o$ z#@ub!VbDn!d{HxYB#zD+f}V0Np>P)C`R=gl}`y#eVLSk^HJt?}RX0$G{tNY@UGEH_n5U*x}+0qOt z&H5qa5204U_)B>)qtB%Hbv`x6)Un#8)3Qvqzv$Zk-Zw|9!q<3pDk(kSLJ70^3Y*QN z-)Gb-kKDh)fCb8(%mB&=(HI$&T1tI4Sp}JRqu+;i ziI{kXEEmrQMyUCEC+f;6|ne;9)bNYoW{aTE)_& zU-H12xHWHTcyf3rFEhSiQQa}ao|@EHm+RQpwf@X!1G^m}pEfSrs5&og$c*vB-6<)# zAH&t^aeR(OX*d4$9x+?>3CUIF`#s2M0|>7%EPxyZj&ZSZtufJnpW?`<M?TMk_3^)f_%{0iAk^O4oCN%mLpUww%0WYUpxrIAaOWqDJ~ z5NURuf%RaF7_D5nas*1WIIZU5*_Jx^HC8BMbBT^@FyQL`NKb3cfGf|o?=&&TG@GMCH zMgpZY+GdO;{1o#u+CG%9+z9;;4afuOUV3Zb*Y`~clh+C8TNF+jGtfraZAhnVGCOm` zC*fON++`VenlNg`tR&=digYR;{izo~Uoy<3l7vwtv?3<3x{!hJ=`?*w!s!l53Sjl7Nk6pijM7LTn@=)XD>%n~CX#2-f*gZw|7 z$=TfQf9BVy=qO>CAp7dpz36KI2jT|`!fKL8%om|nXi_%8&Lts+$v_|GpO9+ARgT-j zgfnft5p~@5VcEHe3Jy=*Lp&3`6X+^Wd(;bxqZY@Ha+pnfW!-gNeZCw;FMl)D2QXCc zL2sBkiVf-bQo3E4x?s;s+5i61^MLGO)+ z%(}hPL_Y(bZQGi8f>}1_=BU7W1Mb%vr@Oy;$S7Q2)Sk6TadoNMv<^Z~ErsppF<*f} z40#do9FMyAU1Y-y`YH)gq(XsiJ6Q@jQLuK7HB@Diu&I4!Pur=@m*Eq;?nDVl)Z)p| zxxELL%yI?>89#75)UOuCI3tk~bsKAIEMk-PeN4#Zp<8)=sU7*9WRZT^(c@NniP*-% zTY00<8A;kH3f9UJ&&lEGJIMN=i9c}?ri40E21w_-u;fmBw0i%cQRy0CMV$DUd4{27 z2l6WM)MC?y-Ds|Alc@~ezIG2wCH=~m(|Wa>0ca;wLQ7BR%HTOpKMB~h!MP!(?yqWN z3Db3cxNQdOq=k?umu^>ehQl;5nmy=+%V;{GY|jt$enT5#g2fuK95G;upn+X?uiC|Q z?R(bjXY75U*Z_P5<7#2Xe04 zJl~lre)x5@LI?l#5ZNLtHH55_p6`r66>=jPgS=lElYuI9D>2$%&>7-3$}1rUCM*uT zByj@Ky$ND3k$I}{3FJq2raKNlQI=MY~^>ID(SuZJI zss2|fso|MF)>sH*sj{yq<2WkO*aI{j12v7IgDD^RUb5e3+B=kk1%HzbDp8z+FIgMu z)~+@P<1*F>*=q!<{jaX`bmlXfv>8^J2M*qTIGbKp)YB=gJSw~Jsxm|v_L8wRQ;IK& zwML9M_o2&C)t2WJ?)h>c=DrLZ- zio)U+g^Db-g=oS;MHi6J<4NbX64yE%OgBSVNQuHob}}7L2;M+=J@%%Wk`iNsQ!lA^ znO7P0X1-rvpGZA0N3T0#ie7b~_X1*+{6{vY?fLsvQ5Jv|N!soXU1~eixz_!T+P1AIwyZ(PbBh>+Q8lo(s(}~iuvVQlg0<7$enVSduWzs0@BseEp+iX$e^Lln9P$G3A@WR-Gy?CmERWDyPyRoF&f{qohS9lBeR2`epyCcI?@s#ZEl*QE9YA9Dw^wWUP%o99I7g* zb~H<`b48g~vt+(V8^NlJn;CsBBB2i<-@9$W69)Ci$JGarI&F<$0H$;28OG^n^64;5 z^J2E;wf0MN@=)TaJV;TO&7)n`r07H)fPmx|jXmrU96%9%uYBAqQ97@DzCB#`0D}0yJY-{TB$!vX#p0(yv$J4xkQxQ8Dv4## z7RB8E>|yw;F9#2XD2V^@<9Wz``$PTTeEGk~P}s!D$kD?7U$w=5(xM8wuWT19)VLT$ z?vP?ZeGFF?P)l_>-cBGIA%Zb^wCN z`%*#;5u_Nkl6L3maYr3Q`Yn;8Yf1Z&V6&m_HJhwBn-n~aRk7xZ3wPlR9~5f7C=C7x5RUAMuW};`l?GJswJlHjdfMhV9xDg_aYqU9aqB>uRF> z@bG{1Uae{aTvTBcrkxhg5%|vpnrL0}_;ZCWF>0P@!i!hcE_r*zH@Z&MPYtf!1)Nr3 z2E|b%CWebzOxja(*inOT;DitArdNA2wWYgERK{r}h#I3wyA^kE*rzsVJ@+7Fa9utj-A{n#~`U2oGR0xJfBOvIZ)EoZ>@PVjc3OF zezIPa?zf4Lh9nUU9^y3b0ryDEJvjqesx>=h{URNXu(N2qi<94`bOr23D3RZ1+d4Ib zszb4yk;h>)Iv&lUcN;@IKN}5;W4B)yh3@HmzZVzP{A1YxOwO+4TLXXhz(@6V}tT^4y`UmeRI8(N>WF)gx%zI<>tb+t1aT8M1nENX`BbAoxZF->4~qB!UQkW(Vz z=$S`i+!IXBfGLL>2AL5HQnUyNy~v<Am5%zclsjObJK}~{Va|I;E;0#li3S&eeo9V*@Oy`z9l*S(u6$DM@ zxU#ukea|0#aXLvT$N2oa_%JAh&+|3~Mz2*9xl5PM7!6FkaSSow{*O zkn6^qc?r8=tYZ^`gg*Re$PV_4nix&*BFKkV0C;ZBiUfSk5)7?-z?Db9$K(!;+;y;Z zu&R9xM#mY{(Q*M0uIkIFudMj(_eh6RmZ#YmdhW0-~JklCM6W1Z=k<@ zJHh<7*0=u^izIAa?X3RiviFbYvXlpLK258WH@W*T@Ckaq39zPr!wq2xG*{=(m*^qR z!$x6CH@n?4WxKTH=hj>=v#G39wL-BGsby82M^yk033%wSY+Te_U(?iTz;$nGv>g29 z%Vbx-YK#ZEp_jsDcf|L#pZ@jWetY-k2F$K{aJVO0F+C{Lq5;P@cWp$c-7**q+oIBM z!)v}m9=z`J40UPW#n!9#}^DWKum_s~g3I44Ub+c{;9 zyQo$5?`w;iNt-zrCtq8V<7?Nwh^1tsMhS5Xj<0R$L0~QD1p=P)N}p&lCtt;311-YM zXrq*;SqvFiTZ?eljJzzsNTpep&MH&#D!}9bB`h722cgNd%C|B+E@kq_6)zx2V%5;E z;1d$gOckG^(h_(Ms8yxps+)?ygx$=f5$)huK-?WmX;WuM)<6wQzvXE>frY;wS2Z9X zf~!whPA6zgIwOECRS-%S+P$j!`6 zM{h=s-hUTs9%+5bN>tekqsGMxy_kscgc}iv2$i15$mu3qPT|8PogXLxaru2v$XRmG zMfwpT057cB#SL?o7|JnD)J#p@ql?NwR~+|6L#!a!;yE8s)3X|ES53pWE@BZ`6(iA_ z&7bd=qF&o;te%X^@nfIm#@#&gr>SN-=I4Mnnf(&IA-r{8P2bstdNJbQE>=F{_FTV# zhq2|F!s@fuv4m*clK4lHat79@~Xwioqz+}dGbp?gLlz(0F zH7X}{4zz@0T3LbL_Nk{kF(N39Nn`00><0?X)tHi^5w)HK)Xb9Td5Dh`#5s^UO=DP{ z#*p;M=C<^{qBU-YRn6OrnXGlF#4kDtkI|Qa%UPeBP3XU8sD&za5 z?Uj@;U|h66?e34EJ*;P+8P&Zf;=lK6D>kY-SE65v^E2v9-3?Gyk&m@}_6+f8?yfOR z^;`^4bLjvqyL9+>a&8T1ac(8`bGfSxdpSPRalEl@5$=Fks2mcfYS}1b8b=_v0w|=0 z{jR&B>T5NJElTDUqQP;#1|j0~Go#>dVgX;L&**Q?3wz2q;Wg@Fu&5{OP%4}O>LvR~ z-evnt-c|cpEz%HN#QwMaEtWEnI_!b*aM`+4p+WtdH!ZW8hN4pLQqEOz^jJX}ZT3)X zTU&%#F$b^>RWn-4+bQ)?h+f2SVx#+ zFd|YB4Ww42rG?QY%9r1n>$c(ag`s#Lbt$X9DlN9^$j%WL@&$HF@(ljW`k=Pe(F0P((5*iR& z^%?wnD%EsM_6wTWYON4%ob@eEGq7sMX@_E6!?1#BD|0oKF%q)r9|Z<;B^4*s2ydaE zxmt-aCW@<_+rW*kOqAu2Tgw5aTFA@+LIU6H|AtZB=ht&r7i6ydAon_w6H`l>UGDLs z=-Uw0bb}S`;Oj;n*Msb9ZS~Dj{-;NYA#3(D8${WQui=mv?N{yh5)P>V&pP}PvNO^Q%c5W0^MBlZ!(K zM^=^VInyp+=C`g9_B#`}+9Xd@3#T0q~uNh%)XxY{Ix=n?Cg*5 zKW+CeNp6ioJqaA>NcMmXd9u~Br32;2`X<$bP=PUtGSK$1wfmdbXTsVIc7PgoF;d*& zaLx-kR|!Q{F$S+`r_6|uhu;o97`RY9<|8`-UeLwhRPLcy8H3pGMyw<@8!{U4D{P;U zBq5c=Af~0%Tosq$DQd%1-B+I^{c|49H_k06-FW*6;3Ja?fMJS>6>~kynj?(kUP@*hEef^ z3Hg(4lXxgwOX?|;+_LM6%h2Ybq9Juv>oa$asdpkXUAth1k2?TY;d5^rI{T#qxsb?o ze!pnZ1oHjP0?)X^i4lN!1c&k_U#|*Q|Eob^sPor5ELdqoJStdAcwJI?_-LTUEj!7i zP@lCzC4OR{2PXGhX^xnXbngZUdddaNoc4zjvL*S}uvE=YUOgZjIUpUc6Mi{a5T5}w za8F0pnPr7|Jo)f5s9sp+Gtsr$EdFij96N#;EXZ*iP-i0luFRn=f%Z%8Z!@ZOABOzV zYO8ka?nwQ;=?4bBQU#1Ji>1=MBs~SF=;1eiA`O5-i=IWk3v3=3MN1x&d@JDeDQ2Wp zolCtmE>&x2QB#HeV1kmyQ8mA=D?_7}E2g!W9O5?}KN>m7 zvO|KTuA@9jGeV0rUuAazR>jsf z4j%+Wx;v#L73oI0ySqE38)*>f?(XjH?w0QE?(mQAInU#9ANBb@;}W^{{bOdWd##z- zduH9A`*CXFCK$5194FV1K3D05uy)OrZ)+E;wNs-gi3*X$&y}u_WdjzGwI409L-0&{D@yEtWa- zf*!06wR~ZAkzbQ@>M$`CQ8y?!eaY(A1U=5!w+&+DlJNEp_S7*(b-AO8(zNDG!C&cDH?8CL$fWZj`@{|Lyd);@<9feerg;8%Z ziS1V#RIT$<;XceuAW*8QVK(Qi8kvMl(n1PEH%> zCrO4aT6mp)g1adR0z2Jv>emlpal`rZ6$8|P(Ci=>5D@+IRRifSt{5a8pRX9qC9NzR z90j$^bxr?s*$^bJVvi(;{FE(5JE4^tDvgiN^nx49(Cd>w*jRY9w3x^)PFF`27O7P0ubtC#Sh41ZotL>~@Z_53V_Q-wZhDJ3Y<5gUq;HfUVh z)%FpGS@e~A@Mj^In9l41Enz=D1ej*uM&AJzNYcGm)J3bGg9~c2wfVbj!Q{l{xhUC< z)XKt(H#fr{x!O*x1dt8m$|Cn?)KgPmC#5hm#fd5vv+C=;^_i!WvXsM1F+grH*S zpqG=H9@#wb^c+e%-fjIDp&c?DVU(94w^+NbRBXK{ek&sr-6P46JvY8Pw$v;`XuvO% zQ{OP}G$-IZSI>%ED1NK{zO+%Z*DA*hY@T)gF1W_lZj>_VNDxMr4VitXrlairC-TY$E4=SJ^xNI|OMa#ezLbwlWqsz|M*n$+U!M{|@_YV{( zTA(l@?}(7_)djOGp?pJJyokwhF*XhVh)p(6<`aETjiGSWJGn<=jkiU|EEve7+&G)g<7*qT?40yo8Op>RdcMTuR699Xsu zHXBMkK-R@D{Lms-@Kg+;%+0+%v`m62T^%cCI*ZSNVbqgHch)^dsw?5;Qo9(61(pN& zF{UAJgva5Gmgbve`ckL{#}UX?#Nb5p74`*!>iVJG`&uJ>4;;wq#84}ZA zBj{mVKFyPJ&{=}xVK}IRs%UsHLc)5#PN31RC|Df4nK1u)@z6>+r~b8akPk6l#;(k8 z^FZF|8+y%cYsZUN&=CIw-3Ai_u~iD2-GROM!PglT)%Bkxpr5b0WnIiypcXRQx3*$l zHn&ehe&y(c0%MIvYYKcAQY5%SYz-l1pq4r*@s%)}5cXcxam^I2lP@E6io&^2lwqrG zF3Xc{nXi;ik(L~&2t1F549eVHc2b?}-!>bYi&x-K3hPo{@l(myhOo_ z^OQ@k7YVYR{#LI>cUd6c7Eyz_Sri(J=)-YaiW*qc!0l%QiKS8ZI;mKty3gX}V^sTo z736{&T6{Av`Ww)q@uW!)caV6B|BT$8i>fk2Iesn~0mY&YGTZUk@4aa$zJy zUWg7{mSW{RlXd~~pw1BXo_^N;Hj}wd>IZPOYHb{GO%Rqcus3nMXu)gr z2RRG~>b>qr;K0oDZ$Xf0!<3^E12%XK4`3G%(w_Qk={IjXM{&EuwZu6gxIR) zK1>Y}G7Ki}9TUX8iuA(1P7~{xI7qRo(piIHF}u6CwVcJ(X~0Y6WUfctq5Xu9Nk zfq{UEKz@Bh_))iw&)mSsT#xAeUq^%@X*CJV@6S;7_Lp}b%NJ=>RRZJHn<=MIv-vZ; z<`M%Hx|@U^c;&TU7^fz9p-|E;n5xn(H zV};ggl+Pw~bg4F>ha8qsp^{?FN&M?3s;Do*(0wjxZK34gRBhBsQNq|4!|~kHjuSb( zqVy-V%PY$9-x@T@_j*Htp~d;g(grx-c(}q;FCW&dr>E9B(bS1RHgO-VuVoW@t?>dQ z^^8S$_OJcTnKjV+3UF~^>ZTFAhbMJaPIGf0?hZLe55po{5o|rb7QbSr8p-&8hQ}o< z%gKaNub6IZEt@uQzi?S++~7C#_BGi;3tFLlP9Y0u=VPBbG-PR@nl zYkrib`Q)<){K$Ov)Y8&bO>7KbpPmb52d_lnCR^$t4Bw!flweS{88F|=obX)?cA09D zcYOvZnCUKAqpjeW!cbcx;{<(lvJ8%ImEx(tn5la_;=l2rJlthc^yF!v1=4&Wqbsw8 z>P^$9!mB@AJ}v0-7_fmNz3fM7cO<2IOscOlxG?!vYA9sq#-~gx+|L%;P<=(u6s;6p zAjU8yQxAy+N5kO(_2+D`{74-hiwFed@b1^2ZUAP(U%61F1mS|cY3puLqEy5mT@uxJ zR&~?L(uk_5Qmr>zuGgoC|E^kZ;3bb{6Q+uZNWuEv}fAyQG zMPuEyU0!#FD0SA)yv}EWd+;twA>USPWeDPWiw&$4rfuFkGGx_ji8~4q81aJNU197d zmAN+gI*?!4bd5vY&O7b&T@iiS@Jvs#o2fAqS}`D%U6K1b6D!^mOe?3b&AbpN2C`n| zJa3Y?qHBsg=sadeX?_uC+-QBwnTU-6CW6MaoU}t5j?vr3J*^HpgLNE~#MDMdC81gd zJnl620Y7{zTNH(`iG3;M+(CbJE?b$atdJHf5N{51UqM^YGzg1+k0%`hV%KcOp|!Uf zJ??;HCPECZ69GS4Y|2q~Ur@K)m;47?NLubK>C?Y&}m@o*>Mkyx};L|*xIY;*PBH>0{MuWd zWiYfJ8d9ZV6E}gnN~MPOj$ZjWr^>RD*DOz}ll3Fl=R&<&*U8b@Nl%gQ>Zp(WtaIfmgN`^g0 zz6dSmY>7`M3FH*_lEfY8`dQD#EMv~Q)WmzI+k6u z6N5j4h@x^%1jdKZyCzKyIm{9V~-we+aT=-6-cVtrq|-xy)PyU^7Tw-r2_Z?n|FxaWd&tQSv@$;!K< zpJcHLx8qWvIhuQJhTSn%`^k@mz5gCNl`>JWbE#I`c~?IuZ4sW;Acz5|@BZ;bNA$%W zogfbj`;ar=!zb}KJ9bIVx+jz83ose^@+K~lCv4tcN@BnZFFuwiPUpNRn0T`*l=S2; zzR6u+C6qMJ8tD)&!*UHp?S~=iMc1jvojYr<2I}N{%Jg9Q%D|{UVkY;3_GFu*_w(Eqj`7g8T_E(uQ0-Gzv!Z(F`QX02N3?Iq!&vl}& zA^7(L^wPV_3h%t_Be4CZ@Ccc8ymF3_><_%=f{p#$BHWE*2VFArDnz?0bCZk;CP?Iw zlD|BCIMRLVXK=G?^AHVOM>{sXY!Yj8iQxyX-4qJPa-r&B#$ScxtPsNfCdg@l|G<;b zGyXmuN>meLB1&AJ35-(lFcIwN$-~scwF{|J@pBF;^0icpIa5@!UQCCJkFQMes-t6D z9onXy?uVq1@`=*Hr?-Tg5KxO{M-W7|t3tvtAL*x(KCp%^7GTF3H$~YDnqtjNC-@T% zz@wJ(OXxCmuk({cNT&F?CYh<3vCcsjqNDV#jw7Pu#*bYkA(X-~m|#ar*q~n$)2QWv zT3Q)+`-+fsr@XEARILp1XOkreLbRXI!|tRf3PPR!G|13t19hZ!0Svi@D?{R@<4_Jf zq9{KFi!eQmKaF2E61wep<|f{EEQ6_fqZYvaRTA62zRt705C0aA>3;W9)**ZEjip(e z=)_JQD^Ihb?k>hSCVP&dQbx_y`G#w>@UYBE_iOT`VmN6foF|y^$WZ&x{*j4EES{y; z@mJct)|i-0bV2&zcr0C?DOO*Pq0Wp4GfsEp!!|Cz3@DQ2lA5sxp`WnRTXj;D;-a7A zf#_|=9gweM74(aIq?#0Q9o>5IQb;%qD||l}FV=CVbTjMrZIC?%TO3cZ$joqolzawo zP_UtmY=%xJCQgYDgk#pj2mEV zC_&Ak7NO)=m8n}!;VwCEU1@XK#fe{QTp-Kws*|wi@|`>p-ENF!rS%%2eB{dV=s%hF zdAyQDM=0U1+($T$^1n9bb;i+d0v=leZK&4bdlS0D-3gk7Q(6|kw*y;`bo!Qe5D|9-y{>k6NywZr0F>c$i3EA)|Z=E>CVWIo5!- zEVO1bF6L2VDJZEEfVc4FAR^JQ8joITjG|p6XM(+!7_}G6KplT+bT!Tue~unH@kzCh zorA-6w;_k9fEkmyXgm`*>vc?S)k(-o1r*1yS4`C_chF=tpBAhS-t;F%RJmor9Qh~p z7y;~Xn;Z6@%3e|pzz;d66+lBgFu|XU%MyKCtUcFC5+Sa_pEf0&E<}u-T0G9rtxJ8p zmX+RqWpotU4V&$vJWC~AccCu0Br+o`v+F9W$K;lxxs&wmt)oilra4sKat|kVk0{aZ z0JVTabK}ukJ@dQKh1`ZVq>DjjA*g-NWE1tYalz4!q#Q%Y;xpaQE!(Aa1o(6Gnh#*> zz%ys~N4VTZ&awJqgr$#Cmbd$SsaCCFJM?1@M5 z4M?&6&{LnIZuzV>k?mN9EHHu=pep@u~&AYX_Si4oWp%w(ukJD5*>7|qQ z=DhSR+M%e8yiml`p`wB6>0Rxa3Onmnwj%Ff$D+c&7#KmYoJYuFoYjv=SBV&>82uejI}R?U)T-O21!`0w8OHEWF3qmDCfg89#07ts~hIGMFt!~XW;o#+*$=MO5452%@akO2BGg|=4v{cj<)b_%Fw?y5dG?D^ISM@Q z5~p&PD@hxG``mqtgN2>1zKkWa*IgZiU8!Q_@%6T@Z|Yx}^xmjmjmTIwdgo}gF2h&j5d%fJfBz=pn=j!XfE zwiCeRofJil*~(vZR0ckl7alSpEp&uBH(tsQv`!{sM;sp>H%2>@=R0O~yj9M6nKYex zv7dWVl40!XCBN#1)1-x@i!dY^BntwzCpAT_xGGd6H$`q3AUDNdpl47-sm{0kA!=1w zfA6u^Us+#x(IABUNMxop9gKn{F-92GZCd;sO7ZfATIJ^%m0C!`z1CM7U*kz>3QnPm zT(R+?r6<-m^h$L=R`OZg%7cf)ApEBkv5W`Q3xh z9({%!i5~*{@82oimV!Yp)V?%s3v@hje>Cb3YT?Mb*zWHwCDX$~PK|j_U0zQMgIH9e zsN7oO7u1=ccNc5S+2=J>AtAst^JNy(8yidHOOK zBWYP|A(%|2tjf{bJI3>4Q+9^(6uz~Sv1jDTn{{%=b+*vZ7`U5p&BE?lD8VQG?~NuD zb)@$=8qWQ|5nxyHa z>+S4BVhlShA;Qan7Awqv=3+_bMeaknSi+tvYZ^MrqCib;$@3SCcfJBIKamhg(t>1r zE9~gRVi2HX&gsUZ<6^qhrD@6YW;>N&V!DjhDMdGlzGi0cAElNu_0 zs>ls08&(YgiI3o}zBS|U7BwI@k@PgLGKOb8m<_e3W?n~nrY)l+ZG?i)nz~2d; zklD(gXdN;7fm2yHFd4wXZBL%8RBzx_Z>D%OdfSIFv)!Dxt1xlO7FDg?!=Nm;REN7V z7KS$U4>>7eNv*d=d#3lP^?#fww3=VEy33A=+pKQb%wKGnB5~I-JEJuv7%H23h_<}k zbmPl~aXh1pXsZRa3?|KRp{=`>Cto%9zGjGTf4`xbD1YkK@J6ifK6uple$+T-On%-^ zegK@&OcogDDOi(3Xg%N&hYRm1Sdyu#k!!hp-0*F2IKPyMF)*yOSLuzJfT9hh`AetZMj{(1~2xYBW3fl!728#dDPi(k7mhq z3B1?$%7Yi;fQ4KL-m(IxxfjphXgKI50NPZ|FBekbTjoNopHzFh67|)O@cq2R16023 zqIa{1yTz>SoN)+6l$1(RypGM})(vx28Qjf=QwBzzL}&;e{{`-+Tex&DHeQOGutuFV zOAS>c*7EVAK#HPfQv2nK`*i+dJ3d~{N3X}YQmFu~`MkxEF_>*5Ea|7OcNNG&nMFc{a`2@(8>_qD;eCG-FpMpgE%JCFN|j)L1E z2a-CqL5m!k+o<#u8oy7`!zdUN)8}9T5FT7;ffgtc(6^wz7uv6rRCMUdnx!=-C4%dQ zD0$*s%!ATCeNrwf(y3U=okI;vd7~hT3YtyhSCr*txVTUhhvLVU-t{#tE)+OeDMTuq zhTdYZRCF*9`lB*CkijH~C5JkGt!h>Nuo|$tLp2e+;0!B?ltF%UoJlij!ezQtF#=6| z83lil?qq>!gJ$b&34!hvsrED9NN@DZ!2b*&e0$mb)B|au} z8DQ#vl+(;i=*(Ztw5E%ck}Az;>nv{dIdYq`8f331)`Gi-%s;S9U1c_gR4?Va{bJvQ zbkDEG7!sc?_gWmw4>EWra1F~6nT{TN-8<|UtoZSmeU#eI-`5cpiV{_cS83rxW{z)7 zlZ?Gd^;q3exd>Uj`7}jW0$sh%(uhn{Ogq_e<_3otkJ-VzEnMJINBTKQ&)$+^z!}j1 zRnLKM_b#ek(pu#k%lFvV{4|0dZIHI+UAwBKppKVUlev-tpl|(tRy+W$>EVyt}UB3oC zj$q#`%js)AD&xez=Lva4n8kBGz$oWDozY_PJEGV{a>3YOIWC4es^2M=vTu4l&9Roc zFMdX%B+^N22s3;;jISx|D)C%d?bINR_+yhlZ1WvnPhatVu*SxoY;bN`e=Fu`h z1L}4mzndkY{fRBksB!&}ri_7NAMxgez3MRIMn@P`Sduf$w~zz-&JsG#w2Pt9o-z9! zl?;oB=0(9P@JvE?0?W_M44%4Oz0%@i%iTaqP&guHI-!MR<#gXlUfqs%!vph#$w1e< zAMf@#co=zb89;2#B>2!G&9RX4(jYI7nfwVj`V{^;RmaOMsho5b;gOQP4K?543uf{f z=!kCJL=yM#fEj|$m;Q^e9vUMK7h;mpw%{yNxwm z5JwK#6Wh4s64uVF8X~xGo$ft5UoqY`w-bMTpgT}N*P+Ls`Y3;&d52rTm8Y&R7q-_a z!fems7(Tp}kTE{1V*{6M1dBkvi5f+)(`@M*;0{nNUR`v@@)7|EXqEKW55|0-lH_cR zO#fb>OcBB*ax?B5r?o3>#5;KI7et`I9_S<<{;G)FMBcofbwt2mQW)VvH0b(kgP=Z! z1!~jm`B}F14N%Iqiy&3C1=(N3Y1Ns(>hrHtFFwp7_D&_f2mi1c}kXt5@ zo9W6ME>IL32sg~mk(27VCkv2oNS(TkdBsnSKrOBBrcNk8TI4@{Q~9*KLIrlCQspAR z)l?AOBGVCz=~m)pF zKfR38>0JQ&nt$vATqv{b2wbSJd<$Hed+Z6skZYj_gryg(eL=vJ|H3ea8FHJOhnver zsER;Yr#BtH{xn+ymBPsQ(W*=%`eqZch4ONMriJo)0Fp-@(<)DM+>)T4HU9Haz{LSh zFcDb9yq(Xrw~2uCSlZl?pm7!cC=QOlj|^dezoz<;QBxH*WY0_E2AKduQ%zeC3uI=& z1mE_)4Ts9ZRjn6AgNuD$>Jbwc<>p6%HVlyQ(f z(kksdgb0j}B@!zIA>HYDFFhKJS5ZB3$d=Hg26=VDg2NVB2?LT(hF>2c*K|2|@Tjy0 z{Ng0eGEvFA-9GCSD z<(a1TLy+`h+4>gJWCklKgx-BqynYPts*@UF0A6PZ_rKEbQ3`OKWT(Lv(j_C!`&#Y! z<;fwPu+h1(V&)6wAd=zH6P6A+ttb%5)>eWuVMjk#UR%C(n*rGLI9qD;fW^XiGcjfz z**n5ycDA%#tJh!@9Bd4MFyB59WyRRk^b&iHQ&J1i5R=fy87{J8ON?574)5edE{hg= z$$F0bShm>@yQNWdxO23c=AX5`iAPe$I^6ogYNeaE(K+qP}{JGO1xwr%^oZ|>yfWhU=t?oD-%f!o;+uK9BK5dpGyml%Stm+jmET>D_7Dd9vQLdnglbs>IQ zm0AQ!&%669ugV08yWhqR53upVj*jE8Qyi$=-;JD!eM=~Nuc{JDFu|WG2H;4R+1HCA zUqO&5qIauzi1bG{N`~g~?F>s&HLl@XgsIGcTybZrrN#OGaY(epG=KhUvD4#g93bXb z0pFh96PiPtNN;isu?tyo6f+Qz?JD5!p5?N{0~5fzOj8jH!o zymub=^n3CCP@DefEhKMGj~%J z9N>Ur|A4Y_JZic)PR4p$dtIK42bL!sonpBBlvRj3_!Q2(M&a;Pl&%j6tRNIPXL{n6 zilvh#FU1KQ^5LeZw97f@bJYf!2}TMfF;Czhp~Oc6&%{!6N*GVr3NoucXh^MFOnM7~ zgX%G2O?0Qew~( zYaJ9CSC!Da$cA?rV?6=TK5^_z_Hq;Yy-E|IIKXne2qPbYs+qGG;ZwCKs<7sJG~5cR zK0>93DIyX~|EA43kvDMxsO5(YhzL~Qwi^FZ`i2)8IW5?d{fq0bnMy%j>rf>vbX>Wn z?KfH&i(#3cRb|$L+B7Bwq@<)|2Uf*UNz2sL5wn9K$fRXLuVHrlwEXnu>r>svL`JfZ zTQHXW`G>V~ObT03swPxbZCF;i7|xCfj3;5ch9*=W ze6p^=n;bb9gkfS!?zT;-H2v%#7}#oBQQEEEZ`CyQI7{tY_8+5c27h45{_^P1-NzL+ zv6Kp364Jy&J}A_TD|M;sR%X0QxEVGZKGkLy6;+n8#)6?=6OjJRJIJPFw5~OisW>u* zn!%|0aLxQ~JbgU92n_*7sjNL~;a5@JV6yh902S-GcB{YiL(X0i zASfLmQ+wiMxP7D)o--tQDOtELMmTz-b06vu+h;18ytt^?OFN3p5#H}8su+_wk0T?C zK-M5Vb}T9{>kNe1#aSdfJwyZN!;sqOKxh!{d-1bbDorj~FcXB`?>w%JJ>8eim}pEc z?#&!i<6GtEW#DC5k;9H>^*EpC8^1xdCl;L%Xtmv;0mEukVH>FeG5SU<#qbW%Q;d5~ z$yp|Q?X-}r*!Se&eNz3hC@&$+Ya5DWtqdt3F=5Atjlz9`CBztd&WQj)47L$6A$YpU z4@)b+A0U05iQZbnUxJTz;?yq{4uiwDC)^+?xs>+5zZy-@#G8kWP(q z(VK*_jXh>N4>0RAy#FYqK%8U}pJaEe`MF71fGzh3iE9K$lx$mc=INS++$lXTxq6ZM zLa!yP7bSOHd?C`!X#DKii@i>8?2EPM0Q3~tcY0~eDh*G{__{yWE^Hqe8|azc>&Q`A$(tr*{Vjb z%S?}=`Z?k(@}{FX{mmLVVeGA>5vP=R80jciY&)`)N)rfHI^(34LPXZEX#Vc1S64yP zH_jlzY zBV9x&KfE;RbcQK$2FZpdkX|@h>)-OP9jg5!wScac_nbK_Tm}Zxp4a9H!FOlYxvyVd5p6v+f;zXa{f>YFQlmh0{_Ky zz|JB#->x-_s^xeFWQpIKMOLq(-Y}34nA6D&8XtNf-Rh)RkpRPy;Nh47cK4NN=~b{t#3jwkM1l*#Rur`2D@FT9w!UqZt#gP{jSkc$AOkm z(lcBQfQV(eOW2-f0cT9GT;~uy^}GR|D}9eztj-}geqY7s=8MA{unXVb-hutPvHRur zx;UFuSFdEAdrJHL_mO_B^lgzTpUDskVr3wxoPy*zrnRR#7y9)HsHz9mTY z#KbhrW|P_lXhM@zLFT2IydH^wpYhO%OgQTzk=p@pI{>U;oT7ZKBA+qyQkzRM9(D%F zSwbJ#Dq7iw5XI||Rwc4T?t^P|;G#Za+*kV!Mus>n;I;eY75k+Gn<*Y=cBAEElUz2k zh!wV!6LNULm4#c-!=8yR$eI(Pz+RrQ@GFBAk}Bng2+splTk<1G^Tl^!w+ZEkdS^pO z5wAk0EBXZSHE0X|FiakS#e~NjM=CR8orWFTGItp=5)9RZSqS`Z&>irKzCs%tGK?RZ zM+0yNcymm$ASObA9LX}e_bS=Lmp{$2G=9!i{E@!cGJ+9m-Vu!&%wphh^=uXcomFAX zgzoWe746VQ9YFY7b02b%bJ>)Ll=SlD3}&o?>D%QM%ytaVp)GV5fE;uNDJjp{=> z%%dk(Ngsp$mM4_t6Gvi=#>Zyx%%BvLYG4!Z6jGf6{=3z|W)xWTx^lY%q@11tE42tB zSt=FaPub{?-7zTjFvA|Rm%o2W5ysV-w;8t^Pw~TW*vC=V){C0nuck7?i0Si z-WB=z>twstn7dItSRrM^VHR$#z|nJ#nO9a>8qXMpNo)-}Taw+~EgTv^xeQtokjJ+U zXf=UjB>i_S1sqqZlm|=LkM}iVT7;8*|5HHC{&1lkVYTI_j`o7KSlGkIa5@G z&V^rC!^MDK1{2GK>2lx8mVTSx)l)bA8M&je;2eAY1g{;ADQ*JnSdh~pR+k0Iu(mpU zqF6mo$$h}eYAn5r-=Y}j08(7-;zsbL;816=Z*#DxY&cJ}+Z1_R!1WlCNw7G$FTtN5 zT_HS~FF^SdWL5+wL5AT&$jO0%$N?j*YJ63dp<(k9P4VnMZFWVQg1& zyAmp255g!|ZZ3ml65h<{$a)sjEG=)qHtD;ITT%rf{v`Iy{+V+2u6eI>@1{4UrHgVB zD=uVKeUA!u(wbluME5*QnQX#lz-=L)$lXgwNTG$YZv&ge2D5J5MZ$~6;a@58aBD%MT*P@nF)1wxnF1G|ixmeL5)MQQ?kNOyK0Emi@~l}*c5EtfRv$)+s*6{YrA(z>X&4fSczu)o3< zccvf;q~;ise#15M+;TmC z;anB?`n^#L7R-41W{Rs5IOSbYl{dl&;XiCYz8;qnewW0Sn4C=88cy`1Ftgr)6IXE9 zn{}?-hYscahJ-o*8F`P+2pkWkH3E$wAw39bhsxT|70bkT8|qk>P^0yPv~1XI zLrYfm%3El3|ANu@IcA^Zp%Mz1?MV6Z=cF)Oj((Qaf7B)7DJiofbmf;iDbHBnp1G4w zTRNEx|LfT5jq2--8t`|DQYPfq*-xJUpCE-<|9QBUu~1W>rfO*3mp5P>ciBqjk%>&i z+;`YQ>8A4<-l{WeOWg2}kx00Dx@=!MXR^=#)cFEZi|*c3ERHSp)UVJy3uaPQ6WX!Q z9?hMfF;sFrRr;>YDqF^{L%~mWi>jCPP~O}TH{y+4*WDBM3Rm+}Y$T#4i{xUNosm++G$u^uEjc3=H@U@qM{XmPIlq zaD0l6$@44n1D~fqW=XD|rH{W>|2a(HEe}Y=-|xN;MgUt+x<*Ef&#sb)4FO?5 z&Iac$&)k;QiWjV-cV__@hX3(uqt3^-U?8ejb>jn(4(iv9BQCYPyd^ajGOj!!w|&~r zb`&f)*t#ldH19PF;k))jo~ z*NavRlb4#CxAGR0p*PL@jkO|WXdmug4}1O!O%8*=^32m8I-Lsml;1|)fD&O$&}rG+=}`Z#gaq4f%EPDiJD-$tpcmsXub3P#VCxdjJq(PD`L;fUQ7cI zMh{}%PrXqi&#Wjzo>Itbr=Efzl-c7ux30F_nC_62@i;j`0NCT~XRnS^vO01=AivttDQ&sUqGDm;CCG7CmO#zl53kF{rg!c9ti!8 zXRu!(#Fd7{4}2sE7B%^e#;0zBJKsbkpraKtxV5%FS%>FNb#x4S2^@tddTaQv z^YGr0Pn^Jxguk9eC*VuUpLSzk?aS6K)-+W=<|}q=aIw6%7G!UF%?p0%#?$^Me#lR8 zh4B}J^L9C@JaLN{-s8ir~akq+8rIDyX^lGJh<=9o3j66jhQjP zgD`}k&;u&iYWeu^Qu{mV`tSYyJ!QBx+w&L!IV+C7^b3e(^IYLiivT;)ZZwkj&ugWJ z1i6E#@dF;a&)CfTJ`ld<7V_0(r~KS<-KADf{2e0M{f>O^>>x;(d!$TS=8vof=&%~N zu49-bBca%<8hp4TSDU*3{PnewZ5mNkT~tW?XCVK!(E9?%|X5*+NO`8|v*`qsxT`_Xd*cs=9+;tjuRx2+*Te)aNQ%89_b8q*Ek>KTyls*=;pH z*-o*gOZsnP{7hx_56~N`n zZ(m&kU_HN_oImN2pZH^M%Fmj$KWM3+zucMs9{+?KFq8kXryoOYo!|IZEg`6S+%vd& z95e)wsu?JcG|)=)m%%N=EwLliz7&2(?}t~6RN z71Z5efAMeH<17-OKf+$%9X|o`h@Jp{Y8ZGhfLiDeAZi`ZZ=6dZAZ=fY$M5_G&Torv zw=8`=WZS?AJaX`<+lkzz4N#yz^xYT0Iq47Vk1Z#j0IEG=7=DHSR`2pI?Kwah5FOZ^ zP>Ryq?uDHq!XDWb{)yM2Usi7$DK-K~SI7hSn#GURUEXS|K7DVm>ce~M#be0xap@@= z$cLZ26fg)100004K!Zr85Q148KS4=Q%f2JjgE(6uoGGovo{nzc*~`^|7{pM)lNotY6aMomJWuO-pg3cO z(=_uqNO2G~5XrKqub_njitW4uGi&iYtjx2tyRxxHtZgAO63Z9yXaF^hsCDEX2bsQ; z00ZYWVlo@CNUCzgDTM-K45YNAyS`MaF{8I;;RGy}<_&u5`zH4f%n6w!`rIhShK{)} z?|rNsmkvfWu{pa|4y(8mEdvKCkYN7jmJ-m3Gb#jgUh>ajYr&A7h%wKV=x zb(mxmODHa=KPA%8;&rL-$)J3!xpo*~(&3O%_(J_MX)j>blWssJiiXzUH}-2$BDi&N zuhxW5=iYSgQ*b~%Eo*@8wj(wA#~3LYH_*9O-0xw9yDXK|g-BP^mCoGie)_ll#2d%iD2?zpUj!JNdd49=sk5 zb~`(DG7h-LRj6r;K_l0-&cFg{rG%{DCEd!Wmlt{@=3jvS6{nklB^|>5aGC@T0KoEp zfm2(%{|~7#DLQr=3@D+upVYEG%{n9}#3sm42^;<$D&GR6*G-k<{|e+(UlNS6O!wx& zT0m1Xw+p79^4#7>I1C)#%VdA$%I=9dx=HcsX-BE&7|eWx!tFztLHyyei5M2;s~ z>57w4V@WSXCS?0(JU?Hy64(@XROGM3C?3!Hv3 zaA3ZGGh~p?T{?ee5llDtohd_vgHlV;2z6+1LFFkh$6sNJkuW=eEC^aAdkR~(uU>m} z2iW`rDhSkYcyb4t`pG|_)})olA6AxFXdsQD@>wJQc)H}U5;1pL>ol#Oc8ytfUeXn{ zR3m0Ei@0u1b%Kq(O=9<(gx1zuBsehRQ)@-IxG9WfKAGV1EHJAc5Zqk{?>??V?IF?p zDTGB*LNDviH*?mzU~!v)mW3AlC&IR1$GSYmRj5Vv2A`h&SN^$A z@oO%O=VR7AW;emr!Ovy3-mztefo_`hqC#HIEfc!bGRo<^q+(7)vrWe?UP&1oJ!uss z4IKXy&O_Y8;O6jU1NS4oBSYTnvlq0+Uu%h))b1qi#F%?{Ghd2hF>-RafAlB7X8i2q zq~yf06@R)@|9dAo_6O*{qH;f|NbBX_&Pgl*0KoV^q0-3C*2&qx)|uATz}m&c>3?mW z*V=wg2W-ec{CR>drW;$_px}v<&8q^VjIsmW`phE|Ss63#Flha5vD0g0I6kGj-#QvU zkYo`AHS}rrPQr<#*B9$%YF2=J-vxieH>L#y(=>a-z|_pTYr@;GW`7+>9=yda=b%TQ z2f-G22Jfr;9B~IvJS0vK*kJ?L#d7s%(4L1Cwx4Z1WR6Swb9|;YYc3BvFw?cYZlA(5 zVaMGW8EhFVmh%Ii{CZycMTkm0t+%mDyG6s@)Sm{KFO~U~ z;a`m2?h=-nFLGY&{mVL+Kd8rt*Zb@499UEkni&>DOhr)#gxxB#dJjAZIoBRygRF!7 zg7aC*)7|Fg83qi@If85JKn8I#x8?$L1wtDut!%|&HH}r6 z_l`>`EQvZW_>BhlX%=@XMB-5alZ?YonYrTgiA=#&bfLw^M7xWzuFu0TyO85$e$)<) z@Aa&d>m2IAG#JsYbq@{8`e!-bF_y)|-+2E+`rnFMN@oDX+6=t~7QT^Yy8+BtvyXM) zXX3%#Fd87_3TGE5w@mi=jbg1d=^!%=a7{nk>3}T|8Cu|*k_E4ooFh*?1+S}dQSd5t z;$A@nd!peN4;~x;N|r9yZ9Kj=^EHj`^$%M5df(33r@b5!YkTo;3C+A?Aj6?R0D&{5oEu#CLc zfGxvt=?=CmA?I%$=RTGmx%=@(!Rmo9r_$fs|5VO~kxj^QT&(-D6fE||7mHQsgx3kQ zj076&97e#QSfAn559>9Ek;@Qp{bwv?DT}18AHZzFW!2Z7YUHJ4yH;@C1W)1b)2y)} zou{w{pj?v=6Q%QK7K;@cLX-XhO(k1TN1094dDIW;xD|J=Cj*D}-_&W+oI?|<_*!~q zNL(STnX)#53~WOb?xQzw{{a6n+8mX`IctNH9jOj!4!5fBtPPEPE<=JH180=DK?$o1 zBoJ)@xC+_S1LnnuN~#m%a>#mVvqpR!el(~tD%m&|5%nAf5=1VY(OAQY_W_I1hG(#L2=hm5xy~@iM3bP7 zHQx>cz*yP$L{Vqgj~;mU6(zarhd`-Clo2QYH8bWQE3}m^ii^RFIJ#Df-dDlxd!#I15X`;PA-~} ztOd>_UZRaHCj^U28S{=wNVt%cN3!^vl|~gSl1#5zmn7PXg|IZ#50;Lfh{@29XUyG| zUALF|vXE#pvCIWkW3|J_0$gZ=uHQin23o~)j|jL$EUcSI`hAW0jo~6*Fqyk+NZt*w zeSbh5t{&XY0$#L3d$v^|?M)J|e--O=#p4NRK)e3DFdQ)y`dD(aKN#(w^A2n$lL3p4 z(Z}PBF>tE?E1*C<5@Q)(%}B81?dR;Q%r_6lzCmg&wF{uGAp(VkS-?$~r(Z$h+ye_f zZKb&M;a$@XeNG1=^C8qWAC~AzTkhHVnAuta#ITa7-ctItH^6)GDT&Rm^IA*W<<4Lb z{n1q%86h@O=7UrN+D$%&&c(^T+WC#!>YsKTr*U&-=U-TtF-)xV+~a44<8HeL1OV@8 zv~@m>i|i(CkF1Ou$o|gY0#1%22YV~=HJf3@Pt8rsEZ;rP;W1N1IDaFp;F(CE1|p^F z*)jq!UE9lJMwUtB^jBiA!hU>Ac9O=KlO~MLP!2;CR8rMd)0A+cwiuz9IAXhJt`Z`p z>&ZF>Fe6D!+gz$;=v5*f`fOsI+%f(>@o^;UI1FBeF2Occgz}Ii0I!O0~C?SS)N7lDReZv zo(*h)(L$za4CpAxrz9#W%1{|HB4K~)?pigF0wVA_%AP(4WcRN{klEw@)JzC2)%tNB z%cE+Gb;K`4wjBM(G%0_2=tFvSWE4SV%3-p)-n3I7bHw;!HWE3yZ{#Z>GNR9l#)8E; zUrBjIddi>@l${lg1&#)ZTc5QQi|Apf28dH|($sCw_>%?WdWv+Yaw-7E|Kne_O0t9C zH0D1Qf2Au33?{;5Bc)4A+(~sWPIby(HIYkEVH{Obhb+!hQ#i-fx+`Q) znF^DLknJWV-oOLj^!5uY4s-rvmXMjIl*SRg2!E;kRG$qL0-&s9q&NGC#cd>rRJhWV zCD1Ah=PSOvb0Q>XKXgijCkvESc5)HYQMTz;QzFYBtPxiiGTFg-r6b@)dF4a+u0Rw@AUMhaOi+>TTnm&OQ6$- z_#HzS4qJY5-e*8fMyME2^+a5?>%E%#4rlX)?@Q0K6zBaCK5o>ei`gQyxrN9X<)`I- z2!xisqRkrT6Yt_p-o2=Y+FjJcP=4!8Bw-T8=q(LmvVmm;o-z-`nNdJlD;OUh>Pti~ z?0oY7S0ieX@s6_=7y#f1=KsKmGB&YyG%+%8HZi8Pws3O(Pa{fg+ipV~!SA#-y#t;U z{sm0XsT9TOpmH@^Q6OUTBDW9}e$TYNdK)xS_IuZE#mWd-Ao4>|rGix7&Gm#m@6&0) zmlfkR$d6p;4Z*uBXuNc1+Xf#et)5=ZX9TRj53i|f5)349lxjf%$*F(M{>Z;^iL2Yd zx^vrelbSvO8f~kwST_O;D1ns`h04XEEmLf={7&V8R713QNFfkEvH z+bCJv2x1Y}`xqOKjJeECp2`sx_7?v3q5k;=$Q+qwm7x!lrFQ>eHfv=wtI`UC4`i;( zT3bVmYXOKGSekT(X{uBFoUQRd(<>xXXiDvjn-G_qLCP?u{sk9!pDok&aU+<#@C}j# zYFELNT%H!O8Q3!|v0%h5b2`~kpVC2h`HsorWYP^UdQHEcXVn7f=1gqbIQfDqb$nei zC$?N1_AKH=9v$*nK;$l(m_AbIi!FC`S9pSyIe7_PZ<4jA9bgx{)dBF=q~t`Ry{L(Y z>e$l+%|m_`>ac4Pre_snvSQR4qbV}!n<<(t$B$wN9NWE!AQ~kPY9K%LDzQ1ZT)G9S zc-2%i!=CD-WZ3t9&a~R0aLBR%EyE>EF(~&?n-wgS5h8WAvbTtO&cjXH1a9fL3p@i$A&aK0cfe<*hCl|J(wq)mTv;Fds=Fvc z-ptV(vX3HwINGT|5ak07oe24_Cr|cM#i`wvr@ORZh`*7(Y+gzDV_VN=?DiI{dGR1dM(O!C_e<=>*) zr#}UEA)yXPhdVdn6Av2HHE&Ly9Hxm|d+}|dfBP*qxc+E|-YJ;Ywu9j?+qpSjUUAN3 ztJW597ORO4Vz5IW_V`F4YN1jqTBNqC2-0G>qS^>>zz(U>u({u<M>S+)XLNV1m5>5Rl){QE>+yn@0qKQ&dXj-=( zn2wS^H(~iiC_bj*j$$f?36;nIK^HsSjTyC%%JQlg#)z}kL7|KsLjDQo zv{WzzdZhJ6v`Q$$h+Qp%rG+is&l@z$oAv#t>B%TY)i!{c)f;H+?c(~}Uye$%|7Hj} zot4#V3z5_?b!Y7E4QNeAJx%c9=h8EvBtWOm27#fLnW&-J zsltY(TSE~O;CBQZ&4>iW<~s@z%piYzwsrXGW-&4pl7Y)e?ije0(`O2}KZpFqvZ@`cmXm9@#6xcHdE@?9tdk`b49Q}MhZsEpxus%( zVb#(|KI{|`IVl^Co7+Y4w{Ot;!vM_9PT$oHzI8EASB`(FamIuM;6NHfNL69DT0M!; zQt#N17yW}dX4L>RmJ&(&=7wOpSFC`5imtZ12LO~`IoPUOHSFLq1Ti11M+-3OrS zOR=Eq$+|g3V$TR>&#NdabtD>=k!*v*XqHzPwdpftYqqNqXbDEV_EV)<&p|uhnpg*2 zXrBlT*AKw)1Ug7s1!a@3)ex!C(~?ZBJoxoB3+Gyg?F%<4$~okB6fHku)#u*gqS}R;ZWsJViKTCP?Abbpw{XPr@CY22!k3%l?P%?@|r0 zG+R@rEPG0EfFN8zPPl%TvFP0d!U7a@DGCuhuf6S)bmduSmM>lzP-iox(GNeTdOp!<}gccal@h7mihOF=|3uE z=QYVGB-uHp(t(yrSnW_tG}i=&S>oMtMfE@7bKWMKZoXj>wP~Qjp?YX}>Gp1I5>*#h z%k_#ZSzDFbE-lY=jM&`wua~F1h#IC|!7Irs5-SaphJ5WDdC@2}PcG>#MRYgdVWf6& zpN71I=1Aj*G)IlQKedhki)oi7wJ?XRUro7p*~}6(=jz44eOVy zsK|R`5X>s@NO2LYCBY)2;`P`ZYU9MObJxwyTF=Wm7Wi>l;i712X{n#yRWrMgr@5lx zz#aG$ZWEO7n&<%aB(22_bq?Ut<&~6?gB@I~(u`tkv7;}H{SJxJc&;({n)YG2 zj+n+{Y?<4Auv@R

      =dtYXLd>*OvZ;JAfDZ7E}7`T`Ia_6HvfX*!4ItRaVnUENCw4 zi)Vp2K-;KKz$l-fonZHFt@*J5DAhi)WhKmcJsf)X7Z?3ny7I!=n6m*wP(7Sa{bIqWgSj%NSR*U z+JE2S(XcJW#Q znJ%~>+HXshRSdqF%s8p-JeR=@#QyiHWinmb$BUdLvdYlGLS~aiuR8Bv%o9ir?xx*c z$FxrAwbeuuH6Uz?Dah+KYiGeG3P-7GW*!B?`pin_Y4ka|Hy_k{C{%9-rBsiCN?I?=SXiXBdACUL zG&iPNGW0IgfDWjyk4uV~R+>z+)-Uyxr5(Xr{$h+6$J@<-!%D&!$q3f)Ul{!Ss=#t} zC0(M7;uMU5rbkl?O6avtmZCDrQ%KlHQOG{gx~A7;OdHPf_$1AmvfNsmkO!Gc0$C>2 ztBO=~^^6HejUniQ+Rgw}T)R zX0BcZ&(c54z6%?n(4oDTFT`;5Tg3ve;(N7#ShjAm_P7G2EuS?yC&_cvVk9z3(&p4S z%2=Z^8|&hxEcy2!tLf!|-2M|w{}j~N1En9<`W1fr?qbAL8Zxte zxJee*%WQVEg170+WogBbe88BfOT5;w;j)sW-r!s{=Q7*tcjO5wa>F`T5~Eg9%5Lz3 zKp$f^@S1dOXo+%Qxaz z?nl96e|N!S*;f4McR9L7dMjsf%s zsj{h>8A#$2k*zPix8mb1)|0e#_dzlVw>RBhMJ_MgN$~88{CKweY3FuscXv;V+n?C_ zv8Ug@zK0%3=S3TB_TXlBvJaqCOSx`dv;M=Hr)jxrsl{5kof+9)i*dSD=;tBm%i`=y zgxGm+xT67HH27~g{SM~+eFVpbe1;S-M51^reR&G`M_5AX)qBW#4-C)*$P6*i>G}YJ z>iKipRMI9 z5&jD6kA1LR`I~?4RT!2uNe^|ot=JJeNWk&=g?nBlKFheC!t{7sq|IwQut5F`LSc3N z{3gc2qN3g@LkMVgOe&7I!2ekfrf(1CQ9_Zz*3zlrv{jALL)Uk(sy``E5-FxQ;Q|BK zwKK^eR6ti07*YyR7hU`#0u^RM+kGFCa>lYfodR+Vvvx73iWq)Dn-oLjM*c04M)8n1}x zvJG1PhND*5l=aAVI9#f~YzMaNE|kI5D!4ATt9AS=xK@<~R|Nl3BZuGq5FH!euM)Kz z_TBcGEy9GatXZ_eoBnkl-zaF@f>o?Zp6>QYz}_@B%QC4Rf31iWfE}#FUhWwD6JKmO z6dHtHy91}oP#0q|AcOcK{+U)5Sn13FA%=0Ea zv7>@%)pE%a!UfU>L5PTDG6d{-weRG=v@cACV{!ySAfIt!?wPxBI-`c3*OfV_Fl^X= zY#HWfwHf+d5ZGHmas}m&Kz%s|Htc_*kx5o^T`P}io1h}QaXxa)pb_g=Ec&J1d@B4W zor3T0j?E85RfFJwv}v*6`A06`Vn55k;r^Or@Fhxd2{ZPBAz$tYnSu`!8)bztT{Jr8 z6hbOw>XSSL5(L-C#TJV2*m-6l^eyqAjW0`+%zab2^|5J+RFzP>%O)5%;B1U7g@fo1 ziSX5c4iO)=WS$TW<45Y{;in6kdZF@B;q5&mYK|JI-PGpSaf4g3;3)a;e+}QZ~y@4|EGPx&eg=x)xyM$*4feIKlKUM8ahrJ94J1swesEYGI-V@!KQ8v0kS|6 zS+Y>l2d-v$Oal<|#58dv`Oc-kKD9KPYyCy~YwPFI84`6S+g`KZT#3WOm9DMFzuTRn zx8HGYWv}NKH!UGJ=!d15wXF_Phxid8uj>|em}?T@V%QKceq}{ zT99_yuy!a!@thT?g&B#0whn`*BU463NrbfF0A@JP7YNP_?bydawY?tVe{F1-cydDT!vftmzIz z5!WS?j0hZp&pI5g9U?rU*9^vk24}8zdw<`mL>u8R(!;`>qof{wfhcf$Gx3ugjU^vq z!d@zPLALCa2+)Owem1{Rz%4k&==>lWn05$JCr+KlBd~yXU0;SEaEW-H&;S7`G-FB` z{jj#exsEIPY89E8oQqLGPwe&g86G+UvPemVmrJP!LieaUopBAL)hU z5aSi{2gH#@A|X`>WajabL-yw(B6fCGDTwHxLQL%8KO$tKP)%CDX=Ppt0HY|^AX9rH2YFKaYnevR z=)m4La3qj93I~8$J8X-Wfi;f|;zzlGXwKKckr@Vs3US1Rm3z{xxbNIJCSpex%;ydL z%aOR$Ag2nmA#OLsna@A(Xtq`x!1MsHTEoejrWUo?zmL@>7An1VXAf6^_9zNGq9{sL z@nSgg5zz4!p=aZ2W}#L!ss@)Gr^*lB7jgUo^9D;wuYNiIWI{LV4M-NJn0U zG{K}SXVTTT%!I`}1&|Y;NY{agl#xwp122KSDxFA#Wcm#-g&fOdI);&@-~tCXFeS(s z6JorpPfGkHvZ8@zy%=pbmp1;e#?+ya?(Q16QbksIdGkgxVxcjas@G?%e_%j0$E*cp zH;(J-g~+e4I0rp6ghn-NhEbtlfK=}eg8gUf+OHtsxUqE>eK2+jRmzbkR>onNN9cdiaP1^U)qnq`Np3BC_}3rUEr?md7%aCCpQ!H z<+&WvB0bw}!AN=3;FX&DWCNW~kCBmiECyM@mxHHqc?NoIKJiHXmJ%28yyy?-Nv7Uj zlbrjOD_)A`h~ZErO5>KUx8=DL46T4`6Rx@)ncv{F8XOg9PsLYYEw6LvKNz}3uk#qi zJH{J&SJq@y`({AEv&;kQt=l;IKnJ;tPY0RJVd;8)u;$4MCp|{F7Q6qbReZx`Yr$Ww zyj<+^me(8F3;alqR^s9=cNdV8Hfu2yP*l`*%q0?#UgGpqP1XA&c9!LuwAz$bjrAj4 zS}8C+@9wS5)Ynn3xx{U^bt{(}Wdjcw&*kUuo6&>*fr)ZKJ#$*JVzYvm`I6Cs@hd3u zLe+=Wg0oto`QLkaXN@cV$#(O=w^A-F9VtrE(!b~X=9n?0JfTUu-3PmnS>XTuJXo@m zY2NxLdE-L*Ki~_TOdKr?tSvnM@dc)KjyC^!aj0%%cff)0v!k~^Ezx=%HWVx`*%_>& zb!Mxk6S>h;`l*a*7}@HPWRt&T@%tHbO@g3H;wne3Q!HXBG4ePSZ8j1o*0k$lM<2uw z&ilmq<{1PAkIufDHrBxSGuS!QhV48pI9T3Ju}6X^-M~VKIMUq{QWcTBjydx9zLnu6 zJCWyLA4bj;i^U^^} z0s<`h8Foej18%_}<}r{q@hYc-rCiJ*F$jIXab$;c@P&-BRSs+HY!}(ZHh7t?6Yrro z6XyI3?{T#76@8sY%0pXB%v(OLl7zgV;e{4qThknc42evU9VK#$R_mn@A1;7Z^Q>5<4&OpMDQFmL`obu;)ZhCND&$}EP{aoNo0aTrI%WE#Ro z!hJac`b-#11Tx9-6G|*)9lzKk4EBf9&l%=bGY;yKCh|#<$4cSr?Y0y&4A1~ipBy+r{&qZ zj>3r2gbzDU)8acruBDLORP#gqzH{sI8u-xyLr3(g4s6^dF$FwGFBq`_;YgDGSrqer zmQB>K8wBo`wzvS_NH*&$DAfA7R&f*yMpU`w8olQ(<%T5Zs;g5EzBnRh3sp9pOzYcW zL4E#K$G>lLVx&2JE1rldap+N(clKg!Vv)LQZ}xEKvZp=p4S|0vN^qwO;O->R5E^Gjn?e@nb-?k<*JHX&tbeQeQP4wpPmxlT;i9QieRgZ(N}^1q~y=C|A{Tqg?Ck zCEP97H@dpXZeENOFDRR(+QuQIqH9gm&V}jX`c&_WbcIuy6(l5j$%PVp~0v0*09IQbs^WT zRJE77)T%yelC#(0T-YuwWhJfa4l6PD&7#!>Zr6sRj=c!}==)i>jQAxyrE0lZ#Vj5k zew>$kEm-ETbp89rm-3k0`(?eey8l`1i~AhfaOS7bZvR3!Agv>Nj$2GP7q_UNQ2O+69F>O{C1qTSl(Uzclbe^5PH0-j zQwY;pF->t;N8lu_KRa`A>=1lf*?t>o2iox#bZr!kZk->G+OwGj|M)tM>Zdtb6vvK1 zv60;d{xq0J$dAwm=oFe!Waa(BOw}f1ZtxC6J@c2hJ8)j-q2pZC^{@AnOV}9>YCai+ z8M-RNe=!y*MR~``bK1%3ObW&!6lG%bK0kHaJeDZ`4`=5XTnWEy{fTYcww+9D+s?$c zZQHhOPi)(sU}7hech5QJ-l}uIyj4$CYS*rR?awQ{R{y#i0w|gqm&c$PJJPVrXGpr5 zX-}sS<`c{7tE*r9ji8AUQH`Yr3OBMx*O!`3vIZ5S35~CE{3kI}8ttyX2Gf*z++$qD1gCpPvQjNbMSa~m{JC5 zQm9VgYH&fU35{1&r&b1LDmiFL7k<$wGXzf%`_euwibR<~vv$HW(d-?mRsqb>%H$u@ zj&cplE}uM`&>nL*1m%GwJ z>cQY^`(idTAr8Byn@;6Mss)zvPVouqy)^vThJ!M_3|CXPRO#1c*wLprvyfy7xAz*DRHFfaou}3qGLd^-gnnhjo zm~d!JlY9L$^5+iQokM=FS^v4kds z(mqHF`HyPcyb|}`E;e~?cA$X(^$k&5rSyQ{fc@Pxx+`O;fq@(7DnSDl5M z4(nqB#--ZUvqz6`L?vA9R6GJ3j6Vcp1>AN?o*X8~QtA)m_T*ARlNG5t;6#Y3fh>VL ze2Ox69+_$JFZkGyUG9S2q6<`ntm0Prhk_8xEO}@zcSwmlpaXU>YJ8qdJ)*{V-{{bg zzoSR&4nwW8PF2d0&5IsPkjxTUpti1fo17K?KJR%aD!_Yi6; z;q=irUO{%OQNgX1UTrVS#=WST{sUt2!?(xR8@`Q&G@#{cubSx_|GJon?D)owRS)xi zlA6(ep=l4B76)ka7?%~8u7DU?eUVu!b+BYBTQ@@HSFdNq!;|eOM_Zaeo=}d& zIGIOKpx!#~G`q?p=S7yhv`Lo@2x-NC{nXvYUvk}Y6$#S=_3(K*QWVeA+Jr@SDGxdy zzs50d(B`++Zt-F7nRV@F*W66lfwyXtQgj*I_H1c7OtDf|!-MQvy1ZPgvH~+(Tn56- z8~k1?6}5^Cp$K3X>xqh_OU;&Yxh&Zirtfc`sD~(ta zntSLH1KAM;B^htzCZ!Jh9wutk_G~Mvv=s;(?dEtqIbu7(pVzh~hPb2|RN86{*~&xD z*>>~VN)z4-acioU!*_bZ;vBwDnKIn)RPuLWx1!d4)<1nuWTeTcr{r2lFSZhAwh`)C z4u05yfc{zQRb%=!O#`O~ce(;Ah?c(dhzNDbVeuSmRw>!Z-rPmPp4w$RZIi`g-srE4 z3;uGM9ycEg&UsZI*1})^Dlr`&w+IVPIbGgUalsUOa#D{`+97orVrfV2HL%q+sB%4B z)q%(IAWz@Ap2k~8hMX{bf!*Ll&933{n8F*#Q=)qa41$3`xQSa;M2Otx)$P<|3s~(c zM|@L~0q&sFgArxc3n5N}j8~mzb)Wm)-k(c|Zp1>`8I^2wj@NOtYcmT2&wm@R5INCy z7yIEIO&<+Y<^;2P>)_0)jytVPhp($<@I^#;; z)VJZkQ2rZ5QG{}b{sb6HUi5F@*#3Xj(!UxGbY>QSl8NEp--)M-lD*Qs zsKxTW(935Su)%S;p50uPJ!;4F|hH*X)(n*;`xghb6ii9v|1+8wvEzxQ(VZ9tkarwARd;GBy4!`ao_(bYMU{tc)S zUoEzo5KB_*0`8d`{#P9KN?plDv{axGb5So!ASFahqQ-*8TP(2)jcS``<&u%&M>wZx zFUe7)B2$S9w?wTHVe&)R5D5`3>#_4CPj1*MA;ad_(5sw29gXW+I+x5Kns)?JoKnZQ zTt=08PN$<7)Bpu`5woItcrc7fvap2r`02w>pgW%ecD#+Y>H9Qj)MG`LWxVu7$&Ly(2_QYW}@ z9*wBUUToF<{Xc?qM3-j;$_ZFKfjYZH26Qo8un|sU^m?m|!C~jZk%;=KBsYj)Zbb2G z(rH735e)cdX^JBl6$$Yoo)4I$w&Bc?AT=PwqRbPyP%3Vq_i7{ZqyVF2^rUMoG?gk@ zO*-O)=Hf!=Lfl|}If7O_gu)BUw{F>q^CZcEG$&11`!w>ZLCV+X%Q9H7CRjGg;~ITC zFrB^)cPZIQKUw`pZDO-jh-A6&Dzu!i(hbbDhR@NDLwinxIjD?N@I%(p7dA%yq%_Zk z4zNb__gsR=a#g7!~U+gyG z9VXpBOR(Zd02?SZ=9%M3y@d)q&bqsg;w1(T9#0{8pf(Cxs3HsJ+#rqDnbCr-2HlN* zVxNwTYeX>4STKfWdj1eEjh&40M7`nA*{C#pD>r91*aj;)zx8&(OQpjmC?`janUubf zYkct0nB-Yww_>=c$9K+4PC$cn44uOwXd-`t`9q2F5EATwyclR(_L@;Wl{L5++PLWj zJK$Q?b=LgE(aBdcBa=_=od8LmZ#;#(!V=;$)lAX3_Nx!tzFdOHssT3OHj;&r5kvcF! z?7xYjnE9RG6}aC_TE%~CPdnA@EVXijdsF6WcZ?=?Qn1ZjyP|+AL@%LkJ*!30+nXE zT+Qb)_w<(dGt{*oeaQMFb~&ri<_UZ1%ZMs^_Vh{zL@zEyH+Re=nbF3qXw}Lar?P@dL6Gy4eTH-2qc_nS(zKgKEYn#9J zDtkFJB6-~UdRoP<=fTl(uFpYWIoVEqsg35`s;jV;<_K+DDQ2PWywK7{)a!TUtE!9s zJIg0RmOfn*f%`>?-~Vx+%gjx)D*!#E0q80FKhRUg{}R+)`F+!c8lQxAg^erj!3y5y z3y}aCi!!ee86(0f$-=e%x#nwMzftztokuzDU!K%TQHLLYX64zJIi_t%UW!(Q@&dcq z&-s=%+u;5RDwmbP7HsCZ=r9jm0U-rcoM<+Au;JD@Ku{qQbd9<-XuF3q(oT$Vw+v3$D($_#c}~l9FqSQ z60eDsbm1{K`FMCC^Y__{?vyTJ5m)dK#%pvhk6>@y)&kUmgobIE0*#Vb?8xCQ6>suaHc>?+C|OSP>;?%*5^u%z<^os zj_67|{&E5xKp^QA6KIRzQj33~{(MpsiN*Mxh$K5md7)bPWDf?VCU`20l%T(NoTfOO zF@X>*qV0ofW()3&8nGTB`sB9Vo;-2?xL+6P8lb2L*)a81`b&x~Bhj(n=VCiFpng@q zRKaRpfMF%l(davf4N_2orOlQqCo=G9!pYT?xxxVUS)(<>Q5k=}^#6Jrm{31Twk!pr zZR4_45@nsVptPDcH3%%VsC8|lK4SJ*;WpZp|27Cphz~QC(SkcfWqg{IKWE! zQth<{&6y1qokQUA5EoN~{9QK^#FbOTj;4T!o`u${vH*%lnd<{wvZ_A;{toR5R=10K zstEg77m`g)BoXf|Q_xgi+%rQo`7*oABeO0!kKZ7xiYneKj7ifZrJ^v!oTJG-L{q4CJW;cv) zako{|OZ4k8pTjhoRxq9=xc611xt-uf9II@WRAwQ#_UuWkXhw69*>!W;_s0V0W=#FE z1-@+%1a?8LJlqR0)0FrR1`C;~;%@y!qFMgmgcI0`ZNxLIg@&RkGEw!!Gi-&XqB`b7 zEoirTBmg-@z;>__P0zhqk7P7ev)UvYsd94-cd#0TXY##TCmLz!13Wc71DsM!87FkJfCYb(xu-@SZM%$HBgYjO;vZ&i(4XULT;)FDLRVOLOmgOSnbgoE z0Pf{My})P+xDL75_Gv&k+ulDzt12sH#&%TL>u-Tq$Te>4Q7YJg9ze=*eR`>`tgra3 zVrWdU_lIc5vgoO6;rqzx+Zgssiopk}d8oP`~TSVvUR(TDt z(JDh~5pF*BWtF;Iug#@1Kl^=^$^mR1@u)l&8!f3} zL#~^sx26pnMwsA1t>)TrbWo%=tC(-QZj_04s#e^vKe>m#8zsEWUmQaB0D2_U3yo)Ms4SO zOZs&Q;rb&Zy9p+*q#|FBl%!0t;%l#SuB3`TL&;t@TLII6Gy@!77S%{h(!?L2mRt#} zf%Dk>31krqOxd(l80OfOJ5JZu?zfky_J}j+bA`s`x`XRcX&AG5x%xt0h(cG@p0#k} z8q6DK?HrOK0;@iJ)k~);$P#1O;BvvGxR_-T#SyT$WanyDwpU6W!v{zBoTdRJN12LD zMJn8ct60!dZ<=q$M7Ze=+)%i)_Z)-_ehFDH*#Vhq=iVqMlrbwWWFx#YN3GmwwYe_X z)L3Z!^1J~ksj(4KJ!RG&4UwfBTprjk7{ipKiJ*xLXN$^^!1h&TH{$e_{8;;#8DW|_ zQ_DdP6GuZHDIk3+-U9OJb<>B+iRf{P-PYa0tKb{^8sAZ2n2j_@st37s?J;ki$ta)C zdx48`=^Rv-`+W@%^|nLiyxf6D-i0coMa5^-y+-qgAgdS4Nswm_Au1m@zPN6+h6! zw5dPAl)oN5*Qq--qVEa`r0;TLx{o!(#(=uMHo@1q5l{pdVozM;_?IDd{SAMHU^5k3%me z)dqe$r5(7cd~yVSiqB>iTQMKQB68R{#eD|*0+gTufs&bXLcjfSrVCrL8)iAemP|lh zGwQc(@J6{IPIgd+Og8!NQ zv1KQIbPvK+ri#~uZXi4Rys|qn(9Uz>7a=l&d%cta{@kc6YxAWuMWCg(iG>ZxlhYm# zn@Ve^6*8k#i|ZFUk74JT!7vNFc+-oCHySySeW?|A$EK2v`F=)%m!gaCyH8r|`O|v> zNJ)8ZZ-*;S?JYh(v^V#m?3@;Reuo&=HF=HkaPG$_Q7yjCU)1_O&f(Mve#J=FrFwTi z^Y3_l{dnGy1@RGA*FSu4b9tvTtt7!P@cQB#i4Dc~xtF&1Q{-55NAcg9TT8FnOA63j zVvK)QoihKYI@Pwr=0f#7RnL|yq4P-3_)dRSilw%nNR9@?E~9mY!W<2?2z!`dlV5)B zw|m$yO~z=`x*(OM8jsAId%e|r=w$8mr*hpU*zZxx9&M+&z)jswpQGQw=j&ea=W~{! zav$dtJPo2_Vx-0hEm_2M8#g>B3RM%!E?{$3wWMe7d z@gEPWkl)4!4n=a2%B+ZKnsc~m(CV^6Q->A%>s`nNp&;q=2B zb?s;ZV1hUS zBqVbK=j&*5v7bwv;OSM9E}sQ#amLo)MkG{mfFxh|hF1#JugFGC$$c+(e8=H~@b-z- ziDH4nNG17YhYh-H@kJX(I;e1HP(pK~NCQHBn~asqt|aOHYOc+)Jc!04add5p0$Vjh zU6EtJC`6_B@ZlZsFT*Ipey+{ER#h(#kf(YsQz*?sq95Tg_e9qYySH)Vf&7Y$rc}D) zR}W%M<~FnLTXwG6Nl0tIgKOlR{0O!dp{Bu?`x%fA!GB%SVrQ|KUbi_@Ha^E8nyD_I zhd^qh#|RBITp@*oqXe0w5zE0HG$P)v5b~oE?_dqN5FNM$y=a7&<6=v6VH#=nP=#t} z@eCtQTGe41MSM148fEm*g=$#71DlBh(1dBb@*>r|%91LngS!vMw(07Gj{|SRc(5p1f1K-bu7Wr~Lx&C%Gbh%vV{R=idM$sd4{B^e zj>!Y>o#&PX=e4_3sh3XOA1XELc6CB6OXbrJJ64<;lHYugVwohzVdV)`ZUd!O+%x17 z485_Is7`!3d?cr=;-DM{)$~t7Y~1pCqj6^rCml&J!=I%%s9HSwW_-})a%eJF9T14E z9?CQ#H5Ir(aMWaZUZ7vC?Ym0&L|Dff$hlX~R8Q0qh_;mUQj)^FsUQ5>Wvc~GTdi?r ziL>~8sXpwqYJ)hhjbK{e_anv^u%lzA^lHM0hWL3u9nT;ncqJ%b?Mv-DH6jo+KYNPoMfc3 zi_-qmTTgFLbWefXn44M*3eM+#ed`z?Bd$VvmE<<@8+*6im%L5)m)O@kO+Q^1|159? zK0mEfIrHOZ=duozYp$GLm73yhgq@GXb~He-lNcU22|!C4Xo}N;z{cIb#F)yGqc}C% zh*=w~pQet64aA~}C<@7@b%_-v{G zMtt`84%n+DdA4BZmD*W-2}ebNm`GHELm$fAA0Z?stj7^-VdGTQ+CmeG_OegLh`kgO zEDw_Dp6m$n!YU>#*XJ-Pi44Lk_U`hNwWcGH%5KcwyqM1bP!tgll!w~rG!o}`ere8=6a>|E z9I#)QfTh*)FSHU?C;HR*ukH4sLyI|eYBJf<)*)KV*zizUnT0sfV&am=2DGe6t+GhQ zzOPIL_fblH*@~4p8!v1Z4)YTwOhvJ0En5Cm=2<^8SdBa4uYn9@u|cy=QLT!!tH%1o zgW5w;4DxU!LCED)lvmV$L=RPUWO+Yw+zIxn;SkK z!7cUI!o@;Kxh(*K0;gSZ;5oroRm8nuI7+3c%$93Rxu&D@eKhS>;xVi=I5h7z+jzow zqfF-}pQhwtojPmU|U+O(Jt zTm}?<`?v+&3nhDE8r$YQma7U>ok)I;vmU6ANH%2yEvEF5i8 zA+gE+sT7l@by%)qtZvm!v+Q2N7AwuBXBCsA6eG|w5~dV04Oio+RhwAosaBgfxIwKh z>2Q@?Rowqaxi*p5ORqLj$V;&{(a0*Ts(8<;_gG z45AHQXq_}L#co213|tHwh>_q|BP)nz@RxPjKo3HQM{Qa z9H4u)=Y;lYsRJ7&F%18J-6Fx|WiCY&!}UxKWt6fnT93m-#QhnsMi^0X`0SHhNdHX9 z8BNWhsIPk=XlR|$Z>`Q9^r3c4OS{xPf@~b{?O;oWPJQb$Lc7ZUhgbW%+D}Id0`))u z`hOt^SQ_2zPcMr2DzZ05ffVgCB2HS~AJ3V~TBqb>(Ca^gN+AYG>MUsexC+DAC42vi zhKc2Lw@Dd!mGyz$kyA}Wo>bd2C z{i+SKnO3rFCIUo6HGh%!kdTYt6Q!@C`{rvs ztO0xv%naib4CRP&Rhg6k$kNtOho<=9w-#W^`3Ya1#T*3jU0m!ciZ2|L#X^tN9J2$0 zeg&u9=$F74a(uLI;AEyt+hPW6?s>?zCYhZf3UU*=YWi@X-Aq?UjT%#tME}OXD$Vej z1xmxhl&B`a9YUvtmJWIjb$U0tKR$>W#Tc8$^CH%_QU)GCPBFVc1&+MjKGFF^7u8WzL4dn zskd?J_|;Tu57vpxf2Uay5Ll`kKXBGlMXn5r&xGslBt)k5d^od@{GjA2BlwHM!vP-- zMt-qey9&R2exr?3xrjC|c9|Ei>UB7Fc?#zYDW`fq$X6xIN-Cpr&s)~ByEGh+jZf>1 zu#~rHqx|FlOy6sM4R+&(h)OEdD2VQo2*MVZ0<0g(XcLyWaAc*>|{-ty+cs(`iv8Ki~qfy~V+;{ff$tM0hRsw!+T1UVttf~J=z2!8=t@BR;=YnO_59uQDF)k9O zr13`PPWwZwB*s{|fBcKo)LEDC(fb%3L7|_L(x+tjw@Kcc1M5nHYNZv5nNif2&u@oI zhHK$rWt1}n!t8Z^;LC4fhUAK4q*#vONFs(%F39s?tK-E7^p|OZ%SjR*TYP^ji zy1=JFfG+yYXYiGO5aa1VZNc5P1`nh|RrJ^DDnhhi%{qey!Y5HH#|7Ab3WoUWl{$P(3aR(VpXkrNvbo1c&FqqF~jXSBpq_AN}vde zbFb>alWSzE^>zw&nArt|lOK0`ixaW4w>;hEDde`>-_Rb)-*k>3&$j=uo0-&_JGM>- z2XsB#Oo}7+QsY=Kpt;5IDhWxTqgz2KQuATfo1_g;-z)4H8BNpf-WKigSt!4$Ni0JO zMuv++mkeibzyOWP{L$$7Oy0)4Ld2+u)Hve_zIV=;7%t~=`pP*fcO!LP7UwaNq7*_}4k|t5D9Fb?z@APIXO~X{^r}%B zyxTM*%6L0}_>5IRkj1V5G z#+QhC@U0Z~ImLSKr%O|bCm3sS_HqRNs3vAi&K?_$e`zXc=uG@8Zm1f60OyJJr37f-dEN65| zWSJL)S<~ZI#(Lq`&oygGeTvi7@mGwjY;~xIz|bF64^f$UFf_}#NZeWZ&#L0u_zj;m zY@eEdSuBSCb~~d@Zv4prfyKxEXKsh>-vX;?1DIj^>vrZkSk&(Z)Ces$te7~U;@NXR zIqerazzqkCd( zZ5RFytJ=&0chrd%$Yo=+1()4+_?@Gs&@V|#j77#|h}pIQ?^vo-TF0bEpSE*1CH(>l z!N;4z;u@=&2zegD?0jCbAzvG{@gK4T`@hMOHrCQ?3GBjsXdhx2WO5P}Jq7xH)=GDV zp2dOJC=|fNf^%%9b8sb~5xRVFk5M;EuTVvhe{OLsgOiwG(!AqjcM$JeYxt>EIbBXl zNzQW4LzM0~M;SpDQ*uqI*rw!Cg<+{d8%cmzHsuzq!$e)0ECn&H3LTi}o9a#)98{Y9 zFOda5GR4c_+Wl2!RsxU15hmFl2*{ZfsoR#R++3=Z)lHHSy$hMBn@mJdA9EXzMJKXm z+CO-a>BAM_iEhdA7EdY0o)fgW1pQ13&y$ehb3OVGyi4(?@&Q3p7VJw&u!9)p%E>;J z+)A8bb9rA_!Q-PpNx+~)JW&v)AgbDUb`g<$YM^6XC>rL2bwpM9+f&fnfGNZG94l1k z&T=0QKbQ5U9&WSzZ!wriz!H)R?3&*&*tM86VW9}Vcjc%{z!(;O=SH3#5>D>ao^v70 z$3vwnloTUZ+DFURg}K)H7g_SsVh12gSa`$`+hc-ta4hN`FR+@iFs9<2UH(9onLK=} z*o%C%7JHdcUf)~l>SV)#1#ggPEwHxM6M*dWpjx8gDiO2z7UvPOsq`gD~m!+8FXcO|~{P zmu|i`H4zDPHajqeSU%i(6GJ}4lkDN1K`H&>W7P_8Z2mV)5{70ZQm8xyN;=OKJ*RZD zgNvU$wCi$nx%Tr4ihw;xhO}0tM)9**cns{E9C9NQ!3C#@OiOcoUhFPwvDAk7yXGCL zk4xegTA({En5uE=Rax~qj0Y&HMTHI?P!FWiyMTeNy}Qbu>Tg~O*-Bz+24h%4``%-g zTNp1zGIJzghoq%e9X%W77qm*VTDbn zN#TChEGqq~BEi^3!LNXWi;VUxX7v!S)T{f{LUpEOoZ8iGt^ixlrEV_XWBj1maq%Ed zQXFhn?pblETD9cR0%?hZ7;(Jzpm;}aB_nC(Tv*L6AjY6OW8!6TexwT9?u}We)l_IH zu22@_!ZC+8>ylH^aa#LX&r9N_BA0YCmTwnxfdjNUHkyC*`MK>QlD28`P~<*%Hcczr zPFuCSq6|S*$H3)WZ826Lv-|wg%jZvA2X~Tphv6!?``{Y%D*w`1)ggYIJ({ymK?$O- z>K9*PV{w?&h~Uds{gChjb~%N>(>AHVP5KeJQ)H8UJ~2s!5I7}hnkqX?HH|+Hg>|xg zJNvzt=J;9ak(a(|ud=Q$q2bFcgI{F|f3!-xT9{K?Bab~%9%r1unxw-ocVml2*CjCD zDNR7>DyQr(>TH$QTHAC=A!A&+K<-Fe| zsmF#}1y9yv=)^7Jk|Nvlm;8V0d|L@SWUoEL)s?E)v#=qO) ze6cVpkgdzs`DMc~Z?`=Ov((Gg!)ZrYhYi|Cz#h0)?PW%f`|9;xKIuSQUQ9dSfXJ$i ze+RBqY8Pa{obi$*oOk=4?}Z0@#K;eAGy-}{g5~j;b4XCBx*#tLuV+WpegQ9f4hw320H^u+ zezR!R4q-RntSk!t=?Ni*NYYg1Jm=g4IYE9jfNPrzA2~uXFs%4Q)L=rGNSajAWLmj! z`7`Kz0EewQog5)Nsr>|sD<%RHk(iIPj0#ffP7=HT5g*%rq0Xb1U@F7gM=-0(15Z@# z9QeX4PE`4YLP$xSe&|&P>eIUnEW;T zqdm%!O=Wvm^lP#3oMIT4_a10?UIDLN5KE?8pY1S@M=&15h06Ed2T#Cm4Sg6{uu$WIrP7t;}KIq_8}^K#r4Rlinvu8h&5Xe*;w{aqP> zMfu$@I^>&z#>Sv=|DY(%z#;yL+jlpQkZ1*ew4{h3n%nP-mZ2`_koiG{Og*ax#Atdz z8!!w?jpRhh8ineyH$ zvAaDt6&?+M5f>ZbOpSd#Nya{k73z+HHA-|jr>v%=L`jcHRa(?y2Eb^X*o#g`S^nf2 z?GQS2`^gfrw>GQ&B8ca#5jxNu?i;WWGi8ii)=y%{$d6K#hwwoggJE|jQhCa`qvn+Og$C5=`5siWCwG1#gG1M;ah8FF66 z#(E*Tr^`Zqe&g(kuL>>JUFC^-rB410A#2IX_ zSRWatvD9D?nK?=OJcivucUI$)W3px(`iRG{$)&Xuq4q4;HPEtOK*lieAvJMiSZ4iQ zd;#TG9Skl+>S1&I;hZBYG*_%n9OaRH%Np%b{kpE7`q=)&y${joZ0ccm{2|8VZ9%-T z-)y4}Y7?glLzE_Nb&U~vm{nK$Fwpxo;M=7OQ&do^^3*~76?zk=y4DvNv%_xr4k3nxz0ZL?xp0Xu_j$!hs39eE(MVme=IH0Z3#Qgp(mjY2|pi zVK*{q1UBr+CTV3k`f=5A*=|BM;MS!cpVvCF>^60fi;8HP2)sscc}r|Ekssb;jZj6I7m`}TbWmWoa+2Lnp2B&QJ;16F|$ zQq`b2_bINE2U$W7XI9ZYZ#jD$#?O!!h4dSDg0SLmMQr^URp~&@;LKH3bv*u?xdm9; zh%1nMfkI;b2)Abtq-{ZXk3~BUa2oJWCtVe-Yu-&JCe;H9IZ-ZWgy04FRjRj zlMXMN~HiQ3L$rkj(2p=Ole4WJsF4whW_{A=ez-?dN-g zsOXnXd7REknPBn);d~zFPag}j322W9(-an+(ag1ZQT-^D-U%1|a!g+6UDwJeYKx}l z%qmKzR#)%+J433JcCfkCpyr++>8wso)J_4{YylsQRR%In`%Ec^XewGi+o7|pxf*@ebzV@*oxE2@poMTD>XrERy zzk1MPeEl@x_2PUI`f*I@GXL@Zh|>hg0pzi%{|shbzMnm6KnWKh=4SS+nW$lKuQD>^Px8}CKjgHFIZp~jX<9G z5DwMtq~9KgwLXB!m_Gz{xjb~MJXNB7wX@pS+tTf?<59O3iftZp?j*lXTk3|4=&f{> zTL*eVlXRUR!aJMwRH?koLBT7sxyLwh7A592D|ZCWeBo-s!x76w7_7u1_$0?fu6&4a z+)BWj^`>W4xy)wofWpSnoMVN?InRGypBDC*i8Sun@kQBT$C#CYh)oSdjqQB*gZ=zG z{JJLiO1nJ#I_6`%@9{5F-~PR9>WeeAfS-<^lnAG83(14Q!dAy*V>n_PoG=P^efmMf;OB(1=0GlIL@n^)PU^3N* z+YrE4T@)xh*+Qlk2 zEQw9GfKRdy5im*?t=Lj4E3FdG2_-H9br&_Jyf+a+`Kw1 zeS}F8JxHDLgK;Xc#2`bR0y^5Wbx2jk$7_JFy&Wds4NMS#v_&zbVE~Y}SOC(7R3=0S z7^o3LX!|bsBw>0a)arF2aPQ%6lEPB2Sw8h*l4*?hCw)_ zL_t3NYkTvYE=ZuAiHTOlI;}prgV=oyLyN06I1(4;NmBncP%*8RTlp7hE9zhY9d=wV zZEW!9#<0FuC|5U&#wr2Y`E9wC3ay#hkWk5%c8D4+W5w4=(!sVXoymrEn!*#>z*OQQ z@s)hLGz_r40{e%lQp~64+NYChe2AG?C2CI;aRZLxh1+^a(z}afwqLQMw3WN*2;3g}lQd+;ja8U{R7?I~UI~H1eAM+)nUfs8jy~dPH(+#nG#0p0wr)Oa6V@ z*NPdqE9WoO_B#lIyFfEP^F^3MhcKt7xPeLe$u}a5`11u+k?n7U7$L|CgT%H;0*zD# zK2I;QX4yk6lKx?*gF$Q!w2MmY%N9Nx5_P$@os?+BTE1F0eIv063$K9E`2w`P6PbFFyW~4=1#qrWvxseuD_!dJb$S+sl!L^ z04pb6ho{v6E1j-XXEp<@-`GwJI};x0#vB)i@ za!H89#u&*^*I4YUIKXtFchKRIJu$4)M7t&-U1)XM>P4W>%m)d8h1w>-6Ha>@c3S~9 z?7CW`en;cT%=p@=t>Sv*;)l0iB1#|0MS0{g5_-;cQBxPh_k-vepFJxlQy%Mpknrla ztCd|Hn@q8c83aP^+p^PaF8C)yY@XKLBj}qa@aMQ(eipE&V3@+;Llk!yx@#qfxS|&_ z-x8jz7wL(>(M*EtLON%B=^#z7#SX$d}BHwyod{Q0+gx+sn02Niv_ZUAuRGu$}js+;&3- z%GZ0W0!&fvt_npZHmxoo*iO5F^@N0x#t5OQD2f-G`n4pm^idkut+S^?P{^DP_~Ju! z?y&b2Nz)IdcP*38?W}P;ty%J&z*}QLph2=jomPzsglI7z;cif1G=<3S4I4wQon=4G zp^w?5K|4bu?{~NeR_SNI*>w*?Z&*Vw({O&->vnbbwg=A41)X~w3-Kfd991(lUJK=Y zmZSJ+S2AQ2BQB6R(GlosGR6pv3;oiHv{bKjeUu6r4!S`?9?_K^qx)erkly^=)|gc4 zNjR|raP<(G-HC3Lh$lXYUt+ssMFEb+X-LKi`BnP^d66{yxR;Afmk;0snWg;m48TqC zWhs9=y&CZSvy zLw{%zlgeVw|GuS;zLOTv;OI3l7sj(E>_-62vHwd#;M5?elg^9P4^^OmiSc+jtEzIm z_s?~C+~oVg==pu?$*P;q3cIYR#lrTdDg(g3h3(?2=~2~MM7r|k%p_bPiwOo^yE{a=&X;-I8XIkSe!>JkPi!T~LE-2Ybzpib^f^Kf3ZlpjNkCca{g`q!d z@<*YttM-^VWdkOi?&m>HcDw;B0MiT&0<4v>)b!^^!aqX#l}Ote5~uM7Q{7thSddyPZhtl=kMJ+d;rjHfMU zSgNn%k#gFVCUWRgOiJy7GKM4F;JOQ^a@zD&jy`?(i;RT|BaIy+-~5P+11(Cs@S+@b zKYXO})Gu^AO%&9bQI$jA1A@aqz9dXBqDc?&4RdsxQY`t$6OByReFZ3Q*h1Cta#L)@@OIX4IiiCJLR%OlO#wfE zSi)Oup%FKte<=adK^%!T%tX`2Ud+Uj7IoP1J8sHQlfN8VqUlos;LTM7cHHp-O}OzW zHj+rwwGX|nm9}dap0Lw!r0BLn)WY!kqwj zf)w(U1nY)1xUDKaf+|K+(;}%THYYYTTQlY(18aV}a*MS;v|uR48`@n*M}!}AgpHD6 zrf%y50(e@mQOMB7F)DKG_3sNam!DAo)1x%ms_zmo*Rz$+|@pTt&3Rp zPDxVJW?ql;)wo=*8;-Lt`=`~Z)x)og+b=)Yk6pGo7*7yJ!M^e|f1wiYTm1MQ@N6Y< zlDMG{gXBN0E0mug|DBuJig|Vx1C;Lv;Hu$2p~1NRPmNgqdy3to9t~2mi^_}wXrWZt zA5M<@eF&1aKc=29(Vv;;^V-WqQV00l%vGPA#}K^!_kYF!WCzppa=a8ce_>b z`cI|L;`iLR+to$zk?HaZ0+pa7=15LSQQ{<8GT1}4QQ;kxi4WkzV@E7n`JiZdd*`q+ z$C1i((~NvUTH`vs%AQ-FrGiQP427fj`h3Rl1ZQMVR@HDSvIZdELMp&ECrmi?GaSu$ zPMm@wuw!7EBF7tkPArw#SfnkHMJ~$)q{{4~Bm|7*rl>`yrE~FeM^~UEN7Ob*1kAy4PCQ^#c?o6f@reN)4)?lAQO^Wo9J?s9^Yr zG@>G?ie^Qel7eyzw}2|A?L4ZI-Dwz*j<4W@L4CSM_X~B(p>nDHwEDd%nAfB!YbUBL zUJDf!z}ZMg0u#6KN?7SU~W!uThSi)&};Be%) z^8OUB;TpzGva8&$p9_mmnC8itKLe@jn<|ILeYQSbW-p=o2sDHWPqOgW0?h0sf792*-WVOl;Bq@KWs<2jWTCeBei%fku?;O+tPy zq^4c_OsF}Ldj@&Yl{&#SxLU1FR}mX+9=os!>nXU}@b?Y4+O$q*Ao=hL{E_Wm$nxH1 zI}v+3!o`MJ{i`cvK91QHBPXkBNz(lvjrs;ffvQq*N=%7GYhsFHFZmuDd5X;s+_P(0 zvw|oz_zvHrO;#ZkpA&gi>XQ)(UuY4pVDw;$lM-0zDC0sGFaeD={0D6iQ8aNlLO5nV zgEzMb+bhnPPd(Hi@yCaUy6^e5?w{eM4oZ0%;$EhO7(_T1D_FGZj1Laet?Ch$j(%>g3|}+qo)lNjtBNz9?Q%?IrTMme>~(BG^TW4YaeH{f z1-YM}R={&sU;1+%KYe&Q*pXZ5w}O17C<`+kS5A2HYd-wJlH~L;`>0Wo)XaU2*?k{` z<4#Q4Pb_h8?H#tvV7|@j*-3vme9XYWHB7%MM|@Uo(IQPkH->Oj^IH{lW8c#ko< z3o;oIDOEoLXc^QO?rkz+7I>`q1eL=@_tpNOL^sL&gS|~%xf0l)TA!4=Uh(=Z%ExVF zp`5)A0XAq+?#nw2)C=fQ<7TrC$4aFZi^&%w;b6g%U!JkFGZ-sYMJ8P2KEb~)mZeS;a?@Zk&Rtj=1?#oVavq^T_8??ub$On9$X`0KH8Q{)t9$4IUdBJZ z)Ew{7j`I$Z2}efM;;&rJQz3Mdax0?Y zza}lopk2nt17zz>5U34Cwwzkk8ymi6a0f5CqLinf->a#P&$JYtsPIx%PxII;Sp3mc zUw_c@ZhP|5Og(_sDj+nE61$aOYG!znKoKI4Osxah9l@XE!!1|puh+1Jn*K&U)9AIk zeDi?gRR%vlnC~S(-aRMlj ztPv6)JaXJN6Zpf)EzGKz- zF}%no8x|)x3uWdq^h<#vku!>^01lTFG(Chi9~)I^V?byj!+{-mgtDDwF97@!!52-D zL!u-|;ixK&xqfCu0(T&+OZ0umGr$?Q-ck9F@~Qqbe+}~-G^Ej=sE8S5o8ar!%kHn3 zRUjh@IXW1& znhlJS<{@PoA8(*Kj3gHFXo>i}%o0OJ3zw!|4fn)CJwEDXM~x6LPH8MGTYKF_7F1`l z0_U1B?LQMH6V-^ORp3VjH=!rK!Xo~4bmB;gH{$Z);T&6a?bwK_mnGXMz-Z7z-Lfr7HUGRB47~uHdt%lNx4IQ(cw1K*} z^J+JZ*DA4hz4ondA6PEaDSwJ1As#0#Tw%EAhhiz{qSl0Tx=THW(NG#lomgmIrjs&L8xtz03th8DybO#Sr-dRsX}bQV9)E<|3d{Ti6JZj8RGhXLG1YE_w&#$&gONpOJU zU^@8MHqw3NoMJB}QA<8Fa2rWh^@z`OEE2%+F7+%z3bbkCD(}d22M5;EZx0&;{}aKN5X_+2w_f5 zdS}8YXoOldhQGo7ZOf{^zX`kd4%+^BNufMKP`up9^K{JUUf==+w-^sYLeh=5S5dQs#}{n1#RG6j-5FqIAs>uOPKt-30QIO z0xNFFihn9@P|Sb&vX-OAnhWV;J>VpB^k(j)&2P5IoB2^{c2s*qfJGu8YOIXG4n&Q? z@tak5{-VZyE7CyJxUP`k+YJkj7Ew!*z%8Ovo2VXj9YaGckA6W44h0H z3KY}6fwMrS`Psic+4+vJV!`g2S+&gLfg* zm2nBrkY_Gf+kpg$Rmj}F;E+q@;Ui3f`2EdC2TOD=f|DsM8BJ--v*)koCI$_-mz1JD z0Hjdri21RAa)$V&cMY0+Z|z{G|ZA1jq+_9@5yeD!hcyhe`O~ z-*JGybhtc^<84ED*hrRmbcD=@F?-_Vo#!-P$ZxIiOZz=pUXI8-1ne_NO6lH#9CcsM zuUxv)QC=Y3@lYnG#@v@6(eA%TKv^Uh0ei8iB6?ZUwxF#?A){Kz`6J7M(pitn?_w!S zb-AK>dFc;-LXhjGN*H6|;QK#!_Rv_j(X=j~M@Y(+g|^%~y$N}h4a~Z4FMnBOczYii zc!buAb|Ed{ICe7ctX^(q)Jp?-m-mbf^sc3D3E7Z0XLKea;)8XP>%zCDUm5m*aK#Eh-_x||Q}BLDx0l`o%S0XzZA@CEY! zsto^M7ENId$f5y*=v<4e<9^Bgn8hdK5=v-NMRXca+@NTvuF`eJ<>R<%yVKbZVxuClevTZo*w%mDdev>rvJB^S zQ63IgT=)X!FN?Nz9`kibGVE!($-F*|ZLOY|y&+4+n{=qR3r`fOB$^dMK|v{&kwh$fQ_uxY-Um)q-pUvUO2v&=Mi-S8j0`JP{Ag$ry-%m zunq-^(~9$OR91^_6C0phRXqsy0W-OF4onD?1WMGfh_{&F;P13HDbAqcQSc`rrYkUU zN$Zm13(?uQ$tO1FW&$?{PoeGOv10mJE*jC;a8lynSGx$g*rN3D^E$K1`!e?Q5DOUm z97<8luu^)zpnYq>@3~VO>7F={l5$cR@5Y7EDZYE=Q5118ZIbpdpsmS6)hBs&d@DGc zy)B7G;B3rJ|25Tn%XJL-cTmNf&UjRBF+^1#V*@SD$K{B%;Pn!a?d)^hGpOT0-!{$F z#n06xds&{E2jT$Q$CZFQh{r5OKWp*RWR>C!VtB3tgvb?bXp|fZJOHSmtB*k0}vjr2wc$`Pq=kJ(s@5my< zo8KO?%j*dXhOBK?tZ7lv#^E+9tC~KYJ1*Bnsx^iHt0H)Pbhd*cUTl_=QEe7b7oo9+ zx|u+e$t{$=bXF*<6x8npGS*oiNN~m|C<-)wu?Ei6=Te0J6+XYgvlL#WrkCP7Oh~A} zoHG7a>iaIOeITvP(Q1laYb;$DW|EIyWT^0;G(S$96jr_nEyo!fC{ev8 z8^v7nP&DiRLg|gLSrP4hJb{1?G)cY5FrJ`&24*!FvBr_Jat#Z8k#jX4EdI74ehzbQ zO=n)0vPg6)a-_$TdvjyrccLa^)yf79(0*q^9>BxCygB2SZ_p+ZYj5t$&J$vKa&LU1O({d?Y7{S5@5oO3nlXPgUeZQD9qZOm)S9vi5^1Ih;b0&!+3hyYml zfY&w?bp1W}(w##X#J3yG?J%ha5X16GihT~`VY1DsNiA8_I6`3htmth*+ zg=$`G$q5u1Fq9Q$zUwEllOEocI;zMbXdyukSssM1ulV+75nfk3af5jqQTb$BElFF% z8mHL|)p&Gob1{pt6&fs{3(+1%@`!5Xi~F$$R}UFtV}MMW4bup5HRy`EyeG+GeT$T4 zHpXzdA7xYWvjPO#PboqZ#5lC|g*FhW_<_<~X_Cby(tfA)Tcx?y7dhY!HQTHbF$Z4$ zda3r%Xzy8@xQQtC30!L>%aYF$uzjSgLG z%pP#k8nZ>Vv*4V~tBIi1wNI@$UD)a$*#i?wReS^6_We@zw|_Pp_2+*arLrCVBVa(TdxYx0I!gadl4*b11D8EMv%33aig1$X zrMiFFHGie(axK_d6Kb>M&We+g{f&)0$27;RiP!4ZkJ9VZ(Xd_eBzXAh>J%OX`Iu#4(D^0W>!!@WZUFp{3}UFKVDTP4gZ|$Jkc0_7qR40o`_*w=zB$ixKBVT z;u+;1ogI`2F2`w$pZwe4ZXh`O7{Sbj(J|2}Lo>gwb`XX7)5n#vmXG#e@jAWqJg#5v zVA&whxw*LclcHM2_>`+p<+g}&B8zPfB2@aAZmt0UR!B|D5qKYYAN|<@)%Pb+0WK6{ z&uaf@%4=!tU7FZm&_uc-K5RkZnLY=_6R9EOfGC-Cqg195C`e&9GF!qna4wZHu5X!W zK1>;HEZL+K^GTiBT9_oEe|BS(HeqaNW$mH#LueX~V{WJPiu+lGjijVACx73Hs8SQX zKOw%O$fM<873ZdXRnJry3__M(5%op^>E4rNVNIo4=}#o6B*1f5>iRMRp=_8!G;Hdz z*NfdtT^YzFMuJU?l_x%fYM7gtO0V9C>O0+2en&UYC!r2(=qEicNDE*mSwn!uof{4H zQ1(^^VHieQH2QR48gU1IjMT(snizjgpmFS}-bl;-QH=re5-{D@`7Jtp;D$Mup2FT4 zGe&Qe^Qdu*S|igtK8i7KpSjCyR%;m>^tS(9=tdBCjt0b2tnG{ooZ!3nPnn!$=;_II zTA}z;0PC5j5rkvcon%E=_OM7K_-h#|M8v_X`fF3%Mo9YpZ327Wp0%KHx~B(t6p*f%IA!ei4G`F2avBTw z`;&Ozk=0R(%RY*+)@sHVWGM)?anEsW(S=aguNYFKkxn8vr6anZM8M2xhsT@r9hQxW2d zrh1}yEuNJ)SEtbs(U5l%9AvGg9Jp{I=1LB>Tj9>QPBXfji^P%GkA_DXaZfH; zs0~JCN^x*GgyC_L3TQjt92$gbd)8-_V*uI{ds?L`z!GV*NKI4dqsJZWOAjDlqh-7>eP%04`jzt~{#ZPDhd)wW%1*GC1u?MvjspNcOjRY-J!GK4*Z%VnC zJL{IPoUZQ`%arBsJ<-aj2BuWjGq5<1=2_0;Pj1>$4}1=51GPaF45)vk_{W*dCST_8Qm9t&xxCM$KMCch-yFvpCuaH1xU| zfTBS8hRg{&3f&paLqKm1bFue^Uthb}X>HpbKbe5`r6E$YOEizaaMEkEv@eG@nKwc1 z?3(V0Nz@^J&A3@MPG|N?KlOq|J1byT7+tXgeWqn!2aL(Zc{@QQf}xM;yH@owAqjxW z8Nw7+mDYbs?-e@&O&(LCQ*C>v+EhBE7T)dkHq_GWxBY@oV-KXha|Wd9yrTEUHVp=~ zt=z>9CQcWbUpj_8`{fyN*}Pv=y(S8!aZ>4A+S`o2YG;g7{`+-Gb?d3K@bEG@^m#w1 zZAkWZD&hkU9-~e}kZ)M9idmF@y|-Uy``KfZx6GdzqzRHt7m1v(EJsP1s1p3S)n@J- zvb4>(#%__amCnj0?3v3!$6X;nlG&5W(_BSPi3TuEbz$lPXZi_Uve8;= z;K!|Dx)+)#{Kf-2aJg$+I4twog<|hmeni2Af)qYm!i7#0`n}XH#Kf!&Fh(@95!WzY zs$5WJhZvGVH`e-}PpGF^EyR03fH02WzdBa`haUtw4mmMDvbuNl$Tzf)69CC>?Qope z3sovGxP=+TbI@r6f{TbGYe0$W=ezze{MK>#y&HwR81=xztsmdR>#Aeq)JawPW7)lO91(pG92~Ibwu7|g+ECSA$ z1a11mc)uN09Y}`2TtToj_jRw+Y()|$*GwdPn`%Xj2#K$MnapSsi37kOo|1X3VnrD|QUuBd7h zWfJJkv2~HP))$nMlQDN^vfEDXAMOUq{#?YcyoEsp6#kM*9FL21X)jY`E-nND56dL(C(BdYyA2COHj|5&V zLX;w&F7xgMUiH?8#;k?P-;T1gWkkJf%#DxI<)F&0x!-glm+7cP8(&3*Q*lt0TOnTj zPrHuKXMMhf9&H-Bq-&Iorw=ho)_0|ys1@Bl=?XhK-zRiuJ#~Jy#9y=^9Jn1s=@&Y| zmUO|YyyewdDc6?zkSMh!ag!WPigYGCN?5|uQjCm3Okx(@&G|b zbKkDc3&r$)zuVgMHQ|bAuq{93jBJ>V{KD0)+c2>=#wNJ{5rQmIRB2XD%$~)($yjsy zEENs_Eh0}Aky|X`W^+R2t@tebEE zBE6h~-YmBuuk3?IE1MWbI~}($obQ=3-)#RRK_WyziCLdYzwdUEul38abGe$W-Sjt< zje4(meOEWcMOmPkbNMy`w4-xF(~>NI7SL0b(!xM)--K8Kp)?_yi6u&h5?`#vhhns7 z*BT!#sdCAT@)sX8`-=~%k?(K>dE|tT;OO*0W?HvyBHCcs844q_H#r?~xA2sgK$-HR209Qw?WDw(jdqe&|z zIjQ~yQVYYdh_yR{jWc=LRQ^D~A?t&dU7&td<~M9 zG)D7F9@97JZs+daeIh}b+|s_%EU6$_KFLYdUbBQ#E49gE=Z7EHOe09(`-F&-wahoBIkjO49#Wb^S0tr6&P0)%aL@d< z(F0;%!8p?{pxA#!I_=kO9A58Jl_+DOpRjwKx@RxJdxM9TK6mf(2^W}Ik{>|IP z-p~ZCu`tIHdZCshF zm+_^%9pim=jao<-Sbv~ozRh)HzW#@0YtY-m6cc!Cbfg_j(HxIhXZ3%?Iq-*bcAFW<~#boed7!lBP}2$hy9n+g2)r;p-$3f7%z^M<~9-vz1pXEK3Ouca`w>%n)JK&A&8}?=uM=bshx| z6Gyx3bobEV;TOC@lr<|AlPL8HgcrzMj8TY{SM)~AJu%vsg!xOHJ8tU_68V;Xf7dKm zC<7AbDnQ~q?0<=K<-f!^g)|-d;Nw3t9(t_5X2YO8W^iFC0#e{sRW5YN_pNK*4hYW` zVe(0L2rKQ{{-JRrWcqEHC=bA>oZF}_-zh7AMH-5K1~W;EnRxU*I{`?X1GOG+^vLva z|02$3+#&xV&IJXSmG7mXA^)ZIfDotJMLGNq@ut5RF!V}wdkVT3t>lOxm^pw5X* z83l&L{8#IdHqMECV?qS9$!Z4tHE&0h+Y4AAW^RC$x(5X66Sz)wW#?=xtKqR2{t%CX z`YJCqOP?0RVAEjCOn~eMbIV>6N8TqI;6l0?@k;5+ky|0y!^@a1;hx44|E=cBMY@MZ z-#O=waG>66zVtm-JvWFD1+o)aVf#)Ei$xWNiDX-)?=ThCh6oifqAd%#>rR!8GrB8# z0?;d%Jj0$8b-#-tw1n`}hpc3t5zeq>!ZDG$kg9}4&6*w*1y;_QtzE3u=#pz&Zd7wW zMA$~wav!MmNN$Vs2)U;4ST_temwjJ)u0wKR!Z8z?a=wX)szhA>ekvq?SSmt>DjG0> z{ki$4U6JS2qiV4rzqcbw2PjdS@W2&%KHUya4j|ojM}3KqUx9(=BL7fs_35(JjwU>S2Wlw0b0ofC8-^l^QjXM2l4$#$or? z-I&IBuRtQz>!BFaSdRyYnFFmJh^Olez?Y4h!4PtGFDgKm85wV~!+f4GKQ{ z8>XjX(T2%SiqJy*E+~ZI$yuO7qgy&m?;l1Z>#!i6$?fSoNF!|-*6Q5Yp6h5gPQLRb zej>;?HVt;;TN=bROr1%#f0{LSZE#?%Q2X*riIQ$vW4x+|8=kx5B>I>$+Hu zF11$!FASHZM*3`Xmkm>ZpE`&BPd)0x*|AN}eOK{0?Am8j(pRK1rI-JRjgg*W6Sjl- z@}Y5w_nJrLClnVMyYarWsSGo z8254ZNZ|X+1N;f|86>@XSuEd}m7c1i>B#d_r>R`Y+>1+%{7TP>zLp68%>!NR?2mc8 z3srayc)yB;Y4EhB>mRw=a8QAep34NR)B_l}B7t~=PvK9&M)J7x8F}^|fp}I0C&9nI zw2AZFyWgK@ymMqg$+z_*W+CF7cZsd}cWn9J`d!jlxwcujdz&k`4d`9O_bW82Y!L~- z(|Wc!G7{sG5X4Fr>Kz)HLe)*1h6c%Ac5Ndxs zr(%#YbD2*V=I6$*N=d`yGyE7IYy$3!y(<>jDiX*uh!IE7^;$M|}#Gy4+roi+uL1*f|OGxJE81<5}T~j|@#8^WwY9a}ShLB$nB9kMZz~#DXajo^sZJBD8n%{#&PsrxIpWduIf4ECm8V6R4oZS#+{|?wE~&q%P&}wMS6j82>LoSD z4|?QnV42=LxKzB|A=QpaYmDf-xXrdT8v?fLYvb;OOGcjofWT)cT5hN@7Mlh_jT$a9GBG^>2PlAX~5Fd&u zhEP}Qr7sLtbXKPdD-K)Wk13&-CIn&M*$ELf3V-{V7=G{V>qXP($feX+VpZJZj{f%0 z5hfd$(J*DgIR0$Ke;!E>aEReWU^A7fRw%10>ceQL(dI&_Z3x)m1Uu5kM+zOTS1GZ= zyensLtQO7L9*&k7gQU)Ow{3?K+4ht6_k&DdeUL(MKiaCNm2ircYVf>xb0;%++pz; zzQGw*r?b;1!#$4zPHl1c^Kyb{+|dsN0W7YD*KBOAj{;7994@yew6U}_F`4`I@%%wA zBeNMN+YGG+!%u6onS@T}dW#viaa!#rJ(fuH3k;UXC*lU z&f`d4Y^)v!4NhTHM-ToD-#}EhK^8m)(}(C)!RlBr7*1)%A-_yb!n9XV3oUi9oRB`s z*MjsY*ZiG?V{KHb$CzSC`Bih%E3S1x2}##b@#Ro65Z2!DBBLK5@Zi1PVAT_5wkIz> zFbC2S4AF{29p`O*`CeX&=i1ur^fofH1w*xzu?-+A7aB=+ z(OCSMrpXA&Upl}`_d39<^DyE%5wIf`CEQ^lZprZ}Xfq{xzg!a`St~5Cognd(4NN_A z_H}`Z+HyQPKSVX??~g1<7DaTRRrQ|oS6cEgx+OS7iAJYtU9YckB&%8mJg6TqJOUDl zgV*m*jP1{Ur1|?4XK87OgEH)?8Gl zbC1Q3^wACl3|}2yOv3@bgfoDc#5rzL=6%Ou_WqJh>!#xJn{R^)Z85PB+*6mSk~(9H zVH5+V;dfXn{zO4`oi<287CrN@(e2MR)oaGl?D{E_O>$CH4!FZw20UGXZ zTMNzGTkwDg&GJ)6cTtNGqAE`!A+;Q<(Srg@Qy9WDRg$r6O@WFGmvwr`dkXlXlB+{W z>KW0AdFrql&-ikIGB?<0dV(K0dD(Wg>tIXufC;_xo57dl6bHA2MhB{9%^d-jCGC6d zD@SzcV=ALI4#clUzBY+DX}ZV7vx${+CMq;r3biEhBsd|!X_*rY7P>Oy=gqvufNF(A zoI>lG`=D64Uk9`Zc@N$?TWd)p{kbb*Ni;R4_6@Bq>LFNh(4_6@K_I^DL!6l}p}j6$ zTS+Xdv#QN>VIHF@9Ia3bWYh5Kd~3u5m?^2u4`YvHTB(64c=Kw3X|DD-uHVWnuPoZq z$gO#41Mwz4JwJI2`G&qKG3nwsrMn=soYsG53O#*+w_pyz&Pne1wHD=MrD6{sq8w`| zU_SaSgIcY@`-o|w7P{0-n_3M#Zf8LL15uM*B$EkxaS0qm>K()=kj>J zh%HF1UP@hQg(XRKQDXL3#Zx+26U3iNeib45f=b!LXw-VLN~}Ou`2ys+>{dfPiCN8M ze@GRUS&lZ18cwqR$D9(mf_XTZNIJ|k#ekc{GE1<9-ivIIOI{>R8sg`G}VZPq!)Y9P+eVRJk0c$N770o3Ia0W22hg^4+<|Voap?$qUEd41Z4DDiYv`sfe z**^Q&MXTFGJ$>-)u{n|A_ED@(icmJ3Q0nt1)6_&RkBJL)^{?~+MsvynURB&!F~<{1 z-^if$C-*ub6sW;l!dg8W2n1!GXDNxrqAAuk4DC-U41^v`JVLTA;zeYMp33{2?H@TY z0}Jo)z2o@Eoz#@nlmYIjHQsk>0L=Bb4wH&3v8NvLHCFv1a|^P5t)pg;*}9BNuFbm3 z&cmG#tShX1b=5oM>%(N{nBg&O`Q-bQ`V4306C^jR8^_Y-oAL)l7Vs_A7go2sp45eN zI4(%$TY)l-D)K{vKS)Ya!uZLAH5gv^(myIE4iY`Im7DMxq-G0|ZTRMgXqi!2eZq?dh5zvu*d;M3OU7wNi7m z@<_ESd?*zb{F_Y#m6|>|8%~dePWy4NuIg0<5!1;7T^>+9ZjW?0E zEE6InnffklWbG1#2UPkzLwlnO4&CMy_)ILwoX^BK6NPAmD6#rF-lb`yK2qr#HKlKA zh9i!D+jEVkj-mC+zu?cK7M($$Y{@nNTE|;++G|k{W^-DTPE*<=Y1%Ene&KsARTv{^ z;ScYmkfHBo7sFqxaxPjck+cqPAQXhcUD^>tk_idp`t$+|SAz~z#;<9QR6o@zr63qp z4wSsca7dEE{<&TK-6eQ^|CQEyg<;E_VbkqPw8UPPRxcNRg<%Rs%6-1w@O$V$?g!Pe z*hV0Q$v#X8E3>OpEay8Qw*Me1;d`--0;sIAv|&UmA?VUbJK5TI~lIsEAj10--oIgRY>s^0c1U8@ymB{noA^M)z!F-)};2 zZK?($jN)a12*gL1M5tooXxMc33zp;zw`5`r0Wx2FU@8qb05q`d670CzQFt}`E@|)1IeRKM=t3QLGZn@atJanir05?P`*%5hw_`ymiiJ~hu-DGhHG|-0t zn-qeK6Hk>dL3$wW&+?ZkDQQe}Qt?i%m#7lX$rlT6Y0S{P8T~8bZYP#;#BzY>0caJ~KU6(2TM%VO1N_F|J7LJvgf6kn5C98P!V z$i{WARDFd-&Gif9f>f?^L{SKUIjurf@J`heYLBKVl*b=0E z@?pj{Vn;Ax_h@XgQ%i)-%Q*3-9@gyg`(yIFGf#s_ByI^7dy1{$J!mp6X>gF9^8>ng zY9-&b>M){{A~h?4bmbKBDq5CZkOSMKu(2&nn%N{ zv&S01{5;r;;9+NJm|0>|65TBiO%!EA@oUUx;lfGg6qu;qM-0i_B^4^su45{8Td4xZ z+;M2ro6uF|i{XKsoO9ly~|5WJqvu8jG<^P7v#kZnFScT#WRoTA`IG?)O5gp?}A zU?5=eUNm>=Ft?12H8@0kg>c*v#l-}Y)s7aYK`6ioo)dXsH>Jas*oHZW2Z>z73XYubURra}Obv~jm}L>b#9xY~7byD+lHARObB_oEp0 zv5OM6F!|JqA+?!6?sWs&7biTm3%rXlQAIR~d1{}ixk2o&r%v2g>`Oh)n$2b&@U006 zU7Jk3ve~l$;g24tj3FN#5Kz!JHCv{j@LVavio+NUoql>loNblq+)i(|)&ak2E_m}X4*W*3steq){t>}YK?M{$OdF}$m&*o|6|OBJE?L5+0t zfqbk5OhU#`JT39r$s=(MDLcU*Qi}e4V(ft%znc77Gv=i2PlMUVL>H%!CwvRA7N$NvOX!GGE=>v>mFWSC8W=%wa6ksi5IX)BHDnMXKg21qxRSnB0u^j zJ||?vNmZD%#%XoWA9U)+u5X7z)*K!kl1?h2nH~HiK+3=bRtUD|SVC6w8;>s&sZ8Gb zGY#kkHnD)<0m6(9db;b&27>onDig0ko&pZtZzkfot5{}-X42DAhq+rl4*S7O9;~{& z71IgoD}Gt+nh-#aiI%WK8`8z7APuYBPc$bN8gKu6essu7rch8CFkp~}QA!xQ*~#PA zBz?MxA65Er@iUs^m*)FlxDNsyAXTLAMiO_9qwbEOC88 z9fGRr_Mp=Y1(?e%Lp|7lyoyLvTCae@-SeurmQ|)w(!FSSw&hrqmelqOBk~Rf9E80*FJYP=NEzF{Njl&M%sd<_;TyBdzd&N(en{^@Y>A*6bVD^WEF9=x#hIRF=|sORX}F?;V(RAHjg z#%|9}Z||PdkYDQB>euSHYxbQ1g%}4-|9UZhZS#ixh)4Hsk&ocgOJMW|LREoA1Yi4 zn9T(AVQ9!U12aOJPX%Wimg+L0q*g`2T7Jp7DvbPsXwq`iXbQ+L|9l^^_73QI$d`Vc zR#6A(c;H~k#+owE7>bP4=Xq-Phb9PGgTE8Va<5+d(-Q2%Z;9fBqu z1VGM8zE^kpgC&vKTVlkj@kX6Of3$?Og@lDGRzWJW_#bzWSf`i%We9nZ8?t- zan9s~hgK;Jt03>IpCRtP~ z%vKkV@B76bEIs7c`B-a;)wn2=@pO(0SLC`HxO@YeXs$$5WNz~=lzb}4AxmN{Vqb~y zw#GIc0hPIkg1xVtk-icmuHyqyOvgWT%_%8YG^B=rgygQx4o|iOJ0SzX$i(xcAaLwG z(849%&-M|<6wBONCEr@%omKn|xRf1Et{6SXq?Y#xpkq+pydz)Gh<8wma+D2ccn<93 z{woi11KV;^x;Ft7nB$8(GHJlEP@$k4`oh2I6OkeSpG$rCu4tAYxg$cW;B zF~;LaW=2``oKYTa$Wh*K`8}mTEZlUV838tMQSVD94V0JfQPahgj=P)cAuL)CXR)xM zA}|gw1rsKtDwtjZfKa>jHx4hDI&N}6<{HuT0=+@<@6g+nbs`9w`k$fqFfa};3?|&~ zJfoZyy*BF~cMuH8hA=RjDZ_L7+wa8^14;4Wzu8QhZo{-jm+(A);X_3I8 z_eWTb4)t$hb>Pt3hD?uE+;bKI{5d|l1982XJus6lub&iMG<*u%fQKDKk2c%)D_@O& zWAw-ViIbkc)kGrHsR765G`pZOzo{gx*Mz>KuE|blJrC`&oRLD4O+Mn!E|Fc%OEcpZ z9oAT%75P53MIWzG+zAHcD6MKI`DRG3|Dg)I)z(wqmWI}1A z8el3DDv(ssC<^0%xXwd0I(=_LHL@!8REePE%vFJ}RqCV*UH%1J#%8a6cM5|$GuTJO zS!oBpJI}>6{*D6>v2hRp=Ht0sj%j2>UmW6v(Wh=tTusMtS75YvtxDeRO`Mb?It|57 zgp61y+aR|vQK*>vqja=(cg8MLH-Fv~yKZ5-X>TBixSn9@Z#{Usf3@PDuaZ$)u?U0# z4cB?XOa@4t?lVY){2cZ*m{w3;t)uTx+=rj*`3m3u3s!BHidsd8Zi7v-5#JtbgJLR9_p1^QzF701@ zXAm{}sKB0ay5Y$H<<;CCn$ej!L7BvyMbNQ(oL8cM3;RN0=cRq~T=(d+@+zSBz`k$` zv++1y=wC)-KqZIQat{pWB(h=!k{13!&s(JvqyJ%OTdnXu`)l4ibAe9L-QXp+d+VB^hpmC<=IgvUlToQ# zu_VxGyLd`hKt_OT`3gA>CClJKJp3rztZ(l=R>>Qlzv9^0rAgZ`s55~uil$uiJNc6*LI%f}kS_PE3C9 zrrw!l?=n{YOY}=q}mV7C;9w|SjTJPqd$>XD-+Zywj1isEk7dkV$ zza4yAI-b7|i$?N6SFU5EX_ojSR(>p9T>-{&C&1#Ofj4(ew7S}h>+YQF1E#7cO-##a zUdOZ9N4$=b$tM%3qNU3;VUNv9#+f(6|1_G~KJ!) zv*uUCLQiBS9( z+3blTG_|rld02xOS_pK|5|d8I(Gnt_*xpXYtI?ZsyNiR)7qJ0OQ#+CC@l*!rc)pQH{hUYbg>_$Q?jQtxOwQ4B`PeMR(mk}_12ym zj4Bdr#?*b$^wARwj+Z zOh%+mC1>@~%}K_9i6s#@Y(Wj2l67{Ebq)LeO%>RsaxvXKi((tE>v?TcN7Q-HV?KR0 zq4GFu4Y1E7s|4eU@}ko=LC}&D&I3-rV3oN*G6!y90)3&KiHF1^J+`eD@8tl`q3GHB zI_72Onr33Q*;j2qWtEG%5tt$DQG$*=LSNJ%%)*vJ?PX(gMJBpYjJHOJ-jFq}&$#C? z)UA1Y#%XLBWbs(YrQ|qr6UDMSofn9=NU@Nf_XJ{|(Cr3o>EvUvjVh~Mml(uZ6eTpo zx^7m^flgG8`$7i4E6g05<1`5-C7l0+E@;3H^luy1gQv=qS1j4{N*o)@N!C-bh$*!Z z{{7*b><)d_cS=R)q>x)d*Jp1f93A$JjYkS0`2Yd6n~jLiok%~joY)qz_O?3Sz4O7Q z`7+hLccgi>b4@VBb(^L4X12E`ya&Xq{WQ!J70a-{Ag!NF-Rl>U&fi@rk(udSFJKW(gR$1a=Bb!z_IY2Q`Qe zOF}>!0gr4;?#22&DYqTrV&fa7N?AQ|dmHKu&-6p-!mj(bH;xR>%5*x>&64?^+AYlQ zUP{nWZ%s^~3jGEgz@i&BBaMt7%?3W|!C5Ah3B;=Y_M;j() zQ4(lOQH!pdC*oXN{15(8&h&}UYQ2$w3c8y?i|`ypyd5`|NN;UjBVmcn(Gb!6Q&m6x zf^I`RhUnCj73;Ag-tN2Eks5NKaw$=+{p^D^E!4V})&Z$E7U)mY&bHCBeH}2 zoIFQ0&&rDQ29t2g4&=H(Dd`FlD$@(0MCOB5fdesBrh&@8oRv?XIB;~}cVL*5JMnuopv;VxA`x06e{~D4a!wsd=j1XcTtVbB6dylCoSriPbsazSh z>*0hmr5H>O@?XJvQ0x+yZfMkpN}2fhpJIw3bV}U-6oq|(M3JedK)xv*#cysBLU@>^ zj~Dce@FcPW{FaC^L3?;w@{ozCZ;{cY)d`xo(6P@?P z7CFl(O8%iJNnpu}BoBB1MZbWM6B;)y2n)hM1oRMJcA8+WiL}*32nc9IMVgu3qGG_7 zbL_z6x0JFpyM$@Q73JN3yo1h8*@RuP%}ju_Mo^a>0FA#{D(DY^g)TWQzOmJg@}BdD0X*W3j_i3~U39I!c8r0b;|c@-IU_;53Z+Yu zTn+*d5Zb4F7k{ER9r=M@iJbD815uPYJ@HK&j}a@Jx6k^jUCiDh9X}Vo@YL2-7HBSv z(Fsb3sFxOh&;?(60w6gpt~5&)QZ3pO1?u1t{>>0@OekQ3Bp>lZEbOm)j(d|$tbd)} zA`F3@3h5UW8GZN-JkV(oS$7i6*Pi8wQ+~OERU5ATS;BRQvc6Ou#~kiGD0x3B|G=xT zgeG9gVgC)w3JQw0T*8V#e|o4^0HYgWcO87JRwgIuM@5vTXuw-$p4c{`oLq!Iq_{LI zAMp@d>7D)(8BHyO2%bD=R5W(JCM`?E2t81~z5K(J@ac$hNLUCyWFqn*_Es?N9i9>k za)07CcqXSIcnp9vee_elJQEH8!Mn(@*W3n|jOm{PA<&E;D-8BSi6P1r+%A*fvWc3k zT2j~bP`}-UR5&Sc?KU{eEMQzV);3iGW<|6mK_F+`EeFlI`O~yTLU954A_qj24@ua3 zBT3f~h+5{Hz<@l(eT+De?^`5t8U1SX^>7F+lpeT!ne4DASa$o%urhSuwK1_U^*DE6 zT&wsHRwR=h-0*w3vVU2z45%{?(nGN`ZHyWM@W6*PQ$A!m(lSF)baBwVia_b|%a+_gDc%~Ie9H3MIPf0^naNNu>R$vwN zn|=8$1mz&j2zVkKVq1=*dI!iKbzKD$3~!f29l_BA^{#@ zlIZP2L~aDZb9f@q3iXT_Q4{SVq=6c+>I(cL2NI2tke)%-n5iFrFdk?lrJ}vlcHY~! z-bz7R%-+-2Slsi|>(LFvshEH~dTj+Tv?Zi3E9l7eC zua`bpk2gBG=c=nXI7|FJyJwglo^r+>t?TXP=i6|~U{F1nZDznj zY_`y?U-DNiA@$)lzGFQadETO(J%9=7G6gy*7x0#t=lz=Q&{t2{fU}RlTd*~P zkfWv8)LXyv3w_O8Gp#)~h!}I91Ty`aYT?R)Xt92l_$%2Q7(gO`bRQTE+m-WWN_6W) zFAHU&CI?8skkHR2wCrr{7Q2)fWl{C5Uay8?B9IpGjI(%U78c0W46cS+W?`F!yo#MG zq?556+YIEC*pQ6Wp!FQ717xSOVl6*IrvRjI8Uq6lTfs-c!3z++)TPsrNgkDP?N_TL zXx&9$%%5yCz|@zfmr?$>lDM}(-xJ@frcr6@GGufqQ;5qalpkG7;Jp4NL#Qqpj>abj zX@%7-BQ$1x0tPrVcki8|zUe@BS%kQ4`cm-;MWnvRnMDCEJq~C|Gb^S%h=9tsM-Vn) zpn5g?ILpv!@G(v&)Fqk5-}E=)A+p3X%e!7bZe(w+m?9z#@KFGQ-X|SL>EMYCv=rti z7$=sc!XV4hFkiVU!j%@!N|1zCWY{h{@9Y_47JY%bX5#SViZAKx=}S~cd{@>{eOb7| zAgNHAHcCNrrENgcxe#oiGRc~J%mF%xp473NzQZ;3taN`7Y)pK=G;MA3VyYY|F#&jO zWRkcjm%0jAv40eGArZ}{v^3S@Es~;Atu?GsPSo})QkV2e(hp&lYE+{wF0`Ujm7s5o zWSE3h5J3!)n7nmQOo1yV9~6ZjBGpMsO&*8MsIC~Q$imx97uN6i$i0tN!uRiWV*n~w zh|&0JFEy*M-~+WN%j}*v`!)&noTVB{)mKh}-Y`>s*e9xv_@CQfhO#?Jt+ z?G>+x)%Fc%nz+X#U-O7_fO=Tu3s~35+n~f>=r#ovN$(0nl9cG+d*X=)(#FpkN&n|m z;tQyUW~jmM>p;&%2hCB6q8r6zI~eKU*Tr<<7SGSmdtiDGx=P>A4Rl~^L;vWRl!n`w z;_cam5b{9=eNRKB1@mf=K5$TuAjVCLr2gX1^{EIA02Pi;IMUCer)Dsn)Oi2vQYk1s zG}u)-C$Ya)s|WISXz4(fn;`y+onaO5fSOw{N;gvKpztQc`)nXMtIt}Sxs92}7cv%Kc45~IRS$2@vy~I|rli)KREytH ztUR4f*VjLpbj2C!q;#aIN$_WUFzR}D5+C(W z!jg~Bv<^mfJYfldBtL5E2F2Im>F!%h6e+*G-UW@+vg4o-N)|KYuUIj<+l z?8C6Bw=fRpI|7|32k{B=#94<#E}uYM@9H)j3)}?=ml%GJS)@b-R(d9<8<`GCgCsREVDan(oSC@7m_pe z{^9IF`BenFKQn{vd09FDma%JHUvC#$3GzZb=&Wuqp*?Bz%%k#|8U@eRqQOJGlN6f*IZr0&54D}t1JeBCs#L%Co)tK{`$zpJk^YhS z=-n=iC}R7Gp76GJaJ(w|d%NBdN2x$8NKz^*jt4{@kJ<0ZqQAPoi(_5MP0zrFETE+( znKK_x*>-*X8>AdfFPsaT7S*23BQ4j5)tY%Lr2mhvYd^33o3z8v-Kr+8@Y6&&R|9Dr>iSk88-2WGqVj^LI*N%g@3wN; zU$=Z$O^~f75KhuOk{#jXLij+eZqNg)dU{~80MhPlk`hrjEP{ygRDYf&K&;An$!OmW zY)Q4$-$Cv=Gokm`lSoZebXDjoYg6Cn6PTT>SRP80ER=NHxATQSTCOmvlQOJH% zQD6-!3y4P(S;&VeF*MZpC`*6+Y^k6B0{s78jdJO8P4hnh0LNngi^0%;Dwaa}R#pc3 zhL%EB=EgQo{}mUkW$k#_g78zL>;GVa={Z6;YkIn2PwniKP-8!u!h=ee?e)MB**|f!Gon_ldtO&m?BdAA_kw_O80BV+V8E*(v-C^_OmzShSt? zV^Dv+60A#s)aE#sZ7rKXZ{Q--n5 zpG%LVOYd8+?2m_k|3ggsb+>yoecs^j-eIZ}!wYRxZwKM%&W12a?yeH?LNtEi(63OyW+)-sv}-qEr5-ZYAHv>h zvW8QsH3wTi^N_?Cyru|+DXf)cc}DqZ&as>6s)L!YGW~_{MJ3e?5A9WcLJ<@vM#2!3 zrqw*mg;T0~J(ZNYP?nHrleGg2ih?N54fn&pH{I(G<3YT}?Aq1tMWrBnl;8WDlJE3V zO(*j6cVCZ7>NwBVdR&`4gvb-4lyog;sA8tT`78q+nPtqd%}614w;L zj|69oTvkPryZRDH5itk2_{eQs4wg=7)iPkbSKF>-m~a;NM^E2bD0cribUE3xN%YWA zSc{iOBVazz_+Jqdg+;1*CSzq9Ky$vBD*jj+KkGl+!zbHYtb2g9#2Gm6efYj9UV{19On)!H_^w z;_LCA%2V9aser&k^p~g5*0Kt(UxUAAbFk{_$%Of=wuWT)=Y+xp3-scTG?Nw!*9uZO zDf#L3zS&$>^J15FR*vcWkFzZ|X<*JaQMR)=ZY2>a2FqRx0(dm7phCssmv%%9R$#T^T}$KI45`A!)+Ml!~&fNh>Pmq^tkYC1$Ug zgO<#=6jVuPs(!18NAlU=0fzx$5YJ4$AE_dILmqFoy)PIXQKZO$qw+nHsZ2iY-H?dO zGNIf85}Cn+emK+t)5*Xx0+ZR zC^o8o-CzfSbPEJE>E?IMYPZwjNkw5Qz??9e`Z(Y}+o@KE4cf#KIuuKbT09_f67O-G zl|HtQGH_+N$|6L!rB!kq6u`;h@|_3&NUn{I$17jzjriGmusc|0@{_VJaau0L9vSC9 z6)9^Un1YwQI11RU!u!aoycFFCQ5e7*=a7aUE$NlOSvv-9-1Ne>u9oOG65CL`f=xE`3|4Ac0qfQGF=02;g3V#d{GJ_cJZGwaKCF% z#45wq5}DTpNUaqV2DLtSUbCn}2&+$pIXNw4Gx26U78_+-bJ*&a;tgA@9PEdWk7>QP z`q%RfpEDS+*TH*q+>r@jNt+3IintEd3V9Y9?spqo4Z6N$92;de)xyx9YcuAnP1V*y z@XrgAwA#A;la$KlqnmO7#qv179>dQc#oeCnTEGeDxuW_K>!Bnc-p9SM1^mhgI?&-( z;dG!yZ~bW+?wKU~!m>ALe_(e$GKEP)F#w>^XTlQrBz7Shjz0x&Zz5a{zMQ$_eGlqZ zI{mddmx9fNF6!eFE&$~|IlvwG>y%->ymvrw+^Q4o$Fj;wwN_Z2zio;VGs?XiT zCpqb>E+sH5n`F7S$R@pXr%KHthGme@H>ag0W?|mce_*VR!CDl6`Fw*>QZcrZjC+-*tac2O<;r&y65_VTj+y$ zpg<`!{nvt1$~>MIS38Wwl$bbYF>UIch{y*%5z>%X;;smIR<(?K#|#015xkY~ijo^+ zo{VCQG_9idnUKFlOJrrrOTg_*m(LOEq|@6DB5z6F2e;fPpcT4xGRKESRK9amp3^3# zx1QUzNLZHC9^df929*68sbBj=xXWvMn*2oK3hXbQtG>r~)HQe(WUvM(t47DGOjuZ; zoVaDtC)U_8*l2e0wS1jMB>%uq6>BCZTya1iC!>9#Otk1!L=Kt~?SVjUAk(WpCTJwM z#JwDk=&stD9-GP|{+N)2vOp4(hluH7!;c3BthK^LG%yKAVBCfVhlF28Hued`FU%)P zehzy@hch?|N~q8U`?1~lZ*$_lLWa-|H`(J^Vfnqqd@})?GaICH>)Kumx zPU&oHV9U_32gA|*O-$_Q8{L(c-K=1XuJo)=pv#)XoD`)?%`MXb;!$EM0RwuhG^xND0(qdd~A4{sVLw#*wdHEN!c8WvZ| zcY3nk-O+J!S86OkLWX?iaC>r|SQXerl++gkf2%1`kfaHEg~ZNTs*Ycm;npLMTdygu zA#;E9Xg`Qrt8!+LU6FT3!Qu;YsEoYPCme0uCH_eQNF?a;V(p2_`)zD}Xb5t_ET?uh z{uI*Q%14#p{V`CrSoXPhV^0b>Od)gy7cZf5EM4bP_WWf zvyF!wXbIsSv9Ws$RJgguriro@Wnb%6ChD>Ydd$~Z+R)l#8r9zqxJM&H6kq_UW+ zBGen}%$|xS604g9Z`8AdeYAlZ7Vtppn%h}7MS?|lax9*@@ZLbSjvHDB+Xr2t5mrS3 z)j{M>V1}sh=8&`zFA>WDECA{R^(zBd$Oq}x3nqKmse_h4QaOkq$Nr-3>+CbA?v^!D zKnWEqIGWASG!#*&WnkUig*(nl>*}(2Nh|+;Q2xGM3|3q3?j-fDR(N}}KpwEGG5eTS z+gN{=H|Kmu)mEb%x5Cx*dfRia&NEv&nOqO6!}vkNB{sPMgRbg|CQU~|DIiJZETDUoy=`*{_Fhm z&uf10uO|3Zb*ZfwhY*yj^#;e-55@nolu&raR)|M7CRC_51Jz(c9j(X8!tn_*ofE9cJ`>)qK_7!$!83y)iZ> zK65WjQ9wONXte!Ga||jdY!=6qHFLdDF_R=V#`6@!05OMyz%XmA=dRkN`dzWm|HH^7 z5)aaXS&x54fFK4}6E*xO%1>`s>WeH~JOK25-Yll)fb5D1ZaG(cK!BOE<<|y}C(E0g z|E?dVq)d$4=>Zbw+rj7M{_BaT1H0qw=P-lq@n;rKQ z6$+8jM!n_wDADR!Fx*mIh=Y2a!(^D~WOg6#hJ+ba#Kqy*W69!g_Y(Zq979Fu)Sv@o zOOo;87}+lP;m2$en_m>MUUrIj$W+yVKN_sjGi$oxf5qR9N1EWN0lzh zv@Xl2yoD1pqFaR8%h_V-i>;qSS-oP6*Tg+J$fUpZiB(3P{D4UovCCq7qQ9)a)<#G2 zPhS3JyW{*GlLL5b+3x#yfn#dVata987&OCa9v zQS^hqZ=ro5Jxe2G6mTKgN7++EQ89QE#i?g;Zf&`L9P#Y$EPg(!YoiI$Pz2Zryp z6bG$cikr{Na(F)wJU$F>wv-)Xt8tXtwI)wRA3Hy(IW!}!-2#ceo0k|ro;F9;^aM%S zc{BytOha*Wnx_tGJW^?$uwYpF)xcrIdL4T15ahiP0Gr^PU;epqVI@T}9oq;p&?q+^ z)&*X5?&kSaZzh&50`^4K)(QN|VH6Do^tL zUvf)>{=~SxaU%b#^1^#Zp#4GBDhDB4?~s(WDxjFM-3GsEOwzCXEDY3@q*+U0D{2vR z@Qa}Q6}4cgYf-FY%RSpRt;vIVLG^>+W$h4QUY)7huMempa7O4wdvCA_(?%8Okp(Lq z6YEL~L!pmoA#QMi6^M!MU5_0=v{|sW{~3quk~AHw4)|sw9-@1F3)WYSozvR z7dcC1(9~%-iK=3+tBEF)X5$bE22e8d21TDp%8cYHy1Avw6M{w!Ho5H43cspT-DOl?#84=6b&RPC!(R0qz# z_|=1#3@>3)cakAS2F;CPnvYbbtIu=ACfAh9$law`*9s=Q^X7d`^zF#WmJe@b*szG} zmlJ$*U`>bT5Etn=VU&?7D8kBJ77FI^a7MqnM99i?sY(jTGdO&v;o#&Ak1H0*h4&pP7 z6>yLErEK24Fxo^A;Ove|fjhJV$QRgAgSjH$WfqxICtEAvlII0nXK>q{pEX=T;}|h zN>37swCo_VsyI>g{$_^OwjZJI;;?d}6nD_{8U^VKlngD@_Dx;vl2mDe z?8+-z=cbE4jpsC>I#P$X4OEpe96B`DJ)4q;slhTtJM^Q>RccXma*xLaIQneTWJiAkHCHD368Wq|~L;L}kA=M_`;xZ0g~h z*dHbvB|m~5$MU7Ypxxt*KxT52M}!W4Z){wU%Rc^rBX&0{$F!R{gR%BeLhw}wRR_Na zL~9816JY}@g&=Mj{Juiqi}RJWAG?e+*#;^&sgsQ|go=Hdtwv5=xI$VbLbz}$k3;3=_fwD z#epjtA8idgmlIqAR9jK1VxQ3RttggVdY_bOo*32W{@d#4{Jl}CaX$as7^?AM38k2) zD5IHERP`Cc1ze%ovMknXnhefJuf_dm1xJRZ3-7B=6c4V}Vff)(%7?dckEhn2vvI|F zDqgU7mAlb-XYB8ar?~m6_BrOiDn=cpajd|UTzG0v9h>=K@Apddd8=zG81mNI@*By4 zznwoUanPREs{onStGPm%c% zGPDvHM}t}N=g zv&*>=EoR%+ei62>skXpJserU#|8^tAE}mKk484J1oy2i6@3+OGrnkupEZQgY@#l8W zYp3_E>!(j$r?-2=n}5wK#@sF2*N*L5_Y2Iw8q3Cls#4|d3aL!a>$KneydPb$&X@n; zb9KtA3}S%*0Ei<0-;$TGv5CI3m6L#-ot3$v{{JH{4bOk%h4`B*D}Nbp6VANmKD|W& zodsd!Vv@{{%X%aj|B!wz8tKqI2=9 zhV86_Z&#;Z?DYiGhEG?=_ab1~F3UI98K?C0jB88}B~c;ZS5SJcW%Ul(E)*Q|Vm!s+srW)!s?)go*}(701*g?y^;9PC{0c z^b&d_f-x5adDia=pfeKq8VmdgAjMW54F7ihbaVQ-fw+t0$`*?Cu=~J``L(zIxjDGP z(Sp%t=i%j8j1b00w|+efno2(#Vu0o*ES+PTTp8dS(K zG~tfuD9Ch-vDm*P(QMLwOwu&S~n8O z0k}ok#Xg1YbE)U3%MS$`&=ZY<4$YhVO&r3A7i8QA@dDcFMKj((M&;)S+f*IF&&^{N zSRPA4=847I`rNH6D_Bmchr|A?0-~iS_>k&?E5<+vlIUdn+8Z43>;akIy<3SUB-5;h zE%z)JLPZ5yCAdnDl1OwU)hBXAyqrz!M^UckjbS;;WLwK(vE;hTi_ByVAVL=ZO9LN; zvdD^(Jq+@20Axeg=(~Grn75EYnT(e3gC6D1MZb}K&}2DGYH0J9w=Uy5SY0O$XgaRQDxA;SVn@F!1DuA?8D2CJla7g*y zqs>ECy!dt{*0h~a6~f%FM9wh5+}``bOBzmjy&)IN5b_YC@H6WirD+cuK>UjASgw@r;wvSXR&g*XHp1P&P=-UBD2h}e7_t=BaKnIv(^-y((zDQu z=vX5q0oBMH-PCos%&SYw`{5ELEY?T{mh_nflTV{XgFWTsrNB&c85b+jergN~McuFp z1CFxv*Pk!R{zH6(9?E>ART=t$M7gdKh`S7|xb`Orukx4q!Epua*;~xaTDm6mTd1$)q=nB*7t#aj zBdHOn%U)4qf^~AwWtjQTFp&51ZF1^q6Mt>u>G6N^SN^I!*IY%#X#+|@cyn^LcKS+H z%M?9D$+>yPmC}}=#ASjrWJYHMELYv)LWAB1pA)bK%Tk-eFZg~EsXu7KD+iSSXzK9Csw|ZG-#nf z8B89fHiKE?jA-+B74otqRgp4H(sb8_$!2wd9V9^(bSMep&ThpJYOUxUpYY8Z*A31J z6iAVHt7gWx!P2jwvH85wWfAR|q=#vgIZn&y8Rwkd^B;FbZx{6GgwLqV4d&1i1FA4% zkLqUD&|0QS#cZu^1`o~QJcZtDpZ@#Q2kxK#2YAh5h3n((t1D1d18;u6KTQ>>ia?go z>r=8PdWy8WN+b)f8PfoDFcYIj6@nxxO>&Na1md_#dGW=4t}=b&SYMalzU=_?VK-Mp zC#AIFYZ?}Xvh01)#LbQ3<>}S~gs$846Io=N+t$kBU+%9A0-$IGw_VFjhJ*(wLCh%9 zax+Xn;$Z0|gWrhMHyEh|GTVihr`Eb`amW^PG$#=!3!sLTj zTiLT|_OfjrJ{LY%;iKim>+lVXOFEYW2IcNMCq1(XcFUb=*f}2iU_Z44Ji6bvubU=? zl+^5k$4(SvO#3Z~<6%Tm=2*@`#xIqa8g8 zLWgDs42LSem-)sRudF98mxFo3bj5bz&Vv^Lg7Yw0i~jH+%`ld$b=S0XipotRBKUEv zdab`zbU#|vWlD|5%SiL*SuBw_BN}vL z_`CmJFQe-hm7zNsJ_msM(t{Ef+~pSc83V<4=cc~R`%1e#TXvPM+fxc|@yF@04DLtD zUxp<4R?S5l+1SnLQq9}ML8s%sfB#t#d3!7=UmT9KLQ7=}^!PImhN^dn)~qyK`clVY z`vUAen7Zoo3MMna=DXS4__sC}InA~ihAmW%OD5bVyz79!zROO5I5ym=lY%6j1rDZ+ zn`|Q+&;^Gi6&uDDkK~kWJQa8qXA2oFw>2hVG#-Aw>d8f|*W(JbKHp~mr~bsNo7Cq& z*Uw*Fq*KU4IwwUd{$3r_|5#7P_QrrT6c6UnbW|H97F;K~yGwk3S}cG6N8>Gc1{QV= z6aWC11OR~0|4Ie=Zx#{LcQSU>cULraa4~j}GIw+`wlQ}2?|Js6BPIvpe}so+D{UW8 zG9mFLbUt9VgxRz9G)7Oxb#BPgG7#~P(`R5oBHh^hzFP8}Kr;EkK<+eDBEn%hS1x~F zh7un={_7fu{JC{aJyZNW!QA%#vGaEJuL?Q*x;z@N=lqWGvA5{u5&VV#&w`o&y5gES zt`4rMhu=>>oHAkZbTMc&J9@g!u1|X_jhRI!Ya!#VROb3EpXYBiatRd!VZofk|HQ`w zFDwxOS~rxcGv)jd1wzyp;AT!Mq~J2#J6W z>L!?x0?tEo)Bw;CRzmX55)*?pdRtt@3)52&_9h(}*)IE*2Cq-R0Ov0kXFFie(K^RY zI9Ou5DYG%ILbqfmT}hTn!ZkYO-PgtObzmbb6mLxCoTpixvLpx74~J9GF{?#=tv^|B zh3R&r>8kdd{s0yi4{^1GJfHOC<(DD7#Lmh8t6cOzHiR%W1-2|h_RtTZTugq6Ch{#d zZJ%U;LxaQ(O6~r1W_WpvZ!0?+;BX`#1J0|4bj4!22mwRLdPv+bi?fdLW;9m(op}#; z@x1x!z4RUSvCZYQ`Ewfj;{BzfiH459I+_(&qUIKfCm>J{3H-(0HxJx23L%^^*58oG zI*av_91<b2#8-;r=ZjLUI>9|)D9!bRg$9xEM%F@1U<9#XC7zC(%$|!C#s5eBJ z60MIkE7!+f@X80Y)co$;D4?*N=6l3q`&@-23fL%tm1zVVTN{3tuY)Wv7J5DFNm5L1 z8%t3pi!w74knizJk0Miz9|6ID^L4+zh)rseNMBg%dl6*-C=BPCjzR z3`IkWYCqU7m{iMj?F{npDf-U8xnrYnk5sifg!le`W}MV={RP&l)`MEuw@%eJDRI`} zlZEhR_3p-@gaqhKbR<0rc~$u>^&N1?Dav~3 zTc?L#kqb-3@f1@aY^RV1O1EtbP1Fri zml2Thp!?hDgtC_t6W}dC_XT8FL{G;T!3RJO`2^rVB;5JU<9}VgUsQO$H_@Dh} z_)cL`AeR47h7$fMLzAOB7W2g|*A^tk^~tfp44ScL`1%()PTVNr3b*PTR}?0K!`qb- zWkf_usjBx@?-*&~$pt#*Em}DeO+OA6i|>s_B@-Cjppbk->Y;)ZM00#Vtk80{ob>Y^ z*7cN%?lq?Y5QpM5L`S}IGR*_$02lDpB27(TWi3S&P!@>B{@{Ons4Wz$R?MtkqU5-k z>XeLNyE(O6W(HMYR&wk57*HjH2gVjXwo}|rX&=SY<1Vt3mPHrH)X33h_cZDr49N8i zqypo$B(T!0%2dh_7l#F4%r6P!WrXR~%HJJ_^pe6nKz0y{93fL@9g*P=)>%M0BpLwo zWCeXG#x@l5qGu8)6+Hk27vS{rf&K_sJ1+T)2>Vf}+OeAHDCcHYb(w>V7Z*g{Sl9ag zz8CCvi*?Vkv0Z=eLYJE%ok(D6Yq(R>!i*_39T&E{FEdWXD<0*pmO8WONf5F}VKKqf`)eZlMdfuSdP z1t!pmqgK$W)N5*TarqmO#0LZZvBPjvp`um#17$!;FN;w-#6Df(gLWY~q>V}`O{yG= zsx`4B!8F!RG@_85@fX>#pZCZxw_J<53&{3R?i4e>__EEx)LU`6(5H^FZlcX#qy`Ti zE@qji%ZN3mLAoEo!C4h)TkD0-nG1T(8`mS!`Bq`?!o%S-Z5mE7s51QTC4q9|2C6vU z1Whitlx><=Xp&jfi+;ChUwUEZ2(-~ZBm!wk8Rh?sm#33}fl_GikqkNY1O$l$NMMBU z#1NsbC`~>-W=kzo_oGAhINtv~svyMJZx0q3AlFukhbon#L8xEH)hHmtFGsAcp>!6q zkMEjJA9SK&5YIP~aIaF#E$LRQjzJAl7n}SHf5oke9&~Q5>bPl5d4-(RYE~9Wl8Q}5 zmQJA|n!*%3Wnk`iWcK?C_jcCi~&4 zKT08v!h;t|I38tMf&c2iB0#*=N3S0bEN~;Y5m2FJFxp|_jyuX1TUL?5xSoDVQin9R zyAmv8tC?CtAV?`M{p0J(yM03(_Sfx395 zP=irofRt|Zn>`7v@6Z&>h$D%RgRiDAkd!s9Y+H93X!z3w^YR7jdD^hs)3Ngc=IJ%i z?cl4OJCX^4)J!c`{T<9RRt%MD-0MRL)zXRued4x&%`+MquOqJ2jzrTd=$S%I z$Ou07e&O_de&pEV!4%1urohP8D7C4SK!$sm!Jo!KHTy9^h^uN-XoHe8?%`ssHuB)T z8$EOz+fr&{+AB9#y{5dwcs9j52$r|_@cl0E+O++R2GX&dd8OH>VUz0vI?(`h?b)LJ z#Te(Wd-8_bPF-{#qs%z9j#OVal0Rl4Uc{?F{fySL_>|Qx{5_qA8L%6%%UzVvkF7rh z^zK{`rYC!U6h>xr@unFiVOQ2;XXGKWh#z&x3MxL{0MWb{C=f4tx?ym0B6^ZKnc(~Q@kaf8%vC0?b<^Ge-hrE^J zX)=ql`Z!ZXyjz>Jd1Uo8tj^|$*}*bd(+NN$e%DQ-;SK)oz^>B_|qd_`QrQk8#?bg>d0zg0RXc70RU+KS2+CN9QU2^V(W_=ZmRl4}+LGZwZAs0x!z*^dPo=CyIyYq4P^&!92Xp&)hLI8-ko`RPa z?v|&k$gMarH)OT6kk4jju5|twwB!A2|KJPxUzEfB?V;izdQ8Z}K;}&)5%Ys|EawzK z&N=4Dx-wR?X%x*4#)w1v6>V(iP%dVnrs2rR4b~t@&_T`gl}oSlgJNl5&E3ODmci0F zKhSKT)?((%#@Zo=$y$^0$ZlPzccMO4wCb7cV}Db4Ff4AJY`(eE(zn>_@^_}LF5w$v z2B=vW&C>^s$MdYKt&g|s7i#u)^xUPY*k}X62|e1YZ{S7B$;D8dt|pn|l+;nWUxpiTBELpcp ztL0F7$3cKu&1%o#co(OTv!m1%%2#2^Ewx$$=Z<#z#*HJr282%*B&S`tvBOzsS9Q(d z`rGF7(H>f;OCI`*zPQH}+#MaAd~GPF7~Q7l3)-^{KONrru1hxXF*bA;`AE4uH% zx1+&YiJZO>qHApDmiHz@`>oxSV=$Ebs8@@S`3WMa9s%nh#3q^WMoN6)&hvE$)W_)E zNjVVd&Qwk3C5)oMfEMx#z)dOpguUD-mS~|(HQRF8d5!04y%ehx`(TWNfY4x#<%v|~ zzZC)v+WCc0E7^Q8p%*`(O6KzWy=*=zX3x*0Img;C?AcEw z7HZi;6MB;6t;zknW$+`l_&h=)sGb&EKfxUkOVXYX&Z_gacJQ)ppTAi#MK4K1N72;6 zWZx&ebV{WBk>yK8$3A#d0Q^zu;2TgIJ|(*M(5rBb*dY{HKbKulPl=I6kdx*=iR|LK z2EMZY&OtOjGAMxhOfV|cg>43}ou;!ye-_0S4ETHxtUxqe`#R}KuAw@egE%{w!u(>L z*#*Q(ZmDpf1PaZ-UU2VA0t!9M@J#_$k|g>z4_=v1!lS8r7m=s6&WI3i__|SY8DS)K zyGR`u&pWevKNiH6|75b)ftVSRZE{1Mx*$f+Wj`)Z?oQbsO9XVAzYN@m359 zZ@la~QKUB*EWaSX{oG$xcs}r)i#RWgv5kM1K#tMZStzeqNDW1bo_z(?$Pff!9wJx? z^$oX0lY3s6jjGNlt=^k4IQ$G=-^5uH;>ho0kMEZ)!=J1A#3h1zKv+xJ0@J|h^ zo63|n!D7HjnHHLVvvJryK?~uuGOg}z=z?W;hD!{uQDSQ1j?_2U8Z0dhGUIb1X zOgL#c1UNrN&lO{%3SOPn7RBNxPahXHPM_~iPTr9>yt+EDHos$WQ>@8iAnNR;M?C+B zvU3O$B~X)f+s0|zwr%^gZQHhO+qP}nwrzL6bK|aNR&SQIt%|6K%>T>G_x1V$i+JR6 z3AnM7v|7A(N;0b;Kd!gEz2b=lQNsh^{pXEZzt%oB-L#hLJC^JEFE};L11>F9Ab{QD zO~Ha>p`mZ{u~?*lMIj>()-S>@SW-ti4OO)gdJB$?aX8@|T}9YDeO_Bj#L*{&wlXGz~p{o(9g&CQrUFKMT6Ix$}V&06Y+kdUH3@*r`vGg0qdY`{RxdGTX4gK1)fko zhv!w%H_XIm3t6XV+rF@W!Hu{~InHH~6{{8x9PEpI0_`mJSD=)k25 zTcjDV-WHXR?A+Uii;yC&kL{6NE%0$+e~a7KU=8vZm8`OVt{PsW=idfT*wK~zL>fc9 z?YMcv6^nKESH;X!S(sKYq0dE$wgU!=%GNorX;E%o=~8w3zSW__CzQbdT-XF!2xBwJyUDoknuA%R*kmb3DByJ$dx0n5vA(bx+X zW~FrFM?DM+I^DY$-wBI-7s{RVnO=_Qn3y+#jeTCMZk`DYyGP$53W{kvWAC$mxSh!0 zz+d7fS$#)rKTP}3t-h;lO-r0Pq^G7$sfBisUeaxV%o>?L{ZtcIaejZ8z+@is#MQEk z(S|~v`rBR(oQ-EipQMfJ>a=bCn2B8xa?`d9{(#mRp29|Pw!jig_YFCH&s?;lTT;3z zE4?lbXpV+W+j+&~)h8X)?AmQxhG^PSdfIY7kd7sDjCQWgxjy3Oz-rg#PwuZe3=+Rg zjRv)?Jw0U*;Jo?VYr8m~m$U!btO>jj@C>6dwdpvv(I%Z#66nR~n87GPe~RJg*Vyb6Q2%JkgF)u~8k=2M+sT~l)2@T{^KCsWyf z@IVRk;9WZCW&`8ip7`qkPtxuqcd>%IrEFiId`DaPj5o8$cScV*Ol@S&0^Zj*8h#$H z%7Jo2P-vMI^5km!hdz^fm|*hmk+qb{mcc3Qp&M{Dnow+Q%E&C-8(SY2MI(A82FgO&c~uDb)*n(!pvvyVc~&WXI9tcNSR#f{ibd zEX_Hsu+rUrHjV)^vRD;LglGqdKq9%r6V#5L?+Y0ZvTy$IWx(O%iHB}l0}znN4YmX#g<4IvQ=-Y10QO4@YryEcAq&w|yuR9XUGal; z!qSr&Ry_G(hZ>_=oI8%`BnrI{qDDLu63(&A3df6Wm|f(Pe}$75V#M4+#)Y?`A=tR( zcVa+lVY$Bnetw-(rxYFBLm`fxj%JP&Fi0c*nCycJF{6i4e6a!#UMS!(R%^bDB?m2> z3Y|Wc`MBih+5%wWga`0i*L`{oW#K|J2)HieuQ7VjUpTd(VR2^1m@O{u8_8#}4ph!J zPxsXU)?3wO;7%>v+i;p5VB(}=V`Ak=g{GVX(qd%4B;lkHyJYIu%wEb4RbPi&--a*t z&ZM_7yeEq0A9V2%+%Cq2aSo|22^h@S@=Wk3fMekE)nnTW>z~)9-~GkTe4U5ZO66nE z7*5|05HkEFzU%QYI#<5uW(A!*Us|W%k>%=Lsp^8t4k@>zWwxWCLLJ^c$Jq3O&?R89 zhGLJRpOxyUx4Z;Cb8=?FAXb`=^l}1`1&R{h#Wca2S5JPJFUKV-)sGu6%fDr0tuTc> z3=$^SeK_UVFg0wN-QZat1%FZYjkpvg10G?0p=&Fo2jtTWe(MMS+2hho(`qXvpj3VI z{R@11F+)2+nro2i)@zDXMn-I}O8a)RA3l4e4DMd8ClZ~7QmGN5dO-c}olL)Yx0Q24 zD1tu+*(VuaW=@%_QAjy9RWvBIawbc4l(Xw*&uP^k_wl5O!AXv0aC0xBWIs=`Q84e9P94MCsv{(#k? z(Bec8sjuA1SVIjj;=6EW$lcQTdtm>dkWa64Hn@(PL2_CJkHXf>^ki-S!DcISnaD%M zVVXNzK@OIhzIjWT5B8Wc&b7%M24I;A%l?w=)}K>O6NDndU?Ep$M0Xf$AM1j6mmg=r z5g($wh~t8C-6D2gx&bTk@}e(t^Iz#et_klGmN^|Ld>fJJhS%J2679g}HrUqugxVQM z^|jRfTIhjaEu4Qg{^0R#Z+(e|G&ItA%g}bGh}AI~EoZx~*ot*quc3bb>e)Vhz2@z; zZSB9^j6TkRU?6>o_&se|wEf2WpVHTX z2$5d{763rr7yy9%{~~>b{ud66W9hib`q25$sF-Sci4qyH&*=P@X0%#2->#>LEpA$r z!wm}}W@aw;%R_yx`ThPw*F$nSoW+=IiYMAH ziG`o{B`$kQ*8`(!6v(+xbjwetlo<7(6Zo-Vk3lIYME6$}Nb7n!bNp1*YbaGKMSL&| z0_Q!0WoCjl3<0Cg5)E26jY)%XW84#5qi4R6f`Ev- zJR}FY4@%_vf>C7ffWCVKXcR+dYY9P#3mhgcPE?&I<1{xZA*J-U=YNtGQ>+pv=LEWf zY7^hL7grzB*EUv+Zd_yI2I_;2b`w0J?XR_>IEJ<|X7v0lW$M2nTf?0u_wg98GKs`7 zXue%8Ga3P;2h7a3_g4>n`?&`ObaxuEW27qn$6FqTv6F~cQsNbI2cxXm4Py{^IN<5b zq{CYNlaYw3S34!}Ky&n|<7Zyt=Ht@@!PbSkc$R0D_cpwHX>W<7YfU-d&|TBgE(W?^ zl`!;%A`0AbDu0WcIVcwnl$@eWNXQESk(BzZ3LxQ-F!jj$A)3aBDg!Yx3#}oOGBVLIZ+W(;atJA1d<7VN5B0Vcu9$un?&=$nBfzI;e8PQqdYM zZ}s3tLkF%G_WMzU@nkh=(3y6)VmDIehrxWg?(bnB)PcjGa*H;9uzbSt;zG2`8 z#cSw40TLdz|26%RaPHy-_A!&?Or>3{^Q zzzSovaT`#&h`>_=LihOKn38OdKS5noHQq^z;FsuVy4qOfWHnvQ!91RmF~&IlITbAW zHRyzs^M8Sl0$-Q6jh)X}`?t!?3yY9flAo>KAdoAgjn%HKi}1{PB$tp!W_5T*AYk5<>V5niEr722x+g zBh}FynyH8rS+cugncKJvvO;H^6bToEim^%^))L3so<>*YkdGM%a=TciqGSc=7~h{( z&~AmWK?vox|Mr2!?x5!|W7Y>SAcuFqfj8v55aYfrDPM{t-**VkFlX#tTFT5hjlXfFzL@dOQw8I2&0Gp8<5TrR8z^aiXh~*CN2R4YkB0|T9WhRiO zKNyilF~WSBv%yybuY>j-&gHF}96Lo1AV0|8*Xlxnet&M&WQOtw_k%l}SA(-jknG4$ z{X1dtlTI`m6>_T)A)`F3A-)vw+{19hQh# z?+SB((DW@m{#UMk?%yYL(_FC(%aGTXtvgk)*KjP@3G~u_$K_ou(o~6oN_dNnD+AIe zO|TLbv#KSLZ?h5*A3k)tqH9RermbX5dK^N-3{q?Jq={%In+P;+m?yTvDTrRvVF=8y6nP zNCOBK#20-P<8l8T){E8~-guy+aH5#3ab!B7Iz`g~J$zmXSoK#aqRQnZJAw-=yTIm3 z@j<=`@{g=)LwIxa{vSEkW>Eh6dWt9Z!0g^~8glA@^uqR!T8USi0#{e?O0NC>&jsm1RK*%oj?Twc~y}38|f(>fd(EK)|jxd$oVgHYl4|80CwHN>sFC z<46GG>_X;ZuAwnG(eCZU(eH7OI0iljiu0Fe4l5Lzr13oJVDfPqxBTo%!Q08#`%VGi z?rxAIq6eag0V{EgA+Bq(e`d&cQ?e_Y>nfn*+ z1YXV_$nBY19+*OtL#;77dDJeY_;|>8((ueb#@TMiXJgnkhf5y9L$N@UYfHNdrE`d0 zT<*Vh+EFLAky1aIz}!djh%ZMF;d7@L`q&8eF(cVY5{u?E_GENmz7ck9Yd?COTD^re zzA{=FFDh%rE#cqX8Hd$s*|iy`sQz}k%2uSR%?h0VizH?|tP>a$<6Pd`F%s+m1|?6x zKja%X;%8`H{*b$g#RiB4=^D-W1{vS(_1mxQjdE9M#%GpCV+0rFnfK=>!Z{j#-4VP;`Fx@q zV=z(-M@o>GeNu@jwqSmmWeCQB7VI>jHUj@ZpP0!N#3?3h{WM%FA-o2hoCTh(|JLS| zRH{%V(;1%fwHA^kZYXCU3sY1XM@t?1vW$dl616n2#}5GbQs4L{Gad$O7I-TA7p8h; zH`-q;405A>nWLe`2?AeXkL$}XqYP0J=31OAruvsgk3>uNIR4P*=g*X4A_F{P>wa$P z>L$!Jyb=Q!X|Cxj7?3B66~Y_~tBcr!k=NEru4!{<9x?FG}{6s(g~&mRwC0C z7(2IXfl`Gk@X49+_H}LnD3hIWEBoM9(VyyD8ec_G>Vc9o@VWl-%f!AWloJaGWnw_3>_8zL9=J9|<^-e#Q@nn*ih7^=diuA2eR!^3gog z4FudS)VUH$0u(vb_~f+T=0rRmuGUy;9LeH#E6hY`$u(7x$Zc0*{|0ZaE0dwDE! zg9Nwegl8iVSmIw#`n|lp6AtqW9hCesNJ~QJ?9(W3Vy7g7wPKB<}ERq9r2 zP?O+&OR;xz;EQQ=kzu7 z_5fJMqJ~CIMKWmE4D_=FoKoZrpo6u`=K2?@OocRtE2aM=wa=nwxd5g=;+K~5*BbIyj<@rMq! zzT#tz!(flD%V6vzO29%`fG&;z$~B0&s9m_t@qSv-QfV=PMSQ zMwPfv=;~KAja#~=ooDE3rQAE%ZS(5%9bL=r-W3nin&uoVEh;1cIeH*Q=vO*lM_rf}^k0t&12s z%o2{cA(5}yGlPRdpZnIE@Fv;!&Zf9K?YEpYw@e!?LY?cDSzpD=glJFu6B!B%76O1P z1U1Lg9WodC{{qNX>i1$7^LoQVn0K_a~ zkZvmQ7%^XKM~@zeFm!d|BY$S`K2|TvqvF=)tc5GZf2lVtg$hscaq7E$DHyoT9!ZF^gw|xI!0Gf z-Wd80%2c+1Yj-plHyDDbJ>LeT&+NH^70~4=sbcpHWv%lDOCtSb?S@*k@O4@^q9q?S z9bj!_aL{QV9_y{VLv|mqX39Iw{AnjQa`!4#+8dqjTjhCW0@of=Eag*%9EOP0+3t8^ zWryYlag)6`5Dwfxr~)7Af{#VI{X5*4;*9xk!%>iUZ;um;T`7u!+wyssJDvNaL6s~8 z*_^xqO$cf&`-=#p_7e6XzKEjp1gD@(P+&^Yqw#>zd8tEZrl;w=5(45vY3;9U79F&8 zq}QX(QIL{`o;7m#hG~m{p0!o)C<^8BMT4uD`%>j+>?h6=#IZe z1^L1<;obD}U-p2eysJ-VzdkX$r4)RiWP=V%AmkQI9xXN#N;FbSrJJV+q$gBm+{VWl zd1@!XPMCSS*r#)bmYHi*W68ez-F=w}OTX=NogPiRpOX8SBrqNrGzLuIVG{<-e?-^C zYVL-ftO?l!nWxHtV%9UK+=V)V`@Hsw$?C#c2JTOo?F1(?=ZePTe;70(ST#Ma!+pu7 zz^=V^0XO?NR|fV;aGXN8Qd@ibsfnbm7s{okIDGh{)cs!l`7^rD3^n_huI&VtmaUv5 zOIH?Pp`|6m+Yx+Q|7|4){sB)xNU0g-$YsIU_`Ak@9x|~id&Ca0t=#rj_1}^7bpi<_O1YR z20}uiWpIU`rSz~ko2%cw<0nR2Rb+CD1!8y0vIkG-!FX@Ikr|%TcgZhp)C(R;hV!#z z#gL=43^RmORpYg{DiP{?({J$y8#5C*wGU6BvpB(|>|rm+F(wb}(uBZo+~)Rdn+A48 ze{06VL{Ey$aaLeMRg5CZFR|yeMx1{g=)@NMNVp^qAdCo|*tWzid@>RYLnE?rQbA^|%-Yy1{e?JKf z1JGU&MZkB?=31!kQq4M)|Vy6 zPOKc}ZfsnNee?V~>Rofc1nGP^(D3Qc>%!asc^|LYOScf7Q8H=!GqBxK$Mou+xx%Wp zTFdp`&yMCj&KuYM?tc4K$0oA}0rn4KKMx2t3bu^P&@()USZ{yCaO8_3>7U%m;J=_9 z^MtBs%m6qZE{&zw#uTQ@*kuFjP*%c6m+g4=mEd(*Z*41qKzBLbn}A+GF;nO(eq`Bp zIaaPVB$<#Ob<^foK4T>~^Y(cFYP9TWHM7~1bF%qF%edfp(&d_RiiPw5J2#vnkmLpd z35Rq|iEfTA;!ScxZku)M{qr8qoL_6d4!k|*xj8!UVnk`((?IjpK;$2)3&(2299VDg z!yH(eZ=7bBS~^-V;D?#_>!%75wKLk@BB$2CA@()|)seb{B zQ$C|^S+$ETt%OMKS_&E5JS73aaHYiJ3@nPB-EhaCPF|Q4VuuG=C24~WNWd0>Ym&7Muj@zpxqy6<=?>-J~zJTff3pQF@l=G4+eEj z_u;>K5&9B>f3m%Xpwe5((j*RaeF-oBdRSe(A3ZVdAIY)Pf!z<|+i(?4%{vi?VXM`6 zTFUgmX@&NvO{W4r{RJkE?UekM;nDWQHdTp!Hg=#&_6m)zwn_EEjp5T)@diqX`k2^S zL_$<*wlz)PCU$KADFJ1x2PymJ0!t)Taffrc<#K`TY{92TMRSOY;ZTh2v#~^Vta6dr zXBG6T87S)DdCy?S1ZRvA^sUcp>0f1k0&vTy(1>aT6~bifJQR}hJv=n#twV79{VqG4ccKHUQpjeo^ zM2dwmud_IDIOD)=ohk@Y^ZTgFkIP$bW?FEO_u=~^6*3CS)WR$J%7j25*Ks#2@UuZt z4`IUPfUMuy0}gL8!}IJ`b<;B69Bco8ILGyy^M^~8@$%+7d?7Js9#pEw9mv;$NtQ)_ zrl}jR#if(XGiwhHc+$%{4M)H@S3j%sGGUyt&q(*gk=uJ1ElJ(*+p-lbWu&v^`Yrrr zfttevv7Sfc}hkVrB456F5mtHMgXo4_G5U~e6nt=-a{F$ko$+_i zWzJ_K#@QVS-p_agW7ZoGv=*6``6H)bphWgyKzJ8Yb6GA~#M}8_dt%!11iZy_CWVBg z5bgpGVMIf&An$stbltEGw#B~Q&V3pi))OiXEYgt101!!pGJc_8G~lCADH9^1<%ZT) zwwDUft12$+>>?pRMn%}(Y} z>LXIq!3rn{7en?JePXA4#ptCQOW|laE%>;`cZ${*LzXu6?*K_{Fgg`7*>3JWXVR+1 zbx1T9qmW(C4`u>|oD9(3^%QDg49g|5UAWX_)Pcw=wK#b6iH+Vv5$5 zu^D4JlP)ubB}nDjaHHRikCKmNW1z*M+|xfCWL|m@i;Hd3kMux$lgI+=yXuA$9mAMLRL{5HTqN;< z*W(~4ex@?TDCC$P07yls5=52B|1GOoG|KyL4p}`4ccEafWygCU)lj~dpWaiBt8||$ z*{hmP&PdB1msG~hmp{22r)69G2*b$Lo0*nmQmj!PwWSYNV|Pxs$PHO0GHEh#62WT{ z&_?1cIA_g>pT1<3Y{ER@sV`s@P|%!gv4uuwxk0wp79_b@^6>GjYnJ@BGN5UKj^Mj@ z`S=dJ(J5ht#+F^r-HW40R*diqCbcv;o0Y)!>5@FrFF1I&I!(X zQ$jL$adl>u10%d|Z*|&!5iL|Bm)eR1N*Uv8U3cM4StD|T#_cT?I!gmg`()oorp6y57eeKmmhY3*<%(y%sjA*sGy=tK;Q{L{gmzqvy3C zD~Em3`%o;>d6+apa)lI71&cv~>~b0#H<1azNdN~!aA%1w4v2K)Hp!40gO4_IR+3LS zGqeIVRI*AZ)=^Q)I(IAPfene%1I@e zolGFJ9DiiY{dPZXOf43?amq@??yLyh;Cto%W`w?7rv7RxZygTOJsZ;c7uttP+~Qe% z*%)+4$LD;^;=b%2g35Y*s*GOm#|kKXy?i4g(pn^+-`Al`B1~88bX( z^~c3p1IeQ|nKrYUM}(VVI5~Gr36|}hR6|62UL}ZplovA=gOe=YPy;~C>&f#buty;eKcPz1Cyv9}T8su?ll?rPQFnZPiL&9fOM8o(} z{gYq)^e?qB&?4ibEjZj)PeIjR*%5`uwyrB`@+mGc*u4O5yNpZ zVxyNbyoMHVK+t6Zg4X+*7m%On?ku=-7EQKIjs41c?bTze^q>+r@s4v$MLuaCe4*6F z=^nM^Who%?#YD?XKEi?1YFbgc{)+f_LXqiF3}Yt443&6T8cvah#3LqNG3ZPjwNPkM zP>}+bBw7e4y`W93T$XZ#r`e4vpi|fc=f>pE4^-R=^1CT5%PVu8s_76|NO9tm>R{^; z?ybj*a$9(oz#{3NBE=8Yp@5-sb5iJnz$h4D*1hbBcWu~>@TPsNdf7aMA(q~fw%k((qzMQMoH6)hvTTPrck*oWo0M6Hlku5dJ~S8}EQGY=>rkMWj<@=0@PjW#Y%3A%*!vQJ0$>@NV*Fd(HHf@23Adj^bc(zu)26@9zi|3nCXiQ#!>*b zE*3`>;OQty{o)lJn8#sE&w+rp&iIi~py9quMmFE3niuNyYu+#7Pu&kF+?}RYK)r!RVHea5R3q0D< z*=ZEIzks89-w1xJT9e9rXbNZs+(p}=!7T3<(`J!0`ut#wl29ptU4Pp^wP5bm0(g1V z`u#d??peBHrDLj4U;8RsQ|P4^Qd^ikb<-euY)a{&fF+HO8N8;-VV%}cP@I9h*JxCu zd0?uPir;De<3egEjTZmw?MbPgz0>p=h)?i7PM`&J6>qHt_y~~Jfw}kp>Fw)iKbSAW(8sCjoEzZ6l9HDgdsE@ z=Z`Lzu@V)8=viB%`M04rk6J_ehwh!8b&b%yqbW!xo}6c=SZ7gwwhw~^ZJiamIrWxE zL}s{C%(972q}@%1Lc}eB^?78`rKo)6m_muh5hC#+x&>na+0-Lr+72{{ERLmlvc22! za+B&SGO%X#?SG|?Y!~%&)gS->c>k?la{u40{QsH0*SNGjWJUVD={2YTss^qq?&@$A zH$;=Yx-LmYGr0cqjWUOXQyDMfpLES~^U=F|m!C?)u~TDz3c!FMR&w$WC7g;)n(4*b z((drxg5NLnbcB7)rLC!@?Z06EhhL?}Fxi2BjFfx3fzXo0LnQc#5Ibes)?y%7WXRKjd&LoH^O?FYt>Ol-7t4NiY-- zu?6)mggGS0ekZW!ApDW$$abRHba5cm+?K!SSP_Sl zvX4oBN_(gJBICv#l2FLX>cVaISYV;~PbrYg7&jrXzLd5L!u!+YcG0&XKIfPy2UuC= z9mn@QK!^4jHcnqJYUS5Z(BYKT)43NZU6evIZiA1Cm2We)7{}uL$e!|0gA||IlaG*) z*v1#XA07lF9D@&fK5C{Tx?>AU(&lLlu_y%;f6vaSs+@ZELCOYOE26KR>D9hlaeFW#M(!SR_Montw2BLLqnb;#tVp#>rr>f zG^!EHovVct=-}(bwgRJ|`>D(@vAEf|M-6 zw7abGM&93;PnHs?+K~&82ARf0GdYw+T}c6H@f^L4+G>w#O-3kS58+N)@_)kkpoeC- zlXf{Fq0@;^43j0C<1HbS#)CC9F%3xo0w9P9NaMmJlFa{w!gJ_zvS|B2BzJCEJ zuOFvR{b+l2=4t)}k&0~KY)Y#S^Hcm5bACDi2?)L+4=I1d3i+#} zQ&scj5;zRr00N%)f+WaF(YitgW6}l|zkKF0**pq@hhvbnvgGr)(5cU2|1X2Vd;Y zNlSAQ)@jf$FSalunRhwdiqkF!)S~%FLZ0iYayYkadOKLZw@j&bbMq$Rdb7tVFKF?e0Q!uGKsX%>sXs(pCT1q*+>8IX86$_-EAy0q_pLl zdCiycvv28Vg7Y^)9;ZEKv?gK4>f^c$9JRqn)O>ihd4@o?aL!uGn45&PK z8|ZIhHB44$lxB7LHwUetPYMQ^Avk_vXc?sdzkn$fI{sa_IDV30l&;PIO39CxbwaX= zJ(^9(&1ffJ6z}%d6W$N`7Q^I&WsoU}akr=n!PV2yUov`UY_zTLEjbCJ+z1o&EzkiZ zA%p%jdukZ%O9WM>gZZ>KAN*H0vJWS8NuSp?YzO$t}^1KnmCnKE@sK3QqfcSaeh@ppm3Z)RnqMuDS%)N50z zkfX^gw5&c`Lb9vxH4?!xate&F4g%e-Em93)1RH5@LGfU}0 zKqP`wM4N_8(Me^GI|A&)*ht_OL-Ckd(t%)Emzf*Qgd=YGfAmErshd60@y*lVxOwPh zr5^d)ZFoT1sy9m!q7a~oj4Y(!Htn=bT&3*ki(uo`7BeQ-O1O@e+PQKmN30HacqZ=< z11j7!Unv*9U2b)ZYUK?xBgFg{QX3;U5bB5i1PNE~Wa^F)4-Zmc7N5&-^Wme^4H;Na$|n2ii`X zM#A6qy5V1YD@qs^V5u_Itu;NYhp8vKyi{1W_kQKcu!GZAU5T+=$v!{?y)JKvcXMGv zY<@2=UD~-MJpNtc)-+2nyZ30U7kk_;jkDT-$&&L-qCF!-uiihb z*qJ|1dJ1MIHT-^)yOgoT0jco&2%#W9g5D%c0OIS*iK2}p3tb<$iVq&i6HM9Imnd_0 z2GAjMK&9C;DPp)8{M{Tj5E%c|X78|^PIIDftJfsmTNzFyJ>PPRV%6VJMXO=F-O%cZ zTy0_37QCcRDgut)OwGy2Sj$|pf|;y?mp4TW+cv!94G}Nr^i4bdx{66ijVKGS;5|8%E>p zDsVN9nzyZ;?CPdfvesEvQC*B#w9#7Ck@0n&%Ivs6J=MAF{h#mwO{&;lA~XQNsp9`{ zPFhk$M#bFN^}pBwwx#0+=ZmLLa1GaD<`qb%`__#t)K-36%QSz2n#cCCy_rbYOh>i%q&eI*YYg0~pi!!N{*0-EI4OmQJJmGrRUHULG4`~P&r_tCUBEXX&fkM=e5VW;>Yo|7R zz`*?Z@ORh8>ASC1A`rU3act+XXxZHYw%<1Pe`NF(O~)1d+<9gq6AOYX2g&rTu5C|; zk$UUWrGU32yC`l>LdbrfJXnKwEOIOx44?)VM|E}&s*%E~CqH05Y~VdAiIpG(8-ngM zKwRSORs#mp95Mc3h<=93)_4-$ksWLyT?Qzpl$_%c7uDPFRbTc!F(-revIR3J6CA=< zo1v4qHx~3trUCWhGxkF_8}HY$F$(|@U|3OqAjkVN1Pf-Fi$l;4jll3M3^+X#avOVY z1h`DnN%+02%hzRJ1Mwg=3pcmB&(m33W^yqT2f-Ks1o=xVYX5 z%{;~(abNIXtbO=w8ZnKp-QfDRlbbT<3crH$=(F!AL~7 zkU887R7L(jf3lN^b?U)gRivDvK>)A)%=UMrOb7{dz}8UhT!Qx3V9`U}k3%`jX3fQk z0JJ)z!~&g&z9a(;I{1Z5Dj#iEU^e0PpjcRQW$o)2|M6nen#+{o^#Ia+q$6&zfvovY zM==0=8wDH}sImB+t0z%v8eN#LXelFDLP1S;ehAy{0I`-HgD}{P&|x%JjiUM+m?g-~ zHK9ownpmBBgdgw1Lb~Ct&wj{)rYv*1+n>l?q@0?Lls~E#sdG52coN$b3Co!MUO|x5 z_Z4zLc5DTOb|Mx_@*T8q;d)Wl0b1~o`0{wHgF4KVTSbxOR$NGI!iHrY2*kf-Y+r>@1@Pg=3TTN%N0=jr7 z1!{{CI%J3VQzTCZ!7h&E=JVdDK7Q4s3sy=+Dh_7`FeEQn9ftz;jyTW+2m*`1p_;f# zgu8k5b~wPT%6k!GS99@d0cCz07Xn)6r^*(x8ly1u56+P5^z)BEbo~Z8E&2BI31X}x z%{}rU^&ew&P_w1E1D_gTQf*NqY;X1`v>j3*czCs{NU^{3@l(tfi6MwXpFW%a`?jFt z%5)wOYBCgCeZ-p5=oydRGa=<4JmeKeg*3ohi1~sT6oEoJL z9>9|W?Cne4nJE3|QT$%4{lLU~g`1xy-^>hIx?hQU`%KTj^$oJp7K)Nqoxl5xv1GV1%Ek{{=+G+f;TO z(SY{=d8%;!Up>+-zrxYm*kileYLoyJHjOrKxJL}IZ-f`zL_r7xJi-)e2}tj7t(%t` zmpC^v+KLf9;OfW|PzXt-9Z0n!RB4eM0ocXR(9@GYtCxtj<_Swl^_zp^_}htR0N|q4u=a% zP!c%k=u4;M{X$I<5%MDme};9XD3wQal-$KSSFho3dYZtyJF~KR2FIY3<^+{2M|oat z>_82Tr)dS=QbtnwHBum7^q%7ziir1To5=h<*_!{-h1#WnGaEIhZFItxhK^^{7C1>wDwkEDu|1LUA^K;%w>m($yEp}a#mnQoV==5-(W6V?Zi zMYM3Dqx4h5$mrz_Ka@l*23Qo8SKDgZr1glW6J?}&;zDq`aD@`P%#?)1?OBr#5HW!< zVQ);n9y(5{!~k}+ikV_vlhoeRc8p^@rPMCjVQlMTbo3dA;$AFEim#+b!`m`2_FCaI zQrYsv@_ekmg8SZUUmkz0WWMu@T?tF7bOfZ+=ry6m0A{<@{;_{)QnFZKuyV3k6~Ult zHE7+)@D{br!@2~XFoY37G@c)XZZbQ^X7j^P_HJ>VIolHAtcKhA}tpa7}$J|1pgG(KtNp7O8w(EyBD471U$TD2J2U@1! zcjp|C&0SX{BL);w4xd`mW#w;%bxo_nbJW%PGpl~T%&Eg%M+a^54k>l}mb^`2gs8JT zT>dJGh}B1DKo~?aX-#bhm#(TNpsr%ETgR2~2zs`UG6|osWl0|N9qW?MGo%2`#Mj^F zRPt|0lZ6IsAkVIrJ}22U$&$q8_!CUPW)^(KhHuJ#=Xxu9)b$Rg>%ZZWqD)d1^6O<; z;~S9%sN2Q-1=PdQl+KG46-`Vt9p8T2F1v;gW{v(IW$zRuTi9;t)+*b!?W$F_ZQHhO z+qP}ncCE5)+g*R}?ufJdYM+~o85wz%G3Wfoi!mMm_DN1z<ZF<+AS@~W`N z3n~oJ3yVHMOv7MjM|Ksb)IH2IU%hzK!UEI@b3_6#aJQ2Ph4;_A;S-hRyaTxfg{KZ~ z$-L9Fs(ZSWc;BfdsVj-2$dRcyNR6VWbl;Aq1ot!#L02)Kl5ytpOMLlSPyHPp6!^6q z&~%h^Zb^$DI=F`yKrP4jvrRJO`?u?kpNOjyEQfveJ~iZJ=r7Aw4a99N*Ks{X)=Vu7 z*%J1$3JMJ0XN-Q+8%iLeAeq->MC12$$UiUO99^DdWdge#6s;DF>E zlM5IDYW^GO7*ME?=;h{m@AR=02$=QTDaM0MXcTX5k9X!jeaJUI+7KQN9xe`s0OEM$ z)(rj;yxj6Alqy%Z#WHn3po6t)x^(RkMXq<-QdGfV7y4l!7>3(!)a zDn=Ct9y0zB2=5iALmuE)j3J{Lb?){`sKhg~r!g%HLE)UGb)G69`9vZh3#V>Slw>KC z(WEef>xWCv^PU*sE8kIW&_Wb_-Oy|JAwqz`AH<$oNbrjB<>ISGwZO*7`!Q2^N(9DM zISwE_VRF{zM|s@;tRnlVKXKQv&+!LPWtgNTx}W<-*@b~IBa{;oqA$^-svAj|X2*1F zflup{mX-bw$KVrjZ*ZrJd#=^hf8sInbb1Ykh{CAbEZwp?PB#JZ$3gSWorFkW5 z5t#^H3z2|{;47;4t^ga&O}v5Y>zst&4US+Q{$Y)JS5j3tc(I425HkwQD53@bfNNJ< z>TDhc)Z1yvj)f{Uuaxx|&=Ukn^SiZRinCPWPAsNA`*P5D+Dc702q{Jor4j~GvX!~p zswqdo)7DS`U+MBGzWBT9`W&J(jTGkAkn;m+6P47{vEmzZh3YjnEw=fms!=uC-sj8V zDR?(HK5<F@M%L09yD>pAR-Kz9sromv%2Y&1nYHxPVA&>c7yo+C(Gf zyQC$HC(FHFGcGFJK%>4J`a@wP8$9-*&@%OZhC|*VDyJ8Dgow>qehS=C!)4eH$;yWh zm<*ZZ2+M*TNtig*@#de0;uPqwnzX4ExJ_~?lKReQ$d!{1M<-^zpm+WgwMGrS0M3oV zl^}-Z0kn-S&VTF)>PeZ^0`N_t!+&&6PI!0G6#a9>&>PWBov7ToV-pgdrOS}LjLjMr z*LL>n#e8P0w?_d5Hlw~<-*1Wfo37P!a`X2C=w=1SU&l!K$ILzC*3RWU$oCXckBU^^ zeF813scXQb0rIIVH*kH5Fu1|ICDehoGbm^zml9euzNq=zS9KmeFC?ustRfd57B;&t zp+n17?l{YoRWp@&1KS%D?7Wl5vRWo6esjp}Gq#&{3JiYYlRX4m82@0F9yGUawOHr5 z%^6z79io!mAv+QVvJEX&>36E4T-KzG+Kicy-q*kM2i~IU#KCj#m#ZvcMfL8_A-C&R zNUM_AGQpN(;mQWUR}$A4latDDb~?fIIuLF_Yo9I9oE4pt9}T5Agx7_;ZG;|PI%|vz z>R;U^Rmv!T`6q1492-gqFd-}{6#=qnCp;%*?1LZwv|G?oYE~B)5hdzxOc!;BBD)Z# zx+KpD2#YZ$0Vs}`lCwYwQOkQG&z0M~z}rm?&=y>o5U;FU%wQ8{UYA%rJPephbHc5r{nL2mGufJl#e(TqA$a%E-zFy;Yq7-0cR z3fVEP)9)zL&E^l^yLxhPQ__pr-v)X-6u-z9@-U)T;a9AG%WRhCgM6dO^)B{*)CE0)hG*JP48V9;_LAjpeE zCY&^q*-9;wXI>*l2UEHD>@~-Lxp}NAZbMv37z`aFD5QsMN56qbNtXLM13oQbB>*vZ; zxl+IsxT2JR*i0WIJWRFWqBXAo|8dgmMCCWTF|tYL)phSoqaNUVd*jO0>;An!g@*~% zHpqf+BmUWzu&0=&wcpY&XT9Ipl`wY+x5NxHN6T{i550xfepo7D+yP4|#NZ{3BgqC1 zD$FHZsv-Yyj?NZ{x_FJGMwsuACGFp5_JKdM`wJ#Rw<>(jA8Jb<3mt!JA5#OJj))si zQw1OMh0a^fTsqzNZ1H!O*LhXaW9b%}If~K=7w^TBN$4Z9k50#&1TpF*9b&&*218r5z% z?I(Yat(vv3FXg8X-yKt%>XT~bPdA6xho-KJ?!#m1mXd9&cf{7?$@TNpwok#$x4suJ z)gxP1D&1ehvIL8!r`!ma$GJKN=Lg29G4#*Xxf!s0sCHt0R-` ziHXkprj#{hVb}Yiqvy^4Ge0FF=Y-Ay1OV{=#o7JOtWb8p2yT5- zV;U0&eQV?YEm(?Ev>glRkwb1is1~%Ybof5=u4TZ|)(SSPOYvlix7I4N!Amn(K*@#_ zzQ0KQlq=hYbujk2Wou*B zB`6h2)CkGRjO+ptawz}uk`PZaa$c;5XT{ydNG(Ni29b$Fx>Oll;?=ueK@TWN`Hv{?v4h7c@kfoYdlb+y%Md;uff!X(%Wr&}w0?7t>BfH}%Al3O zA)G(TU3`=~I;>yi_mc}{t8{8&Qk1a0K5V)>h_~}NnC6iqI)UQGG{cF=6C#5j=@&Mx z<5Vs(rL!T;iUGdZRXD(&A@k`M-M6Dsx+xP&C`Oea;&NMU7%3Z}u|jgf#ZDOq5-G_B!FP^H;PD-FPVkEa)ivA^z zWiz3ZWJF@<0`1_Oc$A7{cTH)hb(+187l||QZC;Lwq1C+AHZ~!y%scUL!I&1sf7;fn zS4#KR9HAbzq7ae7M{;xpqJ3~3&?>l!7-_1--3_pHf=3S;SEgxc=tZ#ti|T3#i0VXnY@oai`sFgLqe68_=eD_zR zulV;o?|+x)I-=j~PrvfKoCE-X<$om4|L^_o=xlAR@8JGl5?x)}aia~zd!@FoVX=nW zFg~TaD0@g@G$nh>2vHM>YVDZ}I>+J2bMj)^o}dw4biD8@~-(YdxmLchdl9ToL7hP0}3^ zSak*WVMODEElH_^3!*PDJhu?UA)y(9?lZ0v$%#>b51H+%8vEn9Dz6jLmO zgsc0)jgTa~em5TJQ@lvPfrgMJt8vzAqVU%dtw?~P)tK_5`tz#zu|_aBsi##ZiO`UE za`|dYw?3hEonB^`83eS|cBNBDZDmR77$)Nv56 z)9Adwgv?c{{mJ31*%?WkRoSS_gx&=G41GhR&#nN`xc<~U6c5*YR z!>LK%HxNV>9A)(OHQDEEA&MTSXx<%RBOgt z15BsYR&!bpunc1h_pMBqO0hT`aB{N3R}M}|)u`)=7=`{m_(2*lm z`NBjxaN?MJ1krAmrt~Xgn}sSyIDyq)mh6j32Q4U&Fuxa79qLwYc&^WI%SjE-o1&y@ z!s_e|lj@ki(z$+oN+QZ6%Xfz>Q_9hujE@wxqL#R>xQCH97Nc!uH6mJC(8T`Vom)bh zNj?uvZ93TeKZj{%tavC^kQ4C%j~3J6((JJ6HmK+9IS~!D=UkeP8E%-*q6-k8r_IN`!ezgoJG{8V%;P@P%>35y!BL%K7$Bf6^)R-0;VPMgk) zJ-k>&%bt;haqVt^WfsfzKNGlM`G46a>P01&uzs=82;H$W%=JL%m@${k~E+I~%C@YYZtKBNO3yGwc19n(8Q+qi-&NYXd zV#v7EJ7Zm9(=hlA3ly>C9J+)vaf>X)v2{wBT9K<|oVg(@A}|=t@FFRSRdz(t&sf_8 zFtYpmgK)Fae%W0;i=~M&FKmdJB!YIb^>%w~m;8x_y!7OP_J_+c z@Sid;y8f!fl|i~P{a%9?*MGi|p73GFOfdL4WlC|K+*N2J1Beq!Y0$gpJC@M( zpsrv;Xn?soe4RB`@_<{mVQ6>w8`V%^tj;a^J>I2x0ck`j@Z<)d&7v59LVjJ^tw z3m!9%06YqFT}ghWed|Fc(jo5HF0`VpM?(nmjD)yet6F{n zqe7k@$Xl_c2k|r*OnHJ?bsXfJeITZV`;I*{QvTF*7_ek&(z`O@DXN1}uk%H*cgxj7 zLe|^39|!QD@ak#4ed^50@}AeDFlX8gZVUMYA(H>Ot=Y=&*Z`U>|r?mGfO!(ZEmk8|74_#1g%=^CZWuglsm+AU;=(QGY2BRM#!3?W9rIvAZU1( zC0}#gR!ftx*g=Ck>>vK0;WW9X!&4}uwZd2cg*O92JQukS?w@GBdW$4+oE^IMYVeIRy zpf=4y-p|TKD+ZlH=FRH$L9Tgvn3;j%(RPji%4}U54^r_RsmM-o@BrUryslc_5W#Qw zb!Um81;p)GhHWLaj4L62Bj{n8lFd1T&z9ME{a^+UYav} zwZ6XXe`^t^Cr!GXo%yR;tjC*kKsO@E>UJR7TE7*B!ora}Bh8E?n}h#HNNq)Iom7GV z0N7yva2LmTWku?VeefUPU_XIdm z0H(e>@gHN?_nnuChG>`)kii)%(+Q~nnu~4E>y>zFT3siPtKHgf7!NeBHo>ph71vhl z!#4CE_wA83B&W;364!M;f8uz9jPNkKDBBKLhY^j@T85n2lkLw-(ygbHFYsx~zO?$mu0u_V;tg+xgQQNQXQ4+?x5pCgy^a<9Y(AuvZTO~gcz^Uhg*k)R4%cpx0yf;mUFxF2R>-D+hT!n18pT!uhv4D zNugu#{?O<-1~dW&_uD9%SYg5UVpa~H@gyh%B>^8Vn$pK3hWnc0QO!o1>IPB8G&#b)V2sAA9kb0WzW7hy?LhK5UKb-p_lwVs0Nhtd=c#h z3!OtIjHeV~nO+}jVBEl}FMrOv`T1*OJ5+OPYOfnhSu*ngfUy@1F|`-lY4# z|HfNSxF6&QQ~`>oCQ6KiGN&*z|IUHjz^o7X}%HqWOzwKS~;$B>tA61N;H23*7{s4x%B4hUzA zkwHT1UtC280m@@}ebqRr=KTf9>?34(#2IjBlv*-uKUe)!n{gy!U?g48)3vwh>>enA zc~h|hepjS%4Z-wIT7-0S9V~C}(%FQKWyR-VfF2YXbp&rq`{0Q()>2lvBGEek(>#<# z-519M0P7`#eopl%Uvrug5Fchn+d5)p-fxW4JAryJlmMCJep#|nZc-14%M=NaJdU_J z);>K4(j+u8bQ&E;PV&cy5^*_3#MrZlN)@XIJW`?C6yv7~(aX|W96Ogp*;IXql-G!n z*-Yx2yV1zgJ-m3*_g+Yvhz=P^lN4E3uM8wud@2FK3fSKEHa#Ro>sw8`${|6Di+`6M zNcAu*2)Rbi1}8#xn0Vq1Ky4LTV<}GDZF$12Z2|^~$2Quv_RoVb)5Q!Se4Vw46}{yV zq)94vSM(Z-(gnc%`tg+0^vv<({l_4@Jg~f!3u<%zA&14_*&IT;gEz`(mN8bg_FmP+ zD%y;u8rX@^9UqC6FbrCzwS~1_@;W5^JVWMC3O`E4NHa*D0+hNO7IfnBxDAftLZc_T z#D3-{55Uo;Y~h%d=nF}zS50<>P?OZT9pk9 z9kgtq;9h2qXYE;&K2u}gWKQCnWAn|m^{Ja}Lp%8c#<45K|08Fbh2>tdpT@*_RJGrd zv}yOgsB)jEcX6uw--~E^Q=HXUPym2|-wd7q*^KM|7SYa*|L4`OrfIt_hU{~qn&n(< zO$fqCZ0?#2SP8t4iO2_WWszIa&*O)6P;W8<-*oP~<7g7)4`7aIjZ29_(VxTqGRYcR z{BogRK3f}yzS7o<)>&2^v_~tTLVd=)&HMJiw zaN-|z{8%iAVrrs{{c)|*O)6Rf_!u5qmpUeAhwHU`G-@8?y0y1GsiqnZgK|`=uM|V~ zl0WZfVU73QT=h8~L^$gOY0k43I#6c)=YCtQ#x@zhE+0CJ<}qpFo*|-iI@ahUx{+rY zepmwZu4=r=3~wq>dOX>!!&nxwg#<}Hj@}& zK*&rQAwgYKih@4!U$FrP{5e!L9VINbB7WLf@lkM3bmKXsc{+*)8ULc25fN0DrSt;M zgEx$BNc}>{=em+|`N{Zzh(TLaGexqK#9K{Pii>=^-B{q%aBs!KxKLU9toQ!k$SO5M zK!e6Enq~Q(nldi5iePbtP7z2r5|?p>3e7SG$ojeY(otuS$`Ro-3g~uj8RVm4vf~TyNLwPU&?R`GaC+f@M7@@ ztn*aHn6?!bk9y~~#-v6~q*(xhMG!E44fYOY1zz{y(hY+vvDNJjBg%f+UN~|6WLa<3 z%MdqRiqn!3thzA5c_XoZ%EsE<8Q?d072#WOk|gL1Xi}cwwl?ut?8yV%{5EXl3x@Ch z-geXgjcGJ!HBFqAc9U)VGTH7pA_JN5wjccc^8O0z`=HPP5r$&A!Fa3xFJ6`Ac9$o` zYCxvHF2(^Z``IOGBf!B+D6av`2nKiH91%5rr1LlR2(034rpw;ias!!6C$dKt z?@^MFA(knJw3XQ7{=MoQQjjNH&4|ycDLzFriQ8j0uDxS_rcFVz;yDeO)z3i!2L|a( zY94WDfGz=oB(X78q9jA=j@j7`I#4gBXtA+)Uk=FJ##c568zLYWfj+;8`%d)ZX99B$ zq{P)8+`f+h2JKE-;H^qDl_@4uioOsev{1H-yu#s(@<*3FGtCTRiZRS-j+Z9P8aslE z;b!bKJ#8nifFcDegluyujz_4Kv+s*R_#dqR*En{bf;Zn`k#)+I4d(;`2`RRmAOhh_ zBvTFU1b8N=RimfXzN^6NAqVwijYy@MN7+3jqN?t{PsMzLbj!IAma-~x1&i&(!>d0_ zhw$YF0)$M6hr3iEAyi9;l*45Mk$@hKOeF(TN!at^?f_;3CzKfjE(7~X)b)nnAvcrL zTk>rUJ&Jhvt-}1UK+H@-PJ<*XWRZQ`jx4UO18^;U7}aW45D*rVq`#0sri6e@BsM;= z;_y(0A6kr*BY#Dk^b?80(OK18!>sBKRch`-`1rmgvt3k%!UQGYJS!%5vSj<$n;iZ^ zxjppWlfJ~prOaHft~Mx!PTgQjtj=r-alY`hepi;`i9#Sv3P9fpgy!ri42Mh7FR?tP z)L0;uTP<__U*jNPxs5A_tM5;!chSSR#H+5Bw*z1;O_(!HQFTXK$G03!%|gFacvNg^ zxtIg>UHl2rA0%Sx@Rkk)>X3v!8T-$=Hz*rp0iA8YEZ2{(>!tU&E(QX)A_j+sC(1r+ z?q`;~JfDGJzk^R_vfB%pU9t`&HI`9XCmK2QFU2kBOQC$4RG`RCf9o+-KHWM#MHz|4 zd%XXCyxm4jA%FiKZQS600Ox?f==|(Ef1TZ%g^+_6-gxUl*?cl;XX< zRFU09C3uwbKoUbJeQ~+6kq3?34X?-8AfDdl{Lh-Um82e)A#ibNug89smUREF6?w<{e03_=c zAyv14?e74e?^w8-pXa?Tx-!?={8rvx_fg1u>79Wq&Xv3Da6@h8wYTwM><^BHCw7}9 zX!6Ps(wHm54cnzKtFP&e_YGaKZt6uJ0ooRgzq7jPtC{+jD{;LVzvi0W_v!GHrMao$ zMV0D#Gm&u2M^Ks?DlT`+l+gw)r>0mk&(MMxdOY49UhyY&42pJmF0;Ef< zqN)&ctk(-YkN3p0C4T5my~LCo+B;d(vL%5h*58Z~rKC?9Cp2bDfT&4!_T6MizVa^# z#c$^YD}}L)DlfK%?Atl|+>P8ko0qDQl2GWy#Zgo>h|4;M6k5#AnaI$s5#dDQ5HDbR zgB6wA(Cg+IIl;hKF|5sepOXh;Q}j(^l|bmZp#^R@IeS#^FN{9$DdUS-!!4ZtCqMNt zh`-JWVd?v#H{)9Wf{XG=b~%h5t#}le5$bEf+L`vA5JeOhpSSy%6>}9)N z477FF1uRssH(C!P1{!gL+Jc?@6SEqt$A`vCWP{<*n_UG@b7}MRt$1?*OqXVQqoIYT z#aqq>DvsJ*j$^jxYnq(k7U;F*Jv2wl82fSh8>0YWQ7rL{(?pP^ByNORonfQ?$oMSM zFk8FCE1}w&O+gWmSY)UUot?&rq0HFa8^=3)6ygL*gUnKtr#(AxcOuDwP7`;A+>Db2_K0QuX-uB@ zF})R>y4|#TG1RwlbbEhgY}fnJKee&)N23>`xnB`Ub0mz_o)5=C0=ixq{@oz&)nT90 zhIqRejcHu)gvT)`!S>7@@`i=EH(<}-+?u?nCX+w$NSP*XTf32uk)Oe4Cy!V228y*) znil_Mq5E6e3lHaU!Qj43_Ga7Z`et5P^de71p;ShMf3@10c^T9E!@_z28pJuH8%9G& z!VNh_W8e>9V*Jej--WeRY+tB506V@}BZm+d{S}6bv~DYzTX+7Rp!Na}b64?J8k^&? z+U_x(v>8HINe}&5GAYIi3)lhgZIE-cuP$Cu_22s#s4K3`oO2{84KEGQ@uFo|I4o-3 z_C4uZ4*q(CKZN&!HsFEO&{G=VJT~F?CDU?;IQ6Nfv>3|?c<3NaXp{BEGHL=zbIPD7 zh1@F;f5uqw?=Apo2c(eW0Su1)$w zQ(MGGgw3E0p`xsTiJ%{?I=&TJW#+VUr^;R_Y`9ov=C>fd)D0-Sz;;e!j?%@Z>a^(&H?j+%q?<72r7P%$!NU?^!) zeB{U&9NU`g!o;)ALgm8P00?UqAcVytG5A>c{RlJsJ~13IGFXnG$@)H#L(j}A{n!dHr#Js*7mr>DOu|-xJxJ66QN+(K|A1|J zftDhX?-Ux{-iF(SOPosX5Nu}24f16c((z>+wl)Pa`JLw=E)gwG)&p5RNSWoz%u(1w zW@NBS9O5BBWfkWpf^Iu7_E!b>)kt2M`bn$)OQE{VF06fo2Mcsr^PzoCt(m^+pEgna z<7uvba+;3I%SnnJ!lAf=JjI0u2lI01C)l|U-vXbKS5OYW|e)HGylXc z7j3ITycj>@hus=aEK{uVEL)KkU~sRN{}ZI9iIfrAa)%=X;>q6JnAxn1F5}xu`&vX7&F~!|z0z#8 zUpTW$k9CZ%Rj<#4-Kp*v3v0<-7oN`AlgC`i%o0cstefP@x($?*7SU+8&s z#%}eEkuX-gpJKp}MFiYI4+K_{rG|T(rn+tQ> z=f4Y!!V9VZ%c-LUdE;9<_eH2Q#O$PLdr$8=@XsU|-?DqN+q^Aqv!?gYHitzmIg3HM z!p?qYU8F7MXNNO0Mr_!SJgB!?LQx$6(XBsIl4gnj?z ztMj1wl>6puRSu5cwf5WtgAHp3CUZYDa|CSEnK(9_u3EvnWmYO}SLVHNHGm`qi4E!n zyy#ocHgukI_Q9dbC&_SQAEDPbMDKRd0T|gSG01SN{X@)L)R}?X&K*B%Z!~Ft!%dAF zN4C(+wBHLm_jd962b-cLjT>j`Ug}vdpE`dzZ4V9Vr#)y?doeuI^`zBD1)VxyKAsl z)qE~~=hU;VvBWnxqI{p1k5XFS#J4w(R&bzz)2qNTcT{&z!%6jrhEND_Go0;d&J*<+ zCPrA36>c}f?Ga#Z4Byc$SF5DubgDehGNZR2J3mm@Bw9(_QN?;GB%?aJ4)#!AGOMhz zMl?9u%7Vag*B$Q>W@mYlewV2)`w6AD8>wHN20hO*G-|P{y+&_2B9fAC4uu+=9aMoT z=*RS(cfW~+9T%mVwFmr@{%?dr{1H{)o#?IBqrGM^8{o5O-xiT)F89nLj87?r*s^sS zq)SwlI6jx0D_`ml3tn9VNRBm{Zwy2jOY8D7=Nsbq5VM^A0t(|O;rH!5d(`?;1=|kH z8^nWmj8vURu0C=??ejHj@4vb5cb&;=&-Sd71WU?Rtyp3ydQDhl=mRRKl{xg|P3an~Cxi&a(_v1SDbg8BxZui2{W11kS9>bH6~&hP=&Fme0;x`%G^0jrJ7RSoQ4C|$BKI9dPsLf}b}fo+zy}tE z+<6_SBNm}idbU!rJSWNsz5){E!xt&r$9Ks2lVqB6(&!oUCnu{ru5qMSiFVE>vY(Hm z2V7Q+f!TW=wn07g@eNc;bKQ0fW*6}qrJE^BH)g=BLCNFM8o^40)Mk44f-!`LZUu2F zR18E_yj-M+YDiFPPRzn|-QUp7>`yWr7g=)|+@bO3w6Q70%#r7{p1`yFyTC0v2Ez(7 zIvEM)tVMV4SCmW4!{d79vh_bV5ABeWY!B<5_TD(4Tbw3_mg$|@+cjiLRA{J7DbtaC zNi_+K4=vJq=B$i61@7EO*o`wN;!$+5&I|2>!4%D^_T ztv@O(i-fXLnbK}nxYS86ESULJGHU$krZHxBaBF1L`n_`h>tI*QsQKrsTJ#*l_OC~! zm53tiBscQz9i7Bkbc6qww;Q;-bh;j+L*<-EA}$jJ1J+to7x0$+ge&Uaw1DQdr*mr& z9L;L>gb~PNb5U~VrHpX{pojW8l%+*GVo&~ejTh}34DDatGo3Xb)$PkjSrL z?hZexc^0Ms-kSeUfIweJ)GY+!=%9=o2dkaKKQ5FRDv<5T^PYq`1RS*m9@a30Lpl(2 ziT*)G0rGKdIDYY`PXGz|XC-`ckn>5%d}1ddXp$?(K=Lc&8UR6a&c`Tt3C2-DIR$Q3 zzl(WTWW6?AdIF-T3xiQX2b{o9ZU^K+ap73(VE>7AB2h%q%Xp&VhfaH00vq|f$ST-P zP9BT+k#>G0L3AoV>8OA#1UC6Yx4CeS6Hq$Hhhl&Zas@S!T|)nZepK@;-*wW_4Mhgq zUu2$fcH6juXjc0;z#*tYm_ads5COc32mnaT@ z5g)zcG$Ts-f?)6Q%svbXB$lvHg$?iNf!{M-86}IHg8i8)a8NG%6Tjo{m+d zqYER_?c)+=wthe$q`V21H^0l@={cvgH`KTRQX_X*Ru=4W0u~EWrs|IsX`hS2nDQQG z#vuH_O92spTj3^$I3zvwo<%I?J2Xn{-|@jf6@gS1p)X(6f3=HZ@`k5T`5teMr>RMKiGw;Tf3P9!yg_y2Xyrw0b|{wh&ncbA?Mg8*H0Zu3K2UlY9 znkmYe?Rqi%xpBj-#!~Ftgl@yo45~CV6SZ)Mj;2vL`JD4Pha^LgEv(f-O|?{f<>lHWrA_ zm_D3OjMKqMAt>^u_+WQ&O<#(AZ4FP7dKoj!0So0MPv;3?=^D(Q;!BN^5c|Cu(@=~P zTxKI@u^)iqi9b3Uf1phuimh$#LG2>6{A>wZw}8~TuEKuTVNou0vYr3IONgg7OebP4 z2Wtvjm*S}L9BRqcMmW_nqlC+0H-ML50Wh*6`pl*jV+jdRPl^`RN~N+wgd- znx9;P=hTzJD4&j}q-|&IsE^M6d4}FhUMIAPRjLUSIKum#XIQI=tA{;9KWo5A)+~rP z2p!#HN{pDfs=={Hy@u%1s*$V=)Sa06p)*wy_UDV)RmJIJxYeOrX%$ZwyB!_2_%u)SZf;=%lm7S*Z{J?~ zNxs_Cm-IXYs!w`jwqfYkDd`k1*lY!ycEl_*QQfsl2F`tF3jPNdaV3u>1=nbsb*p@p zs1g_YpxHG$_=trvpbv?kM_n*J^9`3Ds%^UPQB8hlf+hNl_X&h4`rd*wQYaDnI5*s| zSv8~x<5_(@VMe(Q1oe_U#jM(l*-S$TPGB^B;|)s!&hx_v{SUTKC667ohbL%zn!kJN z-K8HMbHkJ)ebswG{pw@E4u?(Y@}J%f{q&POmGR#6x8-tU#Y!6@m30Sp<|?B$4R`xse(;R?U}sW>+`vPLURi zWWKlb6t}5=hd*EVSI_Ggy7~R0sML_!&P}iy8-+?@GA^As7Q}Bye2bv1Wjk+PYZ!zK z?PfTmFWK(8zNjp8%}*!w*D z2MVs8s}0-G3;>HHse?-L5X|o05ZkENuak_vw}(+g@p+6rkqoj`{rUnHa{Ya~zax@S zkuAXP_9H3FrUple`zl7{=XeO!Ha&yAJ|d>gYas=_^zY02J#s-h?FzKpJDr4 zT(3n5?Hcr(2U;JIO~y_BMC5|tt?F&xbMNe3kiCiP;ivq1DasiB1Yv6nu;uRt3Gpue zo@*Y1US+JenS$kSG0eYo{MAb1J%KDiiIm||h02RGkvb#WW`l1lN?{4h)}|5?dt7H~ zs6gZ;Y9pw&`x#JpZ$q~xMDSWj0C z-iN1km`h6GINsZGGF`W(>kX^MeumE0$$tJH(o6%DIWXt1wZe+ye`>Fgb7z(ib~7}# zb27KJ`LAVjO;6baF~px-*?G$%&A6~>%M%&uaZ32(5ND2W{ZC|dn$fqb z-GBBTHnO^GO|CC-llJ5RYo9WJuA7;nO&{m-qaX}E z>TR7_Vtmb-<#MNLZ-eqwpg<3?h3whPHHnj$N`OC1_IJd^*kZzCMW7x*cJy=I%`Rf> z{uIpTe5rcA9N(5&1M_a02Y3t#e&Eer@5I3Ra&Wx*&4kdd?`(0-1?=OO6gf7teLh`Q ziV}u@ug(WzjWQt`BJX3EYx00M#f=W61=(*TKx=q@RDZ!CV#5q*4nbIJ66(kimIMu< zMxY_Uj~rTFx(TS6O^ml$ueBt~HY|aV)L6qD)ft_j8;efq_W2GR@FUYTCQ4k$R}x7# z`0pY`&7&IQrckabq~HYcym6r0exZk<#v#DQn2&N1@Y)o*U^PFt%=@C0e+s1qny{=g zrjOHsD)loi&sx0aV#O3jotVMj(ip9&^0r-kLIQ+hy5Ps*fpZn?(%p^)kYGo%WkL;H z8H&n)@<<*o3|>73hSkM@#y(*hXwr1)&7mT>c#&vCFsG@ z_W?REk0Ovh)MFR_FjU+GmA5pvIEFpWoV;%JMiD`Wbl3DEGuF!RN-rO?FjPYptNyXFKq^_$6N8Ca57b^$bkEhlGNy$ z5+#I9Ot3nwsTXcRZ(84K(kw*GX8e@tOW9jT^M*vZH&w751>kU_BEqs6L#j>3Vf864 zk>S z{?gp!27Jf;l=<{|2;#_|_CLn-o5N7vxuv#E%LepOw2)t|p%g1Z!rLg0{Srvux>UwS z6oSH2X^Yoo{xUlMzKvQ}2G5^F*jhJrVW&O_d;@cJVABvOhKMGT=P#?U62J+Um&^X|4w**7k0T$E0#-MD4=;(tX<>^_x>(x%Yzki@wi(XE0(br4WQ&3R+x?TNGNrW8BjQF+j2R zC$AmDk)=~UV^fT5z2T;Ai-Oouy`^fVxS);fmGb;0&mP=N_~RJzJz}2dQIbLBK~tl0 z!nsBR5r)4WF!#yx>4~+vZ*uX$!`ovX2UfoJ!bRRwxj)4kc29)_tdp@&9VcB&L2B`f zdA594c({f50)Sb68u_&R*;29R?6j{(?^LdE9LLfCI24zn6mD_7!Vf{*%+snw5x|~@ zt!^ebu#hjBjWH*y{}k%Z>FT3+Gd4Ma?I3P+?;hP#$mEflyjlh~H$#`9v1Jct-kPJk&p0n#QBg#oLdCipTI8wi3D-{<4M? zvh{U+U4}?y1m0ZXj;Vp`1FpXzYym=fEtKDhT8Q+f;3a*`ud@#-!hKaUjAV(y-pS>s z36$Lc8b|0~L`DzrFp2SJDCduE8l7tCH2If#8a3XCvlzYJTnCK&!qXI~o!W0;Ex_62 z81sFLHNsETjts|2RGkFXc<+GS>B1_#O+_KWYAe|qCv4|dc~-RhZ1C`n4-^mx=zeyz zBoIOZu_Ow49pG3*Nn!X|WfrEk_`A7F_Vm?J*&0wTNM>lS@+~lN`ayBf#H$=5b)JNc zAm{zGbimjns+Ddq5d7MdYW|w+q`?r5t3WXrxfB~VDoA2IxO}~_^bCL?ik67Mh)N2>s8z13h?SL`^_nSY7{%vwLjMbZfUo9ox2TR8p~R z+qP}nwrwYsq+;7n#b(9Me#Us`UVE+iX|DYT^65Ex&VJpkx9gyC$6*^Nz^{r1mYSMd z!fwUXh8`G`8!5IYYOKD&7^=A#7LoS*3`$dtXw!-z907_0*0(y@t&ze`iK3uf2IB9J zn}t)LHgX9MI;FHJEhBgxqw^!SgSE)=g`6x{`CkSwz67RKqtTxzH#3c+J|Il0Dyt`p zMpkQdaJIJUcS$iigLc-qEy*9svBtd+L=Pg?U>TU7hEx6Z&!B!zILlM{**t9NK7+xp zCeMoOI`zqu>|26tyjSk#Yq^(b8ooWBVucO7V-DB9D@9xE-0Rmfo@*(@(F2rP>G5D& z=8rPpE$6D2Mo&^DeA({>T$jZ0O9x)y9A)Ezu7Sz2^#&m%*|i$l>}D=vLxUUh^w6w> z+MsR6m?8`QhVn-1_XZKf_ViV^NP1ds9Rw`gJ75{#tQ-NERs+wlTxAu?N%%ye&O;eV zHTzj$PA$@Zl6Szfqr)qvjjtlUSaRsY|5+4j5_o|jx+mX3I<&66A%dH#^3;=?c&Wj* z2g}_=_0=cwH7z)9nCw|N$k|MGST+y{O&HKzBxxJMo~( zHJYwgV3BFVT&z*SimU=N*@`kvN``e&DLh$j`A@WoHwW3qj65NvpZ%!|HK^)i`v%<> zK~l;!gv#j)#|;4%xnsv15!nGr_p5P`3-gEcMjMGpc|bLF7IhzbzfOt>6_IDVPL#UO z7Py_ePB0mLXUTojj{CJ4-p=rqp_JuP2fyA4?tNn376DzOz;L)fKzfPGdk5e{t0W04NS}F~H144c9}@>Z zA_tQLFAx}_5vkP(IPn@e%$he=!5&ISd#DY}`1eeKWnaH~`rh9`Zf;<^*Pp@fOm6Xi zvls~U_Oji6t@LUA!O_qpwyyHf8oH=zqMnTN>{a}Yw6Xs3-y-V>Y%fH(0F1c-F!n!N z)BX>}qySwV|5K2mlSO#gwQ}Vuj9%{&;BBsM6DN0V-}EdaZc1JXfEqqov7i6e@Bu)Ewtw1Vmnbm<1@$yN zCe7W&8lo57<>SH$!TOCsGXhe)IIT{a3yV%CDmzsWl=Tfxqz6KL_vI#pwZB z*G1Q*!{4C<9GXUGp#z)n`;FG2^W{2G#LD6P6jr+&l!PhjHeiXKFhpxCnpBWPv7Ib= zqaRQbkPL>ka0<+fvO>==Lc9k|JRoG015`+~6MEoiR~V4!x7}n*nQu}JE2Xo2K4CaM z!wmCmB$J*c}!pNeAaMoDAQ_|BoyMKYCt&!C+3P5D>4|6i#@VEf}sb23Dvn)rN7OEx?MO%{)Y2LqcKYu6!F zI~Mr=kJ!~YPmhW1S9?|BfB`lUzfRPT#xb0o!tQSPf zqloeR^0Fqz>YHn6ntx59NwW5ZnH=%qXIe_XTlErvfWpT765o7k& z5VXQ=8sMqiV2?hn$14i?PBNs8BqL+$qD4E}?jJ@Z<{g~)a zDrd%%yO_b7he|y_WE16@n^j9|2|me4@q!73|>2N{{aJ49B^DWyWgL?a45$$pT-+`1(C38O5KdJ8nqv}OIxuM6c!W)epO9- zN-G9?nr=jQ+|}YYz0zZp`2%cG^|ff5rheQ~H7_S`_4dBntUX#wwmtIvKB~jUAF3IK zb3HY4+G>m}*$4(<(Wg8J6MZwahIMYpxlC}@RYMC#W)#v5B3K+p_1XZHmg8>B28a2o z9tMvu#)gsLR7D_19-+qldN-sBS$rgQ@ywSruP(B8>dBf_lGiL@bJ?{#TV}s$k|z ztkym(&~>-IB)l&o=>m)0Cj~0G)2#ky4fPq&%c?pED7w#v(Jq-@q4@%0O7GPEm1_l! zi{Oj9319ZDN0P?X%;V!J;QFO9m7;xzeyU%AW<7~;-o%eBGZo!C^jzX>1==WDhoH-M zjmjwOC`&0jF1Sk^vN+mQ+0j^Kc$fSz5+OjW;|zTeYGW~IlGG$bePKcM37GFge}ssL zs*?;5&K8wwvxTWfF3GIhO6`%8{)A5BG=xH76=Pghk^mW9a>uKuBQ3#s@(kP3C>-WO z9I&oAJ{q$tm~ArXunyH7b6-?^cEHUCsL*_?znV6KAX;i2;1$f=5{XS>k zSH|7nBYy;6L;8u!bNC-{#g$S4z~wDr7jMXy1vK+aMCOa1HkT~E4iEZ|x2Q5zh&GpG z!w{i5B7CY7pCT#T{_&R15XsnkegkMM|Gr)hm9&{`IRpc9>m8ZLd4xcgva=yICY}%^ z`bd>G$`{jhqw1%)1U?i*8gtLE8@pO?zEonaSy z#QJnp?7wH&kwNM$HO0~6Go^T2E$fRfV4tPSRV!+%Q`LP>4l^gJ>_w)~meGe)e<|LB z5v~8kVV7u{4{CLF*0ZN0h0+NGV~B_ z4?^Hrx`$*;w$>g;vDr#(QW86#?Lf95S9{@^wFDQ!>iT2tr&kU#k4OHRfAUf8Z_Udb zw1jL~87_-Bu!(mg=M7fRk$VvQ zJ>`P7-%N2dDDbQJ>VgHdi_s;x!SjE`C6*CIQ*ijvdJyR1@{PfKB?%=B zL`$3H+>1G5{Qyel(|VtE2AaJRy4l_Bb#|ZiAen9+Z~f5BR)$oTBZUZ^=f&ZSj0!et z`iyAka`&Lz4nuyoxHu;HJr&Nqj*w*>q#pXsZXZ`~vE&H}PMGHRJc=|KDtbiJF<{Sx zq&M0#yHrq4Ktbop(eT1#_(iv;IqJ&2x)@K0K|paiEHo6nkPh-tD!K#9)ST|YCZO!s zjeq;st#5P__~!g7fJd;q7!wTcG~OMI?+>!ZBsYk2wV9-;Jgm27DSPP#vpr{(x-hP` zKk)w>kI%c?*Cbk767jQ>O=b6qqYRva_-;^uLS zj@)kpjlcZ^bi#z?z+;8*IA%Oog;pdYUWObgk|TLJn@HVGI)2Vb!~`2tMv%za=)b60 zUp$11fx3<#;_M(VS@Z?u1cTuHVWUTxz$OjyW&L|YMg|AFU)+!8JB+v%&(m@*fCV{! z+V@rc+u!?Rks*lino1CVzi<%I;@h5bls^~GSK!1K6aShJ&r&cbDxjsVe6-gsGDGC@ z>mys-sB);XxZXB5F0zkm$atZ}C&8MS5e6JubTP>X2QoZ%5D~>#V9kl10p0I z3K;`s5eo|Zl>#r`88#LRn;o#C;ZCxxvtWp=wg)fm60i84=&t;h@EI#sG%>XONHEPqEQn`3aj{V%tyrd~Nm%)f?G;K%T(IcywsPx*tGclDe4t z;lO6?OnG8>3nB<<$t~J#ue`Cip_Q5H26$M!bKItyWWLjZT@-og4%{U%uyfXmODmQr zC0K#GNluB-fHTjP9~{ zB90DXgLqcXoaqVjinG`d)bgoSu{lgWS~5!cKxnau%*%n*sP%$4c_VAYcQBk^^B%SVmPnB()SE zK674`e$rAzgh|MSVeK;`uv#Lq?csN%Btn|_kk_|YEh~}OM@VR_;rfv0RWR1VdKOA$ zwN0{g(wbB4^_rMbA^w9e(gcA=r;j0n&s!My^y7utIgnzS+lV(=Sja#eiVrt-&Z)E1 zS?iE_g(xQEB?J)LR+LPxxHPGaz8!~ODd;7UO_#76xsW1$pB%q$-D!|+CPZYyxltVi zeps>_(#f7R1IA%aHe-E`0p&zvI0BU&>X9GvFXA9_X=HH42_`PE;6(bOtShV1l}Nn4 zxg&C4C^?F}Z+Jry;| zK@Gj`_EnuA#8u)rnz(R=`g~3LJhP|ol0tA2Fd#OHo3zaeEmOCvA$D zTJ_lY5~RMVY~^soIDE|qsIM=r90c&c_PU~fjJJ|(C+nPp99Sl;AoC!r!(Kg0t8m3> zZ&P5$WR~IVFkm4e4pOQlCK+c*Q|*2WDRAp6#8QqUGSA&UHvqD5*|-vGqNPEhGDp(SXh7K=xS=#PPH`Qz{Te7DriE>$`X7buP7v4$2#u@ zdIkqwWh@<@xC&1DyHzIWJnl}`zqXKdNqSSH5>{D9T4SP0dXwk7B^1Y|O8XcQ{8^ zyE`5biKomk_#~Sa1KaJKRorMuW8U8?D`2*`T)BQI!sXM2;SBnc(#83EsXct+orMKvWs5$w5}1R<_sU=Pj^EAv zp>vbNNXX`u1t<*l`?VhDKMFg8m&yHK3PbJqM`4i@|0+yOg$LU!Fz%LFVN(w&v^6Q6 z#KUZtH!9*IaW~**s;d>^-?>-#4;5L3-xMl~HK^&AB;;|Ox#fNvCcYhF8Syhj5f}ea zSPU4x+{J$sh7k1jufixYqPYHFg+2ePFxr06`QbUr)qKL0>)h_^__{F4 zb*9WBpLc$G;v_>X!>Ni-#`6?#!lH(Suf8Q30y}K1yv>wSED_Hf9p{Di{Bd>NUO2kh z?fT_&p@vVh!C&{%c+&a1dYuvv zGfy2)L`dXdAsz9d)O3fRsaf3vTu`~EH}v)k1H{iCcCX7DmUtFrXFeJV;^Dmu>gI-| z=14bWVnI{s&*HhBTAl1~w<+z^bK9krm2aT`Q(+a<)0vm>KtRHN|I?-hB~uqyCp&o~ zD^p{a|9pbfy>Q+HY{shV1u-?Ndx`-({5!T&lExo)!*Tz>(H@uO_as7T!cs1QQBrbo z|J(a5_Ph6wb&3F(GOq|Wno_V8BneKL*xHrLxN@RM*yHz%K{*jDqK zkEgL_@vE}K(WK{(xNp?wTGX-Sv!4aTL;v}XRBwNA63=egua_gFyaYIPSn$ zH?xp10i3Js?;rX*gr7~(yCwWtMao9S!9PzQd;bu8-aXyIb9G%52d}<|2Pqti5uE)= zvXJ_`=r=bKA5H66;UY+eA|;HM9CZlD3%u-$k05k~Tj~AnKgZ`W*x*_B9YOV%OAL#i z2PFNqc$|Eo~^Fvl_(#1F2=NG~C$7Pu9sTkF1De64+t0$%_SHzD($%oVfKsuvGMzzqIzB?vqU&r*wh9aUVpSE?b(y*ws9R( zgBuHQFR}~&A}pfrd%m}W9Xbv$P}VBJbbET^pwLzuOYRz1Amq741B<30t#=@Xlh+2q zhs#hkuDjTJ_jO%5XX()2j9<0$x)?T%9r&)l=j5ik<00EGKuD+6^7mPmHL^XfeTQC| z!cpZQbO4{BsTJjB56=^-lVw3Z{Jo5pr7*f?E@}*WP2Qwy5eA=Z4?BTqICs%-$mEwSBeD}8!Yr+JQ z%_&be_x!*NkF$Atk;B_*zN<^P{I?&X>jVh>q#P6*%uMracf!0W0`bTXt_WY$pPVHo zsFJ%FpHcKg$Xs(?R+(XeYl2BVeMweJkK!Sn$WOLO`FAY%rmV|w}>nk0U|9 zu(dq)?`Sb5d23_j-{|v2j<)Os`tJv>h})p=`*R?0hfXQN&8E~>;MIi97k;P~n%IiE zO8;d-K)zQ!Mn8MmqAqEx`^{5C_h|AR47Hsr1DR(X9VUURsN%p=){;@+1Igv1-)}xV zouL>)hG+8Zv0z)6XXI`n_m@lrYr5eldBR$=woz+fV#W{!7h#cwhzaT#AUQ8v!5wxq z8*?exbr=%QPK*X_bc=7qm*IVi$Hb4+IeBYLCiJGqWfC3xDxm>!bzeR1fpU=WO4=M6 zz-j>WW%@0l0OAOd%^1mQ5N{G#B+!P7GWKShL^+}(FP#*?hPH)nyw0=`5HGLGf=bdz zTmbZhL32pmE9~`il*HT@99W)cN3uM4*9$Yn021MBbp z7p2U$0ge7=r z0T5(vFLXyV`a^kMdqns$MMas=hm%xXzo2<{l>VU;O-?8tVk1@(M#$le&|_i1bBJdQ znLu_#4dv|G!eV6jGDK1|u?D0@WSL^Bf@qB}dNjE!|xf<07?H)w}eSNZaK^%stP!idT<@rwc2jnk%ogT9ho#uX+=lnvo)MTl7=5Ur!- zFsCs-fia8n7`^Prf2F8YNaR+iG|2}W%zKPo4tS)`Ij#|78rQ((NF*`3pKTlw1g-+e zFC2iA>iZlG{gmR$5bJi=qQqYPFgxaC8^*YD%LLV)TER?l;sbak6uSe;DHE0ZNXr9q z+$YP`*SX~3`{-82IZ-)12XcrdpEah^_V1Lq>%CLKPvd|+G zvB(UCY)G!i-woFJu>^lWZ+|DOVGg^EmS(GS!XLA#^e}^HL|5j3F@5`HH1&0 zFQZtEKZs|El@Ga%uF+;n><1-emYE7SSDbBWQOTEJ!^ZGb@MD-_w?QXQbw0g=8DoHC_kir~6~3wgE{j&L-GK8}4t#1RiovxA9e) zO)y4oFzvzxZYA#A8MbW?T|QGk@Pj&WULfT#EVq}b0~qg!nsJ-E^wqR=xS&4LHT%4u zKM&<|u=wvBY_s%85uHZ$9f;-I6fu3GFVr9 z8UP`*DT3M@QDGT~Z&n17VWZoSm2XzSOALZxIyA&~M(n_$id!%C#UTFq@uARz+e8Wu z<};M8B6sN99}#l~e{OEOWxC@#M|U|8w(cWJpvQU_jO7S|DiEiHrZwpWudb%+)GzFENXbt!Oj7B!2)J~hnN zZQRO--3+G0oM%Lm*-WrvfICI`%^}b{u4~hZzqz^ks2}k@^ORcl9F>?_^2#YKtz{eh ztMO2OudxAY%n|arT`<@uU-|M!cxUa5>(ypmEOF0-;ty4wt3#Dx9F+#MzfK|7KT|bB zo|^>XIi7`aVs1R+-@6y9Y6c9ldz|2gMvzp%cU=l%9j%%)Onl|b^w;%vn`1*ac0C}& zZ0&g;L+i)%l`liHG*oblumE4B!x`|R%5hs2hkaKfqkmt`I@AQwUIXT(qO4LarX~3JKVhjvTBYay>N=EL)3nGLSaW8-%{XS zs{9Hn7my%vQdS?kVRyyA`;{G2fJMhY7)BPIx&z-~-v$P%+-Zgo(M@7Gi;uD%ji)O# zw1b|8q*j=_na3UL?rN2stI?y_iu4|!L7QlPBp!Le4`Y5+ zXwy~=*;9yACzwu5;s~UvR9GgS=>d9)7?e3ypn6K4Y7hoOJlIA`sA`Zy?v}5)UdZ;W zT-KfH#emA{%~ty}lIp{-SDn~E6uY%_A@hi|;bk!_*@9wvadaWED79i~8G5Y1GCZ!@ z5Lg7;aI%zCm!W#5aI;+n%z#u#Q`wYb4G!&fRs=^Q)efSp3vqxc^q6Xb>Ask+oh-7r z$|0D_N|m%yE=^^A;fy8DrYov~=qn8@!4Xa>n}%l7mBC9REOS*#jUQX5WFecX(y4@0 zB+RA@sDiAf5V8gPN@l3+o(NjN8!#Z&!yLj{ zwo+80IB4=Ju+Gu0#K5@484cnMFTi&gmC6eX(avhA6q-hyoJY_Iu2$?>h9!DU8ibbf zw83kWVn&D3kk`^KS&FqxvWSaTfMa!P6-bFKapjn4P|4Gw-(oLVCCanyw)B+|w;W}o z(&$Kb9JAXux|CsT!RmlIi3ZgO%z%ucpP&UYdIq+iZ<=MNVkR8@Q^p05R`?r#Ma=Ll`5 z*+nQ1ljfrOe6(6Is>5Stkv)#L%c9EIa`dm;PG7mtb{F6O=jCzsGJRjyZLk%ZLnDi>!wu9EX8W8WEKT57+c2vY`=mc;>%!HYQ=Shfzm;UCNBa81W~t zDBBo)s$G{}>h#2)Ob`K3&LK=l@7%#{rfN1^bMi)cvH zkPx9AW;^iA(Eo@X@2~{qR%M%(LCNbZ;ST}1)r{1*jNTyrt>|Eh!}W1=KuCi8(J^F~ z5~~;okXzkoFQwy#^1Z9&ulq#0w_+G<4w;s&pCieoZz#Ot$^AOqD6RH2boFDwe_|50awBYU+#w6i;vjFQ zV|y-Y6oRt#vz(4Bu((h1-D6y&Qo9iuNUZCjlkm-k$b6AT`{S`%TKn1KMWzlK_}_$h znj41dq6h%l)%!k6kB~tXqz*uK6?eoWFvjulUd>VTxj|p?j~mVl!){6&T0uVgOp|Y7 z-`@}cxJB;+Ww@ECpQrco7B|>hq&r|};C_n(DOH98;lrPrTEgu01Wk=-hEwZhD%GVz zW$fp2^C&O44DBo~KNWGF3|x7o4v!wsN9t__d`OH9nd;~T*KO8q1~gEl0g+Yo3tOll z?D)qJ&+>B0Vf$!BSr#$wlgwvxh$Ns#$XZS$_;p6G| z1Y)xXCj!)CI0atd{g}381V4Gm6m%W`Ggvh1T^!CZTISX%Ou77#paLAU$%%46`O#8 z{4Z+Y$*j(o5;RvjrpQ7*c69L0;V7u>8t6p%(1>sX^UC?C!l z63-_fT_-XukmdadBD^;Xf~@QWI@4i^4&M*~wN7l`4Xc-;P9lq0!<|ueOE#c;qM4sQ zz2J{I0aamBY89}{>Jgx0zDx+c6YTXgHZpx%tCyjD!0(@nvNvtf5c2NsSED;H@+xF& z;hs3dQtKwa-SYHKIbvisfubJMrll7+Y1@wpz+2ad9%9 zggDMAGW*sziNjULc`3>?C5rP88$lGjWH4BnfX)J6fp;U(u`r|XIBMl4UD`Wj%2p{Y zJ2mN;O5!l^DB+kwIixb93J3Iq7p**|DCrplK#`!zuoiM~fW`bWMTnWv;IGavQe7%1 zc&_2OuEPH`-2eKtXm$GObwQ!iGAEKP)uI)fAWb~u~H zDxWRo+8p=mb_uDi_dEBwyWHa~t~x#OrL=?RPlNgwo=E+9_cCtg=6U?E>v?ijJ+51eOcNOa-7gXAyQE`$Jc?9CA^j0qsC6iC4Kg{$-+>2-g3=y& z5fXY2yKkY$Tv*uEyIA)5<~17ERvm~S@T&s83i@vQmEfl8|GG} zg)W($9ko8jCYOW$DMPlC#u2LSuQ7I1nz;;w>$JG@0Hmd|9CWX$rot>nb=3>d@S+~T zD!dK`b!mFHu7ZHLLIEw~UK+0ZiPBE$uCo{YQ{mLhPbu4qT8i198Z_|wHi6wqN3izJ zQDLiXGt#n`-N}5;@Tsla_jhg+YN}ZT0;L!~34PrbPTg!N%*)T|O=hU=|I$6K$a(xy z{_LD2FWYj9^DVtf#1H=#qmX^P8*wlleFBt#khI!+`|JJX^ZESre0uxnyy^Vx+Q)rv zeLf+j0LxbO8!RKddc)4n{ET5-|8ev_YI8%_{M&!(sV08CDfaE>Ti$el-*?H?XL8%m zd&$j_81=-|0f5h!z0|=;!~4N5(dW&d&6J z9!y3dLuXU=AOD%Jt7|)Ni2YlKSpis3wtyqE6&HPq<$&UjXJAvB%jb#V0yG?5vaTzb zA@}7ydo85{*OAv_iUXg%eC~77vlIA~{ndc^$<`SdKVQy`<$L{S`>F39`~ALXRD=J# z3$k{%8#wW>QpzQ%RENop^|mkDI#-{$O{+F&Su|lBJNkNEhfyRtl1>?Gi0g@~^$B+; z9Jo%*Cv5k<#n43H)&;{W11FO6t^@5OfLf}KIqWY^&nNei=! z(2ix!RzJooQq8hWb8Nc&kelJ_Ktcg`EPByUpsv)os7YW-TbC+?+n9n%&cpAZ_S4Y4 zMbh07XYxUucK{2%;O|_0URGAuQUXFfpe&`Hg!GYIuawHTJIa_8->Ba}#EWnwJ)F2t z_L6e)7k5#{*`O4iG%;I*Ejx~C;l6c-jb9X@k#)q+s%nK*e3c035;==a>=)~`uCCRu zvz58ioS~F=vG7r)!_cV%b++rC8N1T*$0C*=(7nQ|YO>xABx_xMuCJ+v?wk;d4GY0ThgB=?Jvik7^lcik7+hI>1o!lh=ywOGZ$`dV*j zM)8oZ)ff6jqMiG$2I+Y5@mMan(#Qt)QbRB|GjZr>3l?;&%q*+ffUaim!lYe7#m{=EuRdIuO; zli?)`&eOPPn{qOI=QLJmk2cUUZ6CtD576zb`f-<&jbK>=cG-M9UErU?8sAWs$xU_j zU^~!@5CFf5BWUkm(l*-E{8RO0x_ETV54lOaITf(&Mow0P>b@($<*Je+!j2ocbC3$h zZv*j`sME2`wT*)dW2<;oE==i-9}Y77gPG88D*XGxXDzSg##$uB&tF}6pR5`m8*Mp< zW2A&Uwdw9V;*)>1@)Pv`yi?t-k-_l;>T@}9fq?k`FAoi2LmL|-zy>B^8%tBW|N1xw z=+fO}NBd`Aq-ue4fW%bo`kV$03gvIONPM#MFkbK!kxeTLMI!31k-6_4VWlK`Z9C_) zFz^awZWK>Do}Il>>F43~0?aR0*EqhAbj;K9#k1j20w_AB?>9IAMEt#LyrU z7FFt5sKJ-+8zS=>l`n_`K$-3|0uO^2NB4*^n_+l{BSU3+#SnB`C!j_B@$51x2GF8z zF~<^-XmiN@V-dwzDoN>BGZlCg+X$}A$kH}8Ok$bJ`rv}IZ6 z8jPJQr3!LNBJ^V}SdfFl6e~DNPjPIg_xtr%B0@&#IzCFuYn+O+*inYk5fR(0pyE6u z%%~EV1>NbKI}hUEj?fIN??~~@)~T5-ayn;(2TQm=3x;J~8vVJ>ni}O?=7!L%WO&7( zhs+O#JhzilO`%@6uv~qmvmhMwfb@D1LH~8Jp7^cbwh!%!%lSD_SUHf?3>0sLaOYjn z&T+Ak_1ak3JjS?KJf_`#G7FZs++e=O;Ym^{B}pQ11>m>G-AGg@J`|O=of@#gx#X_8 zaDI=1KljF6BOq=RI68zrllkT3>#Fr&*{}o}<1iP7@A)!?(rBnySz;qPLZ7Pe5>*F8 zU^O%JCc1vOr;Aue6yU?jE+$H{F04s6*rlrTZXp1c{lr*rFRQg1sKyjHBuc4rpRGRG zdeZMV9Tf}qaRLD)kEITq2n}s5-j6}o*;GeD1|N$(QiQ2>-PsQaS(?qcfdu;u?knw< zi#$*nd#|D;!L=@2x5B7FOHE1$dwiH>Ku?3mK3=+Go0O|?S8MlH+R*IamoZIGocvzs z6aRdY-DWoZ1-n?F@&Z&Zeb-^|wz_hvR5(t;61oCWay2@h}vZ6l&rNmCSzd_ zpCC3$Pqqz2jZ6xYVha0}8vv7vE9Q^*$tP}2!7duVFy!7>Vp(?A#`$*zvJJ_Vm2XK; z3P=vC`Eqf8Wx#zZF)3J7MEE#1h0$>w~Y%l)=Z6~i(Nx{B4+(HwyTUS}8rfPUmAlSw|RUG()x zGnxJ=>}@dv5*71?wA>H%0}R5M9r8-Dtxin8$6kvXtYjFrssA9R9ndY7lWj%iG>?Fj^< z>fDjxm#JjHg$jr?#N*YU>n5k~yeS}3-u5YbBXi2M;u`QHkAv}=Jxv*Bu zw4v6pr7>ww=3cR!!swuSWv+7NHwb#Wz4|y%e5c~=S8)Iy6cNgDJM0D#MQJ!X%3q(+ zTk<3Ym{Ct@a=-y!4MvVLb^X!!uphSo(hh+rr#speN0!!%N33qWD=mID%H!Mf4nAd# zU6$LVFzl+kH2b@(HvtP_EQ67Wc^|obr05f3NK45)uDlr3%4p-e9(#BG*BzX6Oc{pK zo0*JxIba!>JFvA#xG6%Q#K>=0(5IY>iG%e7S zbL=uC8C-rpXw2NdRTR#vA@L`)OKMFdJ;wroCmF4n`Z$8zjKO<Gy! zX0;oRLl=x7`0Aq@Sgy@oG20TawCV<{HXGQ(=&3Q(;pzmT+rIu=WH-=ps_Y5?+Ij$J zh5!HlZ?d7*8 zsPXf>6Ek+V3F3e8cICo*ce-W99t*Gpw*QW1>@Yn&(oyda=o>wMz7|v4%=PU4E-p<4H+=9n9Gpk@yq5lfgy0vzAUGi`Ten6Ur zIo-!PUv+rhls(vz)8-1~eZ~v!w|KJ+mS$$SB1ZkGaI9(4g|1jKV*7ckOdAb=xq($1zhq=E40L*m*{?_`3xt7krrN1(=vso&dla7D4|Jb34S@&VZ#M>@iKW3N1 zsO6`=6>rZK5zL|DLs2oJ*{7i;i!o_ap$ev89T{o{{V9uF$LJAbM@WL@6JWs<@^z(u zwHf}Xt?on;&={Gily>CXQy^12nRz!HF-fVV-%iefbTBfOdXD*+bNCf^RwmOTClYO9 zr3_#86}5){;*BVu3~=r{my4<;h7mp~@HJ}bPq{Z-zg45n4jtG`p@y6&f$(wlVV{{J z8m5zRYxwP|*l^@KvX6$iI79qkc=)6xwQcKlgf-)B=$>g)$j}O2BJwP+qSQ@V8S!U(LD5l z4_DvLMVl(Dbu(@?of9)E$41wU^59?Mo-l4U=Ikvyw#@V+jA%UyY7FHPcSkti!6kC4 zlq?Pu;{3s&S;}1jWC*-Pt&hhr81}yYgDj6zK)>+^zZRyM3mjYAY1BMtdeMIBBk;fmJm#6F> zqJ{P{`+i6}26NACG-0peJpA^f3Wrn)+;8$^VUiDk!4Xy?z$ zCSB*^%r$>39-Z++g3YwbE1{sWni0zUW{Y?x+ZKdqOD^S&DC#f&1f(GhQY>?))#?sjEe>+V8S7nZWugYG8Sv+enWYgCr8TcS) z?A(B*L0#PMDabECRv}l3CRELf-#y|zoecYd97&-pA~xhMHU{Jdp; zb%6|?1#Aq$hY}k3;=5s5k?s8I7j;n!7x^m#OL`Y0J!0nt45n-xMLU8KE!x5X7*raL z^JZsdb{nwFvWZnn2CLP+Qot34M+4^*g%I0nU?=co4*{F08Ab8bf_?YFq5sSnaDdi- z$tiz@k~^;Wks0RoVD0kc@%I|-lp-^2ZW#Fo!`@|kb+o(sNxH**g1|wI-!(k8Az2%r z^-@B7A?dFb1vEMR0jgtr6RndKY%k#1*0H(ao88Q0xAa_@kEe|SH&2dW^;x)JQ_AJ5>-91 zj?a>M9wZ_MY)ChJUbYKNn^i$Ohp^;Ku8SjTeo2^6`(4G$|AE0z!l9|sv{8w)mnV)R zuE`(hm0Q*WS4Yzkv1BlJQ{;RsDgnR2pK8Z3X4skH7O$q$X9TpPeM-jhzc_@1saWu( z{Bx49Frve$_RLijQ8W*PG?&_Clj!N*CbZn3p85xr@CWF<*Izt+o>ww*0{4x*$xu&5 z6XYwGsU1zd_>3)+Ue#})bfX-ojisMsev(BwNW17#Y)}^b%IH|@U(rYHRR8)jaG*hD zOI7Eq+-gRx&X+WhJ7;EF(mwJ`-}ZO1J%JvK_T@AZLAhdbJ*L{!WL9|&=SBAfk#xHF z{N42dik2|PnutaHvEYcy0Md2x)w+(0%Jh?fdVYyDoW3599j5V)z3`~Hu1tdpPm=da+8>44Sr$SO!gzvnMM)wzl<87Sk>=}`luGaU?) zC7uKIHv9SLwr~gGuGo*MyT(=}^dkCF^I}jlUrs%dj1c|8ElrSk!)YJ1-B^*XZoNNc zl;SyjtvQqfcbwgt!&w(25rx8;Z?27pYhn@@wi<_ zD{%cN{xw#kyp&Vg*2@OJIW}wJCf{lU&$I^=)SS&8q0v^fz2=SZ@JqSP>ifUZ1n=%_lMt|R z%or62NcR7dCjWJqh#R_?x*K{bn>x9fI>}f%yO`RUI{nuKP7!d4M)G^E@38920!qcA zv}A8d3mcN#gp0KO-*3^D|K6fWsmI+e!Q>j#LC}q|Np8Bh|KRiMan*YT+kMR5qVNs> zTxI(3ar9jVpQSv!NoFAa-Eo2;7hKFfl!QUE`iqK+^04V9@aKG|kNB8TBNPT@V1hBq zPEusvo$^4a0^OL)v1siK?M^tKHr!c8;LSluBCuSv<7wl7gbda+O@DliL6TLPU{eUA ziyUj>K!1*8Xjvit2L|4Deh)7gAb7)i+K3i6yLj|)tN&BesuRBJ!eeHb?`=tqcwMgpKP(qrM`i6g9K^qS|+1Rb(#{Y7nVd&QY3(k z?==##!}knYALv%kZW9S#7T!gP0w1=vQR1(2qg7d~yGM zjO(otCveR=ivi}VjJ47XNr@5-DvDqP=79=xcsm&K8p4j~S$kuERy+V&dDX`}X@9Ju z1yup6{7Vo_Il-4}Jas;V;gYA|8Gr9u83YR0>!P#Zr-yfU?u|- ze!J!|oaj#S#TnbR)5^XkIRUc`rF%?mIP9y{R!3poG^gDjI}Go_q8+jILW@Q1760H? zvz%2;Dh(&^ma|i{-)s7^*}$9TLorUg0jHf7T>bb;^D^Y`!YHW{FyG!q7Eh+*{S}Q{ z?d1|(qiH+s^vsk()9Zuu{YM7cqio0;zWVA-F#AR~fP84RzNL!))0$X_hK^ge zkeE@J$G<-21wk-|wuQ1D8N%8@M;3r?F|8QCW}5{~+edM!_u$>BAXrf z=&nbH5RTi{>RIECf_?7Ts1WB)pzs>}(&5p&*yRvc3%dv$369+4d&N3nx{Bt`I}Mx` z&My4PPY;7dNlDv3_l;5kuKB$`Z8BbQQHYu63*1AFMNSwkQ};{fc^xld)z;hrbxms` zAkltb!loSYSCdZ2x}Gh8BD%e zCzjv%(hHyD7)6vkXR?{nK0z%i8BnPhV!=~(9Z8rxQ<*P}&`X#NhPmBfxyum_Zc?+Q z6F7?na?nyahE>_APHKPM@1Sj~9~C#(u@lcxHKA)*yw5#@rtFP(D z{Y0f!^Q*HVsq1D$jU%h(C3?$i7<;PjeG6tV7r~|t$~nd)g@`rt*4j=-iLdvj^b7Ag zApC9Z_~XpQ->qYN#<>K*D|3Cz(o z=u zP&n5CK$%gsV9#Dnzm36a&N2G#uLF>CY*ZB08qQRUXui?|DOX=iRguL-mooC}OvFS8 z@+Tr-VJd;tWmc=V^OG5FKkyGl%vc0G#Lrz^+9Pdu@pFkBuG%*BcDfs&S4a*oebQBk zbw!^rC!&xi57zyHnq#6!rfEUbSq4=DXJ*M~rws8%gym96smLrUY5$wHY+;N0$w_7b za4yaqQhlISLsSM9YoK?laV0DW`wDlmkKQu9$sO}K&stP-uX<_93?2XB<;(69#Ezg+ z5|3Pmaa+1>j&4EjDrBe};v?cA%2huUrz>uqw%d0l(={jp15;BmlGL{^(ne_a-Z-)! ztkj43%EXQ2!;5zmy;Q!R>lBA@8BZS*YyP7dx)y7eanRz=C+Lda2DcbWXd+n-RuIf? z=2d|Ze*b6^Y+V#P^B8sg>D6ijW`|uKpjqjb--=VG-0^1r?hj3;bub*5Vm<xDl)4Ujm0ONiul_OlSr;Py~cx++rHRFPC8qpTZtX12wga}Rg zYFA~=3Db-Xcv!S#B|wTa&EAo{dGfaeP>dyeg0$S5*fkIu8d}WQd!pyHI_vLiPvJ+% zHJwDm^;R5`^}*jB>L9B$xTgNJLkcI4f&uH|tk6z6-^LHBFTHBE=lKp>4RnVgSpzS`0$mfk)a}bYcBL z0vE@i>aP@M8$TKq<+SVCm86b0vN+pNt^5@pzZw$g8x?BI4ysi}y=jA>L+0hQvSbBA z)5!Uaj$-kx2`rzP(Po?&Q}PD7byo7tVC~c)6R@ak#FSt5rDPugoU!D($%&2<&OPnQ znF9>8%#Hh&JxL|-`vac8xO4y!8xn?!4+waGi1#^?MmXf zqeNgMb|7Ks$I)TL-nDnPL~t)ytqd$Y*9SRLk)2rRwF2%`s9n9_q`UCJ&zLg!MeODZ zIB69<8}_p9Y?KT#lq#P1_P`F?``zB;3h*Mlb4>nP6uh*9Lu|a zjNNKtvqDqO)6%eeU?xsAM)uHb%a&X#Q*HX&M5(PWHZ*LoN*2v<&=2Ev-+197AA~iT z<(9o!mg0zdX<8%@Mz{h!E8$;+=5+wkmuh3W6ww0+AA?o0sCXtiK9poK7E&yVfOGIO z9)XScSP}!Q46KJ?-m_e?jz2T|)&~rp+!`>j;O-QV6F+I_Wa!Jz zSKsMfND6`j^%U+SMIfr~^u(L@SP`!;pCS-Z*fTDa_|ks9ioQm|7cM4oj z@fCE(Xo72*E+{j2{l>1Odbl%#2q>0(O-JI*0#V$EWo($>v)x-)yDH~FfT1K=U?3XWvcV?l zSZqv~$n_$x#FP?+q*&{uv^=uL@r46d5UJZ2%>i-_7cwfE{W0MLB2M3pv|`-CQ?Q)r zG<+RRx@xiX-puZ)Y=cgQBbvZIZ+~^e!gCXArj`s&J@pWaG2;4*)^isMhK5L^o2lGX z^3_#?uC-kCY*7kpK3PW!gegN2g}Y34+yuI-shyync|9_s8U<;UnKW$E}sle}WFn_z_~nhe}VkTcpE zbCmcnTTLk!?FtXo$I8DkF931ACTO6>m_-NzBJv+DOaFL8i~bvp`+t)e4O|1HW0Ki$ zwPi$6%F>7(gEY75%L0HtP6RnvGTT+Gmu+`R=^MG3&C>pLA_*gzrS@xp071@5lKlR9Y+c^Oqq_-TJ%z#=cYd_53Q}rIPeYxug&fc*L zjw})wF%(zNh$z2NcsfjhMZ(Eaw3{|izi~afi-M=R@zrEvi})%4%|C%KoRHK+fTl9xs7OU3MdbPHO63ZXO?LcD)j=M$u>!-biMQtKGPom4VL=+$_i&Vc z_I;ka7(j3P3Bht?e!9VUf+*)-P2e0=bWCR?d`x+$cum0KXo|+ii@*Irs7{lL#6zVB8xuci@0fP-4{VA;sSeJU z;;7G+pm>8V<~>kD;Cz>*L3q{HXkNr=T#zu}noX%qaiMu;7(8%mhi7r@ggGD9!9TnyAtCW^%AI(T6KSRm?P?7sYO6ek8&DBrkw#$wT&(# zM^6o$Ms{!HWmGa{GO@6(u(`?&tTn3KSvKAtU$N129JlDeeVE+cZxE>`eW}8NcKJ6M z=JuFbw;ALxo#bM@IaW`MPL}{uJQ=j-LRt)wn?vfYtk&z79I2_JgtQ&pm12lg25B;{ zttdCDwhn!6Bs^d>Qfp06DJ2Epuo@>-x9Bu>ULFpL1u%V!N}vasjS(ZQmO2b*1@+bKLmLSNbZsi^`kA9|T@92MY&G z2lwa+Cu(wIIC3Tj-`A5G1&P2V*~i;hEpvlbwr@o$bw6-UCWa9)R)u{%6m5@zKK?yO zl#J1hJg9@jXf932oCIzsB?U=ny`ytsHjp;wq|%Jz0mRL2Fr^2AT+9U;w1WwyM@FA6j>e-f|u zCMTJls z%4He3bKaCQ3nLPVMoE{3vIWWI!Yv7JTDL~F-^|80R}wbbISVJc{$`;^5gZTaib0iQ zxl^xcLeV5dWxY-Gl%t1v004^s3s>3G`KF2!)k+PF7~B$lC3O!2DulcF%ISYHbo=W; z&p39x0+$8+hiQoWnLIWrVc02B<*!I6P3%gX^r(Y0DS8n~CV2rA1T6_ys9z@8f@wn1 z2tiyOVux)P`S=2--i6Ib8t!;^0=PY^3WpyR_$5b1uz2*X;03DJp@Ai*o&nZKX zqmw5=qO*Aj%TDc#UgI}twZ+*#BJQ7ghtl-VNI2>DW1mBnSzbqQ~4jTCCMG3Cjbl1O@;ndlObZ5eN za~sqJuY*}T!t7H}&Cm4y>nixJA3J>nWcWRRdZEDox97}%v2p+O!Ts|rv=<>zDuf9! z)6klt0j!&S2ZOz+eW1p<0^NjfVP_31RQr6*z5WS^k}C48LW`dC@ab{RyA*nr^IwAd zz((3JzB(ot$?JS{=XU^%eDs_!uOPiV2v2s}2oZT!li-sxW5ytaA#%yoEaHxEWW~0& z-=MS`^ncMg`EJ3SNCw-j7%pG7;hmX68wTwLb5n_<#*3OsB_5>U>9Tx>6S|A}5VOw4?>42lrsrdnxJv_Z`&RrzQ!60=A$M~FAqHM#W zW7>MJF5)+30+r9V;~Jr6u_eSKTWO(KefBKs{XXU2o9=w^^VW=q`?F5bJ*>;6ATJ|^W8Fe!yuKOY%?%qS3o@K1b#;I!krl{mCmY9_s!STb$STbc~ zi6{jK^i=6qITjd|3>!4*y6G6EWA1<~^{YuZivDv^kfKNAm=UI~_bTsKMS}77!wrJmJeVKMnpzr3G+76} z>S`L+W^|6-oIJ&Q9>q6!)ie#8&4M>P2Uwgcnz~91n36uG8}Hr4AFAd-pUgh#eUV|# zJ)&qYh?peX689cHC-Q^q(H)K~oa#|`+pvZiWdqgp1JPrNcbM822wJc8#@w(DM00eV z8=dujBgb%}PV-$o1#gtN+hyHJJn?GY+|AxYb`?81EPrGplk%LB-X!eNVD+J6lCj*| zP@U+t>&bF1VNO#vVEdTABZj?Ok<{cXvnEGeOb6Hv$b=8Zg~xU|^Z9J3tW{+8PMYn1 zo$QqqjxS}3H`;@3HYuG#f>G8gIVSt_es&ZWWN>%Pb&{M=7P&HB=@jtu8)FV}=-nq4 zDaJF8lpf{yDH%7>{`>W7dzjSjqXN6ajI!>qBsHZpdN;<~M3g`KbKao*`M=GomLr-d zL_j*}i4p`v;{OkZ{41lcWvj5kgZ42~cg12!ixTOH`)Fm&Dh!9Kp@{7Ac;Y8w_qzq# z4sT7q(`wznchoJ3!&hYk)ZIe0iK}Jy`r6aeCtJmLxvjGGqgy~cPY39Wdzl;`_QdCA z82Q9oVzz6-aohQeT%s2?!mfxI4KegtcS%@Oq4A8rcHp`TgCfAx-R$V+UpD{S7l&Cn zcimc92f$=MzRGl=cG5kXb4Dh0Y(1FMoIO%PF+&xcPlqkEh*&4R`wlNT_C*z2k=W3( z>F|B`{1<-DhQ!s}NlCfzmnQ!Zw@)Acr;GRguRPth1@BwkUpf<_(271VfyOv|7Xn%u z9ger|qF=AI|k$H*-otC@N=cWZ}^^H2uQ zIv-yD-$YX~UbD7M-JW%cE%B2WzN#S~(Yjc!-c0oG1i#J`)n5_|5G*u(F6`vWFK)QY zR%2Tm659wBA7o;E2YhpG>#MP{<*qJ=K1`c_HikA$@ij4XQJ(YgMS-7~WB%6e=u&7B zNln;Tc(OddD}-$O!~Y_h?oD=Bg==+-Q~l>`PoGLdYiATsPjJ}dKLxmZ#=q5k0r`T) zL_%&6M=8ew61uM*JDx8$gnxG^A4T_-!Yiryr?n>``Nh!Efc6|x45ZSk1)e2mLf;mO zLY>2EWQb~g-C9XYCH15!I9Sb;QKDJ@w%58Q!>QCd1{ZNZe@H6rqF-J_Vp-ATb)9Wp zBGi|Ck(>6?CRr!u!ht6xo3qO&S9*uY=OF_5h;o&Y2Ro`GTt15)koHY!#Y~v3pabPd z(M)I@3l9hEh|EOxs9z;R6a<^{;QeXMk1UXfFP7A{@eMv3d*WHhrR1s6V+ZF`5fAe}m*(0kiYIGi#L~cW*O0kTcN(1nQPj;B z|5$`A7Ix=TRda_hFUsxfa+8?1a_63mbQNigPdQFuRc<6^nARw%gDLIgIkwN`>xn|( z3_vPiFHa}WdunPC-L*jWm8)O}@jun82lwC^O;+tz^j#++j*~SHV2i2b- z(@kVLV5UDZ3*dAf$4e;fx50R131XSb<-olac0-&xX@rXd^DVzDl0!@ct7ETW_4d%% z7|?g~dwe(yD}08UI&p|frfMOH(_J|7$1`y5T+Pre-yGPdJ91}zSulO~)kpN7C{IXDXn1=&PJ2}_*WK|E^&S3kfWVYM5%8dJO^LRe*rYp7on zCPUnfKH|Lk)5f(2mUL9A58?MC&-24T-3K&|jSTm=byKw1v3>bo3 zu(o0VJFBt{=1hYuxsyal1&{eAcm%BZ(BCAJea9_-27v*Vd@@!fOH`$N!0(GqvSMLg zGE*xEXRKewDs-e%bV)T

      &2p5W4`Enm$E)ZU zEleqgh7!g5IKh1v)3Zuin2NO#GLFSY(T!I{Qa%;CV!(Sqp^lS8Ry?1V+$O$BY(E)j zH#ZIzLW(s;j-dm59hsThL{PH95{^0T64#&KkpxX+DGNJ(!DpVfxK(08*p!+V>h-3& z%k18p8!x0>{0Zl=Z5D?{Xefz9P9bIDwB zYj`qp7HR+96^>KUC+gZ?AZ$&wkQR;z+P-S6rdGf0o5-rX0psMHiF(rUNez%YO-a&=46kK z0(b1DY2RHC1r~Lq;_$PYhEzRU9h}Ro%(I6*v<(+$H=HkEXh-L07;#3J7pBGNqB4ig zla~-%BF-^5XU_w|_jD+XnM;;81|^@{V2*Htj=TgoE^#^2#BS5e>8+FmNT$idswX^% zq36Px67{w?TnMatBpQWj&@atJZ?r6)997|Rtrx`=~x%Z#D zcm$EWSkH#Jo2-kTbpBmjw*ss@zyZZ|NtFLw(ven|Q@1pA|E~#N-Bx9T3mN#-{n^2~ z%A}$LPUS9Yye$Mqj}Hyswz)|u5s7|Qs7W9tHMV)zyDi<26T*eJuXA`Z9!+=N*txFD zzdn=o%arru^#ojC6!XyG{{HE$8ZK91Oy65q#LID({tSxOHiCQ_ND?Kg$d5^Fe_>qi z;36~O$bx9>78>(L(N9*5e-_vkp~4{Jxh|}k0cKeQvUKOYA$#IcB_O$`FGb_`B}h!! zs38P@WJ^_QfdzB>6scrJ5I2>|(y$?%?>rhD+Ir zXJNUyuY1oV%&?iyYYd}Be0AYggN=IrXx*7SAGRGa!~adz!ed-Y{l{fWgi-jE0A9RS zt5$T@;&!9+<09}n#r*k;ld=nOjI#=;T206eXBBcOfuNH1bz5_t>#t-U;4iX*ls zzm?MtDI*%-D z$d&efpmaWcVe8XR zru&z4ZnbNdA_bJrPfJ^PQ1zZ80&ZoG*Cl&bE>Em>)2eV$eU;~SzYyT+%Z(V*90qc8 z>$&Wk)d+$o;_jLPM`{+E_dvSV=X;W@Q*<>M0z#F)r1Ls#3CyaZHNbvJYRq?r5iXyV z-VpR8glXYN?y!=O`;#^E{+5+@r7)-}yjB?qbD(r?GvTpoyqgv<`_w_EK(YDO@f%Rd z#E$(F)_S6x)tP+F#jq*l zFR1lTxet-l5Q6rTb_cbML&To7RM~4=xNB^!humw$p^BTO}5Bx;1&fVAuSr zIkwwVYVe0X|BWEixS64!k`;UYJAh68Gr|R5{+s(LTF%8Xt|xu?icV=_h7Wgg7I~V~ z9oQ?nv(|u+q^$JOxj|Q=4_qc$#_{)u%}x=u2mDnTTMkQu`OE$}>e??YY!#AA-ua$E z`Udfup`N~b@Ro*QN>KOv^P#AIIgj}&l|N_~P}kr;`>}o}G6`tSo+K3oC*y65Z{Gyc z2&khJuS1N;XXZ84vh*uVcgHZ}z>1P`?TTZPj7wlAb5JlZz_|q+t88-X2Cp4yD#o|( zMM=oeIi^m2$W2P4{;i?!*Cey|7S1S=oh`3ENVjEE6egFIo~f&UvDE(l_A6Rom!7xi zHXpWRaaSF;yAouisukEn$5Dt{EAvfzL$8TLi(>KE#x^ZZox16w<0z5FX9(j5Hl6*N%RQ~iG>Nf;wE^LZ(Jlwnr-m@!rc z?9LMiWZ771hQIvO^EYf0%OT{gHdbagG2?uzSS1t)+RX=@Od$jeP(DG(VNHgnLN_)j zQ0ZEc)Hh9NZ(mqdHw|D1(S7#eat}zkwDT8Ca8Bmy=En!u!j1E@OnAZ>;_@94DSLIZ zecQPB!>c;MaVYOq?8l%(JB ze_Z?`qY(?r-EG6l#1t&dv!jraU%8NJ1IL!+&2Fy2vJzC+2>u%jp`%X9SZ1_X>hIQ% zr=#vU;(kS?A4Ww7Pajynu?Ak4-%j%%Xxa*QNG2|Q%rHw-7|nt1!mDlTn09#9OZ1us zpGMi?!h1?jpTj+59OZ&%MP;Q5n<}4LgYuga33T=hbP9eQhYiTk*W=f$eK5Gw_)4kB^sRTHZ(I)IO&U+!D_*c6~yA#pZ zVhdslhEW5!I;l=5!PYA#3~Agb>uuE8!} ze%OeuQ8kvEpU-~PIYTHh7JO9$=5@5*b_*kRaDvwan08;X_1rqB3|3yTPh^qRRkn3H zuTY98$%3KRzhfR2*&QQzQB<~))^u-tCHZ;p!-e6}H{sf?>R@}RHzcowL5v@Oa+Luz zK&}DS6}IzrbW0CKS54K_?#&=!F%=}dUC`WB34+wb4*;)QYV(Ez?IV?5PVB-)G}|Y6 zX!oT8$S?7I-FOdI;ct+MBcN#uwnctF`R!kc`6I)$1X|Q5mh`2Mv4JcJV_%hI;$51y z9i7x@y&)&;m2{zQi^l5lJZM=weRt&be3DZh<6ip1N=kbbk5dZ4*E;m1x-ms2G8#XZ zyQiFARO~${>HFAr4&F`LE!&xPxhl&7Ro?@2QMylDtEiiT#LnqN_D7t)=y9TPyjhn7 zZv}1u{f{40IUF?csjnm+;S`LR8Lq+Wj{g@k3X4t_-<^yy5!m-T_~V?_SF?iJ_!wjbf_HB+-AQPEAbsGhNsl zR6P*>D|!$r3mX3hqK7aLJ%C9q5C=~-D|#jdE(T`Czdu?5kJK>!^=4vk{P#DE|Kqq( zbh3A`H@3I=C&T)`u}|lK{EL0sE@VFuJZ0_Jf17mt0EH;|SjYTymt^vg*WDK?)4t>SJX6`D9HtTA}lHuI{Q;spfz)IF}_@`5Q zOzL5xDr)LxV?{tIE(Il8bL+%U$}ws7jAF^abpcjJ_OHE zClQDn4Sd6O@7LoSvf<#O;@_iUGxs)BB8;I=%dUWAdeu%;P&8HhZH8`&e0M_SPWKo8 zD&_;H$u60uiG8mOv*?w)KNrDBB#Q}VnMe@dlVMwuoFPQX({R+5Ow`lwZ2E1#3@t+- zjoO4{kz1Z77>D48XtOw;Z%19HT?5pRa$X zq8_4Cc=~%zfLhxGHO-)2ZMAqSI8Vm|&-|abf%!ciylt{c3-`F@{&i&iX&-mAG!Nb9 zP7I22=@#gmRx_EE|A%w>kCKNPe|!}kZ#dDUZfJ07)k9n@ecA@VRWag)U)ifd5OJew zlMVLA9dp5cG~!>z>9s}z*a;aJpmACsyweV86dS+BNtY2FXq?9Up@JqZS)&gIWLH!9 zvfy5iJDsbB?|hE5&lIJ)GXe*OM=atIcjG3E>qQf}SN+_{UD6$D9qZBoatY8AoX=3= zx)t$5?<5VBhDbhB#$yXyJ0fkiOYxnfHiGG~7U`2f-!#R$HuNkD)YQvTZxpnYf5rX` ziZi^mL=lKR_`0dLYgR6v;kZ{(!u1CLTAZ`E z)dR)mT)Cf?dpnL!`XT3Y+51{{`1umq)~~dIP5dein0sbCO|Mc5pG~+2dNHDwG1>Td zUq9xp%fzVP@q-6Q{b!=`8h?NC({dc1kK$mKYzr<8aZT7lg-Ngj9T}EV<<)QTzd!by z{}A=jtdtDI_25y=@qpDv2#e)6ZFL5JZzI)D)uNB9BTi zIG+s@Sz2)V{yS$gM^F&b2WAl)UK75Iap%u!+TpBzZg;@0Ep!>>7OSLy+oTgDC!evwLD0lsV>6lUI6Yfp&t==cOJ#!xi zTGt>!`#q$~EaohxvfTNpMA=!vzvrYfi^?@O;YLtC`#9FB}?YVn6~vO7nc3oJx>6UXknc9vcH&WT?u! z?QU|eUX5K3UqLl8RAUpBini5PCm413d7z4mufMb;(3&q&qsu6`C!)rjS6+t`IWhW> zhGGD+IQ4|p(*eAJhTrcHWY#L75b*MecwtFJvwDdZDf}`)y_tcIjYt{DJM%FH8WQ&*1tlwBo z$3SG&#l+R)bE;khH|I>XJ!Fj+^|!H(ud_-I^NaR0?0;w*_)hSLfAaVY8GaiS*d~_o z2|sApGP|*BJavn2MsbA`m{+g7(*M(Lnv2oM0RxptdQ#T}uxxlJFy!XjVKRTYc6w&u zUf;X>LD6pG)YSKM8+y5?Etiv3$uD|N(4dpCNRHz~Qv3%>G%4>oZ9fSgn$DviXA`kq zom?|{hQsjONeAL*@xrdfoP068YUL#v#T0}uY`7jG0+O|-`R&cQIEh84|Kj2b2CKiK zFrmF+Ci=>Ewc`w~I-D}s=R8Tyf@ZchnVn-T=WKi9pEY}nLiYGG&t8*eSo{Oj-}I}H z9RtU2a!{v0P+X*v$re#pGJMm@C%X3hN~XzB;a{*VCWd`Cqbuz^Ae$twFn^L_Sj+1_*J#QTVvlospv*n_w-d_*>EtpVm|<44L)iz2`_Y5Vg2`ayM;ucRQ6uo_w&uL9R#ZR2s@fGIwZBfQERAg z)Qs&5*%28{KxfX|{bU*#+wlNrt=Ir$*&~ov$RuJJ%^{r{&s|71xph|`+)bV`b$!%( z4xRxRziLD&=!Cm=x*m1uNqt5qZjt!Qo(q;f5URYcgLGN-URpcUy?xZ0J$9?Evh=Jj zdgFGhze!uYtVqG|`2SlG!~$&C@+oktD+W$=|EWpq|8SZRxEZcX{!cm%6R8&mqrt;7FEyFaA6K(ZJ!+ zDL#Bp^M&2U&U4JW>zY|Ls_SO3mxEHMk;j7YNnkkcNx&8h8<{UBgrHOc7>;9r5HyTRvW_v->~rtg zapVYUgpd|5Ex_qQVjwuZoC&lmJjUbU3R*`8__WpbcJ_1yU5J7=N1v3i7NgSFKg$&a z*}&WOe-G*^_eX*hf1ktzyA+6L;uL02Pp)Az`zvT02JDj|B#3GFJ7~M}vYe^k$szy@ z+Wy@q!vJ1MUHBk`$xUtLvtm=60;LkU2c+L5>S6%~d0Np0j|MFWB(#f6Q@?|HmLw_K ze@U<%J9O$Tl2w7ZkQ;NMu0>=~0qHl%1+2W+y~fta8z}_x?{rH}arU#s@cnhUgM#YI z0PZw2fRI-1>|cE{DcPRAIw^$cBQ`A=x3TJ$EqWv4g_cbO0Jg{=E+rqT@;=o8k}Mgl z37ZN{V*UBBc>CYsgF-VD9x2- z6PTh5gqHrxDLW}Z3B6Xix1x;fu~lqUtTm(=EJPm$Z+)b_{VaXuCQ9F(_MNXx?vq|bvgx4#=rqw4(Z&9 z;w$i@raB(NYf6@{4`JgqzE8rSxVN`_QdKA{CxT}~&QBU3?Q6pSccYBE!j7K@F0fH% zWJFpk43`)W6`$!e$`M5i#V^vye2h1Q{ZuJHH_&X2#%`~XR6Y-PKxV{Trc$&8UUXeY z37C#Es1P73orGWcJVlk_X0f`{3m?tSh>LIUPQ@A4;Az94U=0GRYM@?n6`OxwIMICz+!$F(J8;nTQ_YkC#tmd8+!aw-v~kvpn)C4u_= zqvT*pX`PqJNcii!}OR#B&#-}QP4h# zV6vgZG-1lp@Zs?E*Y)q-k5_%^OXJEC%_DcX!$I`-cD^~8zP84SG3Meq!|M!ospXCq zpe+@WEf}p(R;-op0WVq}FC$)*A-0EdhoPPB?ih+}?O-jXMspbvVH8*4lN2k^DiEC& zdd(sIJZfgnIvl3|o7G1*r!fYy`WoakR&Hfdg(*-OCX0Wu`u@1>eHUw}qV_ha2qYDJ z%wLaT{Z&bBmF-D!;@i|1Sw5^l{cfn6+9k`|#SP-%Zb(es(-t410ngHz_UU+ksqK?DAS=Q3#VZj4y`G3Mt)r(#2PKrUi#Z^$qwnT0M{LBWu5eo&8b>Q5VKA)|d&|_Df#a>@48LI#z zaaaOZ+%Le`ZJh?8T8r9_f-_zFNxKTD51W1biCIjVK&n{D6{vpepYiN0`?Lz3Z5tI4;BU$U51iI4A0wAQ|k8tNVzdAX!THc zb8517a5pc2 zmn2IV7MXxGHpe?uPxL*FP0pD$AHUN*RPh1-k(LeDqJEvRKY{h)OPqX$en6{3S-NvMMHet56S7MNxGIE8JXp+@I<3ZPhy6Bp*7J*dOQq4SQQG&wYCE=0 zBCRdC3RHO#C(>yP74S1^?l@OhiSsLUv@|uF%+y^sGB<}tXGHq3FTM40t-@5X%BIJ> z_t~UO2M%j{{a*HMtf&)9Eb4e|Cd`s^XtzZCkIi&DLXY_sYjn);6@ha2tzEEhKa}7` zB;*WBnH+WyjaV(#H@JHf*ByrrnmvDBJid1OM{{#JAFr9>>Fb>(9TGUp1O1UJT-hBp zWiw^p@7!a`#;l4(oXcEIQH zzxS!Av&>#A0KH8|93UY4{~<2_uV;9)PybrKfqA(g6(AlPG1r-i8e5N5*@g{qptKGO zKs1ScvoTR3rSAMS_v!mhIZ5B?H(wNYBXIj3@7%j45G2T{dUsjpv;17+5zSjq!z{ml ze)|0e&aD>n`IEQ*sma%h@?q;7e+h#uRhAie>SnJoEI3=S;t&5waDN~ESvz3!_X9ej z8sqxt*TbZaKv+Ecq6d}cC>)nLl2$aeDyDo(MX}!-9iql^UzJfzLI^EoA(YsJa)@7~ zU&wX@D(@;1>e<#E{T*BFAGU2tp5_o$T%>)QyaQVNeS1GI-hYAeblMedo#*g(CK%Ba z{xS!8nsi-IzEUf(Y<`~TDSeM>TYHxFRU1sTDOxFt%1^(u*n$y)fO_UrXH$3m7F~UD zrtXf^)bTa?G9h1SE71x|30t`muQ&{biRMMlP+uAIE8nx^wI%ZS!B8o-Nx4p2t2blnDAdc|`Mp(EJunvbN?kWH zrDsd*o~QD{rqCb0%8wjqW&$udtS=&EC?lm$aZ~>&%0>t*R=f))R7HH=HB_`$}spri0N}!eLcr9$y z`#Zb^NqOCE4^E&hOUt6sot_!${W3!^+GMUf3+Um?Vs{sDY!Xee(8v{xK-~Nu+rlM> z$moDz~G#0>E>CfMDcRE`#nv@b$sDiq5OEm3IrS|>DX*Bz#_l4 z2yE9N*!=5q{OI6qKFhabxmbs49qfSn=X|~JA9x&;;+>Eyx&bX@B%+#Q#~^sy*dcF- zy=+M?!XxRUO*>h_D|8@0mzzWtF#C%|;doc=s2x{G&Q&=}N;9?A6^+4%1q;9OXXeij zom1$#KYtM1WfmvjKtE8q{9&SNNQ(Mc)d|Suj<1Z%;Rdy#vxhGn>V8f)_PILU4lS}q z2Y2T-{DSb!Z<{})K5+nztK{7<@C!Hy0S8?SUCw={1IFgyZ!_5TBu&~{6CicczbU?i z@FnO(41vCpydaRUG9Z%Yk-f}U1$O=HY#LHnB&bNBD$i{q*IUU5lJ9_j}d4K(gU`9=gc`_u|ULgHN}K~+zdGp$&Dc@mn&lE2kWz~(Tf z9CVSkWf?5QW^AD~N6$lPCqt<~C2&N&p-G>gta|3l&sE*Q{VX{*7YjD)zgsu6PflG9 zmG|ML9-c<9fzV1CK1t*ZnEob`u=^w{UesA-4|~y!f?TGP-@WeA#qygD{4Ddk(fnDe z5!pDKoA0m2aL8dOEP2RCseviz6*$Ar1b*kTsv~L{r-J2zLK&aRX>E&B5A`VZaf#fK zw3@;j0{=j88IrYa7uH*0g4z*OtzxvVU=RnD63~E0&3G|qqannfS5yT)1t8ce$Jn4O z%HuDYI$N06pRlAu8OQ*@JVR(!mBM8xEH30Nb}9kGie|kcT})_S$mF4?SB_CU`zcl9 zI`L2z10H6>U9xkf=0Urq8Z?e+Dosnhe4Y2+4&eCG@GxwEYb-VKLvY76)KN%37PDr! z%rfoQb4kfl2a3O#2O!C0{wSTq_j~{K(@#yqBa31sRpoR!B*6j(o!t;?m=)uw`@=1- z5ysw+QQval2OQA;te1VP_?THfk z3LsmXtE4b5dx@d{=bS@fK_=RPkABbsy<%{m+sSUZfF1Q~E6Z$Pq`;Sr+Fjdx=RNDl z2)pMm=4Cy7Z#eq#dX)|_wRIcDf|4WE&5J$hS}yl!$XAaDCE z)bP}==hEwTs!{GPd-3zs@yv-JSUJ~yizg#EV?ez6q>*_XE?aiDnU0v(Vy?QGt z9@@Dy7DjVhf20{&L)LPQ+MT1>@BN!V4~raxyayy>G=OBx|GQ(w|0A*e`^^Jjo$qg8 zBj=$;wksirXFWP}mYcFHWuOgb4johEqI@0;7@-iZp*I=mq{~C^qa;h~aB)V@xhx;R zYPHpCLB+$mY1J^1MYQup-aD#p2vB zyO6EoM5*1%+!>^dQN&A(Vcd!|ul#pA5z#-}i9!v#zj82CH!7RV<|~c>OA#WyqAb9e zs)7Rkt({2yzuSq_Non^8500b$ZYMgkYVy456y7kafOLsu&K~m_vXvvTWpf1G|w`N@B3M%G#KZCW4lmpx>pvv~0k-f5fxSHvBEpUS(Vil8*Oyd#ePEcflQ0Vex_O$BtUPa$X8HDtd zwuFY!3s@+ooyNQuO{}5N)N3btMKn^Az&Nk)qJ_2_cT~#Pq!5K+^jjIeWz5}(W2(@h#hpbWk(-u= zxN>$f2I-E58w-zC0aj%oS|2XN&(KW2D!--$D7bWz&K@lIy?kt1vhWjjP*l-Wa#g4>!`yv%5o1A&$$} zs$DKdAvNXM2Ya-swRF-KD6`#0NY*6$m&|t6xHkfn*(!iC8{Mo!Fz?P%!}AK@mR|v1 zSsUi3`COJ>Yo;JA*3t~IsvQaEj@lQ)!ESY~pM=dd1y+Qz8RLNq2&mP~OPLf3*{-#lZo0c+{el4?@vF((QwhjZKd%+|II# zQ5^igRJJ>=2?z8qCj5exn$5*(>L2!wpYQ9<|9>Fc5iJCUYwr$Vcv2EM7ZQHizj&0kvZQJ(D z-gkGCFZr_B?EW~NbkdzvSEoBw_24)}NZW-=+lNcphDulkDtRO+xW>!7#L78E%Q{5L zIz~#_gebbkE4sxixW>skM5y_st9Yg;|GUg3R?R0()hk8CGg;F=OT#Zi-6vh$H(k>| zQzs}#%RfsiAWPFPLnkQPAiPjNG+!?yPcI}_Hz-FpI7cV&uVHwhVML)pSb;%!fqqDy zerTRySb<@9fk9}#L0G;~c%e~5A-9&Dh`ArPmMxctwWx(Zx2BD_RiL1un}C6#T0z~I4P z-2Lwef{wz*n`f+0wuoGBdK84QPTVTe_y z*aMA5B5<{GxHTG$(7`3cPxcBx!hmrk+w2oMn#>oAjQ#sce*94 zG2pb@><;@Q$k}GRJs$Qk8KW@>c{*HR=j+Yp=({_fuNTUdT6em?UayVm%hdcrqak33 z7Hat(eS5s#9R%4k7A!4lQ{e&WdT-upRYr-LO_Nc-^v+svq7}lHPya)wX8kJywWk= z2gT5JpBK4l-u>yk=?XINIvd26`@C;MrW--k{U?=p(&hFG!C2OF@WGguo9%vHbCdi2 zecX=v?S7dR$_GWcG71Pm)9jl4LA|cs{rSAFwc|s%8MOQT`JT-E1QYKA!n*T^kj?`_ zAL|3-ybC~x&I2JA?}yU73&fbp17jQOhx5D(B7n(<5ECCj%DfAvkj{rv8yi6DxC^0= z&WAA*AH>?Y3uT|mhjSYn#QQ)9eF7%E`9a#&Bb#nRILf1_b zTb7t{jPb%bpc)Ms16{BWwbnjHI%lU1x3P0~(%KvXYi-z$v5jjY**yT0kS3b`wbGx= zTq*;TtTCxhR)5@Bl`aQQKZY?EEzFn>WMx*LsY>NS-k^3fZA#R*GQuEUAK`qFN(`AY zB!4s+j*MU=C$ZLp7E(t7Vy*KQcr=*h#u$7#ruX~a(|RC6@0u&7bxn}^dkS^hxnxmG zL;q0iCl|aN1HP0*+Nyp;m*1R%<;BtPkp})(mu38<;{J@UW*)+Mrx^ z5nnDNf}&iIh*HjILKRa$rR+9=TnHml#vQ>dAH%sMj_R@?h*}|KXNa6J6-6Ktf?-AvqIeiGnAhOnG4;C3>O=9dTToEm0uW^=Szov|Lp5mkKGt-0(k2_&I81l;KAJ9ln(@HA6$x^;DB9> zqYp3;#wrN+zfY5YxVFgjqWJz^p0|z4Hu|R(l^>b59+; z8^SRb)ge;G`2aD4{((Z$Ad)035Ha*Fm%ZVFO}aE8Y?-g1 z|L`9Fvl#!a|11CTzX~i1TXPdf3+Mk-S5B|ajdjvTW_w2F`ucCbzkI;JL2w96{=yjP zB$^)_=r=m#`-;remxn2kE3eoMRXllt=#pApTbv(Eh&%;%s2} zUqKj~#v8Rw008|fHG27$p9TdhU!iI6W!N^=3Y%L+t<+LZS{!4TY&<_No7~hauFC)% zS?F9a|K85=EtdE-h};S{?#zpB7s_;mVL7|1n(JJ!cJU2A%?KxDeUGSl#qiS9d9sWi ztV=(ZGEL#jFm{S5pNpTE80hI0{Lc1a&j!&F@>dU2};xqlBg}2qzxi5Q!Fk`)}iU?qQUQn%z=vSr*bV0_VO5q{y=zA0IJdVVtXzO(U5r1bq8 zAXfUTprmjvAMus@)+<|PEB{r_X9ax2NWTTyjg5(hhlz{UkARwm;YQ9u$&8nt{ueFn zZ+tRd4svD+Jhi{sab5!DT~|7ixd;nJlD{~RNZ^be+-ok#69}BLllZHCEB0!ypG4*z=>81BZ7U8}PlgvFfGCniNnNIiiU#-MD zB6gbw;Wj`CuIzs(MZ45kc5KaVb+(ZJ;@*XRHwRT{t>BK^{SX!2zEpWTF#3@!Eom5V zB#IR76V19P)6191L4IBbehd1%W=NL-lvj4Rcg*CbCa57RF>a#ximg>)D{=0ky{ZG( zMEudZ0_twaxEM*Ldo89Gxd*aGtrU9iplHO?KK@W2pRF;=LZbpaytSK=?vLIs!(ZOw zY`1-x_A!II6>En-<#qUb`;-Y6GHH0dx{>sIH!3TMYyDglT8wtzhHuNtT-5NKG2f2s zyn@+i`>E|=OC!ergHUtiT|0x7TKY53{2=$a=dot?byB`90|aLESkg5HFd4=>g>(%c zq@k(Q>s1dGP%=OfdX+RI_maFw{Kl>JCWzs)j_=vUxp)ZOO#|bH^Nqhd4!;*g+*-vx zyq1#7fWdWuGo0o`ojWTLJe7J~^(ak1t-Cy)NOw=TrB8U&xHBOL)Ecbm8ktIdZc!1E zBMB#dTrHqFU}wK0N1;`@p;+cr(oW#Qgx!z1X62rW zN(cJ6p%&{BwPDnT$(|#b&U7MD_3yzQoCtAu*}z7Nz&g^>`SZ;|=szu#5At=oJ!Gf= zXiP~{9dG;}8(sWkO}lDb<{zzB;guH?o;Ze9cG0!U$|hNMsEZ{BGAG&{Vg(Lc#+{L1hBCY3c=%dNkwyz2RiO6fH; zOkHUX)}HIO;FfJjJlf?NAS2(o@-o;xnAs_rC^?Ty0qy@nv4h_t{GOwAz-`Jgbt;$f zl5yi>ogBMlJ+hycr+dp$yDp5_GwCLr9B!lMRKAznnW#d`_b^%wFN0(}H?X3uKUY}% znXRtqq1>_r{WIWHiTJzXRhm*j+PXNS;#R*xX+hh%&3EGEz&@ep-L5fe8p}7PG}n`N zpGWm z^~!T>4`imG4MENsBw=41kkiN)Psbi~|LI7KE@iPc@=jA&bMZONc5jfIw-_dIGhO*q zEHL}?=|CsOe_!&vJIC{Nm7SY7q1^g8IK}H-ddXuKH*O-@$e+ET3b;5#2pjv^Ho01H== z`^qrjwy>!qQC3jVdN@femxq&i@^7M(h~t~H92YOhjx0aEF`wdrJj?a6L`3~J>BD!E zREQ`W;HQDY(qQ4_lZJ<7%kb?fLuh6u&9TGSNk|%mV(a(wko&-JqP+(e|E@GTPN;73 zg$Ag4I_W?tRq9YDNM|*6Hes>!Bwf=vpkL}=gUMe=aie0u+rLN)n~9FQIhxP!$bS*b z@!v#NIhwY2$phSy9`s=K`FxyY0^^M7s1;@TL{57qFwA``Sas)em6a354OT6YNZVK ztg=qkuAi;qwtZ#unYM6Vzf@S|xtYl7o5Im11rO|}X$&qr zlR!p7SuH&B`!O*)odbI~h9qiZbT0-SE6#pMrb&hL@F+)##g!1`dltp6g{f4Hb6Dxi z>fYN>j;z^n!DQ1=deHlts+jC_!)eyhV6b>I?*Y2+rB>7E(Wk+9`G!X<(u>8|Gh3-* z1-|ZZ*ny$Z7|gPYdkfB6)5ww1K~UZIao|(l=;Nfp*a)l9=qct*BNaTaqLb-1eM`#R zLyq32LD_`A_a-+^onzxT?Pr91QiHoYVdcJmm6Fh5@ZcWTx->FFNfPe0AP-h7tYk<^ z{(OXvfh|@xN|)Ted2B&xdl4^Wi^o3e1YaU$(e||ByGf*ES8*86=cb2_PzF8fmT)bn zuUhTgnunS$rq~P6vsUR%q?&gu@Y7H4ns~_TXUxjpl?Vd0U6O~|(U0Bh5#spVx?~c+ zi#;sw1@%tv!RliMkGTC+$Fb+N3iHDQ?pV5=pqi{Ygi>_qwc2jNoYIxG7e9VC+h{b zf(Qp)lAlHQFe7e56Q+(b8yXR`I>h|d_7Zm*q`4WQ$Z*v7QMqRI5&9;PLt?O; zqf!%=;(}jm*aFMVn>87byX2;GBRJS4K^R0)*L1ikFxmSoB)YWnpfVzeLM&Qj%fn_*)!Qn|JVbStgEWD;I~7JiQxa*rKyZy3bB z9{d3j*j)f{b`3a085qA3Eb=eNgA^#Q7Nj>Ql(&x{Bnm9N=LQ}L0qRpItvC>J(Hh>+ z2;SEYW4axwI47U_G{GCPS2!7&d=Ra;3X3`rieVlWkU6{p0a#{$;7%iW_fYUyTNo8$ z5KFP3bwqwip8RMM!e`2!9eMsO%z*Wp5RGhL(UwpT7RY;Uutq1i#{y^vR&Z$GfG-q? z#c7y&OPDulXs6LYHmIO4au9P7(NDlYlJcMdc|JnN7=M7cx-ZBYBasw=bQsS-8ZAu_ zEm10u^b{i!6>4fGeuQ~kbY7ICd0ZJp90YNwNT(QzR0uePut$hk*rLelf{0sN7$9iD zm0UsBATgI35z%&`gyKHWd&165Lik<$y{n{jqq1_6vayt}J+_xQzSk2PgDGY1#WT$H zrCgtn)LsPO9RLXT007-)fY}W{Re#$e>YfVM-4;(QCy;wQ{-`^YaY0Wqknwt#g z4X+rtm;iT{{+CSbo0sK1_W~pi4rEj4eN~Y=pmi67*j8h#d;j8FY2|$_EOH3Z!L2jN z-8^~E5E<{exbn~lWG>kpB$;CknXj^xa)abitazSY%x_=98wZiZI=zM*y)5ych+YB( z0ZxqoVujq~ycikQMA6SC`57FGb@>H6(%m-qkd7?k-U(7dg&IQ#5k)4`>g<=)W z`iV*c7r=4KVu}s;l>2rdXP9CQ=scwJnO{ONKQY8(h$m-Kzq*NVZf)Dav6#Y}l_raD5W6cH!ft&*$Fv-_vxpJjPI zA=z~VcuwJHKNCDxyc|;zh2Z)iYTQz7`{Ku8XWnU(-^m$;8kxZBQYk%BO{}s-WM){v z%zij0eTkC?u_2LD#v3&!5yPigK~FuPO|+PfI*1y38IO9KW_8f6?3@*PQ39v5(X4s?hBMqNO^d z;Y}5Y^DK~CiKN4jG#`;Vsg4T6jFiKaEMJWR?;{NpEQTplB+F9Nr91M|F1|V}YaAou z%tE3Shm5mN2zNjnDLcwx__*xM z*u_r6ok+eeN0&I5M2nU*8%O;VG{I3Q=8PW8nQzt*0GO$A`66R%3a>Ng4=`5inqEw-?Y&SH{-PQFCd5B}3E8=86@{HfqFh zAb^#I0!b?ifM*{~aoD$tc3|g5D+mz5G`f^$Cy&X5qVq0g=yAc#f-kqg8GSvl>T7sI zr1MU)wrNze+4Q`oFtRBY{nJuu3Z&Obugeo&k$qaB6Nuota!+~~%wZ(><33$vy{+)i zRFrt1AFX#)C5=McFSNap5mvZvOxJcBtT-;@avz9|x32=*pooRxZW($A>sCySUy=Ct8^`ox zcF8GQ7OHe6>Tza@OlAsm)~*{H^(wPfewIr9bcf>~y@IS~vdm_y%;m1E7&^@-5zU_Q zX3g*>Z_!qajh0M7tr%9#=uR!z_RXKQ7WU<)pLO%)S6e3jRu0eBa_J@w`sU5?=HEIS zrfzfSN{gsDYo>i$*m-MsMvJJFOZzelH@0(eMvK;Ma|!>J;w9&y^rj64m!2$h=q2mf ztTTW}=g`d-v&rUn-KG@#)-*9URY5L*ZJY8l)hSV{7B{hM8?9z>H{=c`-$IVsPNu_x z;10!sChXAx7>E_P2OBu<4IFP6 z?7d)Eh;+CJbU4Wk*obhr@%~U0_t0HnFcYIt-3+)1;845_IBF*N@?n^W{?MCg7@u^U zL}pxHFbsw^oF5@}2qIc#eVR5JaJ8CX&*?@ll>*h!4Wb z`v_ZRb+Kk`Q_0FkViWg-`yvsb4@df3iqy02>ih1uBpGgr$sQ6^jM-&%#42{OCo8fr z{bV0x;HPAx=vQPCQ%ZmR=V=$q$8*Lj2Fhb5C3te~ar=g5)5jg>BP7?(D6I>+|+sbA5^mBPBS;c0X{KT$z)vf=PzWOps)tITq3@L9{dwDbz zaiUUsj1+lX(0pvT3z1WUuIO@-)O5l!jM9rjOP)q^)<6RgiF&y6_^uWq{nQBB5s~^V z&ZUvkd~t%iBX!#K2-|tbr*Wd~nM^dCtk8tcEltJsF>Bv<%H50F>6tW8YV_Nmd@-fg zv{UnjjAFG@L%vEQZc3f@NHd&Ly*5<)zEIS*&h${&d0cX7RLyo#@1a*PGqV^!O(m>THfAsyZA}U20lM@tyw4*34_BkXyre-nR7Y>?m$to@!8; z?$l6gYIkjJ*J>JmNYx~W6(z_H6*Ul}&pzS1#tlfs^&FaE3B&E^$;RH~iazLc_T{Qt zG(DGiW4$WtFyaBH2jSn z$PwSpp}G9|zV>N3?@rtJeD(i?pMoD!e?_@~%L&Kl{_(99 z(^qOwLFa4!vnM{ge@A^VXZRt%aXS~?@Vogn=J+v(R$p6G=jZgX7d4R3{dOPMVMH2Y zl-4r{u$Rwyu+|alSf=7_9nIcQ$#u`l*MP>ipUu1P!VXz*#nFH_oRuqVcN@(0myA%B zq&1GOF$e4&SAdcnVUIHMS-$3t=B3{wc0eQJ#lvvhg7>&==yoghIo{`^P4Tk{&LQ<< zZ%zBF3dUUkq4%Evq!=~#GXwueiT#>!7vG*;szh7AL@tZM>t?Z32Oh4^H?6E=Qt zb)`{b1zKT(wCASct3QPAz%k?5vGw*Vman;vuNeY0I~>AGU)@h6I!)^vZ##yju2Y=*G#ma-UfKaX}ixNUAF8h40!G}?1Y$2xfWl& zqqI%5C;RonbEyyobA(a&*7vC_fQ*gS-lx22io2DePskoY64;dq{Brp6D$ujx zGtLa+C0&STSzoOcEm6^y9nXsYy_f6AG8hC93|lm-KquO0C_$xgjFmLy3q74$7Ns_Q zDjC{=;u9?Tb6G@~mZMk8Jy-Sggy(Uk9`5;NiE03sBCP@#@7jIvUg=RVz+)_o6ciqwcZxyo-?bUwX~1#wzG?;w~0$H<)ZsQ8Gc>v3}Y=uTW{7CHuinvoTl~ca*zVLcT(2gSQ`beZx8_VX{Fb z28*#Ta@+0Kac7U@U+PuGiNA-w-}bSglpBe32j#*NJ|If0C*alUm^!M#2#I6p#L-5- zn8XMX9ys@$$Hu%L)7OjbZRlG2a)zY3iM}Jyi|RZP3+{57;-*KbKd6Pi-GfB^Q{tRl z658v;)EeX~hi;kox68Q*Mr!?A5Ty-)Tf$kg9en)6-GO@dJ=LXh`!`B!eD+D3$zwQe zQL+NO?D}+myivJ*1}UibpEU4j8m^hy)T41M}eoiOaW8suuv}dKwQIIHJ^cr+$`w*h7qd4EXSSL_x$IP3|@)Bf* zPm^Z`o+4UV-`Sp&haMC@690x-aw^6yVuLxvkoxw%-O;sp^m)0s#H z8CxR2{;0NpD%J@Yp!gef24dDK|d__DqBCa%!Wv{jHK&-@^VHg10|k zT8pE_M_Ouj!t~YpbGl)529`wNM`2chm1IL*MZh7l+!2dSveNUn&ZwH_%C*0c+U)Bz zL#L!?_uM7>4zA;i%PxQDK=_4!Tng^&E(>=WoE>2+O_9=naXk56qBY05_0RZvuv*W} zq(ivic(Z))`ae-<%XQ3b#_t0xwP55OhU<;@sCV@VmEEtU z`7Oukn({xewI+ixt>=-}v$((*JyTnIXy$pyQP9T0GjqHIRE(DjI6-T_S;Z<1-lJC@ zO!2)Z*n>(hBb_3-g2H5`7}F83Rn_15vpL3dlx3{j^Y18Q9Dn=GY395iUv>$`;de8V z&XaS-lY_fGAI`|)`oZCL-sq2^dN$)<+<>e|1x)Ag-Kas6MOt#eeQa_}1~pml(7$6^ zPI9_;_kQlz75t(bMbPZ1L^WuSJ=z%9@JG$$3t_sLCk%{oUgK{HiQ1Sq62}jIh~r!% zinrIw_Nus|uBdJ`?Lq?B7%@ZFmX~s2Nx=L(i0tj23pW<337#&mMmrWNCNxbXP|?@9 z`FNI`-;j_9@f|^$IF?ugg(iBw^{7{@`M#XV$wZHTgK$}|Ds~L(Y~jY^#dc))Hd3xom7VBJckVXj=O&> zT@0Gd$~nMDXb2_&VU;B00-NZX^>(P`9ba((bpCRR4UJwT)lH!zOmTiI&*fH}f}!v9deWf)bj4wj=E9Vwc}6YDES zFZOqThb=^CunE{-oU)M>`b|Jz{u*OcY;)1vZ=-zZn-r~1sHGr- z6-@7#W2~HS(>YW(u_*D!yqx2h1RS$X8P7%pv8OdmNm&Vbf*r=E6&~l)vbLZQ8jjhm zLOro0XVXr!`OP+eh`~*W_AXLXD;3O@%-vKr-mCX*5w2J(t#OO5xZ`=_bD3IN<Dar7fvHa_v2fk;{9O#QjRD|t_wZt5yVDka|)j@!bL zW|bV`6$3UWK_C}+PnO&VS4^#Ls#X`rm8&@wZL_cB zb4s`li=Re+&VNmGac9yCH;Zd-W z7L%Sq9r>`o0Qi)>4A85J23%;Z1S!hWHcvF_DB5|uaXK_HH<5_5bmT^>VVD$UrJ|QM zokxe4R7LaN95gyItkKdWX9u0jYHP#ntQ*p;mo3(iU&VDYuDY_OrnacJ4${8wpaUWS zjrui{-tfxYU|Gos+;ZpxlY{G;LbZU_f*JQ*C;NFnLecLmIajZ^0 z;uRkc_5HjSSGY*Dl94d3d}zbavLk2BmsIpI8v(xxJ~zN)PmH^@UlG#FIUuMQMZ@Ap z{xV{{Pb6R8`@5N|{-H|qhd*&=R?wkpveCVV2KAlP$F|MZtCwg$_T24}IH~&9=2K!- z*6&Cy%hqDcoyT>0Up)wY2RolGX(_*XK_X!n=FKOe?aE3az`YsRzF61!l4mA2_X_sy zS|oGZ5=XpyLq(-58(RPVj%SgyxpTva4|`KaeuO>Dc8kKY^L2$G(#DV>-?150{>+&A zTy67$p!xK){WS56wH=*W<5b>j-;T+b6W4Y5?_z7K!;dRq?y2isp3PBF90vX)F+#_M z_d#XlOm0~hhh(sa3T`)#jnyO9htVqX#ah#so78%($5Mdg^6Nca*gESi?04^@K@WJj z-@2lWT`oKA@y8C=y-s;kz7lL-3`c!6yZzB8gxV*s6}-0L=a>DknGjK(2p|6v;~D7v zZ6iHj0C#WqVXq;WuMpdmog&BJsBch%4s=j<)K6RCQ};jZo^)DrdT7w~=+=+K512}4 zrDky6wswpxHxZiP)4SiOxH~3~Q;fmmoJ?_WU|jYe*`ISQ&yQ13GgB{59%!LXk1qfZ zW96_Pn@_UxS86W~_mdzsUoD86=1-AWr+Wwca^G}1vMoJHrr5KG_U~1_xmmlLe8|as zUyQgX<+R|hyGQbzd%dmf^DcfMT?jWhxt3kRJhymc0O`JVc6|L0@;5Jcdt=u-jaWsP zkv~g36Qs=lxMPJ23#ey!Ii{E;$yk_H2>$>OQ!-4>GOQTKIQzvom|6H~n79UD5wp3F z1dswxnnD{;L7E$}d#cf3&5<*zQL_5cA7ltO2smh~7z_vao!L0YzP7Vs&8j)TQMH1o zBExbRBa4NlxgiWA%DCgCHEhuO{rzNfQK39Rtt)bQoYwu;uaq_VZ~0x5yn*L3rXar@ z0<#B+F;a#|BsdTp4cgQ9))p)XOp=zJyHwX#zFN3sB#0C8kI7sz+p_crf1sUebv@)e z)tfhHDT#uXug`ntvJ5_td`=1f?Og3^xpAg-OK2<^mkts1FI$!j)nx#FJ7Om!;aNOu zO5CmtEcJVult+t+Sjh>0-3sB%TQdtQwD{%cB$Aq>KH?T-(vM`nP(8tac$SdB<;Qq0 zccP5ica`|ht3ZU)M*vs#PeP!D?$fy2b5LHO#4SA~Rb>RtYG3CEB^(|lrD{3xAB8() zbP~UVN1zn4pf;~k$h$B0XmHYpO64yj>w!gP2w_n7Mk3BBqMo?2`R)n{glHdr{7Dxm z%>D4FnPuw{PhJUVLP@a>GE(zqK({~0oC^oo)&A>O@@&g(ot2Q2i1IZ+=IUN;;^4-! z#J00M2GhU^)^|C}z-PpFHvWgF@nX!bpg&K(oFT#L*W#-VGH!3koKyQ91h}n5Xe=3M zBO%aQ48nE)!Eb@-e&G@|Wu#PA3{rs3%Rw>KK{3in+nqpW%8E?1<-ArClowM2wL$qD zW|IYceTY$<3t&EMAXFcwjP2x7PlAQZKpUF_c!(1d)sz~S9CsE>__UzM2u;BC7nu_% zAN1=ijaN+6;bKUJ2`dP~=(s!E2tJdfRgWxPn^atBC|MR&eu+7TgPEirNJi6%*;F=W z6*%Qs8RvGGF5Asw(G|({5NKkdBS%3|9Q}pt0fLYG2{_e1wW|6AP8BH?~R2g3MYZnh>q<=)*~b0HzBD%nd*;KQSy9 zcwQFZhQjdX!`z+}ptMM%Fh&MxFX3o!bz< z{pMx-#q6P}CVs|$yQHqwb`ZaT_O9V;=`!Gua_D-BuyzbK!-Tmc=LJ$knF@-Z9Ep1X zSzUCldK9dhwDU;e<)buI?82E9rg&l7LL}RSMEyOK`Fr`!*KFBd^db zp-6+1VTrXwp|a*E+lk|UPGX$dR3Ow@M$UFPeB!e+n2%aHvsV_(C1-Z-(p2H_PJvrg z^LLr2<0cr@#qy9ZoHLi*%HcdI$io;5x8fKqk0y1tH{5O6KVgTKs7$BC$?+DYq|#6u zphrWh@;ez51^ulI*k@p0PbAOL#k!1j9(jj}Rt+dZ(o!CF!KMe$D=|y@>~EV8AKVye z+j6X)cx__MZ`8-h{hl7t1%W+v1bU=92ThR3U}0Xh5i0OOs*IsQt25d1Yyc zm5in5%k=6dJB3xtAt_>hTFU>jD8{MaAFnh7Q~Y%QyJs)k7FE;fp&&7;Gv1{z+M!Tk z0n-yNV_;CA+i3BY|F0^0UynfaI0AC`Qi_Q$2Xrw&TB%Hm*-toRFGBd)25VR(gR{~` z4iyw}7ICX(V}tk=axj2sSZjDLVRV%7b9adXUr}x(1w);cj>JjIXLiD30WQ$;Y`<&9 zx)Bl%zY(qEP|70R9!0$zprSKL%Cux^H!M=C#76m?&6W#Qepp_ji)Qou%gWmgz%$3}ftX!7xeP$!Mq9{f7xl&$ zNydMK5)EJ|z^$18mqFG{r4cz#DYH5ajj&t~Uu^ z9|3pA2=9aj*Ikb5slr}1#8lMJ^rhe|-fyeUV|rHe6^@}FEaVKL&sk>3W!BmIsaEj^W$MFHo`UFjH>Bcdo&-oDJ zF*IqrL}$8I8vm9OhCwUG+=b%+hA`+Y$|4d`?n+$mufV2M@Jm9OkU;`Ke7=s4#VuIe8y1;fQ9< zO%}Rj%!#(ol-kLm%EO_{xQ@>MZ>0JA-)Nd`2`Y;hKmY&={{)b1|Nk^iJ(K?>VftSr zTNKkdefN?7$#4Cw>wDPtj`~!R;iGBt?CpD)cW#V}s@k1t?N~Oes};4fYhKyGs@_(~ z?5tpJO|h`1UTxd9bFZAeQ|;Z)Q0@N5f+f#%foD6zDO=pptL*NT_0o%ZxuL4YJgJ^nR?1U*|B7VvAEZvW(sTxU(l9 zNtHe$_~@E`>iXZVIm=ldss#5Der_Fy-|Kj@Y-zn~n$M5d zLA$xSfZbE{?EVLGXrf@D1Yo!b;3(uUSHIZ8!R?yDZNeLwnm*sanB8R3f0@_UC&>L% zQi28}TyDBDoJ<=uKkMtKna#ACJvlC^$W%@3)MjjIxLj?Ix9X=uCI<$$x2F%z>Vw2$ zJ>0|HH9d8^oX-bDCZPvbSX*suHFi6lCg((lR>oFRzmKETX7zX)Q5k=%ryiakTk&@_ zf~BvdG1Aw6(&fyp5)a0HkmnTQ-PI@Z_a_2Dj^WfSAP?ztL_*0mhn}-J-PvQbxyh;D z-z4a1eK#1k2_ED_OcH#I*#>#(+l7*mQwj;@MSff77SMM5#>>7z_f9zI~_g zU_#ev7cwMNOT%IK~>Iw^6L<|PUN2PG<<9EapwTSrTX z$jwf%$o-BWHy>Ms5 z`;DGYf2-bolvR4a>u&13*P`KT)3Ey0Os6rq6{45jMlsF)_+C`K|N6M@QfqU1qi|zk zv@yd`h1bzDs007E&&TC75-jhZ!=|Yr0Kh+@^Z%Ki?&SEtt$76voJ?4mY5!x_d#!Dy zwkh874bxi?+F245#ITkxLB?&1eMpi?$ZC{TaHm-;%Y&=dpj#0As>+`~)QXGaq zbbdbP(FbmwRw0}Hx~@WHVa#=V?L;Zc#+Bw(S-E46A{x=wRC_hSe*Y!O6M2<|q1J-O ztBndTqN6>2CJJ-Kj_A45z+Sv4 z_G0_wCiV`90pqi$4wpPyo6Ei^-F-adjN2-F<7Fl;<=fX;{CnDr3!%2V50CKghxL%Da>S*AfMV3NrhPXQeV&msNJL2-chFA!*QM(Ndj0kUI z2qlVcW!?a}yQNj+mn^{{f^wM^|Jc~1UVS2dH1mae0Omf|L#N~}kGDP=zPsLR5{WkVz z5t3p?>`lGLz1Vd=2?yZKd|TIn-e}ikSt881{?(9a+MP7>ym9T9>#1lF(Xst!(8QB# zE}wiFGV~lLLX<3sg3AOL!&}K276&?~qd&OvEq-{Sci;K;LYUz22tEl-#mEB0!%Z-#^*dF6)l9k5Y=n3RlH!71 z1=u0;v?UCY`fu3X5$Ny@MTazSUq+|&uM{$|#vvH!kCHeyzk}9cRWJ5$l=zWA+4TMTC*ob7m9H4fJUe ztgb*1FJaw_4kjZ;sW)j@iFC56~ zV6Xew6cCG&=<5sPJ46x{yOF`lXP_+bNCxlUMJlL_v|JVM=)!P}eu8y4B+#PR)iyuB z2UXL@davxB;hiej5}1Q4gK39eiHdTLOpBP9AFC8YXGp4SPyRXsG(2EY88=wB(U3_n zj1Q+r?}yFp`kM&Iw;%dnY{}5dH7Ew%y#SO1LuUl3pKg(C9V{Jq&bb8BNENAf-oYW| z4^f4^#{$GERlAPx*hM3ctYEoc7o9r?HMtiwy?RPS@ zk<1w^2Q1_$L+8}uzYwM?7#p$WcATL7!5ihSzXArbV(6+m%QDW$*I34q_ zz<@)V7;#Mr56s$DASG&52eDK&N1X83tuAKIn#Jlm0%Z#F0oo8w#q?-|R}6$)7#h;F zhfBJ{vMO(4NmU!;b-GBk&9b1OrL#6@IBa8uFBy2IxS}gE!PaDIy7~FrP+I zvq)RT?fi%|aTGv=CK@bkyD)Y%92li4!3+4W4TP6uzzFLAlTNP2R>yW=7bi2ezZO&n z7Z1aMIOKk)3fCr=mqk&^k^F9c9k__J{nWU1>0U&6^WCuMZHUbpJvf7P}DxbhU2INdq+xSM4O1tx8jc-}by_oSw`uHZ3yeL#E5h9)bR zthXd5TQX$RYsOvBpoBt4{u~YSCLFx;2-ffQ#$mrD{i`mNKVx zbF}Kkw()VolDX6CplI7nCa`no2M%GKUQzdX( zPe^D^0JktsWAz$s62_#N%PyRjGDt=NtyYhD9xki3$-P@Pducud2)o-`ZVJlh#c?1~ z_3=I8xoh2$`P?!KqPwL-CdH9UHSI&)@?q6w%3JiHalS9>BlGc?F-ui_R`~lZMP#aS zMi`cMzPaf$ui0zGJ-<13WLT~wZK*#>_Ltpb-7@#5aq&94+w=P|>$mYUmKyvNedpNA4PMp44WD$%X=c)1K@P^BIb-V8_a^t%UhI?19 zbx$};rJz-i-x`-VIBb4XoyX;Vy-W>-Xh-OgcjKSI&%yZZG8x5Gmi02Uq& zZ*JD+>&wQ)#l%XA`%Bs{vQ43h_2D!damhk=sv=c2IN;eeZeZKiwzFX142UZAbOYaA z28p=2u$wL;)-ARn&_LM*viCy8MDNcZ3_Pdhg3A&Tlr&I>5Xk%9+%g*q!?iTO(;rVv z(WR~IGMAZ_1MoZp90gQV|I-MKbXg?@scHSIPg9BUp!&qVhxm~P!PrWq$HMX#NBuNN zrRyWy_(=D1Dl)960jo|T-Bac!cgx13tQ9)#wfoVoWPi5QC89e1$!7@UwWh?|-EiDK z;YuQEeME-fVOlfQNoAY0#1CxdG9W;XcEs57x~UGEe^b^1aFZ|iKE5d-D>a7v zVuh>kEH$La(8XQgf1wr$(CZQHiZKaEP;&Pw~% z?LN^rx+8A%dEIYoKg>DSm~)Mf`VE>E-Xhi8Q2$;{mvrVk|1~h3s0HPu)@7n|sAftL zlt85&ETTn8f08YODXp-xD>=rp(hjW1s$=NEe0%vIKg|u1vf#szA&xp#%Cc;}YDj8h zN{X7&@F|z>@Q>J`M4e?c_`XZc;yZEu#@hURwSMb7%rG)VaOMp11zbM}nuoNTk2E=88Z`g?+7sBW2aWKe?MSpPlX5 zXz4RqVJ^iE2}y%h4eo52J5Li~31A++7jC3RKpZVb*AAIr+qqCcHCn(#xL;&6sdw;0SP{!zDzFYNg(M z*=p_CZ6;Q#FV#OENw*r426O}?dNjVKzV@qpRNsY0u!VbcPhrSp{k|MvTwpHzz?eXz zn;T8$EZ{LM_An-CUC6tDdf+&i?*RQZ5)KNty^DFdLNWhBX4zlDCC#5!_2>A6oLv!G z?cH(Z1d=*zV}ik6HwzJ>bQR@@j9QJAHB0kSv)c1MgQ7G>n^}yJ$vmQ*b7>2kCNOT= z2qK&vzh4dj)c^PNaLLkY*Yj&-Iv>u(Ofs|SH=|{9cXpHvc0v=p6U8LiH+#JJs51Im z&P~UIOBgG*)rYPY$PM*}y7IbZ?Kj@C6V-up^^5QNxAGJl%qD?qMgJKWZ@Y~DGakdW zf&LW2T23)>`vGEE7{C?hJ0f!f1t`Q`zfNWss~sg7f1IRmyAAh0N&G)`=ODGJDsIB#`UIh>g?(Lsxx;yrTizqCBtW z3qT}FBVv)9q-a*AS?FZ1_K2;cRGc=xz07j^)}6AST$T>8Jqct(3T{n{etWpq9@Fj^ z^z9NMC5#hWN3c=+ z+_pneo2uGUDh9&0hkAL6o4MX>v+8!qu-&E0ujIOPs!mC{C~;a(a;W}i=||}XHN6av zzgLW8_!H&8!vXP>Gom3Aje<8N?bV9YI)yk-X-S?4WWFU;^mDz0d&7MLu@lBFHduq!RrjQjE<=#pU(J-u_uH%~MFx`WhQ zu5%Mu0HU6WFRG-hHbkC~PUaNT9w>_wQ7Vdj=u-a_6?I-<5u)!uftv7Flt{gns8OjL z#57P924q~8#iFHK*8|(Dp`W3Z4V+4g)wvi91W`X15h<-;*D=9>QXxo;<7@=LgYogp z77N(GI4lM~X=-Ibq+CITr%cBL#Ji{30Cig&a<-2gQjqFLTB)fHUjD`!{K6Q9I%x9f z)_`2SXz3~I`C=>p#fzU=LhOU+dFSBfHBnyLuL;czHHa%N78Bj$itH9O;zf%`b1S6I z2k$vL2xIfoc#`<07ZIO5Nb1_RGVz-z(o`%?kPGf$DaSQ zQ#tHo99-i-Smn=58Rq*Vu#9G+$VRpECo_~UlcM3NIy=6_(L=C6z1yVxacVq~^pq1d z;p!ya%eRDK2iF=6SwE2S&C``#c*J6jCqU00(hm9ZtG_?fRa`)3TcZ=w=tbZP7TBo& zMQ-;G{0JW?NO!p2%lsw2*S2uI_gO3Mqmt=QoI!e5zY}72pT%112<@0Xm`moS?=(r> z@T%*1{O%Xi@-ffF4=o%u(wtzU4)^b-$8(1(HUT$S(r#}BGZEh&?Jm3w z#_pHBtGWSAx~eliZQz2LZZ#lFXRfPR0&m3C8-$vf#;JVJwdH6%s0PtzQvY%#&+0%& z=bicCxzp)29`5=$e|kUVmBkOb^JG;Zj=O*mJ^RPRVX=1nGr9Xn&PpYZ#b};-YVoDR zO*mwQEHu6)Qzc^o=OPo_d2PRr4B~S*g(jwoF)VoPEO z+{I1cy2wQ2SHq^1SvWzVTFC?HuU}QfBbD;TPQ9%wNoW?#zw|&3DTR9Pp!lN^9iN8h z(1UvlngUhh?8FzauUCN0xXklE&jy2p>B$oi&R2`IMiF80 z{4An^7MRnQSEyDC^ionQ<}ekUj$VPxZ&vi1+CUtAiqIMooR=fo-|BpX%yv&dJ~i=i z&J$$$r~+6Z0&p;C3hduSJC;@R^s?QbpN+NC_>p|F`Jsc{05^H!R%p=&5Kg0J6tA!G|bNyQC zE@_V&%agxF&CgJ$9M`d-ss^Viz@{}o8?7@`wl)3!pA;@rk;npT7$6{P0w5s9|JCLb zHng$%e-plhZ7fagT>iuE8`ZRR-eg1hrul(U$v>)&liQfx(#dg=l5Cm5$y`iS(ea;B zASEpZlT69kdMdLcQ3i5rq(^+#oLBdMGxn#$`kos*x66-wA~z4d zOZvJynb{4$dwaf4hz+9Iz00tFu|cWvM1h(FaG14dHN{Bh@!n4e(8BTS;6rDc8)+U9 z%W5Qu)Fk9YhD^3h2+5I1CxWf#)h^mlhhJ$P;iQ>8a??63H_4JTq8J8nh0#oKY~I^Y z|IHD%NiUTr4hr`{J-Qndl*cWfBi3wYkR=WjN(yP?@h{i?ybLr=Dy?2DVTd)bC33!_ zKpap}`9-rtkwZn2T+#{khV*ZW8_Gy2j5kA@d^Ui;#ux+Ic%13Dh_F_V9KZN)`a(6( z##~mw)pDXOYq&8?2_H=>?-a#E1zJcwAq+2LjyyGa{! z9vyj`Gc1M0KC}5rbEU8SM*|=q-ceu0jb_q}{c6W)-2H9^cU^hYzl7G_W_R$~&VWC8 zM)ptpcy#4?)7wz)9M59dx2%ml59_tJwNU_k1AQ}}R@__tZ3ns4xCpWRF@+Ph< zZ7V?sccZTLRk8ll5>^qEnp`*_TiY0iphln8VbX;lydVu`8gO6ZPiKG!XpZ!>-w8?% zRv!ZW{fl1b5}!7Iy8z#w@vqOO{wqdz?hHAa`qPE8>65GA(7JjS1 zn!&F;aoD*rAf;uzU8n8Qt#Db*8t0nLN?&VEV_0+ZinGDz8tGKt&0*HIjnBc5T+A~* zQ+ONYcz6cH-|c`t!;_=p*s~jHRyk8J3~&x;0esofa52gNte^s7@BF;;WZ!>sU z5mXsyn&4ov4Sn|w6NM`|`vI`*JXA*bsh7NjtQ8HXr9Ks<(IQ8u>Z6vgn!l7^KeJrv z!cS=={-T07cIgQ-vX^~Ej-Q_Fv(FJ?s*l`XechOYVBqZZk!c|(ul zVu+`U$dlrfOF$38bc8)NCYVPB5p^xA_Vg>Y9ro+9l)AW(c|iv>YK#rQ4!A1ecYWMO z=jG`EACAcozk~unnLxuSR%1Ie-tz&IAyyxdtNtIuyD{u{yPJF?O^`YdV0Bi8z%Y!T z`gD`@})FA5#NpM!$?o{Kq`b&7s*yKQmWO3#1JkJZtdohmn-Egkd#;d8j?g zk&41bFaAJ&ZEHN}@WOBGvrvv<_Ua*~a}0b2Pp7S#W{vSk0GaNTO%y+J0+&)-{N|{= zm=MWf`|DuOonJ7-HNdV=#s4N?P55(|Z9FWGXE65Uo=|NCj19+F8R7$Sj2na3a~%P* z-hwWP=g9LmemUjk(4(drId$lU21yo2hn&Z&IgmIQ^X#?uhUGHAZ&5Ia)e_kuBWvX_ z=mYAM5wnoD=onzJ_&&5J5{|=GQN?taUOG$TOWpx^;r|-me{;_AtxJPLn%dig|q%m z^+CLDPo45>#PzoiVk>r?r-Ld(mRIYNZq*Z7e4nbnb#SqTi!FHcNZT4)6t$v^toexe zCGi09!~#CjPT=t@PE;qCz0IbUnt_wz!A1Yj z6;ix|J1llFf{*;Z-Im2YW-zf z2C&84quiN4cB~wM>R5cWfhHNYT+89E`D67pW1YO9Nb{=YUjCjpz;Y4w` zE<)?vlpD;FyX$n};ZiX?B9r)~yq#58KPGW-@=og|Y;e}#Kll)Q6uq}^OUTYZ0IM+J z6giBLMSPknqHW65+{5F78@`_5zR((hImia3g_uX$zI9@3K;3r*La0|s1i9i{ilqQISxCKy!w>sAjIm4T2qO5s#z8mv9CaQ11Diz>B*;lk3aU=N=2vTN zCjuY>zh&SQt|0Ui&*D;uZ%SS<`71^^RLbB%C3vU}7P75hpYlZZ&OdIcEpn#Y@xDKlIY zXkC!Gd|9zWmQ(ro!?l7SR8n1Q1RpuozZ?kH4ZsSHw!bXPCbdc*H{XA2R)Y zKMP&Wt$)cCp%v;~v2Nfzg!5Tgl)AsRh&4+=FAS;mt5@|=`#|c6J)!xp|)Y!$+-tIrtYpwe3KlKXd*Y*RGlYgYD+NDd-R<Ze^;7Lh;_I`}-DDf-~`uw0D~|U4sP}gn%y~!0GQ&{1-OpR}e+}i>zn2 zn!&O1>+yE?uFdD?n>mh|X7OqfO15`~d3(|96vkE+NbR-|iPpkaW*bCaR)D%4t( znKBg_%o^L@IvR(<)yyHpK({KXo2hpUCrwN;JgcX|SOtQ2q+LG2)gYZ~qT?ok?K)Mg z_$eeR>>0ZwyvS$3vQlbigQ8lSNrRBFiSbzzUEZExyk}}E&H@Mhfq*|K-!lipRFSYv zy#SGO$jOgTAm5PC$+pRD8o_P1o4`8#x{bHkM9%;4x7G&v>B!N>P$YYp()q=_E=Zk%Lz*_Ah&=qc4|{O- z3Rwb37&1o#E&HW3ECZasaO50^qA1m#*9^SPVzF@G~`amHt)E9k^i zBWjAI^ugaJdFEhTF%Ktfoo$FUcQu9x0=}gb5sw+%>9?5S7T*zgGl=P9*T9JoF-<7s z%pHf4SqvR8!klNk0Y%a`jjw2*46A@r#^wddgCy^!pq4_cKg}#BA_Zo=5*lcw(;?x5 zXqp{DQ3;@!@^h@~h6ToAZ9`{SS18zAm~1+^%v<9nkiH(0M<{rVC}(8CvX4W=kf&&2a80ip8PeX=3-hju z_kD5=L6zgo^TJ`)FsaLv+S}LM@$|PMFIYMGpjTf-;*DQBzH;%M8>hOYfb)jWyKfgi zRxc)J%BnwrBe5>DRMu7*`&84E^GoCMJ*1nKFOal&i}>@fE+1dSck}bH?|Wg)+PW(n zCJ**3wC|d#x&~FeQMQkN&GsMOLvi2X&BAmfHlb+NW`CX=_=gV527?;yX)x^(H2LD= zr^-OQQ-%c%cMT<4QtfFBlIqPx2!kK)k^)gZye|8@+l4YwUvk)qqcY*gij`pO0h-H{ zZP)KyeTFv1?fEfp(IRnUHxS0tL*2wketE^wWytedf9d%HwG&kQ^7VF!NtMgAcN|cP z+!JVjuif%t&6BB72}+}r4~K-k>Q2U*OKf7_yz~!%?(yW1Ves;PEn8-Fy#?D})>XAk zdBqo|gVW)?;}5jr5*$jL3@j#CoxjlgcR)8=4lG_*L#C_wVR~!BmOfy;ARqWu;cOtr zZ4!$k1gpm2fx@M5VM$}DTu)2*o8A9FC2L5_blw@b#W#)4&OpC>pKdOH&YBCL6l#`+e7Pn9gci=b}@>((ea%br{uc?^K&m z(~6Y_y=6vhFWvF2#dBu~v^MOJ!sNxFf?7oCrW*CdtdWvxN!&KhwHHnul;VHW2#EfD zeh_k9`$yLWLU;@+*&3iF(VFj=uL@J*+g1eaC{b;3=CMV;7{{(JP;hH2VSXfBpY%Vz zhDqen?fr6P%GWavuOr##@*A7?Jzf`f75BRS!K7Ey=IO0nIWmHetgjlpyJJ7V|Gk?C zg+bx({M!`N_y;!n-@A#3shOdxjf!J;eU0LR8>XiVm2f{gnpybyaTw_`s>~n zOYTv^%?ksEM2Xp@b5ID*g&H>3>znpnzwR1yc6R(N0gDag zasqr>opjXKlk3e}pUJ?8pvkn|Wv*9nlo{L43iwda{!>ds??3~0A7K`P6(9ZRQ%u4akTASaX{ z?vXVaJMEOYKaeG4_zORnNGs7cuw&0e4pxGF_e6dY5f0r%n3w}i;DIvcIg(7~$KM%3 znu_$mntn#Eoc|YPZI^^dcOc7jA4WfzGrn`HD+QJgRjPf!2iPV^<}X8hFdUm_n`EeP z@jEfy5XYW1hHN6}3Ao!h3bYaW$UWwCDU2REx4Qu&;t3R2I~;b)26?IHd-pG|k_4{-Z&RP|vZ|5guKnxQSCCr%Y*x zX_nZWn;=tn9+Lx%0~L;Ar$HYXVo)i0N=lrpd!9M)fB*C%(nK@Oo>V>=y2>FQr4RyG zOkD1Y#gVmG%d(A32W9waDEtF`*xSa-iB8@Oc#=qAKta#TNNrhJ4-FPtd(3AtDIt-S ztiYRY&EyIn^JsYd=mpqpw$kSh>k%71eb~BtFL+-7Vb`IEp9?ul4)oyMLvTAWqPqFh zd3T1V-alrjW45rj*USFugUfc-Y*+g^3je3EBTCCEcgxHcKUm@S{I^KF#C7z(L7~Ow zu%~!@cLz<@bp2<%U^)RJMFY&ArIW{-^h(f7PBgA>pB*Iq*Jx~BGd=Dr2FSgWi;vTh zLD3z4-rFcBJ!Bni7tz@0pThE&On29~8_UtuI}?}CO$qitAnoc8cbm7f?m>j=>h)D& zem1w-7>#nj!28>Xm9 zWE=EE67@gi52dtAO&tvk8eV8q!wv~2;Pu@L=+EBeiN#cS zZRqqT4ByVy=jZp~OgN%&hRK+nYN?wzR1>7CxhbrkYM}O3kYX)Ud=(ZllrM^EN-gWL zm%X&^KM~qskDkKX;$`o2Y_@8en@M8Rvfe7pk>Uos(V^el9@4ZV+WGq-h9N8g8$6b& zdr?bO!1&Qvb8oK)ig7n6*vmCmyKik`ygHebrl=D?$?^BS|(HV20>bPty88s$ZU!v+{(HfGUm8< zytb5tcoCenXgIK^rGjfAJU(0u@w=Qhc&@=EfVP1NpL)%pCVA~5mUq7Fc^qs*?AS$u zp}u6v_{?K@ThMaTS$Owl@SgE?v@R928@~BvcT>uSbW>hb^5UemH;(q=#}~ zU4q;vX$p&C#}kBk7a4c7#{&Fnn#oW|AO}dFMmWwxgYcesG7}3V{*wXTgJoDcMJmDw znn>Rva@vHaqzml%MJA%W!61@iha}zpQcyDi66cfEX}ehCRCWj z7Z}s;>>eCeYy$Waq}w+OuV2`-#4 z4RFe0ppPZ>H)#dQP}e$=WYACA5pRs>4r2#SRPQ^Sw?ot~s{x!bzC46vF)~4?ENPG? z4)nC!U=r^o^A*gHCdawbK$wP@mr9<}0w>IoM-rR@H1{&mL^6qvEWVi9Wgnkb2$2VX zkhAW1#8Ry7*hH+uG<-dZbaM)K>bO*_mnXy1BwP^K$D<#qx*+dKjzP=@`xYb0S8O38 z@S@XDZTHeqz}iFYcWZc@a@4IWY`1-P#elfI1Ck$vx#dW_OH0negBtX53TZBSG3Whq zZWIFK=B3+ZK(nxl}j!9d8Sg4WkE;ZL6p8>93u7 zub#%6X;8#%DlSML2wB5csf7#d!SeXk2`O?YI@6`5YH@*}eW7@HSUCjsS>+=70C*yQ zf?tZR!*&^Ih&}7#0(llQZs6}s3+DB&j!qj}b5Z$W@x^yI zDuz?heO!F!1ABPzJzywov-}`dd{7MbXd0xqE!U!N_R>BU_W77bDcS=#IA1qba#_n) z1!HWc><>UZ6jmC)kgpp#WefyiuIuCAg%r)Q+9#;Ir3HI-QF(xrQIMnCfFA`3%k5 zv6k1@a(1=GMoWk3X3j<0ZOUfIU%8xV4sG@`UL9)IGdyJ+SjuMXGQTbD&f8{WqOsK< zDuH);^Ky*MN*) zv$Y1Eb{xz`?g)OrvaQpMV=$Fxj2=Q+*=+;G-euvuc~)YmeVKyb_NE`LeNPxZl(OZx zSIFS4rt+X0DhqRET_+JQf943~ijKT*F#ruF@SQ zaE%zkZqAj5+!(%P%g!{h0K?$`<8mqJx`uLP#k85n=t{@JxiPNTEHD@2wReHKJVVr2 zqRd^3y&l4+(JDcTZ-N_pqO2c1gG#c_qK{)JJ_$);+ z^`1%uB4bcw2WY=bi~$0Rj?Ep`8-dvBH*JU2Gu;-6guOhH2@hh2Ux%gHJMO_M6GTho zcVbzuoHbfl^?xMVihuh>78DSYU}BVRxr`=``jB`<_*d_-RMvbI;{&IG-qzNvJ-&QB zoA0w-fn>fTbGjXhY0Rhac1T2WI*~JR%kL7cn@!r`Ln^X~7-~9F9tFZ#a_IR%@U7g_ z$P?d0N1-sM&$uVh7v=QGZOtUy15n={zqBV?hIR`dW2Tadl1ho^f|37r%REX`VL)eu zaG!V?NP`=677Hbm&`uy$8M(CJMIBu)u<3_Md?XnczWc|;c~OEPdYDq9EhQL=?ru^l zE&K|kT>i|Qs)gh&w!7WvP@KMbtu*l{b!jPjzAq=N7@~HtZ0Q!VU+1Ww8HE zpyfhPUtb3*@a}VbVpygpuXRa55)u0MRHZk5l_y!=xAl=^AL`LJY_^ymOQkT)5!Z=5 z$an)=2d2(_O3lKugD@$s#NnNm1b$Do7vGiO%L+`?6D#RmUk`|p$0QBKTSZB@V*_RK z-UvpFS}fJrm85D^$^Ofj)r$wrvz}f4fadwYfn{N?b#%ECbUW9H$XLljbymh)*Eb3v0-|#g~){*;K4X#A2B5 zARjR*>V3!8PO;a1KMx?*{p(#L6S6^w>#;;M5mYpvHXw?BC_ORjjR1eQXnZDp6mwzj zfdzFF_sv$N-Ccbs*vwq2u1hbp#g&^q2fg`H{{jo$zQzEpFJdw|%YBu?y~7?)d=$mt z(O1SrU>T&WZ(>TElKGCGNtsg^ub3qCyEbX%cdZ5TMGIs)3WUc_D&Xsis~V+sDYcCb z=6a`4`-Th>3xn?&guU8s3<-xaGbK%f&{rPW7|6BxLf`XI8;xHKe(tyImx>E3Pq@s3 zNC+$ATw`+ZJvvj;yd_|@0@i=vbycW+?Pm+x~E9y~Wes|F}%YeKAzK<7Q|;lzDw zp;9M=y`cU>j~XqQXE%_VZ4%mbX26@l>85Rv-07GsF|yZq2qRh%m zQG&#Slp5=M$zP`;n{te=Ze`8Fl4lmAk_XZ?Fl@JF7`=U}EIo91){?16HBbI-@}eQa zo6ufrrJTGr07sLHx#7%BovccBx?`W%PCcV-Bdk&@WheatwS?uz!iYT>jE2p> zchBPpHK!&-uLy#0`f9g&^}7^Fg8OL2EU)c0?pZ-aNIt@xf!{G11s;JZMyRjw$7rwe zQdE>7=bkNMQNK+rr>!3BJO8rz+wBY9gG6I!pW0Je4HhZyWJ&1fwyN6bt8_or#UtGtR`qM78)5O=X2&C^mj`XbyZ1w6HNu7cJ0#apT3_*Z~y z2yWqh@VXBhpaYeAEmE-ysv@a$wrq$lOZ(MkfeD2du?2r5C{}suyCj0Cu3C5%f9h2- z{Y~B)#4zU`y;`h*q{iajR<~nuMp3dwZom{FaFwyG;M$|N`HaIWRt;`&IwxDC?`JCH zWrKm16bXuL(m46m$b9u=B~I@m9>AVZ^-Io=k*@PGlvN5q)m$_~bxI|i#3UTLmA5tD zqxnwFcn|I!3g|Zfdb=r)C)kEm$J_+j@bx{JdbC;yQnT?*tLpIVn!}h23C<%$Th}$j zc(o0bGHeTFSocNYKWG)xUR;@+(6-|6tDKFcV0Z{0!kOZ{h|!)uz4 zQ!9P^@Qf`>sIr_Vw5IhI>)*u5VJDW}=>KMbZPdmI5v$&`4r`*yFS0}yZe1VVpkfVf zQ5)@Y5`iS|=C zo3ia+bd+K3;>SXF-pD>dY3-}_M8G?T)HcOJ*cN!w5*uWn6MBg~uy&Y^&I}fc##^*z zIF6kaHEp^{lln5_vK3v`K<;nkZqp((EpMFq>I zG$L+*&P-V?^K|mXJzGKodidy7a#ZM7a-Fb>jDpXLOce7$U8-M@;)4R;=(8%VtMxs# zJMPLy`f1O=K>#u}a@Yr~)P24$z9QZftW0Cp!A}^P__b=r7_W#|zPaD0hK_*C7NqLc zrC>nGZYAB!R68E!8vWe?X`NHCvb~~J*mgU_JD}0-_N0u_dFms>@QJwU?569~ediPo z2h&C>n_m0}{J;BxJ+UjUfPYtfNL(Nwmj7FIE&ShyOr;vy&e;S(vSuLoqY zPcQo9-%c=+UqiS3-|zCMKi|*U(CRWNb3Dh@v@z>R<}|+tY{@7^HN06Z=)L&*3 z^^6%&RV?Bs6V{fb@}AN^tB8|HOP3sx<&lO#$tYq-T3F2-g@h;^qcok2g1zh^s%St@ z+(3F{uAA>b(3Zf>eCXc**%;-BuR@=b!Zt?4q4_J? zSy*zx6lgRx&m>Wm_(8%38NS+nW@FbMc;cgdUQfQu}hvEi+-eKM1{653z-%!69fhQ=PvS!ao z>at!o9|k-NyR+o4@c~Z00k9=3f9M;1R8b{qe37bALW_5NjuwlEb5JqW<=eBAroFOY8gu_)4rq}^e0_+}R8;t6l` zHQy~r9>dumM2kJ?a~4nY{zFQUp$}e&Gb;>f$74@(h|lre%k2~-o#7D}v&G{eqk^(g zl2>A|c88T;e-bDSoT8W?7I!;mUU(@Rjui?ChdeeQcGC#g#-7S&+`{vdc%;N&LMP+V z;d;kQ?pw7D>zh4w2fR{jrjO}AZ@|L*QKh0l_#2Wj53+DRy%K)!wf3D(s$<`zodccS z8e74rIcDp3s8%Uo4gbP8-gHH~1=NAUY>E$Vz;W1Nf)T^!0ot_12`#wxq~(l&blU(q z^82R6c;10b%B4_HHKtb3L8GYAWGo5O5Pb^~kttx3ilL*mMaE}P@?Q)p6M0q0Kw*7m zxRZ=uhk{uz}AYt3Xg~Z^8H@v=hiQPOV_2zgCrPLH&H?lFy={w)9f8?#t2Y z8l)iL*3*gD#)B}uxDCd;bJ~(|-3pja> z_mVfmk0BUO-dwZ(g%fTJ3mGx?1(^`1&ISqZyn4~2jSFGsrjM3g7-H^8c!G|Yt~eIm zX9)9c(Ji56x+dNU8=@W|)w(0PdmE$e{1jc3h!ksa&*;%1^WEa`a{hdWtPLocAp`GL zqJOi5)ZOJh>1*r7BB&Vk8>s%h2xAQL`d#(?k~6|+gsUX#(K{?HOH$RTjPByNSWlFQ z0*!KhG4^c2+TIH)`73L#*g<=O4_Dj8?ccDy;Cj3Odh?+Q-PD-UTW0(QsGV|5>s4|5 z4;GO0ljTkEhQe%yEO2e9l2ws+;ldrI5o!~a>`d?~A3Q3I`ZwaVj8G+eP)U-CxY77M zhEwiSSA|0ywL}q+#uS=Jvl&Fi3(OGf)b((;V13-GL$J-|@84j(v>?V27WJ5yqI9H` zh$s6`^gmc|;oQp0E}ROL7Rx?UwR#5$;D4hxl1fjN>1CI+S*s?} z4y?)Jem*LyrB&TCJ3VWbl>FHMkT+VGTk|}56>jWH+4jF%A;te)7&1dET}ZKFhS2{T zVph1HA^a6uSU2MKq$Z%&!E07sU5Y|Jl3A{aaZ^zx>jv)f_FlB6QKwOMcS+ohg)5T8J z`#edIY}O5NL&1PZukfm2U==5>X@41O_?e^w0OkL4`wTU zuWtc770g=99mv+gX2*Z)4ek-4#1A5!h55VBUmameh+|xE^XBDN1BwDmd$eGZNNZGM zE(y~95(iTqCrwm`)v!K%9K%zgUGA$30mdjmkR=0@R`EKJ9NMqDNaeO#pO}9r;N7i& zWnwHX@9M?5zC1E@a^|42Ly!AX<^H8%xQb;}Jwdn2RRH3S-Eyw-FR#S)AF9E{;WKWb zqalvY!R!#!H?bXB4?Lh_>0~%OaMgBKBfcTH)?>xIn?1FQ1+X!>EEZ+%SZekld&)La zWLPbBP>R`?#ZF$N%V5)I-YQ_h|op z1=ZwMe1n1IApK_Jdhl46uNIOdR*!0&7_R;H+O%k(wF{#{+2z{Bl^g?C!G&brSZ7_; zmosbPu|XNTt7|LmC@fIFje0grPkl{~(bIw4(H7E*w{B>Y+33OGP}sjX#w?6h5Gt9{a0q;U>$tSYP$WJ`c6#yG^qd z_U>WsPWiHTv|r64mhqJf%WYLN-F50U&<10IA=B{`{u{%VTZ7*^!o?t3R0iMeL4OnwdFj^szzJwrH`prpLdb z4F?fwwA5JEbhJQH`BL=bJI6^%rW8LmDbr3lk7j|J$YY;}`8%J_(`h8H!LMhNVsM7s zc(rNT&+GlM-|OyYT{AxSL$Yn3X!mA)pnpG+pacLdEY_@?c9%lY?qdpy2F7w+{m+5j zT@yJ74u*k8eS3qk#wVEYO8eF#JWci2tXWjM{D1>QkfDyp5wo9vRSLMg4;qee=5 zhz^)pg6o*{Aw&;+|8q}^@VoP$2TS9LI2MEk90mkuct%;Q8j3&A0rzDvjoO`xFi<`z zya23I=TvYFTSCZDa#C2rFu6~qccP7NN@@xQ@h=q^kr9sE=F*%8`jdecTrR38^%p3a zu`^cWC#|?r#1-yiWWcioWA}*#w167Gqa_Np0R{_vZ;C9#+1r`ZJpFKgAa^!{`PabGg3m=R{fV)M!xg~bm@#OW(fl#5^Cw4Udp#z$9MGKw z#~T0%RW@sJ)7gywAojIyv=_TG@U010x8N}szMaY*Zh7kn_dwS0CBxBgpmyKr58=;s zwm+NMMV*4R=M!?l445-SHRpPHk1k|3O&)b&%0p&3vfrC|7as3+f@GTq^2q#H#hY2b zTEs@cSrvpIO?{oMOr0#D01|G*C* zYiGir+4|8O;K4EnJ)QCdmC<^MfK*x5m)WK+@$m_CFpGXz-^!*xq4Q?ON0hi>c7I>J z>~21_;mAuo!eDNPlsG{s5~n0&!wQ=_jm``QA+3x*xpC>r0QBBhz$$q_5`$oKGt6;4&i__}9&W)ez2gj?v(B+K27nxG%dq zy2J$Rcrv$Bx17#6#8D*lI|Ds{tW(qi+a45~0eBZ_ALufeJuZpvjhKW$vokyw{r1jn z7Wz{b$9)~WTv{H|2?n#9x4+Z*^9b<3H+6GNt`L7#+k@Ah$8O5WcK4tC^zgXn{#fVT zt)zT3$XgHVwM;i_raAa;vXC5L%j{=1YY84nTn018kWbR+`5kEvTTDNQJ~Bq#7EN=^ zXi-Ayqi@J1y?zjJ)Q2_lc}Vq<-mWujG(qv)Fv+7h6tRiTJ3b=dBLATLvK55_lsqGS z^PB9X44;m~H>H6a`Z$C7?co2_#`m+t^($6ehHC*DXxk2Ykq?A@2O2IXX7p(E7PUj` zWf+NP0YAOg59cvW=IuCu!h4ZDuvdP({C0wtGUfX~pRYpB?%fQqentl1KvB7W>rrAK zC}h1MgK<9IVIN`ptW*c;z9U1lLAeA|GBkX?KS$}_BJV6B3L9KtD|$(lSc&ux418aR z?U~R=Qb`7`_&Q1z&rlPkqyj12d`RMotVb)qe@0ykAQ7j$q30#&M7Ri{cme*=ZPuU- zz=5^abk=5oesSzg%c2$dJoo-eB)sK-dCBOx9@qh$rywT~VZFjjln^yyFBmqc z>)*1NLlOv++774<8908zclf+yJv%67pp2~7r&(yx=lj&j3XsyU^MyL&To;BAGjseH zyJ=JcPkAE6)kUHMg%_Ajqd-PN*8c%ESgMow`h@z_<~EJWn1hm6G*mZv(<)?UBqk`K zUR%S^7)|yx6uCiPhBV<$333JlniA5lUa!i3TJnebn-5QEtpz3~TDy8<|GTK?Klp*YqJZ8evgoXgk0!mbD&5yirM3$qmf}4R)J`5TBJe>Oj(;aN0_w6sIJXW2eVJ zJ6BK_(pcw#cJ5WV1|=jTd4u4zlM4>pZXB5l&g11U1{p0~xZwl@x%(11jjC&{Ndfzf z+-fA|S8}R>#_)yAT2CZ>7!D*ZSl<_dYZ9D$-S39ZrF$+@3m`uE^rZT;M`Wh4kA1V8U9p~8k1nErJ3ZE6wBh( z>ZpvnD6?N5_GJK$Icen)5%r%+IVanyn$$}PFSO-?36{5($&3t> z^ne93v#g9o&hxfaYzaTvwT$|gS?^CqjxM31jLM)yQ0CN{qnyF)bwT5o(c=|5ZJmgV z&?;TS*!+4^q8dQ;=}j$Q)uMPoJyj`*;=iG<)Rbqiw**7QS_&hLjtdCoBA)`OA&ru**OXF{WD0`i!-tjxkK-#DpB%#@Wg3TCJO^Cu2 z!-%WJ_DTI(;OcgbEHq%bQLX%cC_AU<${KB3$F`G-ZQDu3HY&Dl+qP4&ZQHhuifvWy z`u92QwzJR6zV~%K&4;=A9O(TUJBEk=_{2;}_!Ofa_uxRy^2RgEtn6(8r$7&i_ay5r` z?Jv6yY$~*7w)EZKkn=Esf|f2&dl0?4kQ3bY!;<6uhvK#y1)jH!?Z^RKi>=B?5aMo+dlYSA)Yi{Je z*QKyf1iTD=F zxLTcn#@A3l4o_`)-T!fHw*^&9K6(AAHy2kQvdm}+OQ5;zQOtLte5`!IVENU#dFFQ` zw;Y`tyWvkPTE9Fd?w~}BBt+QyS5X377NORqM2$&$L0JwM?${g_-DEA-xP zI7sSsf`a)*4P>MdqhpQ)jDeC)-jY-Y?`jH$eSp2^SPh=pQY8ED9JYq|YuoIBF~!o| zv0?;r21;qHgP4WO^i4g- zvkZw6PWyt3f$npIFiG0GQkGcZTM~1CYeNpo2lL$`ym5%cJ;gK>Li~!1b{qFrD49^a z;bU!?KjoPh5$o;fA^QjXs zcc3IyvTEj81z7M)y1)hdkJE%cbIzsM^1XR3Flc26pDDvEv0Gqk60}sy_nb@qd{3)o zvm-syRl9-^;1gy9`Ju}Py`$-p5Z1`?GBuCH>>)3R zvI+(u_2tk}(egOhAe9{9UOAnc+n^c^N7&AcBE3DX^rpL?@Ixy< zfB@nfHdUVE6<#rtXKO;q8^zM2w%e3v+Hko@^CZv1NcQo*y`jYsPIH$qsX)x`7gzay z>ST$@N1wLYGy3C2`VmIIw4s}Fs;24kLmDhk6&D%puZ4s_x%wLArE_sk!xdm?o)(x2 zwruJasL%m2A$D~L8|atr5m^gDEK?*$GNG-)kp$t6hz9^}9d3(kuS1mJ*Fc#DOLOYv zDr#NslBkD@k+8hzO>Kl#a}Q+)!ICp&7IcH*>IXE`_Y)QdMSrBa!Z9@_O|;}w^l5Rr zH-&5kkxu@Jr(2OZp-b>wPLCbq`&)acD{ zCbqcz-PPgB)<+fg@mq|pFEjjG+O3xadGy>O3;k!4^QVlNE+rqHrbu6S@n*PohRu#* zi%bu%dC+QJyUGdtI=TyXpb@<;+ZLAO=+Z!bRPn$)nK_RGy1mmQoc_e&y?X7=-=mV|yP_*)9@w#8irg3IhffUdh)aT9C^z3Xg zm8TNX&0TJ1;#d|EIj)s6%hMi9u_b=X#=}{+XseTVGulvsIph%nYz*1!3)vO2KknZl zHQq+LZg-89;^Jc^_JvqHIv9=&)Pf-{E@3&R=E$}alf&dX$YM+6W;`pRQHYiHMG@RS%mUbbI0#yAPLq3je_F0&=zdHHJEbZLCqI z>|1#axi9k*YLe&2f* z!u{7a&_pkXw>)@dL=Q~04^M#=7_2evAwc8bB4(>Tbc z??`*;dj4Eb+Q7;@iAtrT)){P0A^Sm8#qv~S&!d`QeT4opsChN+A7B1=fq&opIj$V! zX#otr&j30C=l|bq=ii*YSE`D3n}AS2jo)y-yky>pMSU9GFr#LH4M@XLgo={D1OU*H zA{vmGggyB8DIdC~c2eJQY4o$N#GuFuQ%9cNDmPCq(W375uK-PK+}TQDLAhd|GV z{lnjG$fPY-z;|XFc;HiLx!tzEmfC)XPGgz+qin(oBwxNlYTH$<_9cg%?Z^fCLoJ{t zpUq2)vbAx=#TK!S$#vE9wy2X1>%w=sha@?N>PZZf%fLNe3p}QQXWOMBcqCI?=b&Ko z51ud8P&xHO(FG)>jE*JuQpZo8Q|WKI^|pANnmo!N#+#~ImGm_FZj|@5|ICRd#C-o z;L18!F@Atnhxf4L1oH3CL}(crThpxNRm<12WnD)^ayLOY#ZZ??-e#9v*Dv9_>{8lz zO-RqQ10~2DcmHPYdt#Qob{`W?ubLYplt`^ip8bRJM-r_;#h)Oo?Z;%YJafk0h?d^o zcq^DVyMQk6N0UqjQhZS#=43btK7_=Vh9f!eo&%BzQJk531T(Y)G0_1m{k$wW{xDfx zt-LUE2)J|**WV`!-UG*DVY&8bsx7p`x4K8=@gUlJWew9aW@Lnzd)Rg;;@=D!?lHKG z^mRNK(nwAMk-D;J%$(A$2YtlX^295ngU&y58w8bf_0)hX!f8nmhZ4a7$uiWz^n#n}Pus8iq6|KDtv%)2K_0myFEio6TqN&Pf-Q7SMuMf%C?@=v%&ymH61>bo z#NO>#vBSS)uWO%|ZGwsxxbO-`Y^ORw3_xFAx$G4q&;c)4wXE;}a>8U}nr|q}A+%km zr^wC9c3<2fegc0l9O!jFJ^^9TwF)j*M_7smR1TTsO7D-e^?q%Edht@1Kz0QGDlF=BaNh>cUnt{ zvk~$W&3%Ouy;yxKhD#;FSDuS}rA}rG>ZXr+X6gIEeTVoDMEjGqFfr6MU`Ix_Zd|qF z;=$3kH#i4x9Lw41)Befpi%ebUEk}K`ia{J7{?<{4Dmk!#F2e47<`OOn4*a2uMfYyb zZ{1uYA?;rXhpan(>a`CI0&WNTP3KuHMnT`Zz9$uMHh~qNT<@^|E?7Q|fMF~I5YRi) z|BDe=N=;VH!o=-gW9VjeS-VY6K=`27ke!0!mio$&VY5ioL@PpbEKxl<>5mE{3SMOj znGgX&wCMN8OnjoOAL}-Liy&S`w#UOC2Tl?_9o^?W6fuu_s-zPW_^(rtzwQrzYJq)w ze0coxDKCm;jx9lr?r2#ORP;-!C@)~7G{+t3jG`1y{Rj_>DkhK0%K;9BgyA_Altj3S zE-I38wcJFR)dn~lqa zu}6-GqraD=s6SO8aws-S6e5(t5GDG!RO(7xgudX@P4m_;4#!>%mS>g28>osW3wH++=Zbtn^<|@XZ-#*URd0jenfv+Gy0#ra7b&qW4Qc4e`2=Y(mmYVBmNM%`$ExYH6Tq3 z`w2QgU#_8kYNd1cn+NJ$5jknTmI?1P%oW}|LFx(-wq7#QdlRC-^@%ksIk`1AtsKMiHlxZ$*_)XM*Fo@gHZgX@WItz*CIW_$7ucYkwCl9qG-(Iw3 zJU>510a|6Zq2k4eK+SWD4G^OILZ{M;<L z@yN~U7H9j|^HJ{V&RthA%D3ABjlELp$J=n9#jq2f};Ig)bN? z(4+S1b+NR)v@Pg2pZ>wWq>K5QBM>1^YnDye&P!+6Dry!OfkYlfsx)v>qvY6=MAe9J zu|uG8z3eCqW_NjP&!X)JnzwwY?QZ=_PZ_TK0 zfqW=vZmm$u;u9H;v4q(cQdr$nbDOJ>YvxgX3l|p5sBkvZ^4Zq1!J*8Ad6x^Y&13P= zqULa_&MMd*+-S<03G%YB>uI+Bv^c0n<{weg*^i+*0>i zj8s}P-EC`|$0}QFp?tz_llvl#m#sjE>&D^Z?mhZVgy?xl{ehHu6R11Hlm?J zjcC+C(uV^mrx{I@eMTTzQeE;nDxSWRA^-5)9wtCmRCyjfga0dLONEYwu2yo%QBgl* z0x~E|p#d4xkHH?IT1|g`y>72g+4>;Z*T@)=P_jl@aA5wTArG>FYSgCYj1r#;ysT0( ztQ?>Swu@D%KWCNGj7~m60&W!z55v>g@}e?-7cJPTN|uAPReE;Xl52%tbiFfLdFi17GT4{eyLT^YE3-U8tl;5$htB+oTSco== z(^rw6>Q&@&D`1#6ebjm^%#kFwULmpOL@}Hm5VQrYHzXa75wsVgNwG^pzfGZ95j{yp{ zf~T<1CS&xW2R$eDskS`nWP>-EM<#9|Y??W;C(_>aSVj)H97{)vsd})qY1=P%`*vf_ zI8k{q4`$dggaC-pdwVidQGw$C4T&|)saXva=ExE-=W%cz{EAWV>iwVhlRzvdq`Ru{ z?eK#^gO}|aAbNsGI`+f@wEv3A#r+eN!>1R2IqP=co*j4AN_KN+>vR3S8eKe7v1=)5 zzO7$F?_u}V$dW@}ZRY+WU7);x?cn}pFu`M@7v94dM_2l6<>VAZHyyVux2zX=sN6hu zYTS0@yY^Qi4tLeO6Vf_N=Vd9W&ZX^dRK|_07bFCQw%Mh3>x=Q4jGsVE&Q(aIgLYu< z<0@NADUmp{zLMdq4R?+{;54LH2;1`)4)=K<8Ju-KKK{+rjW9EGK zKv_3eZ*QrwpBu=IX3pTfkH6MEZzdSMs3Dkth>uEGFxR_(&p8#WzB2SJR@CnNTwHys zGiCOGd9`I7%;x;?*F*Ggb@6~@dA2HVxxOYypa>$an3z(X(%b7i?VkruE5Npcg4B8w z$G;}G4)%Th(SU%s@GZDblHdmzk$}0hc2`gN3})Hbi5oBkFYfeUecyb#r~gIxckSZV zkP{C=1On<41_Gk_kJ_c+Xyd&Y6=gAaXVCLL?Em zepAOQWQ~-9U{LBah)C1*8%9XHzM-KeG0GwbnN+7>p>8Cw-AINaBy?ij&)bS z1T1+baW!-Ff&C{1k~`Yz_5|cUgkdMKaKVGXKs+_{(7M3#>OW|)(@*#=ws4(gye$j! zIZ}<-hb}b{CNu`#et`1EIlOwW5VAXzlQ|{L($&v|&v!R5x)VjO5K`bu|3Tbx8Oec- z;P1f#{3%ZL_ut&=_~VDqOA$vQA^W}i50braue;UyehqSlFL2z3F1)&bWq^i|cC0@$ zJ{DxD$(Rv6EvBMxv(yu|>3aWEekhd3d^L-hVKgOBA3jl7vemlUD}fz0x(@Lu!)j9+ zI$+Wk%7|zSfu)Qhf^f4ZO`S>_rqNZeqZg8tMJZEnBrJet@KhL8`qRi*>B65in}$qp zh9;%|E|@PCo{g+_UHLmyScBv^oo;NZ@5A>eULWv)^icJmHpR+bz`rZkR`# zEy~vD{ewB2DSa74T~}L$(|*D|ek^e!0g}QTo;-YK4oht_ylxy)5EyQn(*4#W<#{Z) zKHhVF79JNh*^&GAoJ9+@0v0d<)ll>^h9K@QZ=vvZCk8a%<4ihrUFZNDfk;}^Hsl&4 zsxhd8VnaTO`WkU8hKh&$eqs{h5HhqCPz|L-8%}IwI9p(v&}ca~kX8)1(uiqY43)I} zS3HQ?Xqf~C8x#=&Osj*Sl#y{{2`UROBC)8&g9Kx)^@^{Ymz+5>~JdxJt zx3G|Zd$Gfm8CIJ~jxtX}fuNhfy>!LI>iZ`MsLn`2HUn2RvU)_S!5Zny=X)+VWBuzd zF&fF$U@@#MMW#`ZSRd3>)jBW%G?~Qg1C+ zcqK5k$YWBl74j|w)%o|`exa+=@HX|Ii(F-r*k$DkKTwWJ`Q>2g9c70y2 z+e9!h-<+J1B_WF!_`K|MBc7#1H(Rz@VuZ-~s8X&sjPevSZkx2RG*os^O%}G5ER1n~ zElHN&j`6aHOYkzkwc#R7Ik9<#t(&j6LAMC>L< zB93(9?w6@JyC3VW1zGJS*ge#lXt6#p!azRsNeKvB$!Nc*gD(ehXzxh2yp^t?P76lt&EVOy40&=`kgrV^InGA zrv$uru$47Xp=MJ7nGaP^odZHSyy@$GirGu6%7n#Rp>jD=?U=Ts&I5Zx0csL5)^Sz) zdc(Fi6o6=(4I6w^_;sFXHqv&IxSHnUXpvzjH7&Vsg+!nEkvGLv1Dv(mMSPX+Vj(Xv z$YY+XX*WOPrw8*>aKy@yZgAK3QWW|oZ?Aw&6S(;PU`>mt*`$t?>`_nP%=;%Lfuzmf zLrl5oxl|SrK1>a$H)XoG=e1*8TL=bo3V}6m)^atin~`si+mstosKB#eU*{v4}bH`{6P*2=ivBOOQz zF4bz8uzC+lj3wjV_y<;ZNXBNg(LI{d8$zGoA0&8>hzCGw12%!aJuNuR^zw+G>ltr43|W zP0vcQV2x-tTRs@X4TC>1jPS`$GCyAb`G>_jhRLut3selTNs6b<*PP>_j~ zLG%~>HbA4;7ti^|vS}`4xnr9y_MM;D2g3-f9S0IV(`A_VTE{N7Viy(FoAIJ%;)dqk zoiS5u#=kTFh^1NR=zL(QqI@OKbeW`2h^*hW*mRjqp3)m4&PzK|GUq=>TZ=>_eV0$1 zD_7N%@B?X@8s2?Os{NqlTo8nXfpcBCIQP-j;!4I6SsWi#w6k7{OhmtYu>QL&-2s$> z=y`FjYOd{5F1kG4=jD@0x-*?NS0GEu*kXgIa?)GeZM%6C9OZx$Om|mZste`7;PwjJ z=XSSzZ!exwQAq)VKJ**h3NTIenDfK*OruS=o!0Kw>PQ)L_!9}P;Bpi%wqhbWzTxa? zvX8^>UsIc-39qRfFpb9<$Oar90+qQcsLwcsopgGp1@=%urPH>ThW0xf3PrZ94)-G$Sd?&+epB^1RM=c-+o?}@jBII z$IvBI^EBMaJeV4|)!W5ScNzF%7Z=CEaEz(*0-H;wB?pdJZg_6c;n+4yJR1UuuWKY!fvG&#-N$aJxPeY?YsN02K7rb|l z^7hcOH$PfVK&{Rij7NC=)%Av+ll3wHTwm;z!N>o~U^wxvP4~v(N?rvGzx$$w7MB{~)vB(Tk z02=H{Utg^`D;C>I8gRtbgig_HGY+rceTbLR1v%QkO@tY<3b8Fs>d~@=oR1@!O=(FiW2)cv&R2I< z?T{sZahQ}=<8;jajE~JsN;IH?qXIK&1@*5^1HJMRC)KHtn3l#!kn73bEP^Yb|C>^F(VtMFRa zH*>^-8`^NZJl2iPXs84gPP|2I>z{fpP(X(8swzTt}fEwohoE9{Tlr?9@b{ zP$+(b@q6*_V!S@c`5#rcBOQ+8L_m14Ilulf+$<^fSJ;op6%tL z^2icjGv5H>-#LKz_n)&H3LdOdqV7f}_WuL+CksHwBJpYU8d6Yf4r<(spO?t-6iY}5 zhY_m+H5HPG3S($R&iNFZUsD*vi)>LsNW4nK0&E>G)XuYOS>xnHj9S*Hle?u z*|gFjVfX-Nw_B!~!|EG|{vx5~H02R*-*3TWr81@l|?N7-5 zt^|Lk!XV$RaoOhn4GqWfbVmvYL08qNw#i(XX*Rj7*q#FkjcZ)OsA6U%`z!mJ4xftX zN`je|6&>`PIqzaXgZ3X{*G3agN3fQeZNQXO|o+OtA%mOF48JVNycJ12`z|tcc^}PmBTkSRn`@<}6tLT*`D(#j#*80R#tjwLnDW zl)WJyy@MGyFmWbv?Ts_6ra0@5cOJeG6l&%!kGefjnc4wHfjEY4zt4~ zD7hV>RFM;X%G6JkV^qHUJR7W$JI0i3IOm$tNa%zaH2DKm=Y0eKuFHE1kGnBx?Ks+& z%uue)IFtt#@tPd&Z-vjfIW}>$f?)yuxQ!)C z|4 zCEWA;7vkT=#QfGK1{ko^K!^H&!36vxCQ^V?$^TnW6rDCW0S~KJzgS9o7)qWdmn41j zCz?(W&q!HXZlbVW^ft$aDV4~C`|GBaW-z3s-T7Tmp{dfY_p9k0D?8_DAC9+shi!PU zA|&Y&jeb{Ko0Cx{%e7ca^{P*vKqUtQVygix%Pdb(j7=-$_o4@2J;@-mi54ZRb&9jV z9Tuagu%!Q3GM>DxiRQ9#sb8Ko!Fj9EL^BujwUU*pK%Ij*Nh}RL7|^6xrzWe~R6Sv^ zeQJ|%SfEo(_g67oIrY81jDT>_MR$(K2vN1->r3$R$!!Vl^f*Ad#7Sl+9pl478Q>5) z6Skvro-10hNuYu>3L)x40jxCIOmqCip=eA^eyLP&eftq>+)5;XB44uX61HQuA>q!y zsnXZkF|DqbD7jd}{>K<)&&U{gnqrIY4Bx7@g-dg&(T`%J6D7v5Dc-U4mF}-u8665Xi_Xdi zA0xgEAIUV@brcMXG%vB!$porBdvF^2F%`Ct8wA&7&K~#XPHcbz&L0KdmLJ!dAN|Id zfNb042%hULYLDhL$WtLyi$VPW#wto2X+eszBo7Nj)%1j{EjAMStMu3x%+ zvOM0qHA2&#}g~a*DuHP zJFj@q)9vZSrdQz97i_2YyNKbLOWDe5AO06V-&YDT59E^`5P?aj<)e{LT~Zb8oJ>{L zM(d03oWcA_Z8tjagI9w*_Ty@nXv7V+>;zW&RSmzlMC%zU@)P#1+m2|iYwzO+U%?JZ zGJgN#98$}vj{evhvqnK{s)A*cpb`9)9NQ5h^vgb?yN?fGn)K%!fasmf%8GJZUA?LC zg;j{%$(=^zm6NMCy}6La2I)qwRGq-Ood5GO1(9=hROj*`4RwkEFk(?E9-+}aVoR72 zykXzNN{j;OeSCxbyQEwnlW8CTB!v+mDIEWK*8q@||K64UZ&m4c{HIKj)~ioS$}y-~ zljC;Hag|Dl4@MBJ05caCj|`?|N6QD6;K`Kx`%jsorp#f1kZ5puaOH& z&1R(g?V@4~dN62< za*e97dP6Oxj^#a{+F{=I9-|lSV5!-+D$|S7Sy|0_tmCKF^50L9%W3r`+S4c0QsK1p zt#yp2CN4*^CGTPa%Eu#_!p#Z={1Iwtf3`s65r^rL&qyRS+VidrYNzL2k-EJE5_teu zbe*W(&Yp;~;Gs}^kFb78v{hC1AFe1b>i|GmSnH;>Oe(rsW>Y&W9Vz0#aH*28EbprU z-b1%Abtwt20yvs~yoao1B}))LKnufrma7AG?EzfTdgd1NDy9|bCeLmDg<|{-uuD1g zB~tgAf@`KFeA!*f75s-p7J5NaRF9KVIQwpiBrW^~q!DR2W~jtr=HtbTU`><|TJ*h9 zqU#an<3kzpwz~;T_j~ky#i_WoK7M*7axhY=+c#oBl>)9LkeUsR=#DF*Id(|JUmzo# zEfM~jEiIE2A?^Zwxl2Km(U)D^5BL2v3h*ADNy)WEmv4bxyw%O<^74Efw@K(MHZexAj&X72L6r2p^`qu@MO zc5$#1`#yI(1TGn zy50i`2%TS@0(PN;ir>n&1}`*U2ESpOok+I!#!5?j`$toL7H){!@CPtM5a9Jl10dX1 z_DXHp6(tmMi>CE{FP&Zi1Y&%|KE6C~!I-|P2vmjpHCoHB6%Hzz7YYYGM=T^y*aV*I zkiXr&aS$Mj3m+b-TR0jQpvS}t#BLImT=ZJIo54c@`V!akbDC;($dn@SJO_^ zZ+7*xznpzsd>wp|^BICb^V7<*vklk#({4ziaZ@y$Dyu~6ugCgaIm_g7TWo#Nz)nU- zPDi~%d@;EYmN6B&e;)by`+;6!9Zufb{kz1xh36zW0OZAk5D19jKT3>}iL;BNt-PTn z;C{w`c`*Q|-wMT0{#t)W)*MAO&5w^S`?Paw{^3+sT(twd<=?s+ttt%@1 zy;eHUPG656WIQe>5Xo}B;d~;giOD`;+MzAm&OwL`TDV{_w5wy=l3uT zMQ)(W-;iwc%m_=Mfk6(AWdS4H>7R{G>t}eEu5cPRdQe39VBAcDnk$q(Tt|aNijC%- zCN9&ZNv4^8V=(<7%M^H0Z8CUhoFhVH5KTCRZLQfOIgvPAH~prJa`^KN zEAN z?w%oH9o$ev*oW2+_46rX7%=NLN)8PG;vSVOhJ0xt6He(k9UV&zW zf5lEq13n8DDu(Hff*ZnRJ&<@TnT|nsd{7o#oampZ>0dy*#TY3%)ICT-R>T(qM+_b%=;5Spn`DO+d@h=n?XD=13u$BgsQ zLU7dWjo$V;)%$byGyG>^XSvpz9^%OvIkyOBbN$U3}?(>dR!XhP|5$Fyjm`c_*pMF7M9PoMB_1X9gOD?`!#^ z=5HCmL);7IY7TvkJ|%1C8T8Fco@xXBGH{%N&UthV^Mzf<*wvRwZ!;J)yr9+ABUt!p znkvu?Z)H0(f(7~x{;=WC`L!2bsPZ&UVv{-C5L*@7qY5#dVgjl7I zoo;{C!P4Jp=h?cl^vKZf>eIiqz#)E0$~eZCPiR9qax%Wq8ZOx1ftsG%stuNU0hbWM zQv}7uIOj<|=8BiV(s1ng5-e>T-iXz|9TRbLH|oq7a;u#{6iqReybsoug+V7g#?V?ME;r4QV1H+L7>q_bG8+tKGG?t zPNk;ehTfQ1N>0_E*^_Lh3MNSSbLyG3t7s?lm^MmdmugyIAZf;)v_QK+G`txF|2~*c zGO&PAmI;yh_1Me%oi^6UJErl4Ys>GgPYU}(@DGo-g&8A+6K&cilG#B`*;v{nqv-`{ z(jbQ^ne4a>eVSvh4Ma=k7gmkCocU#`eWUDMGzy=nJa2@!?6u*8c?PBfl75TyBJ$pI znbTipDqCb@|1-&3kO^E8&P3hP*P}DEK%LxTuL7(W!fOg`@`KFyhwVWUw5C7A8exj# z4$EpwNZR#L_(Zgs`ti;`Ub42fu542Zw6?lL=ixuo z9v&K{ehsf68=;BihIz?i1)3qlqK(g5hbx$w;T?9pe?nnd?+i10yP1!x&{gX!4q`4OCxx)>^v% zNjaPUixz#ZB04cDU$R2~W-+E-?j9WdwAeSiV|MP)MyI3Y#{?2*f z=x?24zI29@Pm@|vtw~0ak($VUDqCuqmn*xNBMx_t&58EiigX- za~nL~YO36&z;8t!3sno2;ezttf1Z zt>rM1c8cnvyJgiF-FT|byJBkXx&&HS;Lzt9I`Vc15gJJ17@Db|k-q zYU%uNE-NNYRi4h&B`7NhrXjki<5wODBq!oDHVn%QmTEYyS-@*~XyTq_VrZg@)p>Jn z&u=C+a4HNkVYq245Yq2OuQfcD;|tn4npXlUdce=9YeMrpFI4m(1$~3&lJ#xqOUC=* z#WdVF8fQ`?TeT+fPL{9q)aThm{CLmYk^|o-QnWaRECpbRter7@_PJ!>Q+tf3p!=Cbwrm+<;&fp(gQ7#6*;Sif3^?0QRufA#!}NdU){&dPx( zk(}yTSVK>U_Nu6)WQ^=a+4Do+MVR^&&fEoGu{I{wCL5jWqNzlRVB-%T`I>I}X4^^a z_EVZ4|2nyVAiXk7!~d_73x!jcI`|qlf^mF^mkp=!o9bmqxb<(nD*C5&a~`dVE>^05 zlMCDb>qKGCaNrR!CPTXSP*gCZDk5dXV@dlG&b;RD1fiRnYJi5pe3Y(DqwBw z)$=g>IXYD5UN(POz|ux*!K|EK@K`@~_~fDEJwvf-r7R-!cO|yO=65T;bQW14n2&>F z5+1O49FxJ_`eqEJ@#+Ilsh=9}jZUf#%wnQoIAGO@_+x^b#~2Tslt<4qK7*OYLdSaR zHi0~wWH1sD%K86~n+x^<;=bt+yxgNN6e&oQkGe;c@h{d#6pfN`W(*Twga)9N4ZQ;;E9W zpup~Y_{EE!4c1g4o=7Iyk<1%gQZ|4t6^!Gaj;B$7IQATEIXoT-2s(6P&}-9SZ8*#q z=JRE^|1);+@xc1m*kz7ZDUivNak$A`(#4PF%iRb~X&mO9vtXm8yMW?u;iF z+2588=)n}JBK&>{F|B@{Kf+GBB46yOQm!$$-|jipZydpulSfS1`N#$htv;g_8ub7R zF@7PEM>AV@jVguyYmPl2R%`TI3U66_(d{9wX8xmK!72C6GS?&JkfD>aqk)n0&;R=2 zU#(g7|Do&~yDMwdY%7(DZQHh0v2EM7ZQH8Ywr$&X#ddP{+xL#Xr_ZO;e_@Zc*Ie_N zQUxqgi2!yDeSGi7Zy&MsY;GZmRDp;6zQ~0ObFqaX_ra! z6peJc|B*PeY_eP(FCM>h&LOojln75~Otwg5i=H&ADEH3UD|Q#d^b%jCi1^DucTG_> zPHkGGL1k@pUp(}<1yP?-?hHmz#dxu`1e0%$_}jQ(OuNZcJ~g%8;!a71V8W7Btrvac zw1c_?$vC@DxMFH)QZScmw5@wb9xsbpr!=b^$C( zPz0+uk#Y|5Bg0<1oJIp$FGDSQ0^^!kNWT8HoGh!8-k=j%I)tBcJavPFKT2HxOR)uU zx*JyZI&=&_g@&LBVP%E${V0kxhrpvs@g)4|krd9*COM4c-*zynZbSs(K4GteAlf}2 z3{bqme(l){RDsmvzB^XrR8DutL_=-=* zZ#&Tjj=(|trW+Y#*_mcozTmF}g08raTVq@ujS$GYD1;?lX?u^iGhYkpy_B;&18{CZEPCygl zbgUT7?iatP^=uAk`J~{Gklb+?G z!!+`IJj}^dOo2uh`f%Z%WLg(tzNK9tan)R`*>t5ynB+;>_3w0(cK50K@KgVMu2HgS zJAyVo6$rj0PsZkMEJ8D?p+;N!9H@X^0&(UiN{()QoZ zbOMKIg$1nD_>1LD%am%qd2-|CEwF{5eRfF`Zn=(KKoLoy^UO6B$F><9zT$S6M~AD& zQNW-a-&wwIbf-qQHOSzvVV=4%W8UI6n@f{gJ$curN)V*p*dcCok259sjybJ}opX>o z)dE#9w7)VMlL?BP6Ob(ZQkGvblm^!MllSz>gUf&>l4bkl5erG0x!kBOpTUU^x^;6B zUU3K-OBZ!h*6o&4A@OU3Vv7N)(c}3Hed3IsIm<~FY&bo7KAA;o$payU)8bVF(bH#+|9ooSyPG8%?UXza0NzlUUCiM$5CX5u$!B?`xtC*~J-iChOc2x|; zF<1HH(wX^BIcBX!4Q#CpgxvlZNI_!{M+@tGhM1TWnlZ+6_w}m04EM>#jX;LEG`_m? z<_phZv}L??np(@tQcDQY77xhA3c(gdk2dS{Bp9vIFSuD-Rb;IKP`aoLM6x^9 z9_qd{rk6_GLet+d-Ogtawp(Yw>LUyIp_>Ck^D|OUAUS0v!*lB7ie+gw6eFR#~+a;9Cd(6gCT4g(x15K$&H%e~*t#yGTw^hUl z;K*=5`%hXI{yzpfIOx0oD*!I3S_8I?5dVo=_;&}j-<&?o(p4SgBR4fa|Bj~=Ut@^C z|5KVgnY91+#XuD1V~49jQX`3m#D(^BFG{G;#U;mm%T#pa`=#4g^!Imf{JC^Itt};e z0?^0#RA#Q!$HyKRQi#3$R+v;JD}6*NVeA6p1vPL}N02~Lek+P&q?{k`PX^hEaxqiq z;p+>B-IxJc(RE@?F>4C_*ZU{A_a;np+`rgeLg&Ql>n_qS`vpC&-40z zw{moCl36!WTk%B`(p2RPa3o`MaX*Xz1d$aZUf)AF4XJu|?eKoiQ?T{(!-7ZPf~=L( z4KytR_PgVv>fcCvTBMH>hsooN{(18Z%YcDjKV_|O!srHvOgm-rqBRITEVZBnZXE~* z;WlkU9tbl*+yN;$vOk)LLcFOUE+7&JUL;gNoZAvbE-@i>WqSUMIQkT5OE}7nX9(mg zgn6+1T}c`xf3`=RQl5yyJpT&1|?+~#1ZU76v*uD-pH}{d?v|g@* z0q|C~K@~d4H+#%?_Ib9UrE6sD32WOdwXTVN!2~ZrH@AaFVWFTuHzL`gq<3=;WJEW) zMv$1Jd+&+tUeiQdY3`)!CG})k`uGAFpt7JoEa2KE3M`H9qC|1ixdc5~r7rOIrx%-m z)1(5!@hF)iYzGBLnqJeQsULtYe|8w@{Mrk?Z1)5#{JI3-P|~Mv2&S5QqF;r9O@@lz37bkWbX4O4UFP$oS+xpCO_zfXPNg7AGZ05 zu%77T)33?CAtgr0VaNTLKor`e=09`g@N=r zA=>X?>7iEB3#S2NHHlKLb(a_nTfO$N>?trR(MYwn>$S!RKWE(J_ZB+^rTr%RdnHtn zHdBg3?RK$wbTa9PF76HZ<|n{9{d*htWM};`E;6AAsjCd*y-<^F!m4cTpX<-let6iq zEoHWfL}2>{raDmOIA@eQJ@ov^#j7zghUdo7>v4Y|N)=y$f++6p;FMIoTwl6t=!K+j%NQons$3f6eqoSHzP7Uvj` z#*e-mw^{0V3AAS&>_ew>%apr(-p|RyQ9PoK;i0-w8UIlPPWQ=T-HG++%4tGQ{xWf@ zqcM&N!qovC*0U2xTTe`&%`D`Hy@l;F3;(F*WB+oC3cU_Ajm6M78~GHp7?ajwmN=n) z^IN=8dtJI)8dG&10B%2WbkZiy=~`c7YSccyTtbd$lYU2*`$(#K6s8ZPM(#^yiTFIO)f}^;#js<-2N2jK;E)a&=t44Iibpm1$ z&g|8^&;G!>U`(+K!$4P#XfrM(zpAZ^+ky+4R&l&>DR;<1G#o zb0j}B^LVwAlzQ%1yHkD|$+B}v)3+}VEVM`=dtC81c!yf~ZJCu^l1RpZRl=9JzRj~{ zqe0`{&W@`P_H(WP?Uf&K<28>p5hp!3mGD_6l2@m4?aR|!)3m)U*x~a7PjG~-fq3{C zx~OWj5=PI+Hz#I&DWYWFsh_#-A9dsUM;GR=clj=g zt(n*1th41hWpU;k^H{n+Am=ou2zy?PyA|wPdo9RYH{EegDCJW3p7v+^_}^yuxIq_b zB#ked&v?RPd}4C>b;lyX{Yl;N>rQ2X+cRVtlBBsXTn2tgAJvWU@$-a3KgG=|VSOrZ zoT`~tv-SL?CO?|b1NLeDB}#kkFx3HXgHr_hjq#?D>E?wc|e8~yL_j^HUA&MPCUZJ5v$|GU@`d@3KjL&nTd366)? ztFoOJ^dZfsld{#L8I$lP=bN9nD-K5$dFw%U8PA)^z8C$?<>YHABV5UzZN(&$qQsHl zNy?HmTY9y|S@A=rV_9F1CgH^Q0je#uNp8@ZZQSsBdNUufUzGBCWU~2G4V#J2x~o%}Ka6ZVa49IPud&GNm&CdM_!Vs7{{H zT z^$4^0-zOQ=dapl#B^u=fHX}3;^Xm|cwNVc$rbzi0q;YE7bxvIu>uO4g@~CzZ1s(Z< zcFtrE<8)Q=&)-zasZ(iJ_fIiORk`Js7)46z-Qx>E48uu_39R+B_sz)|))q}uOr4Fc z;*q|H?813h_ZHG+<@H%?EFOv!8?ha|+q_U!1WoHhN#)Sf+6#oato)Tx8PrQD(p<)r zOUGr{11W6KYCobFo;WHl)PL(qMH15)TYqxG!(UOQ;;|cTOwxi5 zlZ>xqR4$;`1Mw)+5GUPJiOlrJFczj-m5PYgXnOW4^-h;-h^IhuVs7FBUicOQ|);4UzYn*oVob6;<= z6Z*-43y?B^=xH9StOMK+WSi7!8CP4o6KSN4+ItUDb{U*q2KiA-oZl+XF@92gd3s(YWYc z;Sj9C?=;pz70ewcotWkF(@9?aNR#r>K53i^DF(g41R zSGH-F%ifshjGPk6dN=Nnw`}P|Ju#5T;EKyR?_Q;mT)zKc_qaEZOPAhAkz}o^+IN>J z#m~ylKy+>8Sj|F4MgciuB-@r_hV?MxrEYgEpzGIP-~ytvZbnTnH(A&IA9VTr}V_LhG6EA&)4aCkEfA(fk!sMAv&+16cZw9@ug z+;aE_tUnHA(N;$}Uhbw!yrj6Gc@@pTQf!yCPK&m}AD$9d_aet#jJ$G40^pupMB^Oe zV4Jhp_$cZ|kGMTLe*^J?@*t1}ds~+UBZ`Hf(4Le79i9^=ruD*d)1SkFbUFh}hcYZR zkgm<#v4Q3{w--k4G7X0vUS+yveo>w73C%Mg@y^k61AiGbz6wu@Ybt>izPzqZR;l<;iZxNuMbc`e9px17z=jnrSuy8LcM$nhY7rIwz>JOoZ)Lb%n**`j93n+a)rxqnMs)cGWWq-`3aivX=)d9n#g%Hs~Eu z!|5=?$z*m?y4&;xveb(OD0MrZwOT51Xx#C8K|5YZm2EZDeAJ$1KEDa~`k=O;8J;7` z?7I|X4eO|X6Y2mR8$xipEZ#^4JWQG=1`<;mx7JpnBc!uk2d`oy4;uvj+oscf^Rat$ zQx*A^SB+++k-34s|AreDkeMAcfU74c?2jKz|M}JPe`1EbW2OkaPqWXE%$5l=TyHJXP{UBO%ws=tXs@QZnVWGNTdl7gi~8 zl~X#cMK&tO7Zi%0t0nI zQGOLATiG&Yp0)aB2?XzhhQ$Z@u9143uJ1y&>#+t&pxY@#Bp`r~X6 zMxee6LZyow{=ogva4#8W|Bx!gF!ORN{@oq5RbR~K_r3=!N^f9b?-Su(2r9%(`ZQU8 z#~?|D=;14Z^iWe+EzKRx(}doPY=SS~(z&a(*Yl(%CG2FXr4u%4=!$k@x-bb#xHnni z^1H#E{$4O}_=A+tskph-g-mepN6+DQtB%lKGEa85SSki~JPuZES?!2o%Pm+t5<;uY zU?*-5wh@B73=O1dHPOq!=S4sQ&ZfE_%lf`&-unvnyIx7Sx3zA-;aN{17>UENzvoo^ zJ6aUd^oTxjzWd>P(Y379wjf(8LJU^U!XljX@o}DIpSV$jrwISL-te;xZ2iB}oocv3#as~a_^SDSpRD#mI3eBBLwV9?NaSu+h5=+r%+9z@;#d#lL-K{gI^WyO9qe!@9)DeX(^zC@A$O@ah#D-HohJo#DYtRLWK|P@`E}GwMdX@W9chss(Getla=DTQMzmpk$ig0Ax~nYzjJ8z5{->U$nzO5L+2;W$eP=R~xW16H{U zeP(h-ywNn>&Xl64crhjsJJ8W8KSwQ_h9z@vJeN%WhfgH@?&e2IDD6Svr}altIKDg} zb(QhhBCZRFN$0ZENfk-wBNZVt4xy_@?#Za??TaymgGY#z- zq-)}GpT*L}i8j5soQ0uU8If;@W`^b=e{He}rPBsGS4Rl{oLY$4&_&Dkxkb9`m)WNE zw_}`{!A|2UM;!G2W~&+uyAYtQHx-J1&)2EyfW)JkvgBm}89ZAXK;a`F1A6LP;82lG>k3o{`VGPk_S#Cyi zIYXuqm7!y^c!q2;1C+gGXO^zm9L4u%&RVd+-TZ>Mu!Pn11vk_Ex9+ofTGC_o_>zTWfl*lmamDDEBVz{@rE<}DbOxMX|L?pE zQ)O+FhT>@_esPA9Vud6WrL$tGWc74oVFerWyb0x{lK0APBJGS48w(NjGxz4v`5=b3 zq@;-DaxG)$s^do5N7BrLqHalx55?XwNAFw|NeO3$8MQJ-2`c5Dt=Gp9q8ZBh$S%pU z979{hrLba|UCKaGrY`-toK%4o_42nkV-(=yABIuQV+LQNcxqDj#F}LeH~DzAEBl7m zVEI{WHQO#d`dHkpaQZnR~XVtErk>V>Yi^R#Im(6%^skah(L{71E=aLz2+$B zz|6C#QmBSMHP+%JSrO5%C@j~o8B%4;CvOe2&60!PCeyd05j7AR&;$o*>pNsk`oFrH zSP%lo7kGsa_V^w%Y^~kC&@8t;dkV)FP54ZroDhJ%jNJ|<_6GXT-WpQm^NR00789N6 z?te%b{zG3lQW%SxD)OE*jdPRvpdugX)}00IHi2 zrcA)-AA2t%jJ6!25d&sFxLV15jTZ)C)tD0=ELu@J}IC*slabD(kALV%#~zP`8yMD?F0h^w*>_M-)uJ5LCQB6EdeCD zS{`=_FV%YHfR_*>VhO7!<37(kW{0Y*{z9OBgIcM= zlwk61P;L+LKYE)8HSjHuvZW2j2%}S;*KQsWPea!E5#p5XZdvrWFX>34^_6Ry7->

      Uwj`pliC;Crs0oMb zdBQyLOq;GFm^2)wi4MsFL^TtphP(1!)XMu+6mEJ|JvT-83B$19P;?5v){m#f(SKYab-A+3`yNjFWJ(5@Om#=CaPjH(4Bvr!`9QIhQ$`2ixUjMr*eII_g{A z!JQ?a2nXC?(p@%BOAD5o3_2U-82HcS<(_2AMW|)AXF)!Eta=aBp1}8~UF154($1!* z?)1iyRN~{QQw5{BAs+Z0vNH=7mglCwP^EJ+^9Nw!o?URqQrUk$kGbeo@fo))xc^=# zUEunQL&>k+;kkTsnvTof+KbIBynN<2pp+h&@|Nf7o0V{(0SpJ_QotgePv}7f4vh~@ti5F3CcKV?w*{G}Z4DTf$6>L89osu!l zVamgL>}QKZLR7{X_w9L5Sp;8(MYZLzl$nK0ZW@S$~Y>8rQFFmz<{>w`x)Di#F{+G7i#j-bK z*b3^#-KgY2VB%S75?18Q!+x`iBe3|}gqN3_Uh0lGQ;KnoW+H0G5`iY2LEfkGyrt94 z0N^(`#|{~hNgY}z*ReZOV~ln`A$k`N5~B!Y!hsjy{7zPnB?AmixSucS;cXI2qC zZFXL#jcISq{g-Z;MGah~ z9e`XEhsDq`2gLz7$W4hv4UjF>c>D9?D(b=D%GyH({+h~D z+|^SapAkwM+|325L8?KuNhAYg%L#+T>09Bg5y)@wheYEcL4a&Yapx`}QpBnO@z_25 zI000T7%_yZpfD-lLWyEhU=SoBbD&;FFqe|rDQ8aTK`trjke0`EHwT?Q19zi0NF2G4 z9QP!Sjb76?f(mBBQ3JmLB{zzb_gW+OXWCAOcuC?=qCskU_Vnku+#k7qN)>@(dzCAQ zmEif5=wDCa!sKHI$@t`$F2Rq)qOn2fCIOs4`Y8B!?gVvIauL(2!H9Zy${Fh%pu;Hv zAmdUY#&vRVN^|X^f3zZlIDcI#Sm20v#b?q|On_FNW&lJ>)wwF9{}L^CZV0dhzF~u* zLB9@ZcMHzJtG!)E1K_c3!%6_6)}Nk$%`tzN#8CeAE)=#-Kk7y=YG8C zUNwCVFcMkVi{4*OAgSWMLAWNiaof=W|MMFLto1OUgP$Gx0M)YTpj~vK2wG$8sfj6- z%R4pQEe((^wSt$|d2a!#4Vv_ny+PFCTbz^@TpqxW7u*|znh!DX zx|S7``DO9nswI~ws!crMVK4uA9_hxj)AVQx-e$Zmq2NkBSj!6ZOx>wYwhl78kZ9O1 zKyL@Bh%j$~Nd0bI{DkNv7Jb6o;b}_f#imkhXyJhLjV2h8EQPoBKc`Tvsm+BZ5Fe1~ zn-C<0Z?n6z+;h(RBfG!17QC!mf2DPPL0#lrOCXE*rz&=}lLC3L!b3Mcm3J8m zCcn953ED8#r%&bcZ9G($N}q&E19U2-4zlqYC}H=$&(M!)XjR{=(#u!AEgLY`8&C=% zX!k7tJaoqU7dhW67fu{cilkQ~7ViyCYzUrCp33`s$vg8n^i+~9d59plGre-vthRv} zU|;TG^Un(o!w~stRMQ$pth!Ssn?%#&iIwzFvPnrp9VB}-eoj$duI&1L z&q)DHPMr47W+LGWy~>lbf_)^;k5k*ABsaxtFWZPDO0onAh>GcgJ?Iu&OJW349uK98 zC+5FUDx;2_(Z;uO0+!G>jH@%nGJjx69)+y9j(`u9Q?##!j9E4&q|7pLX%UUiNKXr) zYM(5cUpo5fh?mbKaKe#}SANPAStS4cCqyxvu_w&L`gUfi8UUtci1Zl%}p^c-AomnSjL zj?TbRSShLcj5aAkt@KYDK%!~xqIZfd5f&Y}KqH4;#9vH4zn*;iw5hlCu3U82T zOkBq0n(6>6^bI;M4p{;>z0j2?&i1!T<8f|Dv3}Co7B$bItmt-V0mt&3S-!h$J8KTK z_J~u{^sw+5$20S>=z>YUKfLlhiqmEdk>c7^?ok#}bgR<)3T?_n{=OE({2CQ}qi(u; z;C{*wcDuCO+SO_-=wVlv-TL>DTC-FX8Y=)OnFB!Se_<;XGq$pm(>FE#S6sRTxM~iB z;J>myBINlv;I)2x<@?!XSvQc4z&1M;<;uh~hHyE5-I(G!(gLcjav6kTNyMD& zp`}V!m;0^~wCzl5{0Q@!wQ1PQDWn?FtU1Az_^(@8A*}<0h`KzIUbtS+9GiL--fJN0 zJtLKiKpWkg|C5K8laF4~IjFbi5J~`%?N>R~tVnB$djnE3D>V{2Wws$MF(k?bBaKpg z@?*SRB6-0Y)GoSOL;~dogu_tlmL4y}>2KTt2|1E~%rs3YkQG0Rz`GMQ{GwH{ELP0W zB_UbK8xWQq3a%*%tfPN-p>ZgK|3y*ajIb!ES5{czX_?26#Sf=lu%t8byN+b{m?*J= zOvFNFyeSxxo~$vUn9SE3-~ZKV7g3^*c|XP&eX~0$DR&>WD|Q^r*6X7)NG*;Z;Xi{0 zB+AVU@2Nko&IpS8*5@!jKy{I845^*_IAuFRyK&VrqDeZ_sNf>ea`OJQ9=)lC<1l*+y$ujbbT=$FD4N+CY5}8eLk97+;RQC zQ(aWh^RM7MX@tHQ4IJ*c+K4jsS;Q2wsT6?~bn9kKiKD?!Gsvur8b*-X2dZu@;b`b6yb@U+ zrn;XZ+^rm-J`g+4G-akYCW1m(;%}OLZkW~I8dALfM*2RTc(u;l>DsaMe%(xYvZZ$5 zz$UyIuv5EumN9qETrsx9T@|ib1tFfR8LjT2(S_I4rRE~shkd3*@vf<9Lx)d(>SW@WaZ#?yWb3)&ls3*zOi9=90yyKy6QC%wa40n*n>*sAmA@7nW8G z$?BkhIo?i7?pD?8)0BJA1*YdfpWbg*0WHaCv!HbJ#w)eOj1*m12pgtD_u+scF20Nz zd&o|xE)GF1(-za*d=%;JT{W*z$G;GMAH+YfH^lEMeYRJ8k&BoJi$R?OBWL<*a_l z@ky>N+0R(Ca7T1lzgmiu%sZ-ob%&NyOhgv6+Ng-8vs}&-kyGxaZNf&tu#Z?)zclsH z(N9}qV5)c)^uxH>GCQResU=5xPcUC+G|IfVIV(s1Yj2x(phmyEU(@JL`;VhEYl6TB42+Wc1j&huIxoW?=WTnz>v9~kK;!O?;73hv3@!5Wcktf{1gzj<|1 zJmw!NHjJP2li?9NzGR@ZOw=4z*%!9yWF+Qpka2=^t>=C3MqsYb~j{Qr*FK{ z7ceIT9u=!1jvLcJ7S|&aqt^G*p)vNZn$0HKREw$dj9`bZvGX+ zrc|{Z3s{l8U3`YJJCSX*>n|V77cP+vN#F z`;jX(f_*#7zw3rzy7!~?B*v8JN_+n+U3m8uA?h_qw3Ye}>@c-gM~jaRLh0{qXANCkMTAKv+jZ9a66os??`kfxH(HZo6Tq?CPip>?zu6mODa_>a?2| zy;j5@qzk#g=r$&7rUSSEoLj#(grO!+M~28z2W~&RaV;6T3$_pfj=rfM`c0MpSVFV+ zII}|Ine1s9#Lk({`ViY1{G`L$eEHe;tp2ft3PaNZETQ+f^?K9Vs5t|xaL&gj1XzC( zM+u&T0?Y@@z3`?5WFQvRp9xWaHJA^2*Tkx~4kPQsVmIQR8NKJkU^E+ZfC*mk`IAs7>=mK|N;A6*}-mD;fZ=?e}^J`+^9l-uc8G>ab^#d&{T;s(2^J5I(I(W(VnTvo1e!*=?3^KNr5d2hMcK`b2yJ#PiA2BY; zsI!(amYye&tBWbjXGda|qDQwFiERv65i4IerTY?!mbrba?L8opS#bbhP|PhP3(KlEb8bdYyTgsX8U!W^aqvIH}Vjq<(*L@r3f@mGk!~Ha8xuhKlvX z@bHf`lf}nBVb9RAcrr}{tbENxDZV`n{Jd}?{t?Ba1FiR}{8I##AqXswp4*{qwh~;L;6_6YthZno z#eb%9_NWu3P}>y(jJz+;x3jRp>K2&={ik=TfMDI=N4IgmV#XzkN?@6ErfUac zM2U)F$I7Hwe5F2>0`ujHg|;o*{BpG~dxiPfNaY+Ar=eIZRDZ1C}tQt9)+R1SC35_^xYVss+Z7#jApvw$M9Zf>lcdjj`d!uH?&Y zN2-3h#x(RrdRQix{T))thD+KkW{IiaZjQ9=-%_7zo;5DyY`op>Desonewu4fm~FJG zV5iH*j$b8unaMnm{G*F@ON3jfwl~vSpaZ*}rr6$q$#<;QD8DXTRH;ZrS@q-wR&rzA zg1f%_;9rHeJa=ZSqxzhCK2bTEw6s}~!?tw=UHjcMt61|b z(T!@H9@%OZKz1kj!9v@uw&voZ(J;9}m)tfm{4H?b?6JZNG#1}oCvDt3j{J4J!-uDD zmY~Pt_4Y}GGjEkd)OBk=GFghq@5TB0Z!>V;%3rGTfKrzp-2asi^PkPie}%~c7^3E>5jpdu*Tmh=~JHJQJ;| zDhz0uJ|CUCZL$J$yDw$Ye17t1Qwuw>P4Gj!-CW(=qyl~Ma$@$D(cNLhzIs6je#emc zB>AI`;JcdQyA2QVz$pt6;D?6Fr$nAMj}|Zk@j>K`vI{{NnG^h^-Psfgr%R!i9^%I8 z6-=7JN4d*D<;a^{pwAHTmqiD@0SN^`yCK&woQib!K7)3C5^m-xACQyfH$|U=l6!j+3qo|>@M(o}* zpUAysPRxds0tjDmtBOZj(ZYv6qs9`r5frXm@E;I?oWM`n5MEFDV>k>?~Fe_XB3Vr-%3LOY2&R)c|4 znGZE<9rz+`hMvRmR&%=6^rGv4~J2=%z%$%XSL(rljZ=OH1j0cyTY6#oP~Ho z7hjH#N^2SVNpMFmB0+}0q~3lx?K49DiE2N>h+)F;oNRtW52k`f82&AImIvKzEuPDu z(d;PQiVU3|38bZs{(_rmqkQEaBeG}5;McCY6RyFKqcf#m8|9dojg?|ETQ2=ub&}=H z_$f`&Q?2YnLU0PnLDvMvn#$I4h&!$g#07NyCL#Xz$V6Zu!%kh9?`1jr6ixZzOv`H{ zH}I+-&Szx#N955=?h=0XOuShAfel**okIM8>G!s3ff5f*u_CS8sU4o|i~YtOO3qP; zu?d#$&u;wjH620J2xVz-UvoklmddkHf9|->)!KDJ1*GcyxzpFwUm*UgXku5cNjNISJl=YP>AXKTZG9>wYr|n8$ zG5!_f2l;DIz&|5>4459{!Q2Dts%wn*f9YynAw`+=k{@q%1GSCgbqU@vvWY=toiFtxL ze;Twv&1vS;R!%W=LmFvsYg~hWu90buNGekt@rLTsX<0by+@c<5CD^TX zQwbLPIYm*44f(;|4Z)h_8mM?f4U=sBCY+Er;pdfTAH@dxY=4S)CJ4$holL8Q8D!L8 zaesTv!@ns+;Gq~$;lxl7M0Mxp3JdF(wD6)DQ-SycEu%#dd}8hC7qRQZ9_**rP3j;P zNjhcnG-D;ZU}Ac<8KlG%9y^D72=w?x90E&?oN>s7w;(y91m!9B&)73dz)C_i{kjsk z&x}2+PhJYCj`;%pb5AF7m%oR`TtOj7m;3kIsN7eCm;7M}e3@EqB4VB-`O78R&R=iebe#?p9%((T-_l?yz1t$_8V`p)^RmMAA zTFox-&$->tVYxS8YO!jJ8={C})fGPI7WUM-J&85v2QnzPfAFZLz2$*`YnE%hhS7BvSE!Zz6b$e$r0v3wFh8 zsTbH+n@^U`pqA%rBy*#s$n)cBuui=(=WfOV2~Wm3t`%vUjZ z-w+SfW0(wCXBWBdSIpzmsKUNC9k)zKiX)6PN^xt3a=F_`S?;lCTvX7?FO?WI!NjwX zo24IWFfVdkR46mcSkXWp3x6oQ9EaC}@OcCW9GBIx2w0B0mH$!CH-@jEY*H~&P9zkd z=}ET}I4Uhw#jVpg^Yz1x$E1!F1=hL{0i*QnLdGP$b!Pc zOpLHkmJJ0i)e-t8bBFldn08T1E9oG!Q&MA_qNE4m#qB>(?jbS6Kly2KSc3-Xy+CCo z78HVwb8tjN?B@RN!eW-ZudJ)xrim5Xj+Im8-Q$(nSuHy3oFh@uyuKo%$e%t1{gUw% zU)=%N6a1#TcV`*x9y}p`b$$PU!```ed}|ijvlfX%*F3qb-Rl%zdqekZ{IYVpB!35- znVTq*HFx5DQ9`*#rBARW3JK+<&&R8;9ZHI_1H6KWsa$~^3flF?o_gGle?Lk~;d&04 z0(@840N)kIf6nKnjje4R{zuTWN@WKS^aL~=d_xk-x|C2{&X$tJ0$AyF35 zE+DTdozu>yY-pIQDBPZxSTQyPY_@MFR8J|o(r8jRr_|gr>tmskH04K^Pjp%u`_Q}YfzISRUmRA6J4u;8-%e`9$9`BKiV9u6s70Q{iE;CT)8X^AF&l#0n zt2Le!&0FVCLe+Qb2ZBV^MGK}XKmCH^E*4HmEpOHx1o8#*y0FI|uLN$s{;lsAb^I*= z@LQ$JbSNlS>8D5YpCP-?!Y?b4R+Zlun&CpQg)?S=0da$(3M!Bk8IXgzgnHHI(GV3E z?MwWimM$N6iZ1s<9+upq{Ci*x(}aOtP-q6Cj;CLsddXwzq)N1Cr8JK6vnxxo*9(=#d+``cO&A+yOWbpDk@C=x(2}c%nBmf}U_B*I*@9kOL4G zVU@`O2OL&ojAMKS_Yb&{<%Rgg99W-;|8LC!a=h@oYuAv{8(J%0WZDNlsqC~-l%c&| zoi6KB6)DIGGg6*sc+a#Dz;DGsJ7av&(Bb&vXm$ngTcH8`R$d6xfum{%FU5=wJfhQs zi=dz{B=#g9Y59^P_c<`BMC_0*22t+w>2CSYp>`#V$TpMqJk?sY;7K{cb6TwlX!`}f z0UZa1eU;`^{&?YhsO(@I(%I>b{`jKb0&oh4v;qucHl(h%zR{a3@m&vpD^kNPlTBr~;VLxhk>cWPi?Xw+(c1a`YWJS;|F-QgHk zAbT(y#*9+5)a~8KtD^It+HUrz^f(Crhq8D6&$R8byeqEQwo|cf+qP|AF)Frgvtm0H zr()YSE1v7_e&(I&=l!93{)h8(oO|uP*7|;T-5*T4f#Kf|HXOsm%2==$^aK4?AUeIgROS;0sRvwK^E6{SZ-ch~@>_9ex1|C7R{HNN)2EI} z#s0>uGv;qkjX7rxqNG++p@Ur;CmkizV|vENYq_pfDV1s%mEauj7v#5WS9dJN*x?Kn zYmO6KsRgj;?gh&>u3YPbNh!~iaK(Tcot(413r~-?r)RLF!FfNF&yJTr5AdvY8Psmw zZf^lp9E0}3SD6g3SJiBr#=Sd79r9=7EZ-pH9 z>#U~xwVvoiK!-fsu{5o!lvK>13!i!)IZvEXGCO9^TVKZ# z>1{R$9{*;w%%o(hWH>3rhI~hX=a*Xl_}m`c{g(5Dz!IpAr4I2Zsxm9Eb^W<+v-ngs zn_#rgiHiUU#=mPK89ltu!oFs5YRAqSsT|3u(|0O6>G;Lp-ftVK^D^dP4(caAp7d49 zfxboA6yxyFDDpuXU&OSB2bYPUVhZjW3^JYTE5u@XA`lEiNr>=wwSTnFuL?A2cGl^% zi2W~*>Ls(NbOVAVFWv6rZgbEh;9Vk&{x^P>qyMel|7&4GHwAm#plN zJ(9~o;}Uxb%9K`2R8eC!qdges9V>E{n<$*D1mPol#78gb!j$_%rByGPDuu^198jn{ zBb}_JylgB~)zGGZ;aPrNh=U1|%}Pg*_@25)Vn8;jDpe}GAYj$4e`%?%RBx*m83kdP zRT;|W2Cjz9nu+<`wfIWTa7Km!c0rB~RtogZ+9kXgOAty%&r^+wS`jRO-L<^Wo`KL{ z!n=BlqlcQaysoh z4AqL);P=J^ZI;rHf>=PT(aG+_lt^gmU~j6+A%1wztMB$x|)uDRUb~lX5)ZfGzR^rX9z#A|v&e-aRaT;TBpw%XJim)28aku1&KQHRno5!4i*ezufq;}1=AL~BJgra2Syr00B zx|r7d$O%7l(g3ebs*F-(2m?v>&AiC^GUtSM3;y&c4F(N8*Jd~CxWgZ1S{3BwBLKZslvFe!s{7fUth*D{3YjM|MmzGf+IizazfH z_67HEd#J4`JgQgXpm$FrZ_%9~(tz$Gbrhy8Z&9JnbeP4SQk$PBCJ@?5KmKqkx!-&l zxLvQZbXlU9+Oa4U6fXG0?d~zKCPSF#aCzyWEnmL&zI_m6PVprdkiWW1Qxj6DwR*g^ z0%)usO?g2$N@o0iEt^nd^38Md^P3r#Bn>R`M_(5v(qG&RPJaA(tldPU3MCBPG7Zvx z>~E%E7xi2Ac_w-LotwS-sefggrE~uy{+lm>)9vLw(>Oo%5&g9ifDDkZz=%A%Aty?` z{?jM<<~QLm)5hAl8hns_A96Q=$45(Yoj=iO07CvJ+=)TAw^Hh8h{TU)u>mDCCly$c z_4<(TrTGd8NtYgYPU=l%vNBjD0@CyGy@J9iB{9kkGzzN_SI@=C^`C&U1i7A=UFiK4 zQ=BSxeHoo*`x4$G+}bXwm-r{IvBYS*E{|#RU}rN2On3nMguvmD2!DdboT9W81b>nw zQw;vz`e2`Rg%3vpWnpCusai=(_B$&6tU!mll=-2Y%2ssn`b1O3l_`B4W34J5-UdfH zGeGf(IQ7vv_($?$#j|EEq&82t&D=V^Hdkv)6IM@BQ<3zNL3^ zgga>!=|WOC-|Dv2n00x>0E8+%P9h-xQFXz`IrSk1MQYnm#?FHyi8|hhEfsEtUW01z z_5DG$x5W6IhZ!JR`(^lTuIOVw~E@K>UA{80Ji;F|22?6t*%miHoI#k z*2=x4FvZL^u>488OGQ*fGd-GxGyXiO-_>r~Q6w69UJN`3-Gl0$tw(9ZFl&Yg8(FR6~VZCmN`9kR_-{oHwW*7)6j{tv)MRn#-4`PwL|^05KVTyj}|> zU!ETWine!MxNz-$j2O2`nq8((QMp4r#z?k2E$7Q=s15snFp7RL}`x_fASAq=H zF_%Upb9}b9w!7TDYejs)-tncv_dEUU!tP482+F_Xc!G!d;d;NezYJRDp$ttKeZEp8V3cI8O52HcVIofHZAq;B<@H5iVF6ptAfduCLZq<6KhYK zVsihiyYMyihqi;z`|C4>bUQ-R{aOzRe>?%#mTY$!gV|6Iuew$TWfqh->nv)lycA$z zqmZ8=)93c^W%aAUg{?WXcNgE=rJ-j_X~vz`9}NMnUn+M&{OxI>bj5s+{m_r{@!_Or zzPlsXmnYlz>feh&-~aTz&I4`Szk$V|lK=DRM8(q0)zrbmUc$)L%)8%1~Lhq?Wd>tetA&(!wy)zP40b%o;8n zDeNw*Wbaj5W(KzEMX4o}L>nn-+pL{=sWjD0S?OpXip7RK@!Q>&1UBkx#EAhUwA79u zc-V%miq;Lh^#yzfs|&2%@{`rLuH5CKY}e3Kv03%h5pB6ol-2DLtYBLTjr!R7u=Y7k zGyDgkkQ?2!ZC&mB)1db7Pn~uGY3~s>^z#QCCy&0A3-|}S#yeu@b&8mS`%gjL$_=Zn zl3F_G=iKhe+I}S8@=EXgsK-iW-n1+=w)7~c1iMUUqGR&_wNxvf=&GDWuLJc_Y0i9l zX%Sz;3!GJM3bM;HQmI*!sI}Y%>P@#{e9+z?owB1Xs%O19R*9y)3f1C)D$iZfpc~$} z0h?3kxM2=P80WzkIWo%m^wNUs2+0IgPfqwf$=D>$-~H`HJAf%iTbCeQ6vE>K#mIs1 z1KTT^9b&-)@Ordy66`j=y)na}XUd2l-%aigWry1^!Z1e6>ioTwJ;hj%+g0Arqe~d} zaZ&H2he5(Rg?Yn{;9k@c^bK zGl<4^dq;y&+NH?MCwaS!@lvAIX!pr5SfNFxPijS7>GTTd#}HGL=#>V9<5zZXqCr;; zhwnL+5+PVCq3Xj6O$jC#v4lb($bujdy7)-lvC^{()Uh@h>7$ISjb@Fh9b*fje7&3w z>hNGdJ8o~~L_>0Kl^W2(BN!-293gZ>AhjjQ#ox}PhCB}V{|*Qp{G8`d18e=*S7OmA zZ6W(LdRt4Lv~6I{`tUQ+_iIO${HjSKZ##N3ovI_d-F-R`Xu8!KJ+tjFg5tezLd={Z8lD0K@%rA4zAFtD_+<7X2}o9+3; zG!xMXught!fR!FnW&dYBzPr~eQLBti_(oIrnwHwuE=y}WPDJh=Ka>cAI<&zq-Qc3) ze*4YujK8tvjiy|qRoBGT7@zgeMEluvdLunp6ernT zgxybAm4`5*nGmq3#ZFu>po3^0UIN`9B;1~?~D2!HqH(JV}?+s9U5 zVW!oriDyl$J1_pc=JU`9u!YTD0MN1otax9y=XtwxdEUAh$ax8wCvKVV?p5YLHHq|g z`Q3L+ zHO^8-vVey1Qso-mP^b9KIo`2UK$`u)USy)IFJ)DbQtB9OcG!FXQwN%9qH+e^U|~Fq zbmIYc_HIEN42+NDj;N9|>M(33#VxIAkuEl!nUm)5{Kd^HzfA4GZ7|oYDy!Bg>(xG zV4tLtmdXJ0{&gscx3pu7=<%RF8EA>FJt5SWaIs9vtYNYi>6G_-p*-4C-w!LY?|$~t zAIy$_i{(C%{me>TeI&dS*gZz}|_bbi3ad>IJ$2`dbprC4v-UjYE$ZTC$vA@54f3w(2 zy8O$ELOBqYdF{H&5X{#?W+6$LYD2JJHj7ERn@5u-vb%m^|J@%l27Vdy$keorHMh-V zL4n)uKRA z8BQgkcQ>5+<8S|~Eiw5Z-!-WEY@~_z(lzRdkUZG_Z%*OFOUm(Jj11Ie?%mNG;(gDg zo?}@5JssBrPB1l(X=jBHE5NnofoG*N>h#$_ghL%Ln!idk?Bc%%O`l)caj8?wgZ(y8 z&r?2t4QyM+sv{^j=4lznUUc4Y89p>F*LOX=SQ+5Xv`;&IsqH_!Q52m2Wj#Q1cB+v` zzRrykK{~-sm&y(2O2J!I)Je&^guM2W0Wys$Z{P-Be>yVG=%MBy1viG34TGRh!-8Wz zfO`0DxR{kbV)h6yc;8qSVu3I9-?hwtc)HBvQ*4diV$0fPmTL(5gsTlE2q!r+b0+?} ziUG4-pxvUTDw3TdP3B3DSCZ~xJEOnsAY6cZ8Gs&Uq`y2yduGZ$w= z^7C&pq*mR5xZj7ip5$XdWy$yiO^_|!x!R+PtgO-g%6wFtP<7K|{}wMsgQkLlx6j)X zhYEahWNGKpk4-nOs0p$NQzVB?H%6Nn@%9`^JNgCCI(mT?nRSHY%^A-RHLP9Bl8L)F zu%WxqtFblh-h~qgtH`2`%{@uz)^_!7O$SIIy}8E21u@WR!7gR@NCWD*!j_!Jt(2~( zth(Y*>ZV@ZC)^;;#}nCv>`WBbg+cc>i1v^5^~hz3ml}-nj$da7&+eLZHxVQic&)^f zi1s3F$zy}p_e0rbrP3zJJ`!O~Qoq0@0vE7SP=#N(ZQvxN5_zoq?QwRq*D{x!bh1hM zo*?s8= zxQvxGV*Up1pAa~?T8*6EiyV*BhZaSt+qJg5t0)8Te~Za{+-~}Qvs0QRI}&)tljT!= zw;L(AIbggh8^_P>%c7WIjW+FE^ezzT zAjRI@2BbvJ%r>iRTz^{rfb3he_9YEkYS`@OUnSXg4q3poK9FD3?o;k4Bjt5gMj?l2l4xrdjOaL5J`QEO1kDGBv2(DY z^7Z!Tf}Wv8s)&b8*|_2UX_94d^7ZfIuaDC~o4STa_PA?8=@|^LzaIyi5%T`qr;Kp- zcY%~X%^^yj)ANkYY+!rJ1?Yha1X+{*cLy2T7hQ;iDU~W*xf;3 zQr>gI-_y}H`ly}Uprs+kAMUZNCm)!lh}|;l|2z;0V?3@qdZDJ5A?W9xCfNT@I38B_ zTLABaOzH`(?741qj*6m47^}wfMQGQnGpstmR!d0R6#2|P5=iJk!h?n z`w+`Y7?=l)LKX)__{=d+z?_{J$}mkX?(|-tCmlaEUbb!Ii69;fHvGoYpFfN)QxUbWUFdH=G9Y%^K%!TFLX=~) zaSxGOgr)B2_x*mVrg>k+YhZ%47aN7e;6%Fu-OpfT>kiOrtUv8{J(ANi@GN>4pKKkb z$XX!}Z&;f#rc(q1J3h~Z&6zrmA zgFl9blVz{fC0dQE+hQHQxJ3AX4WZfTY3apJpnRGnHb<+d!dLq7g#PRgU>K92xb%7+ zWS-A<*O^)^{xBxXn3I2ebVi5*rT&$*Bbf3L2Zve8>3iy0Wi_m25yY^M2WIU=&l8B& z-9&*|yYtONVAgIyI4p|>7nrq+m1ek+EgX~pt;w&6P)eB2Pod>}I!xDFi#1DHAeZ2n zY=vs@nxw?12;NS|0*IeRRT8tmtXP}tEnuubq!-zU@02hD26JyR&IoxQEtXP@^R}W4 z-dr{e!lHVCsRpmN&#b_rEXuMEzb z@akM2>@LeIC|v-(7UFT4Wf|MN{ig%$>i;a8d-?vQ1%)a`TU>Rl% z`a$Z=HOQDMLitP1!ow+`89zF6_XfWgNmg+&YNNIKoFr-_kS|qGT9Z4;K%z$CcyURobofWy?fcg_)-v z!gb!;iIIWa26=x(_yzC`)(M5}0n1n{vl}h$(cb^t|3(DQ!)&$M`i-MZd|=+Y3psZX z+9$|b6sGnE5(rI`xNm@2ijV5g1(p2e%s-I$HkH?AYzW1b^SKG*!W_xv8RFL-A23#I zgx;cOphdOP@Uknoq=0dZN6-?_A%;@k=Iqw*~RK{dR^%A z+zBooWsBcePFbj-O88W51AD6)&X*Ebt``TJqHlR3 z{L(Xt&N6Ht)zl4)LcI~zyZ=4@J2|1=~n z*pwTq=WxK*mo|0>ub(QiXn2=(Px8Q{k64;4pI&5M7Pn(IkYN0&>fd0mapC+D`jQ(i)>)C%zqfyp8p+x?E6G0EnzHP#lX^yXh{xZrERIF#E!d9pdmA6h zeFDRI8c_l6_-igJlIk|A?>u z=O6VSGX^`bi0vy0{8r|h{> zItl|wGCtoFF4A7)t@H^61}mz%Eag;exm8YJ=BUGJ)f81{&T4@q<5e`ZwF{%FEWPtS z6(a&cj;9J95h|V>%=Lefj1{ZQHL!}D%W*$6584l^IM^$Gp1%+ZQ#{Sl#8YFPsIfiPxEC;QazPNMwjUvyeG@&V+xHalo!;-QB66A>;$d(-ZL*QWbmtg?kF8g z2d2gHoW}NXiFY;vfe+NKhWQLBnx@&_+bKMcM1daex%0}AJ^DQLGn9|`?~Fhj z4T=L-xx)3(^oMTjIqMthFUa!e>)t6x*Qz^fUV8UtdpI9C!g;L>_2))$JDHS>)j~xx zKj%83;Fv9)_FrnoFhOk>*QMKM+o$y;hM9Cfe8d%3P^O%UO=AWEL)g2jtjiOoCWrN( z`X8RHup}8aEQ4WF=j^1jsX2eFV=TP^e~UVb{=2SYZ#gMzc8>g?d<1cj*L1PhDB3v1 zt6DRqG~*lelpmpHyhEe2-x)G%EAo{U$^s&qr>|0=og(E+5mfcGxE(hrtSD5X31Uvl z($qT!WEyze2=RSGAXqsc`6~Ax<4+eoA7ak>05WHfA>Ty`3>nuKZN$5*PfAt)EJ}r{ z5C@~`K~50|;ea^c9pi1ifYfS~T{^}qyxtJUmdBpZrz;Wp;GMvGk9e|XE1gH3ufk_Y z{tO;G=xp>Pc>dv~Di@5o4U1IR_DeZhIVonTMEbTD_kz~jp3nT6y9_~4IcxW2mf){P z%UCiR9y~!~Q*He$#Rx&2nE;D^A5@#?n4_m#EZu^QPvO!r&KVy#?j~xctO08Eb*D=W z>O=Uy(d2g9B zE-)9n^>o))-kDeC59d*747`5+r?bosZ{8{yuL%S1Q>=j11Ov5$1|DlUi!~^fN#N0- z;>UQc!^CAhgPjOKWu~4*m3?8=q?5(oOotutCh5Jtlp~Qh{+Vlg&VN~Nk-Ua+*t2mM zy{oi^!=2tj|F(tZYI!N5wNubL?cIgFqnh%e%I?wBD5vDCbR20t|HXdxdWdU=zxshG z#iK*?yz+QbI!&A+bYJ7UGlOY>v!2GYPps|^rwJ0xH1#8E50}qyLk*yn<^CDR-RE&M zf{B09z$v`mD`~{3e(zs0vtnfAF|x+m?UK>2*wn|b39eIlztv=>??ySeDc(Nb2q_<9 zC7RjAPCGQlZwy?6C7{nQ-ADz{%9Q@FjZz7u>a$tL#d*EVX@&@QDz=PUnK80>{-E5N zy^U7%VC3N^E~j}pY4Tyu+)Z}f%%JT~j8t{5aClcm%H^B#&R!U6Nbs&M}MJo2IE>R>H;|y<5 z9M<^zX_~nwszSD`Oe#l7aL>H*oRX|Nr(N6+^Yto&E#-So#ZVHPafRZ;Z}+Y`K@|0# z%&yfTl`OLVDO?dWnr4GMRRGcjJQ-9=)x1-^8g*{;Yy6?d*ChKeBZK5`!W~xy&wS(N zgK$KqCXHJV2TvsY4HcOB{;&jAjY}g{{hJW`4ykO$;7!BQ`IYd=Fe-=^-9|lEmpOJk zRaz#;N=O4Z|81v8{xTz4u!%%D%$`WT5iImAwLn`5Wz0>>NVLs&;aeh+Ym7;i;thu7 zsQXUvw;Um7RqDbkR7c8dqRlpR9X8Hh9rS$V!P8cujDNkr&@g|Rr5{G%QL7Oxs!CEn zf)fc(4(mjmnhDokfqE(d>?H>AD0ejy^i)z2xjJ9I(hM2=-K;Gb!S&mpjedGa-A`{0giOK>RTCI8f>!iW zv7VuwaSuK&{WabmuU>-W*vbDr8eUyOhpn}(=G z@sec@bnJVA52yF5m4n`ES)Ad@j)V+ddrLHf30`@bhK6WhVY+mx^5%FkA{ykKSz>C6 zw8c3!opa><&!v)L*0Ki8s`drSM%K8+P>c}mREo`#MR;^+e~j};rBty*`Z>TP7R8YC zoOGp`wi-UM%*n6c{#iiXorwWI&0UIU_jyOcY zAn*tpGyE7MY8Xey*P`RE=-ivEL_uVr5yacCYeFCk~MdPGM{E1O!fi?=sk{hN(^iKslE_V;1oXBoF zU%4Z6M1Ek%Ndm^Yi`yT?iThtzX+tyJjHh=S{al$>?1v#eL(bqxGYIRsqcD7f>h%ZQ z7A8(dZ7qZ9ae?$=I}c}95dprUIq!4AN8uC8dSI=u^=y-+S9B374dR$VW6M*~p&8#r zSp+!YN#44YfXB}5H3<5DA>H$-OxtyKr$tNBRg&Ve@F)VnF$NbGa|(jDg+g56V` z{CW<+W8EU(SnS81S(NFHIGclTmDN1{Bh90RsOag$U;@}H%;rULcuqUGW|_RIJwA*o zn1etlZxUj1VfREEvn6tXFuRPsYX3m6cfrZvtlPTEzCGnnR`7EG!nl!WdgMXABgcCQ ztSW0=5f()i@AU&}ZPq+=$sH1`VOe_jbL{M`K=+p?!QZOiQB)nTNfvVnUgfu1n7S!G z^T5omJtUij{n=kE+)p0>+;x z;w*0p#=M_|`!xG0{KKts%vk&o zcel{|{Prx}B|zc*CoA)oxNjGS`N_&x!9TsG(5+S^oHQx6X*X9!H-47vQ+DKw;W+j$ z;s?PV-|akikc!2Q^r;Kf=K|y*XPk2D#WI+(K!IKb6zGSnZ^|<$ zIpvE{LQtcd8|=ttJW}UPE6azY?xPTtr_; z#e-te-6BI{+}2PntHEiNSqd0zOigniKeAn^9-Vdb`uwh^KI0~Xld!~T^9W6Lsm z)z&Kv`=w=8hc8ZhdX=Pb5oYHu0T9o?ie)o#)p;2+K%+@V+Ei`$8% zt3~8xK#@*yH~x=ECy;=>Vj&z}wlv)lyFX7ikSCWd>a(c-#_{09k<3I$$vhBKRx3DN z1}?$Q(FOVBN5De-Pkql3u)YTlsk7ifma|WULDGjnfA5+y0ZAMMlz$&3Zfvxm4g6^Z zwGetz^Fr=+7XLnA!UorQz)W(N0&6;?9arXxJP>z~H{v*<&N4Y~Is~)LXZW4ixv#Ac z11Qof!JF~$TXz`olS@+%ng%0ydX)< zLzI3zI1)V=1u81sOea2ha7vJxvjS76>`C&t0TI<*7;<5F-G`@|6(vg-@#ez6nLzBV z4lGn{Dcp1Y$u2t|cIQ9XTfTOGl%m31X9lhBt7d%+S|>~m;^3}B5Ih%EBM>>ezBe?O zq?E}$H8{@aY2In8_016cAj<}hB-?}(tX-%<4%diRk-xL-7t6MX;y&=;m#9B}0_zU4 z(g?%Ef5GG^TW}J>-8t)JVenV-u>H1If(!0?EpDu^wzshM{ams4fRgx7^y=)XgP>XF z^mC7i?1%~yq`3gp%R6Fv~_e%ma)q$ICNS-E$B69(Yxd^xjj#4X0H6%dQJb7hw<=J+ie1L9iwHhG>}N1 zm*nOt2P_J9#@`Te!qMBHxaRU#|IXM_D>1RM=I#ttU0Ul-V?-ACUGTA&%9GH|HugK* zNYN};hau&V_-BoR(mt=Ht%reOY`Xh^uPeZh*^7l6Kc|b6i;s)P*X`-`;zF$sO1Qq- zkmO4xZyR;3#K|$7HIKI-*7qiTo|wepoKJY(h*W^n@3PO*^VK>;WZ51SL37EiKrLj5&dH~NWL|Efa>*7!aj`(}(zUgPpS1_X71BT*33eLtNJa$KUYJnL)A1q#n(WYxZ;8O@vdd8++5 zJEPt4^XbhG@*#Dy%ab;#(@k#8SyG_e`#CE1n$Um23NPhRyVUt!Qr(rZk*Z=;Qblusd2p(1^%jSV}Eztgs@9w}>jLqJoSZTMeK>Uy`Xsrlrb1N6o;IB9AE5UD~DE z3s7Mhr-(~ZYly}YDJxG_9d%AxXpXL+x_=`Z&exY<>mhTTWA&G8;L$$P%DyR-rdu0* z^&L4612acItfDJ1BOq<9ur9A4RWzHiOxSj@JeNsb9#cLG|5x0=Z;Y0ICpdC~Rhqp4 zuX`Nq(aX;kAZWG`0Hl92;xz*4-{mS;#}nqHjoR&#hMKOizN5&rR;Y^&vV~1+*tP7^ zH5bq;(6Gp3g)yeok?z4Cruh=)van;o5!X3UD?5!7Jsi|0X*p55ySaf~+z9W7Ht`a5 z^_|*BAJ1A2awSar0cwyNG+|ESwD?kPV&&N^3?)f^UniUc+%`qDMUMv^?Rw!YT@ZW5 z*2NO0Mp8UQuF0yOV5p7J61*={ZWsZDe9Si*1?*UpP#`Y4EW-f`1i&A0I*HsT0iEmi|LC8nL2P;?l2t*OiyowRdt#SN=2{O|fA6p+CrOtYIfK}VbGys} zi>3MB0>GVIS91kFWtUXD8dJW@obhUf|>h=eu&m0&vN z;~qY4%@OvkuNJoM-*F!++9bestjnP^^1ebn(b{14iEJ;ECAvQz{arCVx!FP7W z_Wy_&SQOlB5bL%Dirw_}S<7Txwo9{Y+`a1zo3SkDpSZHrGsVey@Bj)N>hM9}SO@(m zDhHD;gpfm@6s@vp4c8s-*+Qus9wRo{!?yJFnLTIdgO8rfaz0_=Y zXUAN$D+^x8BajY%bX4q;cMa^-%tnl~e8D7uSLOYD2dIv#QKxNS(?bT|Kb_v$+-A*5 zszjXQ>NIGlN-0%E;i9LUVW>H$iMgp%J|R1ui&I!_D7s$RZls%4Wa=ZQq>Fi1hLS?~ zZ|&W0kJO&y0c_oV90$h2Axz(K~ifaza|Q`!awerk=UFnWL5rdY{4uL_`Hz1l+f_hFK*J6AlGW+ zAw02lE#0Gg6q(ch)#InBCZKHdJ%f@`E!e#ho`Lazf>*iwL!_)Q%R5vkpRzdW3JLj2 z=5%i)N93wdYEBa`#=5)FF-U=?as>ZGfuZ{gXAf556-bfd3*+`!%VruB{wRD4;kyfb zGQJ?X>x@=XMe}^WRE@qey)2ApSCuC$Hu8>>V`((Gw^+-_9L@rFYe2K`>K*AO-JWW{ zY{@m3%gnvoOEMMH?!171FznEAo150y#|JduMD!w1@%^$=(Vu;G6`i=Lc0*t=lgqH9 zBUl7X=&xTrNL=ful5Z*IGW3y5t6-avec9iD76GEBPb8BW_*;cF2|o)EYq($ULbl2y z$g0k$bfjJL2YFa8cuz*F188*ictS_w8lCMIlxCYS`y!WVG#9f9uZ9Mkw=(YUCg}1GGQ|P8zBiB{q z54%50|DrZ2KXN7MTZOC;{O_q>^}jlg3cv{#6xdDj-&axo{~mHFdv^z$e{EGSwd`E7 zB!Pg}kH}a7+X8r{i>i;NNS82j+eXS@gm`=n>J(v1+61DRTjLRr^DnLQ2kud4Ne604qYlwkW`>!6jE2Rmn#HZGYPX5SVrb0x$A#Hw z!{yPg9-a`$TJXwtuM*n#yjJb_E&I$c?>0VJb*jw_zcQ!zNg1IcR@yP87po zb`NnDb7u5YP-$X7IBg+KfemR;Be2H4a3+D{iYsznpzKRUUO_X_Vwr&sNC%Qnunt!D zX{q)|!9osf2dQ+54yG$R2D=Ol@^=5YqAFyTAlhz-X9iyuOXkKt6enpQn?~3CY(ES< z&VJPhwE^Xy);*rfOh@HxAC^KLp1N3To|^LI^e9E0f9$KHVIc+k(&k^fjBp2Z&oD4afDxQl~sV6AW%3)o!}q*V0rF$_es~rOpP<{DN52) zZs|nHR%v70yyfy^lLy+h3|x}RQXP`oRhlgNCLMikc=K@RCy%ZsZ`;`K9E<_`6-IjL zNyNs2ore0EEr?s~OgUF>X}p$P-UBIebo-{(Fy1Fuzn%ZoEBLV~nEZnRa=S4)i*%^OO~jWaWAG98UY9PT^S$ z0G?%^=Ww#NzL&<1WMp}l#)|s{PbOg$!Xz`0WJOLAQCY7kqLP-&g-jx=`(4ULzY~oi|I{qLh!_ zYKd4(Bh>;y^no_cl-B5dTyVn0IZpzTxi&;t!ZoO=&NY zAXof3`V(6hznMQ%W%SX?XlT{-Gga3qhv!P-V;9+nlhs&+h{t~BMw_pjJti) z;RP2p!iK^=hF5rW_LqXEh)8sfGx)p@Ss? z_RcZ|Tr8vB_q#&kf|qVVnNXZ+u+7!7&1%+1iehfrFR+ zR`e~VzWrV* zjNUV|qxT>7$f1q?5qFU;Ccx;GJT@BEm!LhzVC@A`cHuB_*vK;FThNxU6@Cyug zBh3gz`c~{0-6MC?0<_@vX#i+91>d;cLZ+C}`i5Y@>nY_5>5K9Rb=(0rNR0df{xTUz zrMBMYnL+@eN%zoapjjFG7ex74$rrpXWno`;z=HH0molP1?ZJn&&2pvwY)>`Z&A{Dj z8$DG~tyX)84iBg8^_Bf&b(94a8b7O?bn28l>b{t&-)!-l0ObPB8~Ahm)cX2p8& z5u#gm64K{N8O6rjRVYaTDr%RlE_&%F8u+OTG=TYUSJ})RecDZiGdQBz+~oWlGgVVs z!mGA&!|rTUI)#!iL=O0N^I>_AZjn9N*Rf1FW#Pr8V=fg}-YR;sV@3NalmTCh3$?so zU|MtxBwq?A$6B8Sx~URd)FBHxefzTI$~9yflmrY9mC0>+;WA9W_x3gY%F3fCYYuPY zx0Ay$66=^~Zd|$7Rx^a%ydB(vdean?Y2gt}&mw!@=yfYMTQV(hHuo>i+*m~+4-YXG zZw2RIrwR(C4wMINNM|qH*lZtpw>>&9J1>RG{60910X71)jR$di7mhL>VdkQ!L}x6K z<8g&W=dp^)i}|&&keP)+T}Kmlg`I{1jbwDWYj9YMKS_P>KG<4QA7ZGL z7CkYi=(X~;S45P97V|JAao0`Yqp%1;4u!k+^z5HMi3wSlC031`=(`Co|K?#2f97|o zsXlGX!1k+Fs8fAzqqGyt(*`z}L^t^9$IiW@w zvlBhv=XqnXw*Sho^Ug>ix9U_kKEy)#93PfoF!{|Rj_ab?K^U2sAQB~fx6?GfFw9Q9 zP5fGy{8{WJ74B+5Yx=$|0zzH0xLs3}4sGN`>P>`xDYi{y!Be9OFYA4+{;Rj}ojInm z`_q+PGUri-=1~^Nl>ePK0aYGk6-@kQ5fRm-(?rH?;)WtDi1_jKZ%jsa(0pVIU=V~8 z@!L1S|Ceo}A^<43?G>FJJOPqM_NKP~`vk6G^ber%gZ3-3hF=OkWnR8_L7rRa`+AyC zK4`Tipw+H{n?b~NI(if>rO^aAZ zTU9yc^J!WdgFnm7+i@N|j<}c`B}@Cp9F_Jcue1zv<3CrW5L&bPgs}jnaQ_!(sfkKS z;8{tHOAH`eN=7S(jr`L!UQ5b)s}!vYyyv_@ zm{08|Fgll60iUI9?yOc)&uSjfxsXg4RPp0k|Z*OlL@Z>_5OEM-0W(h4~wtmvrYzjGyP{cUH-Y<9q0 zX{1|(iSbvd%FV{uZ#Ct#_yw3cF~DF6c_*V+&LjV2rv+=B6lZ6^GWnOBoi8BUcOms8 z#zv565#MMD&~@j85kPdB2W092k{Zu;l0NMY{&nGR%;yMpgVe{lt+*B_JkXMJCe@z4 zCcK!S%Sa{}BM#Lz$I8VJ%%3a010HL;+c{9qjTQ%G?>YvH+=+Jua{DbS!;h-E!sLxD z^9_}8&Qf<*3M|bes(~Me2y%%0eZx%@ruN$?w>E=Dy#|1~$~}O`MP*v!#Gw6Ff^@m}3A~y%4lIb<>LxK3pQrl=whkHL) zzYA6fNlqAtp4X1lJ;B;nOZ~-O4j$v&lpN@$P{Q01;D!zm)A;Uy14W_O^tbO;pak^Y z0-Ei>;(L8V7l{;q2TO*N_x9N7G8w}*FS@$aThS6IT%835}Fq{LUh?$-?IqW z%@&(hFE|%ckS+sv1+WX!^F2-q5`_+*ZT2Jep90!bVSvdJEQ{o7P;y-^HvD%Ir0KVQ zOQx1uoF`8MpWXb$WHmJZfqkK>%Q`~H&>RaEb2BbMZs~MdHPr{^4Y6Rl3CtQ-qPy72 zU!O3Q$>Rqfp1Tc0c?wVXMEvVSD$(F2wtIWyP&MH*gHeq(VMEn)p(DB1>kNgq&+~5n z4$41dhG{@&Dg?04me1X{ak6X@g%QGMH!1XjsI1t7(`p;3R*Ep$<4Vj4xo@9PH(IHR z%%mMlTw6oFF1ykmjlx_UmPquVxO&8M1Hxfuu?ZW;i{1c0{1HaQS)4#%9VB%r0UpEY z*B?2y)axQiE%KGK6?EU>+ma8=?^nTmnRVd5&$rUlq8+ma8oHdjSQ%i5YPy?ss<>o6wHM}HrC*C(ib zuJ8`TGTf9bp4DeZv(~O&3yRwE+|iuAAkRuRlZSkgvx2xLr--NX}R3BiI~Po zAQ~Xhe!M1Oy#3<4^Eq-MyGUrPVN!W0l%5Z)eF4f(U;o_KeT%mUk?t>KN63HSGQl3V zP4IH+@={Mw0&VPsn0<`_T-?>l-U$i=w+TYsr3qiK1N z;!n4@$2m!K8055cIlWikdoOjlyJFtc2T0(L1K#uXe3OoaXN&b+*f?IrxQrgVVuH?a zGKu!LnkF%Wrsj4rM6-Pa-L$n2ium#5@+WnOc+*v`~oeDaN7)WMG|cg^QQMI7hJ z7wiUE5W&{d$FN)k5{GXm28ut#*H5G*J9diO0}Xk;XKe2bDq`xIVP}oe@ue-4$?HS& ze0rH7u3CkCCr`bc{66uE{}TJRlTN&Fvv%Ob4(t2<4|~)*2Q1kJ?#dPQ7+)^tw*kptEVWZ53&O-8tz>){;m z>j@X^noH;A-8NrOTRj3QRQYR=H{|)VH$=VCzg$yl%^DdL@=%9I5W%;9G^;%P8j{D+F0#dF{E?KR zR}6(fWo^b>D3)3RfZcsv1L7}|WMxv!NWr7_u%yHsEl%#HFLc!uNtU^`jYsaQp1pH{|D za83>K0BNpb1O3iIs48Ce#nn4nikB{)6AFCYr=3AE2$eyMsCHzmya-7Kkw`e>S;Tn` ze22#8#NRhD4&$;}2-7HPn^y&sj3Iwu$bJ$XqbOlse+Kn|G-y0<>^G7;7r%hw;DmOm z3P%NhZ)mnZ%UvmAUDuaCW*Ljq#5^hm)M7k>namaVBS6hiAf9z5qz7A=7q}lu*#_qO zzyMpGIpV1jHYuIn9S|wlQz#Y>^vH;am~9Fjqm|`) zdtgDNJuJ{!mTiP4i0!W*7Vr#vA2EC!_53SswRDJMd|km#iR;Jm=J`#ui)Z=+8i!No zIy>}{d~MypQM{0z5V$yT7Os-MgGPgitj&4{VNZ{AZJV6cv`yu~2t?_btH*sNSEcjU z_UxGSKd`(drry#VS2#oYnA$A3%~wUcCdwHKptPTOr|`4%UQ1z1gx$PCzN$Ub)zMqzH}c&i zKa!|j2PV!7JV=D?gP6SW;c%h{OHd^AC_9V^%QHhn01m|sfMQ}YbdT?Ag*0U?Fyn0- z!|W$Aa9Nu!g}&}T1GUOK!Ga2h1#;t0MJWNv%QL3K&A{~iJ%HyTr`wV^ANwmSe84*` znk@~<&m`aoIZHtwbbF{6WeWd}zG)_NzN=NyQ#fXeh(cF(1hLyya$7c1Anz}|W=a38 z-+~I>rsL}@Iq$}52y_aL^3sh9Fe@1mczy0Ac#ZDzQR7S+GRSAEb0>~8gC+~%S{HUD zxU+0D`HT{6U6U4a4O=x8(vJ=a^ObUu`+k-d3h_qO+ZBvg{rt)w?^ZKNNt3LCR519c z@H4|7!iSof9|is0rYJwj9m^=V0j7)I3BcpaWY*IP0!i z)Ig+f!=SH`4n&dj#3WDA>TaEJRf7kb4+Z6R7cz6RjSVshtjL%>jz~C}jLNGcs%-AY z|I$rXxkNJ%1Szed57B%J{-?uJTo1Q^fH27qaK)%KG;7l#FcCTw!s{dV&p3+%=_=R- zX}_*zYvOsJiDWkq4`mokn^Ksk7t{;0xU<-te*Ze#Do7HH4`Bw&@JbPPjJ7~m=tfiIp@nA0*5p~bWUQxdyC`C1-O`;c+O&nhaJPWhzOwY|VC znXs+YwAMO~4c0dy`cc*@BVuOL`?~Lx3F`)^{uEJH^R^SlNmj5Gev6c(io9S_D45C8 z%^i>utGZr8v@b!S-^&V(MyE`Y93jndM4LFwsSg5fEXS{5#tWEzoTjM(iurs@SG-gO zZc=q6jHK zU{I#H8@8y)GA%4dx59?()W*Qr4jeO%B8+ySMevZ1t%Tla3u?)WKC-fc7g16q35BIs z91`zYc_pO95@TN=-<_|lRI6tF0NrDl-e4)C{&*q>TkVBRBoLuii z`X*kb!f=M{;;?>EqQ|Y?sFNHk(h$%km}UWm9VX=L1439ngqx@TGmU~|sI*{rFy6Bd z{g%FDP6k+*ph`!IW(uZmf>aeztfL7h6gN_`M)onlhh;iN4mz857NnUcQhlf>qCwt{ zFsjllx-#qjqY&&2`rGNsx1h+1h*LS0%_R2|Q7D}Co=3Y@QBj&46wIO{X-DP`(0H1C+U z>0mbKrBK$5_Csq$-f&QMSsr--Up0FvBxU0UTT4?Vhseost7~v`9=X$Dej)`KRf<7D z`BnW$$R?A7O*Wbs_QYsN&a>dkJ>~=~eIwHM5@R=YXYG;gN5;ad7C@Gcg z{BNoR@J6%|q5k@;XvEpg3~!|O$~Y!9sbeB0nKUWLmm#|pJ*dB{GI z!t**mC$6_2nDCPba4tpPe4w|f;y{lUG>o~w!8=BSLw8=_S4}K=_XuCq24Pz@cpCrK zamTxYS04VO;)u9@`swj;mVeK$zpwVes3(Y?ZR}Wadm)vOPF(LRiXxxFXg`5D@s5@AnmlHB z$u7TiP(ga`%=g=`R}HJuoy{zyA?ckMzD$~X-dJ0Wr1Mh9xLS?ZVaY_!Up=G85p)<6 zU$sU&X{?T4sAJJ#2Qrx(ZJ?zRuzo^dZUpHe+KyY>;^yU*U-XKhQ9%#Lzu zm<3{nOIVsyU#@vOrQI0wH#oG$rE9FJH~8zuZ#KH4x9~o`@b5GEHZ{3_1o?a4XShgf zvG1AO_i-Yan`x!QfQ84#iGWSznM~WMxwjB@rGA!eqD01J(de^C9BVamWAm%wwaO=R zZe7c8@+{tJeYnaiSCnlYX50-fBSj1IxSP~CAe8#99>dcXKfqJaQmtDr@ zi;&CeMa}+NQKm1_P%K+@NI4qHrTA3Okmj9rdHrUJb9iDk^_+0zK{r89UPuScUUBEiV9$+2ge>KHZ^>hHl z$=Lz0N-qCt3~f3A0@HoN>P{z6ZICZa@6qbo)TIWd(&!pG*Y>H|BY;z&B2z)NgEUb{DSE>%@YQq6;BYL&T%+$#4^-_c${M^emVdM(F5CfEje;_``{AO{05sYggPT z(==u1z#~fNWP{P1m`a0^nHC@NTf%Bq9XLKTbZz`d!jXpO68~*n$a^N^I0|43IVrD` zFc!dE0Cq7nqRzc&uT$#(wF(&o9CfYb@Lru0?vJIBWm2u|29$2F+3^~(nB91xG^SO!|#1zd926IXP&2qwx zdWZ3nzArAd1^?YPep~{qVgUVX6$2Ed_Dt;WDu!Rtzf2+BXe9GO&b@ZHr(#M^lg?IO z=b*X~DK3(nvjh`>DP$Mp0U-x3h-Q9-B_K!Ye-4fl`@2cZkSDCYwT9S9NVn16hO@-b zm}@5amlHVTa2#6(X-uNY>aAiexz8qCN9|}TG)7`6D}eZz9fbKriA#?CWdo$i|92PT zPn2P}Z)42&Lm2TV{X$537Si3`)DzL>k3@!>gK!+C<#ACT z0g;rYgNFpL?b?VJ8h8-&Qgv71<@-DvTHYokY+C^o9@b4O+j?HrcL<$4K@(jH(Pv|! zHdNf%d6zb7#s~N^QlLQdwN^3J8|>C7^3`6aq14MPj%$D=MEj2=WCO5-+8{_Ib$hRH zoT}34nGs#=rDMkH?Z7Ava9MHKkRv1nJ#fU1ON{bo0jFa*u5Bz|QB{Gh?}!Frq1t6M z$Kww3mwC%=llGgtuCZuWZ4*;XylRH+H{A#5W$1FxT&q~JYoi_m5eX0F-N_q9QYLtg zkMUbJDsYAsu+^8KF~n1?u>{f1j{%wrnGhuUlOiK{b>(4I#8=tu6ykJ?DSGTUSCOte zx>TNNSJER58zc?ay=MMDV7aSk_wxm_UY~}**E0e#xA}$>2KMMDix2)>y75jwp&Sa9 zCP6w*EguT`gf$aQ0^yX(dc9vAc$!Q`JE#hV@0>IoViQlz8q|lqt+Qp+b1{<^z7N|11}F zepd*vi!9w$9#FD(%;3TIQ#!@Z!J9eSbe0dN@eaWf5D-(h#cyrse#M0A2=`q4-g|t{ z$gX&g&(2P})F!J^TcC1)PoaOv|4!vWcPrtN@JG_TAQt!o){lcR<-_ZC&zWq5u-l8D zR&u4X?jDz6QCFRlWt-ZUDsFz`+#5v3G*!EzsGo)z$Y9jCzXyhWSL58XL^P5g*xXnd z^87k?;viha(*sxS`h&VVt=137`Ob!YOQyTvt`f4^D0P)nuFfB9B{I*ATx?&k1xcJf;2|5=Xjv_CQ)1mV&_%g4&x_Tp-N*EfO80gI6fe^hX~FijKu%8pW~s#^@tql zSVAuMmEhFFAUz_*rb6O5k7l?^2W*RiDJEWY<@*qzGr?e#@+$VnuKwbjw)a&OwkGIZbOlS#;^_~(W^$QYa+@0JXCnk~t3$q$G|*5q zyd7;|x!3I#`xzm8JBe$$$eux3e+qFkR2VIbUVHyb|K9i_L>z z0WvV&!V(Q%(|9vftYWZMRD23T#m#sd?nC+z+Ro$V=tMYA<#+9mBKlPzdXYHKe%t?I z86flaQVVM8sofCPdV2!4XPE!+@ck1d#fIO6z(L8Kg`-xdmkOwf%#^x?#w+TkK>lZTw z#jp3?T3bo@4Y$%b;x2_5+wVJ1+@=9h9lRK>%x_U$%x%MDxH>KhqETO-tda&7I~nL{ z1=j^}r7d;3(j41*lV?R5L!Ge;IDid0cdiSxIbg<|!oSineu~`bVpj41j#(n2uu&O7 ztlmhzN^J(%ANlXjKJ89Dj1KGx%g4{N+TSdIADLKl@Cn`~)zB!9n%?)U6X-$Y;!WIH z10Eb(&El7BbUSPuFKb&b)|%tiFbEz^xu9F7T3#HSeGeB$n+6Un@82O0vn4f@yG$J2 z^XhhPzXeCaDANU}%v=wnlNtXJ=p)0s56Q4m&lnkm#4(8qipk(p9@8#{Cu>Qtzn_X- z5(scG7C+>R<3{!IVZ67V9;PGeBmVsk&3quuITC*bk8_MMThiK!RbwFqa$Q)ZE4w0u6unj{qP62Zn5EpC>geI~IP%dIE zga?5dP+oK`g>(^BfnCHvM7afXcPK6>*Vm+Bej{DtB%*#Z7}^H7;%e|gQ?~g;niIN~ zFftYezF|6pC#+zG8x+NdGaO|$Ymre!RZ2+z zBpDAMxb%2j?=^*?d9Pj~8~EX__C~5?RaVs_wNgsgdSrGrv2n%_%RQ)xIgewn1R`zA zO2fsS___+HrTY;oey$%qeLP}lyjhb>cA@=Dv880#WKG{*f)p9gL%d#FeSW4D3kL$g zZ2H$72|p@Z25%?RQ4j_03sq)agu(m-ng)njdr`NEd_)6M6-D3ZEhsa$76^fy>-sFMuz&MK7O2pD_uYL$sagJs_fCScRN1R)V4xX5$ z-ZDT2Zi&dR&k<)YRLxVt*>#KB+5pfaI>NzIFvP)PUw8Xwj_|paaEwBP`yQUSrnv*T zA}gIIgJiZtgs|&A#_5NY2~Ja^s>_4y+j0(UJ$>8t7uKF}uI7}36lPC%+^L4fM&k*^ z|A7AF`YlWh0`m*z41^(Wi4PN&8E-$r*D|(@txN^w#egdR0ZK-*{UBmY40;(-pTc#a z5TnQv+>c=mN){pecuQE6ASgQF@QyyvL=2npd=O2n(DAx&hIs}S^p(QqxBA9 zm|YX4w4qZY+7$$^fPrs$%iEJ6l-6_BxGZBG#}RVnrZGsbI^2f>7on81JgU=zSNYZd z3z+f7ued%)Zg{8lW&cf1ny7SI4{dfl4d;S;-EWRi{QC+E|8`)nuUTxaMeYcCRa3uP ztispqM9+HQ*Kn>0UsX1rzUv0M!j8f!qyLJ3pDX*)WG6I&YbE>57jzaowjJj38>nnu z`~*Gz%!*g*8o|ZVNy+!_HP6;4;n@3xP#l>k8kzN+NER?PYm8j7+dInv&ta5~2FM8y zQ%n(CLL~;q1)7qCw<`7sCuuq6qz$AiN&Ty+r+}&6Fx#DAeimTXm?Xbugz_`XmQxU= z#-BAYIP=K#WMzCrz_244V|bD(NmSM4x@5$Lwo1zdlHp_A%uA&Y$c&BtvW&WrSks8* z583SBKPx{sC{DCet0E3d2M<$ZCrhh4=5b?7_`1VOm1`GNr$(KSD!$O7OMPWg>_cD_ z^&dYogoh4M{BK{`x32ztBk)}JwmYDgqCJ$|q&Bh;9!#MFbp_%2`B4J|as(Mw`3GHy zBHRcF=tD^-k6T8k5jM#!h@;@o2~yuAn#Z@467_9xiQva#3*cY6(VOO@^q@-lX=X$U|Edo#uDAY{3qYgvwnLq_DFLVp&awY7}FGf z*ZyG}9vM!QPqHLwG;>EqcgoieC1ui+RRy@GIYU({SyA~e*Q9YIONmt0>Lh%yHj9pge4OBw}_@Q45d zYhyW7?i?vP=-C)RfNz(xU^eu;{+8#sPL2$=>BUM--L@+;fZ*?|9A;F(XecL77>Y$r zhN5Od(Ln`VdXw+Lav&L+9fGNNbDe{>Z+N*(TR}?smi`CY-T@el@J7gWn+e8u7Jew= z2c7|5UGI@+#0PanoJ!PLlR273UISa)U858-WkrT()F21hRw`(l%bi+=LuDo$gubgW z#3S(xf+j6_3T|#G1{jaY5JoQeYZ*5>Y(zF#~Xt0l{UKPv*FJ;m@Z>`RMc>Q>D; zQ~QNzZ&klX`pBz`KZ)-TqjOw_UQT$|NbFe7=VDvbU03SD);(oun%7mj{mM4BG5Ol+ z>WJyMICJ}z<6o#_rsF5PKBU=P*>yMFgHzi`-*aknPmy1Ib}?Dug$wB#sq-8y8fOCq6do+h$DlV=&sM4745pLdOO0e)c(66% z`Y$E0NuHdo8Sjb?)H=_&Et>K<0>`-QB6Xs^Y1PiRNOCCHH+G5H1Ra!P0~T77xy&Uk zD~Oiiu!647Ooj5ZXW4E4>Ep<|YXZ0Q$6)2^l^dPT4c%+iT`uaPS=U-E6_u9_?dUX0 zLhy030<`WwE-(M1jrm^CGBO4V1T+UA{{iN@Kn|X4*7QsaTnx;N|N7I~$jyk+#mw1- z(Sey!*iOON8W1@0&+Q|C-hIe{#6SAeI3c-5F=EV!C-~ok%t^ z8sYl2KRejkT3ubALV=I{W!$XxJKlIoZG?Uo79fN~jiYVyg8aN0>a6X5Jf4{guDgJa z%}(;viZ?hTCMnHiwnf+2_L{s;RqHx2)pm(sfPYo4^!$GCbJ)P4l7knm6T&^coKu^) zzG$KD;^bPZy#)1GzQVRS@xR=IDs5>IIISd$0h*BSS8}a!lV1izPe<1}0?tRn4PYFz zMtZI9!BW{<%8GU{TP8kP?%vLD&H2{`c}pi&GI$>lmfKM(P%Jyz7}30*hc(M-(G9;n zgRK!^8R@pNJ#zZgn=Gevs%=jT-u3xm`gPw6r9K91eTBmGBWM4S1(yhIsR>iZKGCCK zqU#)XuE)A4pwB&bS%0t?{1kqjfCaWNLLn`)L07tMhU8NK@gg0m*IA-AHEM5x^Ad$eGHWkI6C^`&-fQ7kj@F-8c6> z0b3^uFjMZEM~eN3aQHi>)SO5l4;*swsYP-D!O#%49*21Ec!(saFEgeR${l;OeFEqc_|up&v@trx z`0fNbxIZ%QvmrF%2{iAcXb1DG7Mb*_CwgJ@7z?zJjpmS5zW{Rz+Hn_<(Pv9iY0#&- zA;AbS!sJl2(^4k zrj;l2RG^^F9xT+Mr2LV>ct{v4@e3D#Nfc$t-l;GPxOUugTN* zeV$!q`|WQo@HyFG`px#be1mHa?PuozU{eP!qtJ?A#HL&MMgo`jgYOd|9Jf60X2+X4 zTR1VHj@$&2u-%_aBga%x&tih&?$3``tUR5~9}5XNs1@$&wYX@F1IJ5$CXW>U-HXQl zdFC$((1S_9wj%BSs~)iYryy){A@u+fEWNB@Y!@Ed`R7q=Q4_=&rH3Dt(oREa$Z$|n zZ2)@E%i0|5@!q0Yz%#}cet+DQSx3vzD_ zh1lA%&1$;bb4kky@?9sQToUWTGF);~mlOt=db6Dw&Q1{g#QVn^MA>}MZ=;7r#m)96 zcVBkC!G6T;jJ{-8#J%vB-{nG7x%ONA%jiaAitre5Zl}6Xt$~MognGfzH;UlhT0>Sk ze~#X~fB5cl$>5kMTMS-X=>PN`wgL0^C{%rin3XpXVlNb_1k=9!Bn{Q~IrDXBkz+cR z0I@-iVPcSdn%3~CH{MpSZ5n(;^fc^&71Z}Am@pTy6D$;Q7y>~21Dgg$R#hzCkpP15 zt+QXi0}f$MtskM*dQ@Z3q}cd4G(f`$g4B0es|dwPVGSS%6eIO*e+41<5?63f)ex

      m>LT?M_FsjB*LserqSCs;K1x) z>e6%~{wDkjLo>cVjGAp`#sb4`3F5IgU84C`YGMVDL>>BPQzuriXlqFAOrcvpPu~I3~n&AiA z^U}2w0_PtG97-o*ksI4Z{-p(dOuW(NlO)y>Z6zDX44KH@iG6Ovj)d9zIlr)n#}Hy6 z%8u{I)Wrp-Lp7;hSTJSvOLB^m0$N4xt5lvt%2*FneJ1Ojr0+~t|B(Xe3;GD~`U`oa z*F__d!OXb>KEd{?Nf%eQ{Jdxb@9n8LwNG$Cc!%y!u5A2r-{@KL-tQ70iVseI)>rmH z*9320ZS>vu>eer=*Z)U-CYNxx-vfM)nSlEI|NK3&{>S(DU+}>EB$0;PR;^8RPzu>* zIUx-GM459*ts+#nMoYz{V4KrFy}TR3-B*|pONcPVhrZ9gySZbxzpvkM2fjyKe2r(m zha)gd$a=7LX$;79747 zHlX4v!n@W?@Xf96rBC)8Wz<`!HZ_|qzOXaGeTypc&QXZ&xG_MBSe_7m&~Ql*l5Nuu2<6BBZUzNpJ2 zGOst)YVPwNq*iFxeotbpRi@z_^dhbv_;yRBFaW3S*^07IoY?kYkPVEJ7dJdrL>B{N8B5|2N{Z_$#eGeErWa zHoUCuolb;av~Xk73#m8Jp$~N%>)3hUpB1-N!^uB}zn36qGj?9=fKcqi2U{q?YVEQ$ zWApRNhVF(B_G4XF0#_D~dT{kA{0Rt0Yjz1rtsfR`4%;};T78?+xjZEs^Mn)ZU!Ix+@oJ+ZJzaQ?6`Z}Eg@g6+Q-?!PbjITe*3(pzNW z?$P{ne`_~+dP&Y3m8?sVo-~^n&$?m5A72Yg$XmCc^+K)*RtT?v+AVs1xBcuZaf4vo z2_v2f}I-4&y-{1;DZuDNoEuvxg&G+X#MIIf_%`zHj(jX_M zdREa@4}0r1_@{;n$2~%sc*bBZ+wb{VUj#IptMIl?Mk*E`@sj%2KaGz2>ea8cCg_6K3r+$CK7076?o-AV>90t7ReJK69Zve0P z|E3gp=qs%o?q$#YW*~!d>6(zy{lJ|I*@e_P&(uCYc`P@ec1TWwE){&vLze*uo*rfAE44wZ8sy(;jhp2;rXbr2P z{X;l|<5fsn9xOOb{y8rN_nrs!eX5iPNHRMF_t2{o{jg1~QC>N8I*ex8o=sV41{%+3 zBDm&dM_4`;(;U4@k^V(n58QX$lg1#eQ&^8TTi{ z9tC&?)_If(Pitj;)#xhUP&On3rNMAWzY-Hp$b}p=Q0UbaW9f;dS%F!Pn+St`r52M> ztRfEpHkxPA{wVM9#E1@H*o6J7d}|i9C-?|Y-Ba)XFfMAI3q59M<%eY>-p4}+g~VlgQZ9|k_=@l0ka%I_StA>mew#+)yjTih9>6E!NIu#yt3208r59gY=~ zOi&fR13!#%8ZUGsC<8=RJ!p&&0~O~f1ZDKAUrTN1zWCjRq4(OEdn0w3@x>}&`{mxU zdDC8-FWqC63sApbk1bSQahZM|yjB}n)E1t(y{PrSj|y*X?4)F`^w}5Buao;F%_xHP zFi2W>3=rNjfJkDhf$MbB>h!@s?+t*ruz=#fh>+c&ZuxZgemz+Eyx+R{I6Hf@f8B^i z<@gbP{*RA5$;zA-AFwIH57-nT|NncPiyHmYs1K|E11ZA$6P5jQ>5PriEy>na$*@su z6wGuSwY22*@AoZ_0p3H#9`^OTt<-o};<+9L0*e&Bzs>)=HF!8LY43lbfVdM>8YP#h zzq`Z~foSC8^D)ymJ3kg6t-8X=lKdNfZN6oyxEX|XQpj6FaY81ojRl(o1{iQERMq({ zT`sp0uZSR=hfSVk8B0G*@{lSz*FnReQsg{NUJKYuG<{X~hd z9it*Km+`$pg^-3((SbSry>`_%LoAn&lY%+T0iH^;ObIPl#&!WJ{0@yiUn5}cJ4^b= zsIu)nPn5YZtU+aEN+No276sKvy+Cy^gDPzi9s8P6IVR>-wCcGPb)Oe`Foz2mR@do3W=*@jHU zJ{Q|9SI~Tgfxf}mUTUUB1qV~sZCb;83;|RSKEfBDeFwm2-g^d-0}a~#ox5;)+qqlN z2&fh_VDN-bsI8BciZRd^7kUTnc&g3(rH*?a0kPI)f*Mg*hI815g6^B(p89Eq0G){c zK6I?a_t!qa75-AmqMB>Iv@y*#s%tN(ziboLVq_xH0lLKA4GZAj<@(mNg(USf+bBu1+r0{?PUPlC=mgW zMtyr*_ck?MhF{=q881L_l+8|sh7c6c-(>y+Qv?l83EXmS*XJXs3%2Eqs1z!j0>#e6 zys={JU0aL2eD^^+K+gQ2h2_9)(jFgv<@O7_SK0{*Z4Ty%pPBX?`vM)7f!FvPGX|7S@JNsi+nh>a63}|DLs{ zQpCS^t$$r`FMW@LWD7@V^s(>PRm6=x2qS3*0Wyj>!)9FvR0q@fB%Z?lsNfs*BFZKn zh^XR&Ce*1!lZ|M)MPu#M^Ea<-r z{i&38w;BQMWW)O7gyZ5N$@OHyGGB^6d zeKCJ)m`Y97;f0Q56g+u9d+L>-)Y9Z|@X!J)2O;NAc>f_YGtavl zRBL=#p9M+rF==zS!thh6hl|IH95c47Lxg04f%pHs)2t?ICq)EM;c#L8AG$3E7JwmY zWoz`$CP$6>f7+pYDvgS{hy~#sr^0EL*#UGNB&pQ^RQsq6`Fz4Dm~g|N9@dnF_H`bK zK=y=?Ow;7FTj1 z88@!_ny>1(HAO1qD%v{7pa+Ay)#^2DYiiWUFLds!Wp&r|>~7OVaw=IEJ3WI2*hzIbll&OnC+Y+SQ6gwL7%2#4NA}W^68Db5EH_=PuQR0r;aViSp8Q%Qxz?PEu{Be((rW3c}A601?~ju1^$W zz$I1~Lby@8+^098UaBae6y$?zXj`+)Cu%!0#>!w~%{k~N zQ2n8L2LKYI0b)62LPSY@i7|zb?#W>($AI5KybT+`8g7bD1%D+4`A6pR)JME8Mm&Bh z+Ks7TNG&|}!T`E0Y)>L}y1iyY18kYQ#g_2;D{X1xe!p2JK)3ZK>ksI*X4irx(M+jz zQ0`47xJa^L#?2_Or;^EBdSTN9ZUQ2`BT)8pCc*COBOfaMc}6|5q2|E3)Q%>VTD2ze zMb^!QN;X6B-!$;Jm{T?^#@LacWjTz)xnmIUG5obeM}zq?-~L*npq@GZvP8)+j5*?8 za)yD6Ek%Ulb?d7DnDzp^9Zvs;24{)8?F!;HKTo!)JYJnW)|){{`J;DJfDeM?lR>zJ zu2qkQ^iI-u;4V0O+=IFRrg7N%gJ)rzPJLbUu0oZD@Q{Z{N;tw&zcI>h|j9 zPeJmxExOHa1R8}NfA(Ka^Itx{f?Q*Dw=+3~7v8>5v8iq;j?YTI^S}H;A2863Se>c# z3R)co<7IEPW*GGycsciTxxMUaEHrYj{dh9@Y?>$UwV9d}M9M4vp+Ks{@#zzSlLC)& zkcXJ@T|K!yXngCo9)>+z3qgF>Vyp>Q4lm>a`7>hyDrE{p429$LUktxw6J-R zX#5yaZ4dX+8Sb!Az8Eez&p$57WW@^3$8QFBPf*(aaaY10^xvDa)@~lnK!C1P16V!m z|MzzrVE)1=;%aVg=JX%Asj7hC%Rg=`!GF&?$_3Q>TX~I&i7WI4Xun{ zCpS-bcao4l>uT{bENZsLWmdd3O%;^QN`dPt3!XXVWJotHwAuj@(?BO?V>xOD-^Fw`)_F-sY}?!|DVdtySn^~(Qmo92b*VVEi&z9k?Ux)(9MsvOub zAabHdpL(fA!35XYCt%6es%W9!YNno9yysi|HkJ$$7Yg0{qdHBjw0zr05ZY1!Kxbb7 zC=9$ZdgpM35N*)vh78tuOHVU6#Ykch{YDr}?9K+}`6(BvAx`FEh@q}@@#2Eb4E?}H!~?!(fV@P1jlj|T>WPQDKovmG}oV^#%ueOI>&nPHt5l_p*}Zh!F&o?5$Tq#`I{F=o%2(%Nc_ zwfHUneiD{1BVM}>>(*#lxQQpplT;iS_``4orMWo&uyg}bqgtN^-2=83DM;rbG%sQ2 zDC*t5=p8R2Yq_c)z}MlRQwUo>1Nb`7I|pf~ul<>G4iqQ?Gl*9d6+a1<%byQ6{TRyR z(*DG0fU><`?;0qSn{A(B7%1^*I{A#|Z$0PI)cnTorm4<;M0}DAx0rlP&uP=r!Y7V5 zCywR6QZd%>8#^yc8p(Cvtg+Eh9_MK46mKi@u13P26*MtqJG^~Lx`c-O%8{r1IcLp- z&7bcY35k9MH`9o+sLB(KCa8*R*+@j?oK%rs=@~0ReVk|8JAyznUg) zVregGV`XOd|B2au&wgfVMm?HaIt?VzxC3FUxEqX90t1$-rK2FzAWv5hB>|{emy5VR zm3ZI_krMCU0W$B_<2oSQhBJ_JZ{o6-{+bh}>=kY&Ki|Lk=6;(O_E@WmqX^}Zp!-6f zO=~?+MXjKT*HWBv15#-Xpk|3w6Y?#;^y*~Rg;vn!+(9NFc-GrG=4jo`l3TnrCtI)R z)>5rO8<%$*HA0jpi*(aHC9p6|YXWL4e=Acf^|huVngt?SCsThQMX0DbX>dFzOqNkq z{8$ukQT&O}y(3_<5~wSvJYg2t;ajd4JCQ&WXzqR&C=%bxrkvVQ&!p}KrU__CV{|x% zSq0XZ>lzy|YJ5{5pKxDq$3WFE_?_N?|AU(g2Lqe=++vyHmf2T-3im8*Tp!A#QbgZW z$%|FXDWeewh8IGPW!0#Sgf7=L`uVSD8(!h$fM&9YvIP3S{&aVmOu&3qa|<{JK+f1l zVg}@Em*d=AGLTPONy?%7JCpwPV44CXI7FyBjhjRaiaQ;O!QFVGhpuj!5KP%KkM+T8 zv>_YlQ#fG15kwn1fFNoFlO5c%K|Ag*MK;`}>`@(qwe<Iy!3)DJyV$`L`K-n^nKFwsNPpUWCD(h}?Tp+|24!fmEHW|MWco%)jlHt6f~ zL1`&v+Oyx}!K&D_LmMPWeF@_AcK0==ni2tXWVh)cumjoAXMs zRI7)`&ZrBn9!~S(re``xbs3XqfxgC69C_kV^GcwD&9J;So-@Q(x6E$AUyQ1AWlhYDl7|YPLV{Y2u<}o=rpARX zy%OrFWHg`=gRwguf-wpq5!AgmE|})B=4+Tb$#q#ouS3B()8SSf7lVcMQMgSdV<0-m zxlp@`^2epnK-D5Qo$2}*usnj*y4H)MZ z8^s;@UopyS+EcK@Eiw9GB(xYkg4|u%W1?j7i;1<+``BxNbLoQM zlP#D=;h}jfXfCeaiW)nds$E0JyU(~c!g1g>qGyHTvb_3Ld}RWv1>BMrp(BrKs}5sR zx6)?E2O6m?36c z977Lk>B~qCBW_#$0<40l(}}*f#6*L9n-2k!9#0RC4D!h4?yMjm%jBW`8@xLFGQe@F%Ur{d;car1v`1pi6s0dh7)_*VmqyY%~O!Tv7xLw5D(i35SIVbxTw_=q2Hp~?mTPc;?&P9AfWk~JQczpY?6_1ILyp^Tm6=0Cs_ z8UUV~&8>AT9Dl)c<)rBfw}oL6-PtO)jU!0$Jbxnz0G{y1RRP(S=X01UJ=Ke@CjKDx zldwKifL_pBkG(nWyqMBL!3<+@?1SmvY{^2NIu^Q&#k|%IA4HhtNx)fZGJd{cm z$fVOi*qD6xLjxp(IgAv$4MC6`si-cM|Lh(E*~dEtpl3a;$H9}p-()Z^1~g0-TIa3h zawb&(XD?P$3y&R5w~(%2ldiHUWUocpen;sqVu(y9qe_!+Xz{Er2fT)2)S|EeB_S1P z3B75k25=p>vO$G?#ip_G|C_NH*hJJXYd68C;(UR0;zC^l-}{rSQC6yR7FK8O=l6yK zEDXZEqK*e-a86x{TQCImZ;>>t%s2awDt&5Z$Wwca`r+RgpzYZcp!5ovhr&_hcE(Tu z$zT{dK%ULAX^4mvJ=4oX0Mqd}xA?!N<0JyoAox}Ge+rTF)rCiEo}J4C5!><0-g8)kc`ed_9m zihoVV`Tomvyh8%O=2e}=8~;eCCW|fLDz^_u&I8Aot4uK#*&jXXI=)q{w-;x&?6^mX)<(W_ z?Lo`1-6n#!687@>O*x~jDZxc9{&dQ`9hY5Q9k-y*$Q5(*!d3pZwQ@y-)W%aepSNv{ zXZ?9JW7XIhLo4f8i|{c)GM69$vDD)1M|F2u98ylY4g~GCWjFP0Eh)_hPqyq={;V@| z+vX{<9DCO7&-v0U_@PzYPjZw%2p&EP$4D>e=8B}X$*r$94nw|Qtk4zDFX+` zt3a=RPy~8%uUiJBQsD_uwD9 zZCQg9``-p&>-;^utp+9u%la5#5|`MV6A`^D#5&p>TFTLv99?uW#oOcIQ+Z$A%zisng|1L3LGiUfCs32GT^kur5({PDJY9Ji42QSo8zr zNH4KOvI@%2?(QHKDyCdmi*x4V1EDF)6(FW;%I7hPnkJ>|3%~_PP=I0ipskHWAvLuQ z*?p{-a@-1-7BPy^In~6a2ZFCyEWr+|Ip@&-tHH~emvOA}|J`?NEBdqdRZ3>k(F9L0 z`{&JCZG=Jx<2Hrs^U6iE0n~kz7G+>F5a%~EZH>yK9N&=7`iNcxLv=R{-6%^3MzU_u zrv>Byr51aUV_dz<^yZsAn$x?5NN-2ih%amPHG>PTieS7xM>P6CiF^ewCk99KCfD65 zx&uF--p`9SCeh<3LkCvv2z|s0=>R%j&nEk10a(|o;A#{l?a#4ZTH^6R)cm4zw+4G( zM|orsR=~ShR~_U^ghEf&+9jPGsVuLX@V3>Y_l{+EK31k+k9GXX4Tg% zq;QZYbCDb2YkUmEcye6b4YT(VeW~ps8w-jwM`asYH?qAszE5EtisY=7;UiY(5Z&T5 z#~qXWj{tjePrfy>Q#yROTF;mH8j1piu1yu~o|;h|!^31Yt0k);4fHxtoc9B+Te7Ab zq~91VeMnZp8?35fa9uci&apR~m8j-!u4hNg zP1q)HQJ373P&s$O25eFP)D zA^beh@oUz^F0L-gf+zmKxk9Y|1Fv@iCnM#8@SS<~G#!Nw{m!#rbYqT%A`Pv-C$VGq zFy~=(#O*@ns+?D(x-%L-x30u!cutrfe&H?V$pE4A`IPfYuO7!#Zl6)&RH=u(49~E> zAPv=aoNP<~asT&vu}Wt~18ioUbTte)n!@}GlORis8DHm#w!2c3c}U(f%)%wg@63Eo z1?V#rlZ!Cr9kK5~SQ$)pvksG=rj#T`-3y2;raOCGak%&PpmrGOTrqMXH0o=?Tr;9y z=}<+?;p&mQS@qLk69e@h4b=&!F13I#E3fW<+O`)(ZVt1vN8+!LPh7CY_RMz(MkdV` zoqo5*SRO^FLW;lMc9d_2ra^weO-^w78lP)<^d#6mH5qMbP;`ME_K&K zmQ)t-PSUxpKm?E5F=Qk69~QWXY!(fKDyD)g74yG#YY9W6s*Ica^)o{6tGa(sK{2zFKra_%laN|W{B$osw9C?|G#=%boP0VXAR97a+tN^Yw2-MoPy{Rs}#PkPQ$D{QpJ||Lc(P&m#K2h77g; zHDq*2)3cJ%al`5`>emgDg5r)Q9u)Q}q5@W0(qwbVr7eK;;VzyZ;fJI;%S7Ua^BwJg zY$GB$ey_*68lsMXa9uko_j+diA`&+z=ZC4G=j9q3Oo}3naBWnrn*^LZWDWn!;~8@` z`3T7*S7Il`U_asb(Xk{)1)MT8QtH4qWI{CMHhQJm3)(`-F*@3x8p`a}g&O|!nQ}Tf z!c@-WsmwBv{UL>@wdf{9(MVX8Qr;9S+G#3=H!_5>Ip@*J9C4P(KbXt03xw0Qxpf5u z3|0Mgx0ED*hP3@OVZ=@(Q~{Pje~S@)c2`&%QC^`;L<2eksDivxsXYe(1=O7wMuB_# z!2iqRwdL4f!IaJM9fKP5yR}aev@?<^cNI*YbfO8cNP$RYptu2)n4u_DV#gX;2bHqX zDvUmcdK7XpYCtJ#gB3s&WE0Lr3;GjyZjMm7Sn2{giP$YK+K*j#zK~sH7$6D~)xa?* zN;MThyYv7pZ9nx;5iX^iw2D)9k8l0 zA~$Ww2~&&@Lt^iZ;QGmVuxQKk-=O*O?nk8$k%^pF3TQdpjd^^2THq z54VvkeT@Gscl$A6bQ6eK{E;*Evf|%9A1w&!>vWRz26 zWN;M1lsi#k3t8{Z36l>Vy?5`%n%?%2t;e??NkNoGw_vFfIrsjG`*<=JNG8* zQ7fA0J~9>VZZFI^>xDbWn==e68`xuYhyVK~Ku+BmcAn6ZnIkqWTGlXi0|mMmZm~F%64~8yoTT%=p z3TOCx#tGUz)^h+@Zno?z)jdU2ArO$xU0C$m*=?A!+xf?w*t8RLcNw;$RcfbcQ|215 z0MK&#j%b$(IqOAunCoX?`>dDal1c8=;CL5&=sD>>B97LZn}_@JXqVasNF*_&BFsL9 zTC0*-W3+Zl8D;0ZYNy$@=!UnIbjg{wg|^6D!$}%hs#B#DA})|)J1JucB3m_=$(f5J zBIHR|S+Ip{@RhrO?y`!Ihr*rTex%`*;2o+fH*{BHaD@b<+Nc0Ysd486X(XV2;lo4t z&RBAuEL=4V&>*>zYf{C}MO#M8+c7e7BAqxP*2I|}DYqh8oli1+FECo$B;o;^qLo^7 z-;&H(wsYsGZacjGH2}WGutcT%oN2sH+AeLTx%hSHNWo+frXjP>LS)1?Xr>onF6fOMtbmPD=A~p@Hr(>DDW_1Gd4;*G} z2ub?eQy>Uqg`K#HEO^piKbqswVtyv6V&`Jm3#$70&}0;ko9fTf%AkNXXMi7z3-mZs ziWl%lPfSV{)ATqr_B?<9;KQi_#M4pjppP|pwVVo4EeEviW;KAO1P_o`Zzs`b9pnldss=`x$n-0ZlkbTcC% zogwewD&EdjH(r&_Pj8w(KJ?P+zU;E}a_pARd1~;?yKA0Lc+q%mUwe0GavTF8Rt)tN zf7|+Wg*{Q@Lqg=T%nt1oM&9bShUVWcU?JSvz5IOnxA|#KoyBYv0R01q|Ix1TzqXA3 zAM{^$To;G+PU!-kOk7Yn94AGn%9N(Gw6O^#S8+5Mj%964NmJDR+1gOya(4}uk|fI2 zMSkl>iVcJN4MN?I|K)Y~%h}(jSEZtH>Iep;vk+l8G_)K3=KjR|TY-9J@Wi1b+`b!q zBqvSps8j74nYo6fH_bdz-Z$?c=1Iwd`tvd4a=#XdJZdsyu%z<2@P>r*%CJXl+a%o- zke2ZY;H0MQa^|2qWeb-?GA099!X;!?MFtilV`!Qx+j>-j31apaA|#6GMIF@%A_cLG zcm<2-l(~S23Dhy;aNHdYRb>4b{h(PF zjl$RILu|xh}nqJm>J+$-CMxMl|0R`r|IMA$E#nfdfb&M&0@gJlzuulYT zd_^LA(7lyp10ArWL|!2yzbOn$1!(UL)BpsNrqX~tT^R}^Em)mI@=$e;5~vu}w0QRcI;czXgDrK06MGU#@>A zAa)f~T+IzY&=;!_31mX1fo`P*BXNFV7Ii_snv(;gb~q3>c!}6MAHAsc@XLsxiy;Q} zOqbw)2aHeLT|2bc-=1~s*pI%jwtl651Nd^@I?>C^fP&mf>Fa3;+e07II#axquRanYenEP73h{7* zH#*+n>L>LC#D0)Dni27oj07t6g_{;g*fXKPP6g;&yT_`#_3EH!`(cQtv}z21pNMiE z>$=A?q4$emWmSM zNZYnuNFf}05*8<@)&Q8A?x-|J1E!~VeTKYJXDtK;Q+W+P+dO!Q@pO5*ooDW&=9*Rd zSI&l2SrDvk{Br|Xx^_^#s-(?VP>}52Ac|%uvRN1Tt>3tl!3m{YhHhq zklU0kaIav_GN;6%L6rzd6|0HO$ezmKTM;XZ-Qh#Mvn6O(Q+D4H0BG=EEE27X9I-I$ zagH&^JX?e`!z*QncO>&TxuegC5-Me;HQDO+`BwSDWRGEXpk>i#GOM@0cBPz39`r79 zMXN7t87zgeFP2tTG#jm5!S8a;rVxVLqa78}l{e9T%CB>3DWYD#<(th)Jdf`$mYoMQ zEFy)ajM&P(%g@t3EfCO*vHaXPKrT5&x7s(8%M<(4q-~gIk_A7mJ*IN~vT5*#ikr?e zHvU;B4gvyE!I^n5ywYTWaDQc1(*oCByh`;+KH1bsb3K~6O{-;SR%G_#%N^~!g%SDr ztx?+0gKFTZ_fa-}vBoM~v%0W1HF?)3m}3I&_d+u5sn5KX=4t`7TY1`Z%$s-S+<6py z_vD$@WY?4ydPhEFyD+!`|aw{Hyp^Un6)6)2nPHcI>A z$X|fPLrk~m)_wT5%nC_~C)pG*)#P~GA0hI|41uge{gT*K;sI_-AJ5LQq%(%LtqhmMxhwh1Q4@zFSqV`g999J!`zor8s7o|oj(=tmcvDqQ|Qr*M?sI6Ed zkj9yvhS~vHVz_!_T6`>2ZGuSexU@2w+%L+*XbF3H8}(oU+|RGNcW&+BlwodrjJz;ZSg(+rPb=E6vDao8BP%L^bHjur9o_o=h6--L8>-0)aRw%WM4Nwvz ztzK*snFz`13N32Ob@E@(_)3YtvVUDsmGRvbDVkiz^EjC@ zb_(W!G><^4U&pL`;Ntn80?0mGrW-)>ae0z3o zwVx#bhooh2%(e%vHe*m_S?1{#Yl!H%CsA)Hu?@0}7szWluwF*~Fyj0P>>1OospPsV zj=_EK8^;~>3Ugnhwjp4oQ4NO1?~q#kE$>PEb#h}ySN*Dtg#`&@2<&*`mKgGQ$?(>= zEvc7cRJO4`*h&UN76oBGyVc+5d(v|Mh&VdZp&{_@+;1Ih&Tj6s1T<~wnW`-zgWkR8mLieg z-k#p9xDl0cWb`y^aE98cYDYh_BgYz0ZEurDA5!o0hTwKj#*+%_ju^_T(52IxISg4y6B{VBTO) zw%`aXkSqLfO+lklE1rkFKrMUih)7gBbfFTHjgNR^7F#l84n*0Avs+CO>>=pi{l?LS zlXZ5qTmH)L5qHrt;N5)eeUnSB#A?2`ArbGlw^+_NglWj=CLfwhbllvIAI&s`(~zfC zgdTURFC*!SDzEfq(GgMaNN#t}ZhStboY^M%;A|!pF2JsmOp#h?tYL+>_dCcen?K0s z3X>_a?e3+@*L`xoJanvMk9GUVIO-=gP|X>$owJaxF2Tb`*oS!6@em{fI!^jUEIBu# z&yq!X^n1lX&pFDfQj!VP&0D~C%}9d}VL*y3=c3S;mHX`6OPPXqXmbJ0BKo;c-p{_` z{6d2|QYML#aN8(i?(o~tMwaa;-SO34J~Px`!&o-xA;nUZV_#ws*d9~2?7z-yx}ftn zI`-5$Sf7U5>&lX-%vygBZSB+?UfytRt?A-Sa&6u3Xs1{(zny!V*{Q#xJ>vBmFkN8| zHQzSZ6kS{zjm>!2AxH&rh&!+89&!D6q-(LTUL^Hu?`p~$mV z`-ZyUg-f*8Ztnlx1rn!&S|n;DxwV~SXF^{e^K)O{E0*BYitF<1IM43DL3-b)ci#99 z8X51)cFk_LuK9bjFGwDox87{|iOc@WV*B+}f}7~^?zZ5uFidbN@+5wmX0$uD7hP`K=73wjqD+v1_!(Q8-US(s-}^k8sV z$J_@ z!Z{J!MiS@Z)@58h2 z;BbNG4;QPt3F@u=YypWvvis)xn1=CoUDqL&?_mDPqjT84oBOX@s2K#JZc2b!lNxYz z%lJQ6NEW8UMUT>j0S*X!X=S4kg=m+oOzob7Z5Va436 zc-4Yig@9>OC1Ra*1dN5Eak$fUr?XXALDXe3Ts*{F@CDyIB`yQzlLaW@ZFk*J`1`O+ znNjXv1Gf@AOJ6~EJkv2|?>NjxU(9hM6)5zO_pcmUp(Az1N@hpW)(3H6mr8Z{Uvp3* z++;Mlot#Q7kQgqI1gLJ>sh!+gS<-17KW8X+#HFymNT4qIbPd^CGJ)Ojg5u=N zMJh)u1?n+yC8_s+&!o#k4P||U(jgRyX^^vP!$5S)YH6x3?8+j5+M}cekP?m30^+KPP(>)-@vqDR4@I0uSWZSR;DGTZeNYJ60N`Q-UgFuGb%_PenD_ zb*M3rj0dDHC7MMT`n6!@Z6&=D!eW9wk-<=fBg4BplWK-*x&s0c!rZ{Rm=wl- z0XO7r;|39FZXKUz8}I-0OXFL;Gt?*XY986F-DsfY!_}@_Q3V|9d3bxrVkJ>qJD4vt z7F9^m;}FOoVZm>vIhm=D*GIN`4*3R%Mg|esoz&9Y#XPYIPZLJO;z%sxgiFUd|6*qd z%L9bDORJEzdd8Yf6@+p~1$a`!n1!4Wj57l2!a__rW$DJ z1Vg79ApwVsd47B}(V#W*ax}@bSmmxDm$GI-3U9e=u>8OzIOcr|hMzq;P8;p96|pe;Eq|uu)SeFHRtQEc=f6Ch zJkDs)JY^|qfG#ldtnV2HpD&}|ma#P_WY{7({Oy1wX$(VYajSKj zsoS_0?v@XkCai}y(Jm`K4aaA^FdUcDpYulPE*DOwR`P`)pViLIFeLICA%L)6X^l~i z_Ap=HA07^J-F|u?2kNGQY+_uMj;pkk9G>0X;%-$Kl^EQ$On$yOb=t^Nw7@gZw&U zt+P+@u^4hK8|pReCcSd4k8=2Q`&eSrYI3%VMSM3*#2eZ$jo#DlBFM0Qmqgr>lP_tK zrHP6p9B~N*@{4CBNzZ^s>>A&j>~>pWH*WCP?%MOiag8%@N!ZIfO8U4HcJZ*HfoX~$ zUz5&MN zIYRy^;0bZ5eA3R%u0G@iCWcG;`$@e&ma*qXcUvCmKISUg08yyg# z3t`RDI{AqGeJb*twz5^E1)B4tjp<{?w4@vwmGf$=A%r)D&?8{F_j&=s` zThin<8(!%jrTBC zsI$~lC#~Flf$-5P*{0s2<9sEaL+5cIZmMW1Chd_vB+1{3%A%FsaAl zL54W}4%Imzu43RqX-7!zwk>GKV+P5Hgk!%9wJc3g>Tx>ZPi*hui($R;_UFQM+cdiI zS1twZ4<0Vrw9Q;vSn=%8;It-~e;T`n2keurF z?=R2fIKHaD{tPA>^pP|yT87feC6Dcw1z!mG^F>7o3T3H7T{ykl zo1Y)oLQS=7UFq|hkH8n@7()voG^MROVAWfE2jxYRvns?hpSpb<>!tUodHV2X(Y(Lw z{i(3$_D*b1ps#~BXnkIBw6J^DoIUrxz&L%!tUG(3nC`ef<`+An^M8Gs^bN^R9(Pd4 z7MU`$=nghXk(w`)zF$9R)=nAcL+`S`FJs;;Ng4N0;cA*9Og(609#yp_&WKI6c**S7 z&M_4c4C1TE_{t#uw9S327mL}tI$#bj)6AQRXiMRlM;tznkru&b&E;sl8Zn-Aaw(a%+_c7#u z?_&z{Wn!85CYHXIZL1NXj@Z3*w z)U#yn6Lei}1I<8yyZ=rZeN+_k^a0&WOw8qZ`MAjg`IIOxk{Gmv>p#&nR!|JcF8_!OnEOk6|CDTStn6#5I?T6>fbI(N0j=; zQ-i9Zap+JdafGybB1j5`DqM|RBnlfmVV8)hx4^9Uf@n@5bKg$d9^#%e2xUY4Sez{1 zy+HKU5?UGL_1m9y+GwFlGWfO>GB08Fgu0t-HYA-^ZB*M?J|)SRQOwA=;FLM?`R3QK zJ2n{&Q5A;6uWoPx99&EjZK5W%UvXT*_j-APcm`S170Dw>rs+|;#YPh%F~U$45>V-) z7OiC36q480ptW$cqCe29K!s*&X(lMJG*Jv&kd>hzX&~YUTL%EjA8frtNxyJ43ri#C zC^0aYq;$w}ue>YDf}hcX8ev>2kXKTr&ZLXtrsajSIJJF92!uDX{ppu1J^ z!he8+VZpZ!`i{*C5XEy?jD5Z}@eHVNJh=N}d$ib_L$ut-O~g;QF!*_=ztBSV-22B; zz<>8V>1p1KtN#VEfyX&dRSs%`R;%a1dB#^v?K?P{bw5E+Pc*n8{&lb0Y8un2W4dB2bcbha9+6;uWZ- znrN<#E1rp$e*4f3v%t8$wJx|gG0{_J!n9hT5{#!{1BoWzS)uy2-)^=&Us?G8cEUXN zk_EQz6PzzCzjwjRNcbu{AzOb9K^R|F1l%l#W1$0LLf{w#{h{X;cyfbPAI}+i?I5S} zFFEo2#k%vCoPbLQq#_bMZ;%)pgrE{ZuwUU@*72zgM_Idnfvu7;JJWMB;xK6s7qC->wf1eilkTR-p?-TYia%yhRw zb^3mc{zA3V?y+6vW2>$s3myz z>Mf;ojjc)!ScY1Nr=03%uqMmfC_>aVc?F%KFvNEojXu$%XDj3$0SE}X3$^AIja^s* z*IDjgw^X}pUe8T;qu4k)>=G!}GxBNIA!%k|)<~0JbI{P9=Ds3S1?%K8M6vew`S29}Z@Q>QQs(vVt&|UqB5C?seZqar zXq1&R@`k8GTm3*jakSanI@L*G##!4YY&x-osmaXa?VR^}GHC>IUBE}@ubQ+C`7y!N z)&fb2V6z^iH_-B-C_5`d?LCr7WOysn_MF`l3nr>dmZ_`xFEgJj1&P+T9zV2ALnljF z9PASIVSrh-u(eW6$4%JBO|Y*os2xrUs4bhUp1FriIm}V8?|jqgnyYy-@0xEov4;0C z;4Vm3HakFG(l>whWLCAMR3Xc`Q#YJs8&BUPFozpAP;dWeSNQ50xhCd?M)6M^C5>jN z2MxCHQ9KT&ZhqW6D1M(hV5D-=I4)w+k>0NxII^|`wL6qqe;aGq{1aOeWV&DPe?@lo z6un`df179>0+QA;8N@QMyg?`PJX=py8Ug&rh}JBALOvwN*tujWm=Ym2XLH$Rxf`0B z@a5WExIb>I9q(73jgJ@UJ>>Ajdgts5s$zk;N6ko^)~-rU32~p#=Sr(Uf8WNB9LyEg zjg0t@v!Mmee&(wolk@ltRVmAW=QzF&^A213nwArGZ|C*U%*09rX47Y*?{bI3QnNKf z{dSiH7&p$YKXU~4ZX7h6CXN506ci6z-ZNR*Hu6ILm=%eJCMfRLjr4$pHTdXdKGCK zGTJic-L~s@s%YHyOK?<7Mv^$ptxczAT!I9T+l%3^5J6x4(apRu+h*t}W7rzKu8*Gq z(`q&3jFB{rXswj+`oqonQT5mpybzZt5*q9^H@l~ENS*2XFeC@W@`Fm-cwtCYf2#u+wsq~uXh1Id2br4iCp@qN_^t{M>_ zjnQcQpnC_c8)k=eq;=9P(B(hni5Bee26JA<){ z%A_-dlO$c!dl*l^sslxhz{Dy=lLy^DSL{3Gx$C~~#H)v;Fw=u3=a1Xk^Ff^mqV-1; zlId$8i>8l3s{lMT7zzQitV&xW6}h-VFSrd|zozX%j%vd^Fj$*FQ-be+l% zo#!#=R9?TH%uu!B>2JeU+wBMJ^#(4{;@~OGT~DuuSbcL&)<+QfZR}HRdDCVJC%Xe9 zu#AqPe=Rkv^skrsVSsPEb}AbCkq}T`ig6s#rTyAdP*yi|cHkp~|CX7qJ)*YYt)M&P z6$?@~Cv~Q6&AslUC?4L9C8}dyXR1xy`}A7&&p54kqqo|ak46l0);+WP9!Kde!zWa$ zo{k>+bR6MYR=i_c9YcFjmtugW#uUI(L!^!UIUNnTW;i}rXMj2z3e?Kf0lW4YmMZ=0 z5!B$Zcdr%boljVxA<(9J0xW^GYaU;lnC5OZg8KO=2t~A>@W23-S=}wC>K&i)Pl6V0 z%eP?|=i_`o8O9&xiXOtfO?g%udTQgW( z#_wyOQZw?F4T069Lhf5mY`-%+$XI^&gKOyPqA(rw3E`{TsSEDMm}tb*-~(>k1Hx&p z7zVl9wYj4~VfikqSA)H@1n+5>bJ7UL4>8|$pzL)nZC)Dg=la&WN?4bo^K7B_v9cYx zOE1g6clfQnAHZAn8>n6w7ww36VaTy?WwT{Qmx`W6VsbnUE>Ebn1W%{Aw^}LxqDI2d z1G>R}96ZF8phC@uG%2RiReLOB`bAR-ZG5Lt%CcnZ(_&@o$ZLOtSY@S7aDz)-#oK?6 z!U$?7fxEcFVS=@N0Q`jw51})VJ#ia0$W<9HYX~ZTE(ti#uf@GsALQWk&dVtOsJ5cw zZ~i6s!SoG5iFZ6t4&{uY0UxLy)iU_GGlXeK!hn%-5P$lowTBiPhCp~# zMjeczTk5RmjETRiAm=A=)N*%+`+bT&cwlG(zE^vxMHZ>y@)0&{j0&jAyQP1u4Y*A9 z$Pgi7Wod91TKTa-)bQ{RO{`O`L((5s^FL&hT@#9~@5m;Vfb>IVOKrs3(QFyJ!E6cDTOABYx}oE)nPK zO4_AUwzE>1QYPU|P$LO6inrJ|V)1D~?I|ac%D@ExS7M^WfpQ#Qz%gbm*3`|Ur=9r2 zD2-fAx3YdGx{zrr6R0(G_wF~4Ap2^F1?cNsZqR$j<2k;0N)|;30hvaT|Nt5B;V>4 zr(wy6D?tN?6tM1392kvlvNBK%`_sKbg0NPaX7SG<=>4ko}j2 zs!67gF*TR5mCsTsvi&;k^kC0Z(~~SRA;*mxUHevzG@G-g)3z-fTO1QP7to7f zho~eAB3Lx3_FfDJ~2*=o1K%IRWpqB!-F#WCEcF@h_=I6f-bwX!9%C-}I+ud0grDJPRC(OD5zY0O z&=YNtGjJcK!CEj~G6AE_P4b1vrLRNiN_x;!Y>U`IvrPFCyAlruQ_2G^p@nex#!8OQ z(e^}x0I)Oj2eFprwM0LF5&B`<;f%PXOQSF{q6c+kfa~RYBrAXLh~-%;ri+}IZh#*R z2QMWarIce2oJe_BbsaU}xy?tC1=lmyp%ponYMH%iRSbUYXW_gHp`R zNrbDqfy}E7j_bi#Z`-#_*7pzrxZ2gPr^lx4)Io8Tmxzo=vVPYOhnGgGBK$VCFXa;h z%kGhFxGlQ(3vXpWR7-TL8Q>%RtluQ6$r z8>f7^-QU=bDMuumu__9H5$2~(b#`PtWp6Ei0zEID(OtGcWRpTDd32)aR=UVxJVeC_ zrWh)RKWKSNO1tM}+*xzbi*U!z?>YnWO%jyCD$YRCD!iYFE}$x=r^k^J1#Bn47%3*z zRcO2+f0_?xV%fE(I$sD!W9T#V;x2bv@{-sgqng0EKP7DoRYSAom zL~dDrZPU+j_su4#Cii)^==cqa{dPV{y z!%|~b+#-$vO`^m?;+U3@4o-?d7FKcga{9zuRo)Y(jNscWv$sG1KLL$j!O1x8;QyNs z89KedJO>9l2`gs4H0@qgB54$px^sxTaAZOsA3A4nmsSsi9)^sK2C5PTCN43^Nj7h! z7N!zFP$&*C?kzxJszUQ(m0-DmInKWDqTwoc2@CUx53eXQ3gwJ3%2Dj4P-{Lx9>Cx$ zg#OZ%u1Zv}BQ2M@>p)huH+|h{FX;{5W$JnCahk58KNmhmPKT9twLjJ9tp7Hp-9FAs z09z8NOmR~=#b_Td&*8ODnaH(=HAPnY3=;dw--56H>2H(A=e&a@tIwndGR|~qu-*6f&(dbF(V}#C6q&0P~cN zB^nfk0ZJRV4d*E=J6C>;ZV1kkbov2DUum1nm}< zdNiQlqiNA7W;EA} zT&jsMqu^8X6j}}{NeOE^=LXA#(Jxl;BYk;Hc^8lL!v#z*aA})x)tp^!37ebUfgHL) zWB@r~P8zKWt_$A?g;f$JK_NhaTGc@@qFLpMd?H#=ZH$VNeZuPnR4;GKw)q@;L90p? z@9K(dy@<4!Y;X;`0aeRC`a&fqQke5Ym1tewHp8m-sUX27p`(C%B)d)nytltHbBjDP zdFK9HIEu`@NXn+IDkBmfqsVlX+#vuc?K!vjBfI4s>yim2)SOTiwo!dq zGLP7vMn%k+xOckI=uCUl8bg~za&H~N^|zX-SvJ;{)G&qvu`C*1OYLE}cu2jS3HUn3 zz4BB`R=vf$D+TkX6CA zvKAj(A?qXFf@Qs0XovUUwv7qS7+k1Lr(QNbQ_u?>FDZSbX0BF#o(=vesXUm$JA4d}52;2IfpD&HziMIQeWgth!Lz_^ zncrlB+p%#Dy*AZUgQD#ehWA%lg*4o&Y%;O_-1p;=K?f}}kRYp!6(`A0*jR5C<>Vjw zGuGRW5f*r~|IvRm>hx4)w?BS>VifNp{ut*myGuFO{~e*s?DC~w$7W-zalbiut1_6s zf#3UC!VTLt{J9tU`soh~q>Dk+biwy{Kh9HBV^*l|`8<8T$eqNNALY%xXXmwO(haeszVKUbq3#l8JSaZRXkFj_ozU zzqb5FSeFj20K%UE|Jyg-|BC`EVqk6k@AdT(P+tN3{}*hx;Q+=pYGB|?vsY>T0*I^8p7`2rnrGiQ-mg1QW&(BsWH6hx6;KQ z)T!E}f{BoKQ9yEpDDK$Iolb)&8P)p4a~ewlWU0%9ik~!&WgQcu0~EZ4W8Y!S$MmP> zuu>Dm807{7zPH3JEAL^52sqIGKa{;wkfaN@uG?L<%`Tf=wr$(CZQC}wY+GHnZQEv_ z?74RAeb(G3;;jE7BeLr1%ZSXucRX(jSGoXsLKsz{6!v7AmnvX@)iuG0kmxLtA122- zOxRm+AOt?*PwUY^z98rXD!jw8b!}t&;dP@%%1WGYYepEqb-%7HTHmSMt+X@i0T*3^d zV%bTfV`Yg$qXK3>s$#aEIi#LAu>#;~so*|64hE`IK@nSD&x#@~uaYk?;bWnc%M6Xb``?_BecVV5m_Y%0Dh zBvY6&K$v7J(@VwCCc-UlE9 z*!N=az%K7Z@P{+Z)000z%((>NQ%0~n zp4!GXxRrfA^oQZa=wYnb95<_vHe0%&4OYJ(AGCFU0vMKPV31O%I_~^ya#4WvN*2L4 z*^a!D!-@kk;G8k?8-hXrPO9Xd zx;%rP0Po5Com95fDif3Er%~n^{`)je=)`CV1-SWCj&imfo^9dw45+o^?A1Xcq@?-* zL&W{zf#DvaOoHeD&s2xe8{)b^YqORlmAiB~ZOc1u2y!krA%q%fOAJ$$DO*gclXHE; zsLiPrUPGldIy99?Mvh7ya(Xpb02yU3S5HKIpIo}X@cs&AM9E(N_M-=E$vF^kKd>Rr zt|l*b6Qg)qb>gNN$Y#7)s@uEoeij{5LHKJpQ2r zI~yiM)vD%6JfGko8hjw842(P#Ac|xOx1pu>@`WKEDPtZwYRR}FW6-ad*2%gJkyAcs z38x}$R5$(B$_Q8YpAwo?s&W1v=NCt9PAhX=?SR$D53@qFtJIh1V#aPol(XVZV>!kQ zmqA7f@r|ACDdS%)%gyD!SD7Yr%g0Sjw3b826XR=ki!bNdJW34G#fNoRWYAd06WLM7 zK6Y+QGAnH6WP>aZQb7*fNY|h>NJrD`Ckac%ts;|mKJI$Q(R>e zj_bsuk_A(-w`)x9?0l8VzuRyb)v$4{LfE>B?6b@#TGn!Wc?Uk?GClF%?k3`-nqGLC zMIn1FwxZ2`FZ$EaK3jSe?DRC3 zvvEFh@J@IRgnpdihkh44&j7t3N#=8+meSyub&)(Yn4?CGp@cjzy8m)0<md@T_X@Ie)>UVUs6zVrdr7MWDQ>%(yk~KK%KWl|*j_XZ{&3m04jmo?6 z&OSrL=D91*rodWl??7_P-?fcx6}x+nEJ7Gv$SrTKJ^!E_h$(A`c9q=q$^F*ytLs7H zZQeJniEyfA)vDu6^pM_n(s4&pWjC2MeBXPUxI5hdra_v9i}e2CG_}rjdXUH@G+tWq zcBuMa+(#Fm!4cUPp*F2WPjjU*43;Me5G}R6NGCdwCAhN5_Hm@)^1E%$4n1t$gstqW za+Pv?tNNe!lU~~}!0zuy?>iaB!}qSo&0jvI{cMS6F*Pq9;6BX< z0GQ5BwTfP+WnAacoj*EUfmvP}x^b0}`+BK@Qk*g<6LV;NfCdxe@zFtZC1YD)aNPM? zh;`W&pV&PWR+0Qt0Xe#HjyViAsSy6h)W7&jmhCZH5g8Zb-G1Z#t*~drAP{&$58M z*()WHwLieXSkJ#XMV|)EqJnd65@G>6=Nz)HRUtNJp+UY$QDM2OekF-CPAJvtY6#@U z15I>!fs7~0c1El>(u9>ETMAREu$Rmp$mV3ufnj3K6^RsB{QAmt3zVvZxZQ_EPZwv2#3=T6wk4+e zlZ}9rMen=w@OSfj0`1(P0mF>t+c81Zy0P96BN480x)Jbzq>Yy$5w!-D0(yd;qLWaU zn@uX$0OG3YkwBWexiqcQu|mn{#9#YC@+pjehg;ECkl{6Dq6yFt1q0-%qyt056N{(J zP-7t`op>dXi$GY5f+Blt*$i00L`wQG0Ok*^$h*h|Ia8K=6?yk;s|L(GH1?5V!12mM zZQ-zi>UeyfAKtmAfyzSSG$lo8KN~T^E`Ixk%=|*_##A)q`EQ_~dOEpc; zRcw>>2ecEC)-UG8!Imj3SX09M&5s@P1dn+cZ*N_%?dkP~zx2mgu$2p+c2|cwoqfVG zx~ZwLM-j13)yI(0TN7Q5XK7{1;i}v8{H%$OI9@C*Ms(ey38Q}A^Y)RkMTU3pr#SW^ zyP-*gJ`y^EoL!$bw@I#H>AL{-K5tbT-EiyZ_mQq9)#U>UU1-p(5sJk}S6`M#^Ocyv)R$d2{FqndN%i3-9!No)PYsrbR?t zO%?b8>bxV$O*NuP13PLoAO8s3-1|C_G}D>IO?GFiN<*jZ(Z18f^3}$8+;rMI_H|^s zk^wwT+!zKYqxKt~X%@S6*4!~8=eo6P@ksrUOC{*RpQn-5i=*~>cFU!|ajp%Ww|QZ! zlw+sbs8VqyG)cN8t*{PpPvCX4BxR>pha-F|Lo(pb%zwZI+=%9|MrigsnC048*|MFN zO1S0)f(QCs&5m4=4w18*(4kwlChp-RuD{pWCz;(=#@qV>c`Z*|_Y%gkZ>O=UZq4D? zfgjf>98Kvnj=QfX_-`>@9?qwhKVM}oP;LK_MSXZBv{dmX=n)Z0eCny(*m*Hqrvo(| zRd({6b~J_^e6a42|ua%x!J{9j=3wq%HU95x6;f1a8HPg-f`t z)mEf6*XC)NDB2DR){sTZh%n7jSgBvHsU+`In~U`5Co)-muA?h6j%__Sp*^nb>E^WZEB20Hc zPXExr;e*|Io#;oQOX?NU?Irjr*NXszJ zL!saY*iBKfIu&iW#RxoM{wb4!c?G2=480@tk{NCpjfi`^W0lWbe-L#15CvY?9-(_B zCqgCzSCceR%VkI7(FaS(7D8*cYIN3`p^0tX&0)-X^>KzpEi3XXKR)7CC3KN;*xN-N z$g>8s&T&IBc?^B+JXtBFFpGq$@R_%^*c8Jy@)|25#J1SSgHp*sF(*!C^9yzcZw>fJ zyZ{D0_Gx_eZnbV8ww>TyRZx`_8B_;NFXsW8f3dR%CnU>MUTmL2hA6v-d2`iAR!k`DMj~;D% z_4@O)o<#WxO;etCN(%@B)syP(rX%(s!PG`{)M$o|W5##z|K91!QhSl~0KQ|QsQ=k0 zS;XAf%IM$Uu|w5$n{^Qc@01>cJTgk-^p#7+XrQ9frY1kB^3D~8IP?syQ77?g3EC=% zU;sQ-a;#uaG7h@D;4yIWVj?Tu(Zpq>)BE1rH)aQx&7rBR{;&#&2(xdrW8anZlZQng zNA*nQpkSq!SSNyU%A)ojr_mKQxm4JhP61jTVg%h>rGm1>VKh0HIxK{JV1B|B<=r zPMG4{()-(X5SaQyZaACF26Y^}`4%8((FI}l>;Uu!IpP2!G(dk~YS-sw8u6!b*lVcY zGW#wnl(vYo--~6!n)g74slhu<^lJl}3F|iVrZ;`{BS4zE^7}ndhmj3L-$Kr0PLU+c z(?|^SO4Ry3rsX}sCggKEG7{iGJCM>K8L+=&cVy%Kt3X)&Y6`FQr~;_md}3$dFX`*( zE*y!?y+rV=;*no30w|xSAjrao_(%H5jqI-6k*hp=KP0becTh}p@{%KgObilcivJ4< zojWYwE6!Kdj?K6odjt@!S)IafX}aec$t#aliS5Fl!%I| z*Cn8CDX-B(>O2)WFDIqfUtn!xjzv7;KbZzf9z8ftMY4^%#<|{lTN%$`u|@{}BUDHo zI+Z-4(t@8(1J*J)ybE4gYK`8|@N}kmi|e?r>t&|RVF)Tv;rvmx$bPlr+{wPliGsgx zkQ}*Ya2c3bC|A=_tDv0OHX$KVYhmM{M4wO3HZGLIVJ%tu?d9CekLn%|xj#{FW1KTf zw53{1%_h>j@u4sO+;h6Krh9Al*Ggem4gG@ZbSD4cq5?uqpKVp+$t+`MsybAY-0TTy>e?CTY0mFah@iZ2As$xGBua$Uqz>9H zlsrg1)=K5EEp<$MxgkpyTTkZ#D69h_hE84M64cY~3s$Wka&Wt$)kQD#A9A!}{rm7m ze2JuC+x$T{YQ&+4v#Xmp~gH>)=Vt$l5G5|FK9-!0fv^*x_` z_0Mkq_skIK{RbfqkQqt=EhFE5P=SaUTiMC!n;QRL$wB4+og9?YTgCnIMEn#M)-;qz zm31mHZlI;BiQ5kn0BG%?-vA*riLrc)Be23let>?Ji_6pm`%j#!adh4%MPu z6ycs*&L^p3}hN3VM9iW|;A zu>ilP!RY+M!sED|5fk?1ULI<$qxT(;UOIHg7&naGhAh9IzAzhg$Gb_K=r12Ze^-o- zKBK@~Xsw13l6tuw0iJNF6Ah*>^BFhn@YT*Ej(BV+S2%zx?b=Si_1%p-Jwy-aok}(gUHCfTsIZj>sE-YA z&?k->_3$wi>O0RFexKF--7pdjIGTl)Ih$+$5Z|W)Kw>Xu3{#WZJZ0E?*lvOGg1!er zFK1h9dLL=qXTF-!yLwk?ois3jK~n!OOe{B~w+xWQVz!^VPAf#~d6Zl*i9`^V+6nif zK}XC{lttSXGr%yQ9nDZ}3ggOfHakXVN1Zq{7HEzC53z_acOag+-O6B=F8#2Hku#*( zs5gs6#jrO=|N4$3G*PIKt}3kKUvLxNIRM-QZs{>d@%lucuP^)3d5*vFDCoIV_K#b0 zCl^KGi>wzjUz?|+>5y>F{*SV08JW3GYDzB^^WOt6wKbC+BE@5k7G9?0Kj(BWYQRD# zR$Fl4%0;{$PbeiC1Rpw1co^q@XE9YImkRBLLmZE<9#2SS5l^RiXSq6jVs0vo$g?@(jhSTLRfE1{J|DZVsjoG(u6?ql-4qd2wB_deY@t5xGW)#Mg+qE^n-%9!uva$ zyhaAo%H-gi~|&9*#CQzu@JZ7K*{Ez=0T?gGro zPPW%Vl^#MP2VvG^fecuuuwMG-Ver4ot`-c%lf%l*>yop}V+E|Rl20zX=pGjomo?%k zwMl9CR7aFjqU-TNanM*DAb6c02M2FN#;~Qj5LuD_gk(@{#X}g_VStImJ`oR|$7(GP zL4?Nriq!h-Siz(@5UY4pgTIv@h(;Q+`-y`uzQMCMluy11#tskmSt(j*zdT6eBwG1P z{KaRxub6VpyO6UHaP1)7`8#c8Rl#`fN9fWDe%R3I`m)fQ>v_Ce$>GNXbIptZ5}VfsuZ_L4I#Ip@&nh@swv%%9%$FMvq@h$ zw#=iC)JW8AvCpwF!nP|NjX%#N7mw?%*I(HN+*$v|f0qd>Fa-iOUecJ~zVZGC(OyDD zTE*Ph_20q%QhnQT4RG`_`0%-VW`YnXL9=kxn9Hao} z*LH^|lE}&mY531{lQG~dTic$^>y9pN3U&agFBjMi)PzyDX!k)wfXM9k=x{iMZ#SL!Ya(&Ec`P)Pf2Nw9RR$oA~(cZzh*AMm1;Hcv%P*itX)cC=J_SW;MumPNAzW)NCWRSjRSQIk-~?Ua^p_I0y*!D! ziugF^F>UKpnX5h;jH!lfC87`e=j0yi**ej{ z42byf+UgckA}t+>vneKMHjN#XVWCc(8~Vkfoh(GifQa9L)#`2H@Q)%H zbE+t$w!CU3W&8_ghoez&_yp5bto}BNt5x`@SD$-qgNFpib;gDJ3YH7Zw0Tp$y<-B? zA+e3-WUX4kW`_|r&rogC3ZFV9sp+*&i@ zXSp$FYLKKsY1hooUe+#`O}p~kJx<_(ojl)sMfE^wX^AQUG#cmm-A};%QYFiGZM#(F z$jyD)s$Jn5Xka4vuEK?8yQ3_EvqUbb_%iF*aW)eGG_ zJ6GC!pJ*qRUbly1A^50ONHnn}E>2K`d%~W25ghW4e(o8oJC4NW$jww-dyUVEE;~#5 zl24?pec6g*RqD|+UG|H9)p3(S*A2>LWPmnS zD+2gk$h0n71Q4~|2`5q|dK=ohSdb zsrAh5P5Fzv9ns%vA+fNXqmxcz#|R$?VERnltR=%F9D?LpcW&>@s>eCgaAV7lXXgen z{9pp)DxBYEAHRAO7^lv{eyFl!_z5BxX zxb$|LAzPX>P=#Mz6|-p4>^05a!NR&w{Bd;l2V3ZtP!stLhdCc4AXeuLyufY0dbZ~o zrC-y7wVV3|{l9lJW<4CadH~5JBI$po=#VmZbo#%No0_%bnivv5(NPd78%I*_ue3?h zy{SQ9?*ry^Rmc_()RDHKy<4GLV+listd111Q&zE!M0|2^Tr6ZEr zc3`icg|W(`{Q|)t#L!RD7WvZPIm5Mmg6yQ%MgpiN zX(S=3ksalc+>!sPCVv8Ww38GDt-F6!lQ3L&`yd@aMG!pUprKs41Z9u200Y}7`4rQ% zAw!132sCb>)1=tHJi@#s@lw`;PRY4OloZcV9J@-#XNe}~-eZ4jZzJw5@J&mUvpn@c z%S>&pb~|%S?oAgyX3tiOc9wHEZD@3h^1u{k0MOk+C-BQ$?>q)L1-6ox9%C;bSd&@1 zm{VG>*aXINqO>Lp`f+w$BYyQ8#&Ys{At zMrreSV}AY`J;oaFc)VGXZY+C7etq|rB0b=)^G$Tqsjp25*3b~Zosy{6hBTpXvOHA) z<5?OXd4z4Ukz;;daL_u2R>PsEEGBkxwtvQ(w>yG`9Eu%<#B2$XRZCn!f&yUfJg%W| zL_j3x#TDyMb~}vYRzILs#Sk6x3Y=vVtkW4yjmGN%S2rVZO=iCri(<~zoEpTYo0JRs z`GJ^QD#LZ+Wm}S{)82ynXq^=2#VrtLP#qscbhwY=;Ks9c%TCy^T=3s{c|6;i%rgRkQP5)N5rOZ>|5xlq=dk*m7a%ariz9V-6-l+7O?rH(^N? z+XpmAa85vkeyY4C2>Z(|piIN$yWL6tkqyb5SmimaLl2T1q1);z0jZmt95T$Y+djgx z!7^PRP;KS5Lf~u-7jDtCa@-{axMK(iOC#CZ?C`Ymr_j+YNtQ*8tBpU>4rk}uBD8kG z5TADSTDu=jPOG532C}Aya4A20@W%=<;t%i(EzMt$3W?a?%PQv0Gcon7n6wmQKXMCpZ z%Q#|Ws#Nir!W3!BihO6gAviQ(t744{kSG$v^a%=^MiW{%eT6K16xp{j&)4fAB^1Qb zpl+qmlc=l;Oa5l5N%!qn-jAYMs{3T@sj)rf_VZxtz`M$kOup);l|1Nj2l6>NuI-v- zji#pH=X2=n$KWT2{LcaDzPCAPO{{nYTFNxfbaGo@r}iH{8IXz7m6>#2#Ailo3YOP% zmWg|+XHe|1719K)sSkVVjce!<9;uMJv~*VH9!t`SyE|fN#lr7kh+ui#Pr_wnD1q!( zrr^_I({7#KOO-zOvH))L+VU=V=_ge=fke@rAj9`ptYPDHG9C2MaYNZ#aGo)cn&sab zjz!XUmsQ7x zkrJUw5lin-Zio1r{Mx=4w`E@bJM5XL)jMILzY*P*TEkiQ+;f#rv_|lIuBs-8&Zv@0 zy)Ur1FDPfJ5-jgMW1e68&6-ZP4o|k-?^|41d?S@OIF;9mURRg}s#Njw3&!ZO3IxZz zANbd_uT>9P@o~f~bJKmJASE=13m<}M-nDFEx2lKNp5M3n{iDirzkY1KeYqudH)`gN zUgkDyHnBy*b!J;G&2Sxf$YgE=tdQAih??!^eV%D?t~zDhFm>G}KmhD3_9h3s#^yB) zMJWiKyv=B&q`7}#Ft04%k&!)QvWox_SOi^eH?!$?HgKRC_;F#sN_uB^W5-@MrK#L1 zXhyQ?F#cK6!k{w0(e1vieygVNC0?zb8AV-NXGY^WA$+T@lbgQod5uLnXHs1@^U$wr zQSoeA>2c=2ZD?qyoGw2Zat-LQtvtCcZp{q_pR)5c!(r^>H@%+;!3BZWyaD6bUK|04 zSJPL?esx-Nn+`lJ8FXRTYHlAYWp{5WS>t8y+_dR$TiESP#=cHmaD(}fU9ZW48#&%9 z8cd(Re`+HeFJqj$-R1OtA-^0> zuCtPQ_b}^cES&M%7~6xeu8uH{V%N54&TPRh)W<*Sn4lFP$dNa6$~*gPmv>SUQ2aE* zhIEINUR)|$;CyyZUl3JOb)HSL8(9QaYWmV_ija$-9A9?NKnQ^pr_m=nCsdU%sjaB+ z&e;{h-5`6`7j?=q3azfGTM2v*`@pv7Pu+lK`Toz2>SI|U z&ooWhTysRhJQ)%uP{$CTl4(hfSYpY&-g+wFT;z1*yZ=uH{}e_F;=dQNb-5r;zan;Z z=7{ywQAZdnEV7^)3Eku=AaQkziYDn9Vo)k)ml&+dn>ge|-!c77$P_aWqQ=dgEHa|& zGZTt2lSt6G0?5{8q4y99}V$<#K3y}A)H%XrhOT~$@5zfsSq>NJ?AXbRTn<^S(dQaj= z^Ur=tzuV7D-*YR{DZCGKwG8aOmHSaG75=oH%P8iD&T;bJ^;8bzYy>;NbDmpJ!o|&$ zu(DePh}Xb(2q0OpyDZ`sO&DT5=tE<-A4ySirzgrH&k+fTRHIJ=_pfjU(1ZJN(X<26 zg*4jO3`j{4S&IH?Mq6`+5dV74ckAh6)PTT$K$hD@{A}Z1k!i3ojytKX|_1%(ug{-$0yMZIQJ>;jBpp4@C0(6OOSHm z?mz?MqB($1(I#%sc!-D3Bc)#%C|opG)hab>6&8n;lT%T;EU#P0;dy>!fTh%zo)ykN z#_#OrRv>OQcm@Se7u+)P0&{mflJp)ajWrws;l7z;uFwrGvT{P+S>Z!LOkR$g$ccBw z4_G~Tmf(0e;XEtn@DnL_{v;K$a;nDn2~@!2DKV3x?<*~t<9q~(c6^u_{q0_%+;L9m znc#g8{0$vWB?gw+1H$^`^)X)XOnXP)tXpa7#IQ-14E!zotY0@@$yyuiIhIiw7|ZDH zFdw}yP3i%RXUTbpOhIY-q$p26RD7? zx14uf#TSlfvR>Echwm^?49T3IxqcTJoC#=niweR}_*expb2il?MD5`nhlzN}B^Cb` zs>m-d!lVHPhjrNBz6tyX;aS1h#LC$4|4XT|sJ3piCx+lnswin&24U>(S6p$PrMS3x7?AB+^S&0~s%Mm_ zju$;m!W4*h;l1)XQ~bQi;^amGZ1hi;I#k)=;hjdPJDu4jRz{;360UQ7CXU zt4>i!4Vn`ohXz!D5>Yt;oNqE!kaYYL!`NM+Q6Fp6am3*dI;;o4?~1|a>O>58n^?^v0*K65zLg* z%kKaUfkh4_=ov&mH92y;MDpJ)5$;APOf);ORs`3ceVthA8u?)xN-rTjJPm~`h`4;c za_sIhJN22qXa+7T_?Ivv&qMR=NrPVzP&nOoW+=8`RKap zop1`SE1RL{E;~hc+rX)*3c`*+UxWe1hM^ukzNW1xeG;{}pi8WeGP%lL7&gmJX}j`T zk@5cOLdl;jt%4?LOnTcGA-_1utb>$|2b;B?T4K*<;IzH-p6N%9h)(-x7`;Ma#oE02 z*s8bfx##b>!MNA2$~!wI*coAut^~JTl#jepo8kxN!*k^5t<8q)hh3#5^2XW^8HR5| zbIdhqCVn#%LpbIYx~h__H!Dq~(X6u*Xi0>uMY7=FGiD7-x18N@iIzCErt$S_>V#LO z$=Xt=)vDFlKBXDc*RO#%S+!RxU*??7VKor)F2%2w+$>h-H(RtirL9}o_Hljd@-#|# zv0%vGO}CqT``D>G#}0LVI@RgCvNW~)9hm!K<~MpV4^K0a00J#u*VBYbl1M$=A%RNT zJI=EN2oS`~UA?9u3$1T@f0ERUp71rf_T)SLKKkfd(Tp`paH5511w`$rvEHB4W9zs! z^j+%0iuOIdsP;D;ZM5f@HA1u|TtQOo?L+&JgSEbuX+72s$-G+m8Y^hs86iaq7%uu1 zG3LTHkds`CyQ3L;)Fz;V@`#JnL6kyZE6;@keQjFk$#?RZmEP)e9k(@aI0Soarz};P z$S$fyysUL+5s+8^Tzn~%_)c3npkpKt5ghmRYn9MBS_AF_t}Jazn8ncFI;spn?q z#_6D52q&Wwb(y|}P(%Hfa4YB}mI$UrU(>S#b@IN`YG+1-$>n;1Sq-^?GlmLqO2F}^ zRYvttCF*ez%;JO3rDUedd{dmy+fQvb)w`QMZVF@EEn0~V)P^O)tuotB{+*mxM+;QU&i>DbEWhug(7tI+h1q8>J`%{!YJ zoo6a+E0lzmS@H>JjIlQAPA`$uRLq+n2rq@n`|)Di%e?o<%IM|1Z!aG$zHvngS`yZU z%&)sH`;P)|e~b=xzm6u24(?!?qN}jUvhjB_}jOAz)+Fz|3B{;yXjloSs4Rvu#Jq&|7JaYJ3A|LL;b&R|8a<{Qtq?a zXF=c|)@4{poG%}>sY_)!@lP?!^rLPgOsziVEP!_I|wfn#DLvl=U=wn^}ukPYJx zgbRdh$8*r!+K`pBHlhf~VqX5Tpk?PW>_eb8BP`s9;o)sDqG?M5_&hhCb;r;QAv#*q z+8?)smre8uoR!O;MQbc+9jJ#qwW2KT;Yz2f?Y=iU{Z_*$g&T29gLdJJrQn(o|b z7RNd;@Ytv-IgoCM#q@K^Riu>uPk7IXelY_WSIhuwtNJ_ut6=3XBx1<}wonJZ(z=$O z1=Wgyh0@989KNxz1n>Luc?Ji_GNwhgzegc<$3WXp4GRk55wGeiwZ8Ae=Y&0|kg|Hs zeLq6kbZw#P)F{=C%zr~eQcK#gCal^7G&gLd;51yHNTMI_6m72k!OA+2ny3h;!Y((RFa8UKujYJwy=7BE(_ ziFw4RRL>XEDz5Z%RK(`!hI zeU!5ygu7TI5oj?IT?m7?O2*u`p00}LP-3hS%=W5fn;IxT zC}xKTN6{(N?Wj*sWFXIE$9*_#a1&67Mh#wLsL+c0tS}$2z)fecLwQ0Qz)`oM3i(I@ ze5@cTWIh8M>Z4N0o=>}8+Ls@H)tqhL?tlBd-M3YJi!Hx&UE7o*Exx?m89oKYBMUtv zKXqi#tK8nczIhxZ8r?-N!kngAdhVDngZGsoMK2+qcQ2KpP<(4(4!oC7bIp+Hl9cLc z?Vb1JbuvWh<_@+YMbFjE!^w7G=2D;E=`(6C8wjlQd^uG8(BW!DJfdMy-tL#3Iqlcf z0w{A1_L2qc_6@b}DVc;TYxShs1SA6C5k~Is59t2;?tudNumlnSkg6> zHZd`F5H+?jcF+eL;QX&JAs@TPg5aa$Gia8(lT`1jtm{$16Exsg5jV54%B}QBfY4t| z!VrN=UVhcH9fjOj+tpm|#04@8IJsfnmbwUa-F{;Ab457mZ=-OfLkr17^!oGl8H>lm z>vf}-!1^S7sq7p=zg@;b2mA8Q&1MT6%~;M}ZV*Kdl2l&6)J(BlMdJ+XuFYqOlEjR$ z$%1|8vSggiYPxb(%!rpI@eb5(|EIkvNa3_#BPBD7o@)df6ns&ssHxmfurSuxANoB( z(~Epy_4=`_(zSNhSzaN|ogedAf$@w+}-elwAzFuMS1>aq-n~Iy3t9GnF*yfKw@J`s|pCTad;NY#s z$-`21Vb({a7iS@a6l7llD?7xb@TzzqjY`8| z3N*}!GWRp-dv&V$-GCt4Z=+xlF#dY2A}^Lydf9@wBA^epflakyuY^$$)+=xS_-DZ6 zf>-DJd!tl3Lfo%j^zkqvJW{bwH3w3z9eV_0f>_g+Pa~u~A%OvOt%6J`t}#hH-`F4{ zDCiV0N3TaRu04lC1(~)eicOUJ*V+fg(Lm}uMKzOCMnt&iGw3!5!dyBvHW9cq^cCbt zlCU;>VO^S3%2o=Oy4ALMid!2BSa#-@9;qai8`(ge6=m zaq1O1%(r}80pH!}+L2vfz_ATfeG*NWJ0l5BymZh3V zfP0$ITT+gk91GLFM*^e_5XYk7=p2)#M9u(_(k>>60IXLOT=t6ig@s5{f|D>nReK%` zKJ6ItzVgsYCtJ)jN+8$Er}OvJMo|Y9njurzYiQ3eo|z!ely;@`91s1VzvEz<$7HJ1 zYnl0AiKZrG)v*hd2b9-qPpI1wAgYDzdw=cIjt9Nh@qVN2zP#4A8)5a`S^O4y58o5J znCtGmLUqrEk+jIq+Pt2#Yws{N9Y7liW*zF%H9UoEY)3A+YHSIyk*(K)=OfYB|?5TjX=GNb4MaP*M=;x+&nD#C-GAz*VYUR% z;D`ITc|cpsYj!i?@w+8&?)Qg5jusJetSmc-j1~(osaSz5HN#ekU|bse8?0>r_L@o7 z8yH|MnF06QO5VWpW_^$JZ|0@e8>?;IfGXr!kd^di?>F)bfzLYyg2XX)i>tFR2BcD8 za3C%RdVuu~{fp`H4k4HFcV6nf#wuU?PojlxTO3@h{rstr>*>z!uc%cBpFa|1Pq0Ta zhJ4KV0_7kYmmfcab-Yi#?V6Sv}%~PMDonJMSi=8rOHzpo7s;mx#8?p)goN6`+ zW&BgnCvhGTLr8X!IG+Zd)%MHx5}-netyjI>U#vow*wj%(pq^3LC^0_xI;DljnEtnV zmKg|K$7W7o8PQ)%B>Qgs^ABFqK9*_C@}v zm{({b;qK-ic7i2E3qnz@4Rpz{R(~w4aUwlHx{O6ZlxZ#% zGW$rX(%mHxrAl0FskJn7zB})PP^a#mLNG(1*`R8 z8M8EH%|VhT13JVvwR^3x(sWaIQ3gh#BEX7{>3-4lZe=oBTT#ChQPua9M( z0*WzgsJ?n?ZN#s2mLr#qoft>NPiO-`YA$qP9~9{TL{%f(H2l>WqjvUYiRZ0x?)qwe zxo=+&7#mK{=j+gyhI8mrjBCTH{#uipuy(9%f_E@o!z;sz{2oaakj~n+Nkh#P1t-kM z3v!na-~?A)^_b=jtLECIXZq`3zhFpAEh%=uFZb{3e?Xc0-$jD3jk7i5KcxYcnADVv zl&ms&DbwhjWR=(i&4@Tn^U)q9x%Akml#19S)ugf=D3S=&qE?nn3_k~y&DYnGR+HAh z9wd92qBj`u5S4%z8{lQ<&SXJDPs>KjK=;pE3w;-TIzZHL`sYu&|Mnol|JCLEj~-`A zoSIg}4)9RU!6Ot(j`sEfB#QpOp}LcQsOQm>m1OqxFr%B&2z(c6G1bwri2s=esQ{WR zIX)pdAx};#PW{~>Kr({yeh$@(Xg~Y&Ge8jF+Y*Ihg_^{rICIKH4NG zGaeV95F-KAKDAw|MfuSijf<+~y3pj)$&BY^ScpvS+@vFX4ope)Wk060RWt0_L8E59URGR`@DYe(AN;D51wor#A zIpr*@;45bH4~lMO7jTXCwp}ejb0_s{O2RT2iX+hhGU>(jD#prNc)f$#VB(pHlgWKdYLR^NTSm zJb)?zfpW^S`H(fJ)orsLEv|)p&?b?ojt>E6btykWlEYiDBI?w;%<*e4e%DoNG-Ge3g)iQxfO2=?W4r5>2Mw zvwl=OW!M9|*_}L|)MvwxFCZXpnI9*n8yUKOW8`C8KTw{jz!wC~>fve+O*we{aQ;VC zPU0u)h3_j2JltanO-)@Gn;{QBh7g7WgG~f3cb>Y{V!_JtUK|;X^W>8;d~_v4&M#+8 zWW9#dE0y2QOchx(*ql?e^$}c$5&YBIyB95;BfTfG91yB}&gR_P`tP@ekJaXyHpaV} z1dr}zS+zA!MZ6F!i%(s&yiOl40rw+guTCLM!j)uldV9BX=h=}5TPsK9d5IO;WqBDg zj?`QJtBv(2*UW}GG4&M>X`&vn;|zlBBB&((~3GlKLI+g@8dCr-VrbYe++-I(H_=cfHgQ=qRteFisg1 z_`QkZ_S~KqThl4yL@wz%;>2BXxumOe_jCp^O^hS!6ay@=iuKg6VQ1K+RETGLM2Toyk9D7|!;P~@Rmt473YLp;*2jR8h!6%KIcx%yycWWS46{N?7&FlqCI zzhGj6rus$=&CAP&BC@Hu;{-j*mHPwxfNuBMxnqznRExzk%XCp`v*BH1~sWj#UD8+0_5!IdeL_Z+;+OdBA(SRH}OKwk}G{IVGZH_D|>rzA)1OwdhZ4NFsa zt=A%W&NN%p3NK4m8cIV8*G|a{h{&afm1J6_hSS@#wzG_RV-)1C@107)JWw@!fpe=C z-XAkU(Laxb988Df$$DYV>TcPAEjY(>pl=>-=TnU)PfT}{>Luy?dISY#B*@H%V2no& zRB+rlsp-OY*J9uz7bKYg$;!asPSNjdsm5Ho*Q^3?Tn}lR+HxAZMoTx1UVAPOaPpu| z-Ijf*+_%*I{^Jw0-V>Jbv2%pklIJzi$9NAzA!WPC;KrHqPqh|9(!9#SYVm+(5!k@B zJFMOQqgjke225bCkKraC&xTy(EUkh@_}CWo8oM~pqCIrGqeQfCSc;rTr=i)Rtb%K; z0+pQ;wBl_yHNtPd%M;5&+tJpxBB=Y>ub3gh^-#eR4kg6t$%R#+?Z!P)n^9%Bi97DS7iJ#B;@X16wATs$L;>MVokMujsZm0(qeB0(blDh+ z=c-LQuiEjn0ZnfaRIPeGi!m46Q&==jG;^6 z)_%XCsh9lUkJnhagUJsUO8($}&H)UTP3o84Hi^0jA;1QlJ{Dk!C_=&mR|Mrr+eWxw@pBo^!L_0bH`jscXFqWUGJz>WveQG=ASx#_Fs4PTjuB*NbNDB}aB%ddFga>1dZ#ye@kkWt5< zld=hNY55*`+tk`O)wqYfYHgmh^y}Z+&aDt_g6b?qGopjPBtm6{*u)nXlq)eJ?8SNa zH}=*; z2m;brY4^{Rc}H@wQj=K~XVCdZ#8OerG1Fj`*;N#1uY9x9Lx?tB*Aw^RXK?sDpq1W3 zctI-xLyH%4j_wJ#HO%hmXAE1xM<)nbiVTa>iFq_VAvrkkd*j3MheE%%O>L_Hh~MAb zUYSB?8_!K&Op3KOR9jhsB{Qz@H$vZTKQ|z)o<}L1VbkzMey9n6OQ6B2D#6{M&}Ds| ztmnsPsA_=4srh_~eFLc{+DT(+0`}pz6ex~1=d}o^6B`7{1O3M(@Tx;+8@t&2Je9f* zkp%@y&$~&2O9pXBWdp9!u%j~CTii2F4Wu%{QutiT+(&xJw6-vnrCOxfJMgApM~WLvbM)fvKP0M^B9;S-0sH(&9vzpmDg=qMPU}|qXhS-$ zhlkv#4}s(dJY7v4SDw|4-B9z)wG689)qk4)$Q2D9LzDn za8qZh70mIg>8$LlJS^%&hQZRw*YB=NQD7s_ZjkeI&&-rlB%T;wfaOjm;g^DI$%s)I z#SL2WPXUq41dZj#K*2r_L{5I|E~eVq>-TbJz$1(IQu^&z<+zyT{q(l(S=s7kF(4X= z-$=`+{8))8x_Q8H@n%c)2Db+Le73{x}93qE!>&-9P|@si)Jo+%`ydX$mj!eq=*UjS_;XQ)dPK1DU(5KG9-eg`}8^L_8bo6sc? zauYxN{n1jdk{2Amcl!(G>gR#x!vi5kqepjJ=1dx$zjtoF#$liCl9BUF<9j!s~c`>CBo>=F$Ym#2ES z#RDn;%D*hbBWEg|zT>8mjd9>q%$L?5X~1o`^C=ubxqM~B*|_;h6i@YHEQTGgxR#Tk z$tOp^wr~IUH^2Cvfw_>0TfSx~-4w?NFI!C9?6P(&%em-_;1H3ZF=n8P9u^S*l!ZvA z_;mT94#gt%OiYo#C;b8Ck}e8=0F+a1pLMMQg6#4)kqqK{rg@)Ljt|X5bXU{U$Utvs zq9(r7P0K1UMVr*$exs1<*R)C5^*w2z!4n!$NU>rZBBY$xphTC18zqu#QS7=#b#b8W zFaHH*t;rOWn*5;)+W7gXOID(yBKMhRK=t(k$L*=xq6G}amrDtEc0CmR`i_CkRD+Bi zb3{}Kn6$GZLv>KKOQpFYPY;tiw4&g|;3~3rC?C`v&qo(eWrRBURtbX!kOip};}~nU zkKjIJL44QBue54OZxx!~E`t`;K8=E=8+igKIR!$uT=9h$s;>Y_4n08qM(9sUZrCca zdP|+szTb!T3U`3eh`RAhtjcdaghmQVQXQlleHo6@AC$C?C7d+jL~xzdbofF&bG(K@ z&=ZN_gQ`Nzf60Qp)QKOmV9WluBn#T`S8T1tY~e$o-cyzot=+H8tx7|*TX9%97>t%l zxDPc)PvV(Lk*9K|y1A3GAF5zhSkMo8rkYK+9!hn@DG>^9UME=!F5<#%oZv*sIQ-V& z)xy0chQXXBg8i-;h_Aavy@A2ZIeli{wknCk*SmZj7#j}`IA>+gF0U58Z&yVd)?@fe zO*Z4M_`)x}mHjHHRT*}md}lV7V3``TY?mgeT0dkK*LSqm&Mudqz|PM%K%lFz+S>{j z=UDeOH8($dVY)YKVz)2r?gTfsw8zl#iB2CJ{!Py7nZj~20YDoO0Bz=fK#++8HX~Tt z{Hsl~0o=^qnLhnFvcuV=XVu9KvVW57#DJy038Y+l)M3FwI!8BfFq(;Ua8v9cGGcDD%mC_YSU#h z!F*$LsTWgDw)i`hpR&epu^bZUT0O_6> zlfOZ_u4iB&Wa6H%^-oA&en8s)v*uhheej@2{}>DqeGe=E(se?lGyj6Lg&6?S5G(B7 zU4KBDPnF{cZh9FUW=F0Y-hezPZzG8O_|Odi>6H&i2>BHIJhg^V|7-SUxWIGYzL;TCY; z{8TIL{ajVQ)bv1Bq96*VC8tvCKY}NMLz@v`Na|gt#6r!HR{u2d|Ir;+=IF=2q`2E- zsXv^`0G61p#|utf7%4XTgHkoooYYB(*ei9)Y@#pDgnbC;1o-Dp5KxD%+xrJ?q|+xu ztV*}Ya!}+>!HyDM6zteh{tZAI?L!@+$%4^7dH`AH%;8P1`^Kl6Mlc zC6P!?Y-krnS3XU2>#OICKAt4}E7DrL;X~)YC-S@V-{E8H;232NkNu1dCpEq&ULNk9 z22&$<=|;ETF2*TL3RDrazfg1zC2i7I@|dQ#7q>fDN!T=dt)CLVsDa3=yOcq2{U@UP z-2jMAKH5G7AR4gZgL_HkPW1KQ)60$xzV&HQ=}MEcCwuBC!V0>M{KAfOZP_Ig3QWWT zl4s_tBfMF=$fY%QnL(tcmcjgkJ|qa;lLJnScdF;#uH^V%OcpEvynZn3?EeI}){aK@ z#(D-uU;n`F{}0VRG)65>D?h_L`{l)Zb?tUA+CSt=%_7dqycY774Z8{B-Vew9T^*I zqksI5d%eE-2Kb0LS~zmd{}McA6eM4f9TZqu{8}yk{%Hvz71})k?j~}8!tqbUn~yP9 zIY4d^;7kAOf?D}-qj^OGRAAYA0Vn4OJ}Y;>KceL9g{jo?4gs-pPzsVPLL()ANE@7? zbsaff*wxZ@!6p+wu;8jTeAXr**98sXi)1$VIWSdF9`Abx5rE#jVRe~KnnqK>8VOVV&4nT?>UuxtGWv|LZx#S5r7jfNWE z{SidU?+dII+22I*FfRmRFz`~0)`w1WNqC2(tRe@{n$lgW= z>>=oyoF3U#8elS^#SvEIIm0^<|3a=0t0A1>TMt{@$oAR!!(_rJ)BU~bW@a7}Qv=%B z6o;wa_#m@5_W%#bLk`UeGIRD9*XKJCZpW|&jQZU;7{n#b^&n0V&8nNzq(PeCCGaTN zT+5E(Cw)r2R8q!Kut2lOwvw@@^6yyI?J!`Oo}#vfFtc#glP5o*fYjhU8)P`P^?Z4x zNKJ@7Wr~O+-8lMP$PyBclaP4!leP}iWH*D}FT{Tz*XzGF4U5@|B-ga|9O6u-3`#d> zN1WHAb5kVIgFhkCG3m*Z_h!V9N4`djC>jFV0d6xY0j+_J-?f=7fL$lyzR`t3FoD9^ z)@x;)R3o9RT2Kn4A{r(6RAY=(1okq zOJqW=pE(XCiposbLPHWDoYb%F(tWLg$7ly4PS$Q8qzeQdOZexmhUQE(|9D^RJH=Wa z`uRAMBT%Jg!7#Z1MA<>@Q<^)#?l!efLQUk}1tQ-z(q1nbcXt`teZmiyiI}##z9PV7 zLIOjnLE_mx6B^4(oXt(QXjuH=G6~sqArzsb@X`Ue(P-$?I4=R&aUl$ydqUo}A8xen z$+v*E+|VSGyAL;7*2#2Exp!|F5PZ=EUeWPGE$3-s2Ue7$14Bl@;I*oZo$_fB)I`*R z*F|R+A58e+s93zHkD{x0dAg{qpg=L z@7o<)|92IY0KNGOXGqm! z&oo2r$(V3c^z$b=0)6n_p#HmyRsBV(WePY>NC2fSE8yJwUv-K9Z%0YKT>M&YG8?&i zt`%_YVVq(+9~E+jt;(O>>)Ri`)-`Vs3s9zUtWLL!ak-Ya`x9^DFFxO&d(CWPWPvJ= zEgs+BUhq*oy&u1KwPW|hTUojIm%G<{?TP4G!7zFRc1JtAHjcyPeB%AXQ1W04JMkcQ z2Zt*PQwa#G3t1k>fLKuJg}V1X901kUbOq+`v+;oqoOv2J(*pQff#-C$bVDYBba6Of z-c)g;N~YSwA0N}XHRAxHmb~a#7nX<-TW#pNkyVFV!%d7nLbQB8foB}Yu%Rtgg?U1 zCLmzq*AXiKzE*g&Q${vQ+&8{UQ=LgeY1&OVFr*D>F?t&vb>uy1?Z1wX-0zuQNuGZy@3JtJZ+L+vPRS6QG6Okzs^} zfT2=HZgK(1=ewnb>4&iwv&e~ow5Y-an2C@nCU?DiBJTpvYZg$y4(}6T7-LA%1yUBT z0-rE0G8hS|?@M(TD&mw7V;_wM$~chC^R#`1SH*bbrH4I48@>6NAPN0W;EdO8-@VV7 zb)2&*MUja;bJr;G;c6w%X6Hex-^AyX&^J#eu=K1M61d5d9P8yUv>Z>OxIWf7?M5#s z*tXE8!wAe9ZDH$42=4thLi1eN(uzU3;PPGaa|Si5Y;~xfVy4^_8|;p|w)t0Zkp-CE zL@IWXp>06-n)}hc&Z@C!pvyD2lFmij4o5CVfHqP`1H5Y6G?Nq}&ZaC3tjVdWXm2bS z`Z=4Unlx^)_Tdu1%h2OhAGI#0K2@jDUxpGZb!}J&WPqg=hz1aG;4Sf?KG06w1hlnE zshB(f_CKyoo}J3gWnE=~;L7lOw$}VqT+3A#@+pS9eY(#9X{ zYV_}L!K~e{4sn%Ui?#6INe9RXBs0bKAcA>^(07XCg#vQpv4!6s*5oC2SWa5uywda* zQ7_KhF8DH|2RFP{CLs@Da@NnK8(JDRD+Vv0cXvhcePm_5QqTw1OVwFDLc^oXBT56Z zE4$rXbQO!nGN{M{fp6nWYLClHl({vI(8C)8wTiOMb7r~glmQWo4M9u2QEbu&y?iPUo{`8s|7ejZ&!nsCuYJ4jtQD0U-QJ^={_!@~| zthpdON5T9=gfQlkJqU2A*RB#a{rrQ5+W6*|)AOy$CCdCy=PBmo*SzU33aUn>x#yXn zE)aV}2+*!nA4LK7hx7DvCOObq7-Kmqz@Pqy;oM!saU~M9in~%^`G>g|5mZEEM1w*hhUVhrV7=weMmRfe1WM zJ3QONup8N$0&x{-nr#JfS;3X_Bd2l}~^9_4A|*luw52iSQWY ziIZd+_hjl^c)11IM-^n2tbYp@MgZBdtzMev<%gwybRux;M*=J4) zj4Y=F(z5R!cssKD%GWOIjitN)W5lH>c=dVjEAz{_rC*jHqLwYu1~&@!ec-0x@6F1g z-lyyjI_lnmXBqv3|I*&~?0x(t>{*UwMcv6HPkzp_q2{u)nVO7)`dD07swT$H^+!j& zmmYU-PJTK$L$X0|-BITvOsfa%a$}oPRNbcojfJn_g4xx})tQi71B|A6NSfW;9K4*IoWxCdCJjx;O&fYzzwS_z*NAf;PN8m{BR54w13c!e?HD=& zVtgO=E%3cBx1tbiw*0=wTpBN>q8pICjeK>ZGxlWTa(5k(5qKTbC}SIW1s<5=cf#l+ z$D^mXrGlFn%29flx^N5hzQwIuW2H7~?Y@}J@w7iQ0Iw34Ptr!WNprUh2vH=#iP{eMdKn$cxm3vPQ|dK-L5<>+%wFyniW}^uk>L z>*vH0a|Oc@zY0?tT_}bKyR_O7{;hyWwN!MCg2AjkKTmRZTtLO*iD0dht-nQ+?jsrW z41L43VxwfFo=A)7ubqd$8kyEuBHt=Dy03QGC~vhZ4v@LTRZAVqXcCj|@!smcK#Zo$ zk7R;Qm5M95RoGGKf;~2}g}1s(Q(Hy%aVR=MXsGBZY@~{=Out9U)hVR%5{6QrPEX%k zDy~tQrJ;PBbz!DdHft$J^P?QCTYNWWwf?cO7_=&kS>vSRd0i$o1njppI3p>t$XI$i ziPVTsBRy*AIF+FtS_4v$;riPVMqPd(pfxxW6M=#}xBh(xTd=ZnM3j*vV4mHitA@La z?dYdfv8d8!-TF4sK5I}2&6*9AMVi5EYp73}>a8`q+Q`j}wDGQ3Vd&%Y2M$|cbUF+^ zu<%2Yr@=vSeXDT(7^4knKhU_%x`EGSfamuFTliv9$*r^|sS&9BjW)llGR3Tbd5p@% z;Z)&vH3pDEgT0XY;h*ld!MFqyh4>I7sPB?8YN+l{|Jn{UBqB);Gh~S%`Z=dn{TS>a zrKx}zrraXL03n`lF{^<(1kOAkFJw!tTjoxohD$CvoM#J)W=)91VvBUvq%W$c!Tc+H zJF&k(zrMrv(0QUdrZB|IjmfzJ7Z}+6`WK2?P7x1fk^mdbV`LCs-TV;nntiv3)Ts7)KV_1g6=1Gkk;nC$SKO*82Rr^X*P`u`-ST;f4JI-fMANfR+L?8^oA(h=4pz<|5q256b>F9+ zVWO0+cj|VLjNPjnG)Iw&Xn#Z7PnDINl^nH)Th~_WI(Z&r>pk?4Lg>-9!nSLBYwauU zeze36XE6AtU@<>OiDrC(j{!240}(j=S;S2|>T@X;ctuXoOOkhP+32Vw0h~L9#$-%8 zlX>3lWc~^h*%_&AAU+<3;)MuXcQ$pS3%FqcUdakhbfq5uVEO$@!;JLBL;hL3AjYgm z)GQCabC>Ba2#_K|*O>Qs*$$ior$IaiS63FR1T5`nT&}m@=D&h=me)>>O?xH==_N@M z-+fH)=I$I3UvaG;)?Y!QUUr`4FgPnua2^rP&+mBC7kOa$U%DSvv394Wnd=>vwE~Pnp7A_TxUb@ z@$YJMQ^q5H=GN1h`KpSWUV3q!fHyI%88d&_rCry7O19&!e>c^6E>h9$qDy*Vs55+rv5-x4e z`^-DDz0?`Fi$p>?NO66ULAs~z9FGo0go*6c#?8&_vI?*y81SRGOZr*hX?VD7%sRTr z@c;DqJ+X7~fH!{A_>;Y$1dnp;LMme=`n5Knuq|)iv5+@K9~>22LRzn_vw{6(y*2H+ zHQUQ%X#Q7=R~vKhNL0nmeXO;Xh7%s|;(Mkh_iL-yBt;_iX~o)J1NxLw2>W#G8w*#Wvn3@vJOg8rXyvepUIR z`Ec%BC0fq|leEPFMBbqVKU&~{G$5BjE|6t|h=M|lZAFnzB@N(~4Off>^s6pE~g0Y3}}O{#Rk461TkrEzYVkWhZTxd7Ko(D z@SKn8{w~iq{Av*kPW^f^X+(18Y})djYu_2>_Qq_dNF7{lx2S%2MmiUqWgbHUO{h~d z!WJCs{5g2{n1Vpvi^Nm>K8v%Pl2_(yjyEpVEk*Ww@Nfaare`pMF z+^gdWkTWK=czGW_)-__$l+KG}=#Tdh-bH{_wm&VE9A-d%Nz<%^nbtIwWNWhx`6v-K zur;4TKENRr886K=YlRuwQV?_UMoOH`)|PjkNV+a6N!$}2qf!rv=TiPNyfOJzHkLK% zQIg$Kq#iv_t2R`*!G}{e20wq*l$6w;F?>3iE*!Dxpa;~|tNcM>^OHVnWZfHJB8cEMx)0=u|G~U=pVY_3X{Fv(- zZ}!pTFqZZW2spCcOciu*QRmBN2Uz0L2}$kzjbb*0+tn>f3wQXbI^yGQHBEw-b8jw@ zFnYGgE{AhF=WzDX2>o!$OUf;K`};er1J$)=W3Hs_?Zu`$7?HYp#gvs>*O-la*M!uhmfsMzLTot$FYXvp@!KlO5_sG$VBP? z_AsGnv74bsRmHbVV+Ob(0CUnWTPNORmq0{y{CDmTr@HEET}}A>Ua*HR3ZHIjT_#KA z&!D#Li(0>FK&FFxHU?>{&EYHY3Zep!C{w)SX0G~TTz;Y1cCw(CK8QowK-;k? zV8pSZkmw{reXkSW&FdyQ73{{bP%#zT0bxD|TZKpom==FLR=`AuYsawwIMPhg7w&_Z z9=Rym)uCdbDwlRLRXS)N1q~K%a)k#E z;H@uE=%B-SF>mIl&~YQ&R7&X+Q)vt@c3pLGH_L7ff5__JvN`+QPf54*4ROX zbD4aerV{6ruDDTK7RnhbvLcb;)!`gI@ zH+kSOW*ln*cx4s`*P&LUT8shP*Tdbm{Y)qS$CNa2c=(TsFt_6w54NKkNZf55x;p{p?X! z_sc@0sn%1n*)W(t;uFAe|8O-mE1x+gSSy(A7u!De;caS11n|;ip2IVt@+`#J5Oj;< zw(cK}`!xQ*#TJ66_g~*vssS@fU0Jl3V1ZY>nbu~;)w-t-vFSP2q`|bq*UpQwuGZ@61n+Q` zs&U??eh1g+XnMFRUQ>C5aUL6fEtxOmtO%CzyId7 zPH?fT<^jE)CP1Y8Cq3UsllP&({e6hXDyGA{j{z~z;!MGkvh9-fC^{e?+6<89N7yg2 zgh`r2qwL{>>-{j41ceJ4*f-&+Z;aA{e~dMIx7D@-6U1V>4*d9CWSWXU{yMc1A_+zE zYUaw|5`OIq=MJ^WT1;%#H!7kMeB$*_W6+-!Jx}FTMmmLzo`?o-0=+xadvNOfv0RkS z+~;}08pj)(_aIN?iIVJ%{fR7>(P0}OQ^Y6;#Ip|EZZI+$>g`;y-^jtQr{_bP9)c=L zT*~w+C*g^+t|n^ht|vnwQJG0jm+y=bD`;29*C6KWs z`a?pcW{i2*;MK*?USi_ArMLxCop(quCC}*+u_E+Xn*V9 zbgo$R1X6@-4aN~$I-qOt^Tq&w?w-7?g*gE63?W{87SkPeEB!+CbvA|9+o#0n{@7jJ z7qMHB@Ehhc{L=X-^?6E(8KDsR2|5F5dbS9Lbv7}=b6J~wlXG(^KG~vz7CFZ+sAVUA z{-bO!+Uo@IN>{V?_NjbbaNw_k<8%F)WC6_5Xu{gmvmmrbP&C~@i>(UAs?1_MIuie>dI;sTpqjsmU2y zl^bu1S=j9$+*%dPou1#PQrL6eI`7uyd^Ervd!_iQQ%EfigCxlWG6T>Cvl|?SR3~c5 z(NjZneF1^gn#P{WVwup8!l1$vzA!_xMCr`5sdh-l7OMGbaX6P_dm#C#?@}-&a&au% zFnkDT!9aQgy8X3*lEXx*Mn@cXuyCIaMRC2fM zpAvkHJPaenU#xB92~W%4+}*JbZT0T5$ktKotWlua!=!Us-~cWe?e@VLAWG|ep*18>?q)!2{`AqMy_5P z?hu9O{S^^p1u6xtvyopbflPsSG;dF%tS-v!Mu_ow%-7h1r@#wTl49 zWF~;Lwg?Z4df!@S>v6Uce@Yh{5wqD^_F#>CowgzDHV6Qb_w<0wKYRwtfRs-r^1|}c`|&MU zGs|aS5lXS?9e9o;Z)y-V)0Cqzf>{LLzP{7g_MA36m;;UOX?L2ZCfS?UsS1|9b+_9YFRmEev1ZyAyu!K7D_BD8*aTRtB69{ zK2dqZaz=JgzT}u09hfgD`MlL1i&p-GbXJzb&Jx&SICCJ0G(uz$1va8K>+2rm@Jr8c*c5raq*i& z(Gp+9@(i_sT($kvs(aO@64gDheR5#bc|>U3{4_wOm{a04W5w$&nu-*xVex85?4K#V zS4ZJH1n2q|B)Dp|g=i1s2qp0%v_o9fdCPw*P4?kKQ zt(r5?NKzO4ZoQe>#Lb|n$D|PEiMbAB{od8`{mRkToW%3X#BGd6Oin(=HPC_-RFmdA}!*nMX||4iBhVXLZ7Re?mB(Dy91 zz><~L;Wv0vA?4kxb4Z<>sNs-7tl3el&X{!g%xrjN?(cF~s8?Ba#Ts>?{VFZs$J0=l z3G?VVffW#Pi`Crsz~GG*F?-AG4fQxShHuNA81fm9EpWA0w|2~Rc+rAUPy=PE09CFHTn&ZBH%Z|#ns z{&vesD>L?Y?rF-hO~cQ>C4l-gOqNIhF6>IE|0%NB{@?o)>7|VvO>GQ?j2sN?&1?aS zWB$!=Hz>>7Yy(nRf^`_Tp=2r@IPdnbl}c+cREHOTwaLIMdD)(V{aDsErx(vtc=s@w z7n;&MI?AU8Za+Qs1gwwQVN|*3ys&`0=vsI=Xk0yqjl6;Kbp8FF7qGFg054THf2vq; zYyw$ZVP0DcQdQcOd7iD5VUDk~1qk3GEv4X|O?rS(I0mmR^c5`1w#=RlVN-EF=(Mc8 z$6{-(;D*SQr~_GBF5|-#C~=av9^oN|g{W2T)^2bzi=@)sI3BF+3R*G(|B1S-nCMNF zPP^MmDPezHu23;1E2_=kt~c+u-u`XG$dnT!^1VF?aHWyZ;IRgO%aAk%uu2LU^<`;Zr4YIDGo(kV= zV1s`4j7n_m>RS9({Z%7NuZVp=jLQ8K>F}Pa2>$EOIQ61}b<<~nh{vpM-whV(N_Np6 z2vn&e?l}sA-sKo20`A?ePVU+Hy*jklmlU|2qH9fz%rk8SlU*e9%OnyYoH1g)i?0Hi z473mGa*t{^Gz18lKJEt;gzIjISyvwCI)6zZMH1pQX|k)-0DM1bH9aq3W_KLvJFt(t z0-k+KBcX|=2P6_2W0*seFWJ5>3MAsS4C$P<|6W6oeQjcqK*0GRVA5tRN`(ALjF zNvv1mr4lfh7Iru_e1usz*B(!Cw+v-IfQBas(HH&%=%7$~6WR^W2+BWG+mS3*QDzEq zgRak{8^rBY4F!;1&j!+lx$zL=#>xORyhPZ2J6u)3{ut|)$v(Kin(C1Pe@;vxchtZ3 z$D~2a0ISat566*ciQ|c)C~m<6G(4VL`0_i7ntA|)!W0AOevl? z!EE179{w3~ zmsqadwB*-sdVrH2@pX~EnfjZLonqE`#KlW1_Iceaf+tJY`_mvqlTy0G9t)OraBuDU z3HpU{8YAcWxyHlRRKma=m!~-h9VsEcVQ`;Kk@#|6sl4G%#!{+DW@^UT z-t9`$n?-Mv?&X{yI=wZtTrT`=TSuit7B3pOp7LsSAT7QwCY>tw0Oz4NumE>R-rB2iL4oMc?3i7iil&< zs}-L0`MsNaDsnK4EMeE2`3dYpjwcKFG4)`@%u6Pr`x`L@$ICDx;09sQ^!gjwrR?JL z2zM#WfJ; z&pBck$eD%lkT6{<0w`2NsX!0<6}hnHM5xM0CM~M{9Y!HGK7*;WngH{~@IK67rwL+e z=1qb2e%A4rUvm3&P|3@}i|=R>nkTcV79hU4-4%AeGls$>%Cy!3elu8O^32zar;1^) zBH(e@HJj1JhUu|9I8j!SG*dF|qWSOqj5N9% zY&mOzLsp}vvfvHYU}y~VPw*ni*B9@CSi@~hpMoniF!tpHVGl9JBxj+z_!q1^ESgcj z5-<8O7;8Ltu@&&cx|q5ZBFSl{lY^)UB{V|6FS5|MGx)GDlB~5UPnVlgSMH?$&dSCc z(JR(DG{MZb_r;TnWxN`L5jc+?isr|9^h1*ZMw!rbp>TbWzL;T>`i-(!NuQGbSQst? zoUs~@mJiqe$dmGjjzppA0z7*|w$o*w0bgqygh~j(HGv0Rt0meOX$c4pTq0!>qEl?< zVz zNqt}+Dx#R#)4XYCFWjVAy<||Rc@<;%j4KL8deUxzqF&B3^=C!RxF?s4TVUy_m{thJ z1ebX99G^Dp(0m9Zz+bf_XN$8Qq>fw|5l(RCuScD3#li*-oP?2nr6kKN@Z6y;i{col z#WVSJqH%`%oqE(1*wea6<9A+>bJ?%<9mYlt+*or72q$$~7Vze}9v?!Z2j8Cw6Xdvb zwje}ylIp>vRM~PU=k;LJyUq5shOeCqyz+XPmlAX|_1G!4$*~(VvVw&)N;5VMV#qw@ zDc!T&yOBoxZ?IWg{l!EL8WjV@Y>woM`QA{gIwFl_5T9KYbEtC?CRm>;blXcAPGoFi zU&!eNqoo)Ns+{bYCh|*OKrS-G1i{69j^9^pn~0UgYU-nd$fWQFYFL46AoKK1h4>OO z=hG#F)hk3xGY`R0ZyrJ``Y2#{GQ| z1P^!gM*F}g6=fFf;H-t4O4aDn8wAw@sKEe#M3lAy1y@jgp7l7}L$~H94r-~hqoVl;6sbM)wup}#D zv(Gabc6X&|Am)E>?fo<`%E;_Nq(9ZeW>htdJiog=CjqOEa&g@>yC$HD)+X3AY~>Ri z@YWc_>R2$dWPuYf`t)K}QTON%*qJY16$)N3H`d(=%=rBmL$0(NdmTOkgGQ;>Zz(Im zs-$X1`Blr*Sso0Ico;6!MY?qt(&x|Mb_N@;^62k{sQ16ew_bJ;vEbt$?<`7}Y&$Cx zJQqCTqxTrtP`o;or#(!$zo=GJs4Q_WuB6k9nu$K#4^)916tdH6g*7aADVKQ2Nb|N+ zY?i8kVtjtQRD_2{ngP-@;jN(O+owWxjgj9X;X%T`ZdQ^_uO@2od^lAz>qb}l%65OM zQgG2#`MO0dg{N0>Co!)zNk>Pqj^)$sJ&3A5vGyx z)rCC#?@LeDS-khvZT$A3L#2T6cZ`~+nwO?4PS@@OrGUxz2{l>Cjq|26<3WmZe?29RiuiT-KJ@qf0t3tO9*SsVSkV7&6=TSZ2o0 zCsp*bvjTj-_Y8a{HZcyb`KGy;p8+M8LptX%iXK=--guwo^uQFo5xf zf2ouq)T5NdQ=1Y%1+G;?EF^*&OoeoV)yDmj!~|&|=9cs*6&=~TD4E2tHOw=&Nr}q5 z8TPuHN|3;b&?A265t!qb%%ECrhzH#btV;6A2zx0o#^VPdoP>~kpm!)AdEf<*B?_i~ zgy}c{(%`yj!1yRBo_M+%)TpMgw+NMh?*?9D3!#7ZX$JDm+R{luF$gg96qiN@V0g;q z;k#}9Ag##Yk<;nN2Scc58v4G12Z;T()RqLZhXFY^aX^PT0(2cY4v`>(!=U?_y5fhz zW=FPIfW#rjY?iV9)Wvk>Tp}|QOj%%cWCKj*5zzd8zkVQD25MXM?gLE*xqIlNkB zIiB0i_*SyEH;tEBU{qFkPgZ}Pg5bsd+}yZ?)q)QH%L#%M5CiG(yondN5z@HDp}VBJ z4*Np3odc|a-(0w5ste!xCO?U*x%H0K!ceVwn9g1pF{&v^+yUB12{li;is}4-ZF3jB z#J=WHay{b+k$;YScf|0^YT-2cxfQNAmbV@^OyvGfs%T|l94Bu<0KOO_Qo~@=-=X5+ zMPr;4W z@twEFi35T6P2{HlF%HCP0O4`8<<9>@**gX47PQ-%XAmo-TDw6lhV#fUWWXXHFNl4dX!i!WqAw-mb3OkStVcju|$Ib6>L~(r;`y(E5MirsGSL? zpe)~nvvF8){Kj#Si}1l{F*c*UlDL7?0P2|Jv`Zslv|=bQm}4*1?R&SB>0J9{yTt3R zj_1S#YUamJ1eiKm2Ui^;LvXTW{PyXRI+M=Tzaz;!GG@bT-Fxas)94zWg^Yga#3Xs)gzM(3Oq<8Q0uevsT*DEQ5sh2C1E`ur*P@|=Br;uSp? zr1+(MzmbyPkWj_N+!rC1NGbM(NBaJoJVX#e810A&=56+r&amO#pa_M>00Z9j_UPn>MN`#Lw!5%8GLfZD>NP z$Jm6;6lbx9FGYvSbyg5{0e#+3bONX-3<68O`fj+pS0=9jr*1{J$7BEr zp%MFl2sy~%nT&bt9CTgmD{qB_)TQ}2AAr_AD)AM!R)OcM*~}KTXz7z!EEDZTlsr43 z3NSNY2$q(*tMNeOhdImdB}Jb_CC*fx>xNWAoNG%WO6X=o%A4`9g6NXSp^+>vJuTN_ zP`Ep%+93uOQ5fFl(E}}?ha&#-tvW#wYjiKTR7hWRF^qSM&iA}x&NZmyeEu~=49m>DcR$x z`^vS1%HA~XB#PuLkaOwk423YSx`)-_SokiOat)w6>o;P50E&c;P5yt z;0y@W%D+C714@KB7kRM;*9-KGNOe30DUz5bKviUX(cYXll@*q(OKb

      ;oj$21%r;gvjIC8i}7DfQE?Ar&Z0GoVO zAV@lhy=3+@wDZO@yuXOQXA!&r=2F(ll3EP?rVz0#S&u!&e7Uq;R?pq)N67vekmBkw z9UXtarQYEER0=iXDb|K9ZI0~*p6Hq-&zO(UcbSv=>x0lMm^?T;-7#UVp$2fcqgj-( zZ^7tx(IpjWkN4>m#oz6`me;6d)RkuzrNz)OL}aDQ18Te(;&%3cODBXK6(#G6XeZN4 zoHD40Yx^z(aEbMAlaJoGyW~5GAGrgvANe!X0CisDX$TSUY>E3_-e0_TEBuHl+R-5z zHL)Cr?}Ii%<$OiA{CUBtHOx-Xz0!?V#sxj)?auRA=YLix>;gY1^?r+~R#5+cD-?=u zwnhRr*3L%uzwNXC`?IF%*AcPL{#!}y(O*mnQi`5m`lsbIpEl@Q#Js0pqV=n9XRvz4 zKSnG;;k^3sW(tNpjoI3^a?7nYInKDrz<@i!+j&_CirLMWYNoZhp?H4#nqOQBi5>Tcy0)iB;-UaN(f^{R_!O)Jlx1nh>&(P~cNWnPtTshs0c1 zZgXafs2@M4p_b~h=5i&!!bx5KVyPlYY@!%9N;3)E<5RX+J$2VouYg)zJnp8x)xBYi z(807*rh1T7Zi&q(C#NX2)#3cZq*_W$B=C!20iSWeD!%Zj`(0Zj>?FcYAw%%3hYV=Y;WryqwVr$a(8 z;#OcK+x9+Cht&x60;5mOjrPTuJP1k_Ll)bi-wFan0Ch$w5(<}v-|a_=MRF0b*VmcC zem+p|$@6FWwx$OG{U(U@6Oyz(@(OyVf)S?geu$w6*7?h7LIhb;K9ke={fh~7g;wFZ zD|w~E3X?5uli)z5%y%@H9MK&{B>7}EgqWN+M3o2;eg`7>iVdT}g^mS22KAmI0<|CS zzZGgL+!eeW^$YU_>_gC-qUYnpxM zNU?~A8>OBiY#8X{QXQG`kb5~%9sHdbIVsmYhnfLUJ3{=6%NA0+=OFDH%)lhYn2VNx zb;rRRy+?Hm{&26q1>d2|^e>r%V94A1R=N1wQOC<2?dF5y#=8EK?$gY7>UpImK%Gv7 zvt-J4?JD^qs_d?5K6>n~Z3niyS;-8%vQG!*@8~Apw@ZY@@x9e_x%Cs^>u#Z=!JYR4 ztulIKMztBQ727)7(!b!dbnM+6>y`)btoE#~lP(H7lD(;Q@-9p6fS&a*N`=`#FKLKZ^)vr*3Xc`qh=_h7P!XR&7+Lxpc zT_MrR7R&^7uMMeOXdeVWxBEyFJCEJ>3nNE6Mo!5EBhu$B*FFDp`8tihf*$x?zW&1f zUjN@lhyPY!{m=5HQd|A=)SvU(kkiq*n>m{?jKQhR(v8G}dZ?|=y z=czmGu<666E!SXCi4d_pz)Biw2l3zrKhWaO6Ia@-Xbd3-K`vKJQOc{i5I?~fRD*OX z<28{zc8Vi!aHwi%bq!S0rWS|7b##Jq)2nvV`t4-OJV}x<=&8b^K_-{0o|el~P)B@f zm$a5zA3*+gfF@7g0;a9;#~KtBWX=3h%ipbrE*`)d5bxKJiil%i^G4--NP$y z|I)G@$L__+#`!a7A#9_6!?m&*CD>kdhYfKrf+*c>?t48g+_~<0DHkx?u5$l_6H+Pb zE$?TRIS(4AEMWB0n`Ourh#m$)wJUwJ^>Xr~cZzjZG~msUOsyc*)w&5y(?0^I-efrK%KU zZ49MS5|1!xIQD@A=@XIu6Mtxju4(@pf0&vd=Z6$oLcoJ3h7OuN!4KGcvtLaoz#VGn zixtw`J1XE$lyN2FrxhhY3IalK8&Nm%ONR_>rsF_DwcWWRO!+u*{Sid`Z*4a6f{&OT!c_5v1Zu#P+HKcGE(k(A5OWp%3bPJ)XED+o zbT9GO$mM{FDk=OQ8QqtX8)Ku`uaS$>e(OI*F8eCg+qJflnY0`FdN2^?J%6W+g#&+2 z*91>}gX7R~wqGMx`jI-5#<8UzT%R)?N1V-X{GkW`tQRX+^e0A``KYrZFo%Q zH%;s4V5LJl&)6Yp){y0bFQQvxu^@}A^u}?kQtS;H?**EXBbS~Wo zaNA$M?rRXp>YG}JXMy+49l*kd7+!?e$O-Fb;B_zKPeU!j?H~Hm371SCEVFxD=`T0| zbG!EU+VkQY(;J%}-^Pxfjm&Nv{4ZCJ)7yBHs=9?uideRk9RdY7nC&@CGR!2A8N){= zVfqc-`Ci8tsDGQO_g!EYwJ;0%#x{&h32#2~8z>!UEqFMJ?HKRR3ynV848SOhJgD?W*%AH=TBFL5j_~>zA}=Z z10g;z*`}&BZ_?@<%|ss>2r4{btJ4ftLYh`Wnbty{JNIw*`w#yuh-r$OnE!V<1OWQa z-|z3Y|McHI>JIh>{{<=6(J`|&bJY1|{<#7Fx0=^~h7;4C7(V@K<8ppgRXG2L;RZHV zwt9}ggR2y$qBH2>yC0B=o&e(H6-iPco#2#;p4UB;2(W^8AumU9n{V~K{JK@a4ZQ0w zjbCQ5^*=Ifyt#IEU4kf*x(#RLp1uu`N9-Pl8ddnj%Z29vluDrFh5wdD21!?l>p>t| z+?AroiuNF|c;c0^VoICA@u%2Ie=vtL&M}xXy>9Bl?Og8gw6jWGG$<}N#-1pNWDRkF zFKI^=F#xIra-0ZLVY+}2WSMdHd+H0y=~7-*tIm2l1{f%?RG$&4kC2>*s~nGncMn+j4%02+ZAo!1@GZ(KoPcO02rniV}_Jf8gltw4GAx4ixu7KoZlne#*Q zN$$@hs^DtJ?WAe+h0$YkgAVLhXUG(ed8=h#WWBZMnB^c1_`E@2^pI>rfl9H%DA?3M zZ0`;Hm|331c6lPq$$5AiSKlKouDO<8402E%Cn=@NNZoVk(W2}u`SQl+ww8uA(SARXPuAZy5@|V5d z8V~(DfJxuEd$%FNzbTzL5yS<2dJO)zsri_ex5{dC!_1A_Wb4mv?yAx1f>wwy)@CU$ zkIh**^2-3zQDM=ikJOL z*{^>Uc=-Q^_r=)ETJJYQ`&Vvf>vpALWwFQ$|DCPFun4z3rX`Dz$lIq(@u%B&S%z?> z%g}z!xdF?Wtt)%q$Hkegjrc`gLkQW5TZ{DJeUgc3^Is}=Dt|d6EgFO;*S|DwhxpUo zU3~X~dUgC&f3;Ei6GRw#_PuzBZ%~#(e+*zusiML?2l=U_;ulahR2B{pQl@!+(le5A zjRA7^yg0QgOC_pVYz;D4f3pBhJ|KczavQPdQkDe^C^1wCf8mlTh zR$y&qag9=v5=2{4c2MJfN#Cg`#HJy&*--x?8uD7ICKk^h^&+&X5%Z@*E@>q-xwa`2 zdzEzZaQ>o_?RVe_yQn_VsTqc*17aX|@&O|DjXYhz zQK?*Uppk`T=M6#(LvcInQt~D$u~ZtoL$Y?{!DvMeDl!AeKDV+qlqxDYgaf1gd^(@) z@(^&6!U4V_>Su{&Nt14SgvCB7^!75mcsDtQFQo)Q?YeL(avKZz zJ?Y{$7`U?(jMr`6A15|k@+#6!KKLcYRDre)adVKwFFm0`nq}sP ztGCIzV}C4C7+&BWnkOq=#`HOcpyodQMZUQsbEd>}f<&C(sCOptDJ*#IS7 zmInUvi_o>@%_+-Q2J6n1=U)9I_j!dcs;j&-Pq6HQzjDSgM=wQASZaJ*KCTwRaq3>G_^ljObNC;UqvrW{cS&`Gut&UY_K?Bf3AD@Q@n3mouUdm@L>5Gcv&yDi&8`oI-scAhZLKST-Wa+ZE(dr zg??U$%0@&V!VicysAYkLEW--P!#~3kjo%}?Jip_m4%Fbat{Yws*yw)3XtJzZHaMb8 z+{jf4=asOF`ars84nRT+sNxKJd$;eYV0#8+B^&UgSD6pZDCLufF1I)}bR*6#OQ!*` z==>9Difms|Yict>EaOe6Bi|$^R}H6(^`{@`u^c(G3IuPV)%mtHaO+fR-Y=J~x2;PW zW*pE-)=YZti8VrHsaG!9yeSBA3z>u55b4rxW*(?z0&g00+rkqfkK_OpP`%L&3HS8Y zO+4d=^|B#!za2IY(Lds&!#$vfI$mPyB zl@2PGd3wXo zQA=x&jD*jQTw)F_O+ zK!FQTG>>^ah0tsdTY#+ypF9ZC&9QvT=^f~&?*>EjrzM&D!^C-SI*h1lV9O%y&mC3o zN)pto@1;*D=KJkVK!|8aEZOIkkKRe$jvQu=3Y-co2ABeD0+EmZ;6;bB*Ez$OoqGgS zo6gwC0|Sh3c6LxwJEm)3JrEfPjF+++^D18pygY=bC2$)VWxxudJFG{Kw(x|UPntc@ zi0^30big%70EtC(K#8D2pG|UQPq`-K#;|yqCwFEe#ee|Mi$I|PUyyCYPfrrW_RMf* zbIHjE-iN@W_dvFVD>%>8h+8H<=o?HqHlp4R0Jsr>})qPrNW-o$;@#{$&jX&OuL% z;8{s}vwZ~>F}xIo2F&c1XODxL38A@EN-|7!SS3IIu4m5b-RXOy=Ujc@Dgzpdm%XZZ+%(*g5!qhrk?6lq= z!F!7Uy$>V9VD2TPt0jp&t*X*7=N)(Zt9GHBT4C>8qYw`XXT-Bta6MaAfbLe&`TF&&QzKLsV-vc-#Z z@k9TZhd^>cu0+aEv}us^F^>9Ekyqt<`uA(3OC)HIjG-}d#q?rcpe#PDtNZ+8<^-(O zlzYRS8jQ-Cyr6R+}qjN-kQz#+TIzwF*Vi?s2`8}k44+VjQD5< z@p8Y|E-~C;qLl2vTMY;6*C>xQwvuB}89H7Z!ARP5$@Ik}UAw1n{mT6>OL( z0M1?I%a2=_7!1!Im1%N*`I!+SaWceFWG^>!kiWGoa_U;+dA+NzBTRzTzy%NS=q}}q z5s4U z0RTLG87Ust#aheeHz*(!@ci{f8-=g~x1chfPY%Aj(DoxOmC0$OA}~Q|qqZS8PjOeg zLn&R2qYkq6t0sNDVrnpEp#rFAXc`$@SzmV~O60)E{s%kGMZ#PQIfSGd^&U_gY(Lix z?oiWnZHp_sc?8Vx48}@lH&7?G{U;aPfP&oSZ|KFq_O}e4$0et)T(P6w_-(Xv)AAEZ zSoG1*>c)N8Nz~KTjG$eRhbz}owA0sl{wDL3B{JgRXQoJXiFZ=7pvgR zY6d=_@Md@$XGJ<7I)G$fJsM}74uH(zO7LcvsQ2bjjiv+UmbJuH6okcsM~3)TAZJR} z5?>N^YIeL5j`Z5o5#WZy1C!Y%fWE;D>oz)`nXjYZE)ijLm-fmAlpU&+@?_g_rv~Js z?JT$5WBGU|`)s-?v~V_!rkL=aU-4V*N1d|X)LJDiMEQbBw5;-LM{**Ziu(N$BvW>W zG)PjSc;kk0cugB7y|$;0F11aG)aWa71U>cwJMG6&uc!_5PcWzh6>NqKD$Wo-xKQP# zmnC*F0j?DBHtQ@ErfH3w_g42DZD%#^Mr-iU446|Tp+%X|Qhw7)!>gM=i&X=T*dAV! zm6er8O2I?I!Cb56xO@f&`;VRV_rc~T-}KZE1_543_!bPU(E;UR~JWbXGivKHh_Ko{avRoC9&`W zGk^7YW4Vm}axpk9+LQT5WiKy|Un0e-{ zgjH%}O;zoagIpG2ZfJoSLUO^%eI>oKfChHTwKQti-r9nt&U=uwF30BM`mpBT%w`_i zTDYzaTu%xW!+dF~d!i6>j_jCo(lZ;@>k;|D?2#od*df26C_aPM`UGvT}apPfJ z%{JgNR*aNBjA@m-C2svR_HI>p+YzdZ;83w!fD-Kh{*1rvtxYybn4 zu(8SzRa${1 z*>qX5QFawK45fzFZ$!Cr;c8(N}EqzBa|k6eo9esQ?UPBkh+M{88v?z^2iXiZDiCXeln-V**0U3 zFg31^P~?$v(LVYWbGN!O&Rta??2JBJixyYH7(xgh0XzL*EN+$9HMuV8eQKyUo3k3V z&~^;13|&ZjzdEb@&Ok1HUTHs(`&>-*Hjt#!r&u9eQR)@NVD}sPz$vX>zL6i})Quod zyG%u6^+e6boRxZ5Ic=Rr@ig_sg4;7*z9UwLw8?(dyCF)5^&%*AHsx2+(-lH7_LzXT zIN}?j*z@yqELi!eCrge`&hbJUd53+d!CQw*D_S2DN`okMeLGYE#|2VUlENEI4vFvD z#sr|W{R~QhA8>O^3BXR+JIyD=%`-yJmv*LfNqH)aUj=+&a{HZ#D#RF0?72ew_RZB* zSGU`Pv1^iM&i4kqx!1#e&bRlC;p_3~@fA4-Cr2Bmw$8xCbVbb%fj7j@OL;+$wzg)} z?!t=C+e6NerS>=1`}aOqTW9<0O3oGH&MD%Eh|NwnH|IA)8?JU&KyXZSOh`(Bp`$x% zXKUZS&sxXlTLxbI1)dCD!a+ofO~^>WLWDy<;cMQ&k%$W0s-O7c)=O4XDd$&d!|>1I zV~#xuCthxc2m7zEnX7FpCMM#L0ZGP-V*>&{Ig>;+tK_ko5Vk>*46ar@KR+@Rth^}I zumKwN>xvUIR26SMZx8kcv5k`tUJ3bRH3#Nc)|KjiwdW@v4+wIEh z%LQC&L6$WrJ6tunn=6oaT0}KVQhO|v$)8)LzbPWTB|7k^UXl;zVW|0k z@$EqMN#ls-lYX8{g(-M}4#x~Qbs#Fq&Ii4hj6%KOba^=G?E8M2_Skocw}2xuN%0mK z0gv5xSQNP_nG&l&QDTvS_HWL#6a9UFP5{aQmem0b(sFYnaQu7kQSiUMfsy_-GE5w= z!t8VQbbnIOXWR2+qZ|=db)bq|rM=s=gQziHm9P8B2%#L@yc)zEy^k~ z4g~h{Vs&*h>3!IJl70IKUIUDM$$ZeaweRsx6i*u}CDY%6lZcAJ3tCUnkO!E8j1d{H zm3TMmY?Tz@BY!SKKt`g>0N(=nn;MF+Mj)%+K`1*f4fW9M>CXXr*PQ_wN0#YjSKp6y zsn=>1-uFppev&9-T;9B-rJd@qOApmogWd3?-enpS`niEHpZ8(SDg+W#v@8PLR^7R5 z4yP*ds;XCli7WLRg@%q0DBzrt1_>rMaaEG$`IV!H%$-3iveQCBCiRck8kZ&&5RxUo z==-H7Qest63Jry$ILyFJLHZ8g-Ap{O@$9OYUK8fj~bho2g5?edRg>EbtVPO!lAZx= zp;f{mxl1=-LYCD?lRYKcvpvaRM#B_bMIkHn6GU68C$nSZUU`;JF#Eq0Mgoy5+TH5- zFF2($F*)f}9^>-`v6wI;2KK3{po=Uvmd1hR&vPq_Mjm8GWeCNFu=gGyjDFkgIEo^6 z=5xzFoj6nmebTPDF1^r`*xM_L1**+;D$NN516PT*fWWA$DrVH6w)B2Mk-`c5B$4AS z1(I*4O_lP5zcYi~gWk%<*f?a+<+GWwwvfYDV(jk!33uhU1b%4^09LPFiv4I93o|xU@B~S??VG;vf+~&>a|jn3ty7_ir_hM4<;Gas}>oJTR*P{+l_=B)!(U28GA z!@mzOYDWHmO)n=eg&ou~B2yKLDcN^+S`i`*GrrCum8j=Z61mD#9`{dVpiifVHS{i% z@uJr*87jy9{ZQkn&-`9)aPYq{@grrkNSV~1(u}*CP_|@4(;uhKQ>STyeCags6QVEYJ9^t%t5a)W%eXuP`eB@Y)mg;Mk!jd;twz}(6<1}PUf6HQ}`72P`-_tJMqj6!)r&JW<>m5>7~y9=yAu6baR9bvGc zRrBU+0`AJQDgn)&UC(T9lTmoCrsjKH4r=Nw@yw~mD3I6)Di^gvtN1~g#CyQ>@t^h* zE7LS$oJHAsbMiaFo?h}|Q)x>jDL%gm&hMQ;h+o9(KY=ilXfloRf|T=Z_E)7A)*hBZ z(_dWD>W~W{xT>WsgN8vwP3<-KC#f4Zh|?T7~+A^hyn_+7+E5#PGXEO+#1zfKB$EHB>L_BJ?DQ ze8%V_=p-^+P9xt!kx#z_ji9`0RfI|bsxb1B^3SoZ^s>YsSATA(mFJULV-th{{pnJ4 z>%38D-W4U_n#J*F>fc3s74NCWUFbWDL;5@&+@Rq52aY<%*B6bYYiHjJSP1T?1^Bfn;E5GsJ#5@ z&IPQUq4-07o(nc)KQyb4N?=8rB8dGU!5 zN&BnLrVq2?58W$@XkfR>VfqPS_~l$U@CSHHZ#Y>Yg(II=`XBq~R>SIYTw%0gTKB18 z5HBv}$(g*oc1?}uG7to&ZhN58Hq%n|rjB{WK!@6}L_qvYuH%<# zeI^O}VC`$5%*W-~Faaf3aZNJ8HW}|1)4Y{x>us>5H9*X(Srb@ZHCflUYl0s3rFqgH z^<2i^0+2{G#EQXk8~W!0cM;z*=~5kvLBq<`PdM^Bs){8@gNd$dhz9J8vFiYiT9x@B zov!*ROrMlVVb9mlEq_jcrr47u>&cWYftq3_A=ZEod=Wr&EsS}odsS4e1ccRpp`V#( z-=LJpGWZvvx-gAb{)fEM+QKuS>s+p2a5Y&aAZ5~FtuOFB1 zRprx{Km+}3wNgyTQf?4}9=>8rqhG2Yt$-FoXPfiAD@2FVn^G>K#+0aXO!g-{U2J8J z^}$o@?M+LiQKe8NK*ITPphr3UVwpuHoT6$_anSX|DtwllrJ0rkiCF5KPDU3Ln%eU> z#BuY(l6LXmD7RJkg9CS(YEsjVA9=V;TuJ7;wAn`>%9D2{uo_<)O`7Y45^);qv{%Q1 zZ_sq^=!x7BV5UK4G&HAtiSoz9c~}wTonP6YAQE?oL1xDaFud36ojejjw9l{sdFf(9 ziN>Xh{F?W@sNz|#!MKc0YkL+CyQXtm-Yomx8L8fX!|`5rDEXwgE}WOmh=c1k-|+nq z22Yk194OD$`5R@oPQq(%-` z+Vhfp&)8}nBH5`4-0sU{gbsUf+_a<8>Xn;VFySk@+tb!6 zArONF5qR5KWor#v(}fk}EN9Dk@XmB13bT!5@_YrYIH?ziiqQ%f@r~kh59ClgoPoc= zmY1lw7EcxC;v8WN63+}C%frq*lyj#svO7{8f;vp_#fI-=s^lnKTb4<~bp+cuQS{|X ztpvBUg9J+JRc0t%akG}`Q(4(+daUgt_7~v6O1<_O=~F&iWNQ9@_1vt1x9W_)C#YlM ztq>9mpT*6#ChS8wCKcIgho+#2Gse!$Mnlnv<9d-ZTqPDRfavO~Mmwr^05bQMmWGnj zRB-CXK_|nLnr#BCBwE1j)eP=y1qO~=P|s%7kG_n|X7}uk90=xxagzJ2E|5{_V!KkO zPPqhNng+AfBAp0|coT*+q;LtGvy)s~{FQ*X`0f5sK(s2!XXX2|%AqqAC@IZLDq=~? zGl&%C!QMmN`}QvC&ibfAO??B8c+(c%^VTm*VyllsSi8*IMO;sCE8Xgg2P`&Zd>7%4 z)ZyYA5#~n8=?&N>=%4QMle4CL}bq%B9M~G)04&lm1Y|H zL8=mA!W9?s1C8CDZaCML%rRlK26;My5pDuM&kK@}xgMt`~Oc!JKW4xqEV}LUjq2Zo6`GIX& z`Sot^zjuR}ySIdIFW;o5xz#bks>cXtI}8dcrj!6M?3Z`Uutx}@{!u^^>GT`l4q zR>lD52my3Sa40a4URY^24em4#8#^rJ$|8!>o7@`NY1Q1`Ma3|&a$Dq*zry-P!}E`# zwF(R&eV2Vge{I5QHungbjqtL|I@U=y#trI-cPa13`=Ir&F zvXDiqDMaUPlj5kCr*#Qwg?p!%%IR@za%rS9Pt`KWBip#?y0Xa_X5IvHAT zPMPk%m@qz^_H5Dyu>!3Z8){1p0X~=q`39sBF_B`%2lqoJ^HNN3ts`G}(#-|Gp9n%Vj-U;SiZLEn& ztU{I;G_O{%PaZ2j#Am9qjFisj)mWoOs0yt-Z(7M94YNiUu+3^~|B~EOG66KDt{?8d zNg)^5(D5g1ltxR3y2&$9xywdZ$mNTn66ej>TH#2u^QMH4$Q3|y8~d|92S!j`NW=5d zR{#NRc}#UL!4G=22uZ0;bTOebFyB|*c5B;~h{1Dr2FeYZcboGK=3$OM$JqwL+sZp7 zu)Pa*av7KK71ec9fwp0>cGW0y06)!cGJDGlW{x{qxLtL--cYsBZ<9j-xLL9NiAkk4 z$r9~=xWno(F+ED<>Ft+>`CpLj*0_+R01!2_h_egb2z~TE{8+#C^v4yJq?Ww@q>(CE zMs}|k$Qcg5FXV)Lthr8-L8VDM+F1=3~YV;i^I(uI}1crtI_4JSpy|ePr`YN)2Gig2hv? zfQ-cEH8lG@d?)i#)gxNyvuk;TgVGQ<$RX2RQ~eyLi>tt6^rLM9?* z7>vk#5U(Txj3v_1<0aE6kK#Vi8no%X_$5XaP$@*Z`5!;lIfmKCWq_#5l1Fy~N3Af{ zDv@KTLRHZ!44=$CW0%rEqG%XkpJB5cq+@_=T}Pr(?*%JQz!PY!7;x7HAjWY?M2Jq# z6&cf}1Ow?pddMz5!sYAyn1kK%RR$b~`#eqsdVl1*dT3z)ZB5YfFSw@jWMq@#GiSV~Ud-;|}*jriLIWlBri65v8=9AXr$its?pOegcqJ zL*acY6AM5@)74nH2DnXPJ8eMa<~P_J_S&A60)wcp%yr_V~$&y{1klA>EJ2_BlIp*ZWKfwB6X31PJp z26NIVJKlpOhLkW?G*QLniyU#pud*a&$$7V5KP8;ITq3G68qP zrY7Bn>6h3|-5f_Akcl&=M0~WsY8~VOa;k#iKDfy!cq5AIKJ0Wny$|AaEJiyHuNxms z_7)VebBZ=Ck0Si?k{&PyF#E5hAOs*|UHJuQN*iTprgBGC=Z{Xe)uk;;k;7@26iX zwWvd+Yc*^`9aYw0gn=yA&3H(IuNUb$_(&8anh-Zpx*L5il>U& zD3Hh?Tw-Ldav=ZJZHh?W6r|@6GMw+otxt>NT{_E*EOr@?zU>B4{weYAQMH-_!q>79$K(AvSqgR7~*DvzUn42R5u5!unnBreFI z3(4zc{%QRu;Uit?`i~Pc3zsZ~KDFehe1&yhy5fVh5^9wsbi^E*W44{YqSWK_`Sd*< z9;Q?swRloE6XWBr;lIFln|^BSjusA!p= zW{MW8%f5VP7Z-!3R$qVIoa`INCf0OnxS651r_=LeS4X?1f%H%RY+S7~wo;uQF^qjD z6zA97OXg{;Motr2ju4i~<KE3k#HHGZo?4~iIer%UfO#F1^L6?xSrsd;wGGjsy#BX)^vwjUrQ27f?~Px#?O z3Gu%atA;Zk%gIk24K8V}=C4(8JohCG!FDJzl+~MIa}PBpyut+>5njBjwF2SYag*|H zo>K4RGlpl=ht{mIosQ0rk8xY%^AY3v$xF{x7w{YULeh+M-a`V2$Hp@Y=SslwN9}>Oa?1U+PyE`D8F2jF8pT-7t$=mP?L*R zX&h0xGs!bwvboc*{0LPtqK_iHY;u6O^9t+LV4uB*JxFk|4a6HnZkgTlDP3oRAzFp` zhZ^BPaZTmcytzb~Adg!$rWWE2QVRv)-ThKF5B$t&=>b#y0>iHTMx0D3SvVEyt2D z@^R?i*(EhN#2L4BcY*`wr|2H(rmBAtlNV1Fde*HvnP0(pqp9mCSjM-4dn?MX2?Q^@t3=kz95H%P;~(RiS+Nj6-ERnd;XAn#+FP_;2hY4rT{Mv zlc6}~1^f`+KwQuVk#h1d$VZc`{zHmULa0aHUI;o}j*`JFAgAR|;2}F{oZd?QPBarZ zz92Ft6|PQt?pxEKRKz5GSq&~aW@D6V7}C>lpCY*28uMW$RaORGg|^;`5@tq1=_J~Z za9y78fb$_Z z@cPi62w#Ca%gFci@q;rW9Z^X%!Npz=X_QC&Sy zM%Shvx6>HO(f+jJ{&WuYDm!C7Z6Q%reSbrFCksDQGqZ|vEvfp@dnw`H&P#^4N17Wk zN~lyg_>V%2Txh;n>-hR|Vfh3_;><`GQk^t^J}3d_0!Y}^JMmR({`aeSkq$W7L*a`%Hu!`;}t8k045WSLiz z>P!jQu=amE(xr9HvTpY~93SxAX6WnER;e-8*^Dj*0|ZAfx|`_x3l9?+2&XqEH^C|e zdrM$e&Pd3DTlftmnW?UUZG$?$++M}`^x~${F+*wf2bZS_pB_bI_y1E^3 zbDogHRBhbCF4L7usPsfrE7b;yK^!&!ytTS5d?{Tf=*82`okdA&C{?B6{$@`D8(?sF z%Y1kaSPQEhfJiwDawDX`ChXK80|$d&Hqps9hj-_D+L4|PRscZ(9~;gzp5aiEYsUnS z|BJA9iqb5Kwlu@GZQHi(KWy8!ZDlw!m|@$tZQFKablmE`_g44ls(Lst=V6cYu-907 zpE=hzldOyM*QxtPVop67Zs+IC)GB3wKBYX;=8;M8M4J+jc3Wq1KpwfuD_-gPaa~P0 zgkP8M5H^$hHOnq~dV`C65TtjCbkgz`{j!NH;@EBW#SJ2O@<}dYb!CUUfX`I8T7t78V ztjwqe^DA($%1~NBqAkSNY@!_%kOPEKLDJ)MDfRUTpm8G40Ss9u*cx8XunZS<3@dKEjO)y%mi6vxA@8p=E~Bnq3D`9KDWlKtOgaJMGM?ON)Fq^M#QUCc-2;3lkop zA@}f>?&+&Cy=OFBa>bSc()SYDs$cxO)l&1Z75r(;exm(hE(dzFV}TRD%(*ksV{Hts()7k0l(QvC0<}1d zeG0)4UktgmSaQY6{UIS_hjn_t0V^z*R76iW&L!J&kI-324_BwEIX&~&oJI+EouopV zD}Xw}`{>sy{rweJqS&5a~6n-txLF2QY>Ju>f~Jh%Kcw?U1T24^DH z?>66w;rBzaPT0winZT7ELd> z>BKv_8l0Zq|Fu=ui!;w)zZqiHtBeO_Iym;(7PQ2LJo;d1((C8YXINIqBveT!%>O-{ zG1Bn2x$iUQ;bzeXp`Mi?UO+Wx?T9(1F!FlxKDL0})yu1E%cn6E4rSWuYq0qX8)^At z&$lfA-YGjYOH70u4|xs!#OHNnx}f{zf}6%mX>k@Zd3?Bu#E-s=Zk`G}-`xO5s76th zOE-U=!^gt%7BhJ^=U#^9CXi^VU`HYYnmnVSooRjQUf3Z`N4qK;6({G8xl<7J5t)q~ zJ1RmKom_7^(;zN!mCAzXt-R|8X>)YIVIAorDeP#mP(XFUQKkIKLEk8H1$;d%100@u zW%;5FJ(uRfndu{d`S)9Y@|W{Hq3Wz>I`#Hv(u5vA7MWqJ>B=oKHWT3216%6oOK1;K zjc5iC)G>8eanbd>%24T6r!>hAN=Z|7nMugX&Mz@|Ot^@{v+Z@tb2G_+xYdDz*B#4f zzYX+F_U37_-b-FfIBcEKzK);g53T=G2^CMO*TiXpg^lOhNKa*e!OzcUnV9Evg^J#d z-sL0$bCe0b#l{7Re_-u&19v*7T_P*bw^v696zBqKYTnx|IS6csh6_-f8%X@)iUWhmri5HI zo7zLO$3JAUVQhh(A)!JbiU673$4_U}J}3E^?1(Ag?r)!Ae;XjoSco%jl8QHT85KBG4`Lipl-q+aa-&0%z!E`#!L@>JHQceXGMXc)O1&0lBn>RuKgg= zo?2sI?plOJ1!GCV4u(IWA1l3!K(djg#$bB!&|>0YvJvY^^`{`IDFd>sjFV(RqB<8- zcab+xpMEGbOiA9F7^`5yr>>VC6p%;eacM#XsZWx$-yAAq{WgX8ik{Ua)MkANl1Yi$ z%}vD@R7oTr+o%c$1Zw&Kx;R;4#T|D%6v~==pnRG+F9^*@o4{TpoUw^b?c~N*y$pzz zj&D#j*|atiNAB`{f2L`%f}E%M9bG?XC2thL;?nk6mX@8a1rsM*-j{IQozB$A~t1_eh4Ni z2DeUnzpjU1L!JD>SHsy;_wD5O9<79pfd)U5P!O~s_PQU3S!@&~ADQ=syP%{=?Kcr{ zWsW29k2%8fUuc1xhAYRZ3__hemjEEC%649kULV!^`1mkV70+K39uO8@OzhpAWiOcE zQ(bsberjs51@K{FZbcm*N3BGGa_gti*W#AOpr9XX|M)Aqz;Qp~_pmfURW^X6in9z7MR zEpoN2_Wl&Unbpeyg*NWq@Adk165c8Dcduwl>XWDeU zT)+wCyZ+A1=#6Pr&5NO9oO06%$&>8HE4?@;)4;ORy~f58lxHxkwFLq!=w~^FV8$&j zp6C<=b@?0Y*=ucX|4z)Eudk$!wxrz`FmcCBRS-A2J-C;ahoX;YZl3qUd*Z#^?u+f= z9DDm}aK1sFt7>$+J+lq%AT^!YJKG5M*By8~4D6;rFB?6xgYaub?LQe)yYtMM4|%`# zGtV;xX6B!&+WV9D`Dpy4W2GLJShVTCfAbR8UkaIzCu_}ffdNSsVA5>429w@3&X*f$ zr+Ni3K)T;lE{IGS$0#4395jEG9!?b5cys}XlyOJqlHkWJil4_2&gR7q54XcH!7eWV zzyIQ|#jVe};&rk+U$IT$Dn>Uo7MUS9r2}=Dy;YvNFV2cN+gdJj`TejmJ2E_tI|867 zbj^3~o!v!2nAUzQG8gG|uv3>C3LPGk9J4mFS5KF&4zEOE6lm_i^F>-8A-p+_hF6#W ztanAQ>E@~P#?K@(NMpYXo)o!n@&$PO-hNk06dBJHbYycj2SFwka*`&;Rx^0b{|t`u;nGk0+99Hadq>Vskt|B0X+$MWb>BBIBr?JgZRT$==)z0WVDdS4Y9-i)!z90TY zCp-_H)A?P|C))7I^Ce}q7W73l)aK)t2Yb2w7U+&ZAOdN9N|RtQHZ+k;y>UX^$bG+P z9e4gG#i~SGL!q^ArWW0AN|v+q*0zQgB=$)muR@s$Pw`x=2CC_m>gm#gACX+qxcqyY zPF@Xe2>NjaXGea5oM-^Y0|lIMtnO{Dxk7Gz0^&*kr4v+ug|saTG0k!T=Z)fuFW}5p zi$Q)c34XZrn!SML5Km=2f0uGgex}06NqZS3SG9La#l)?)l0~BcmDneHBfAUdy5uH2 zmX+hcL8jvQ(l)->wsyyyz6GaVW_YeAZ|xvUQQG7rfRfiYr5#^CbcCZv>aWem?C(zI zrgc|Fjt4efUwLf~Torl0xbWU~x-bV>t3WDyL1&ognebiL51B2+LZyL)m-V5LH8y|0 zr4sM`gKegTU#Cx5F(|6WEaxmoNbjo5Ea^Mt6{;e0b?4A0Cx2mK4r|BJt}k&KTCO>c zX5cYvJ#OvPQ@i5vWwH=JX{4yBDy*Na72&y=|TRYO{QDtzn%P`u-0I(oxG>(~` z5ehVvW?g^DSBDp7S3Dfrdiueu70c>euy9VUZ!o}b3DPzZB_LmohKvU2h`Te``Y+jO znNDqbw7%^W>HC2yz0g9e!i>i%T67tW2-E^Q92gt0?a}n>FfQ294EainTz?cV)ENETYgEyQTaq%kG1J(qbK4JWe&yHzcTf%|@_vNz3O9HJ#9~>6@Ge;ZifzMK1I_G%GaZx{&%W z7j^@8!Oc#ABFy+|BYQfscW5fln&_UHzM2-g+mXBkOSc^GLM;`SoU!hj^sJa6TJE-b z#fHoxjlEr#yfvsa>T1R-)NP`ZAWosBT940bur1i1K!&&kFY>AII%$xxN;E-ZVl}5o z-q;pnUFnoNpsI0tn$Cc3_k|MZr(f5-j`}D2DV+@m#agGUqV)PWz&(klMmDj;CP(Gle)2z{fGj>C`8y zu+5axfLs4d(jh<1=6Q~g)cR<1qFRezz(1hfFosNa1P)hC|0(naP&q$-43JABvVn`n zaf(JC7~AM(nU~iHsuW|uK)i-pyz|q;k)!}D$^v6Nngk$5d!FAu^56<`@ygY0} zW>?#BYlq>8M=rlo=hwUdnI%+Wwys_nAv`ACCI0;}m54?zg8n4#W>b% zXVYwCI*~$mmy#$p=8VZwZ(2cQPCYiLj|8rD3zop9A5EVA?ut8TQ!l%gO!vI(8Ss-+5u$@qt?hq zzpUl4e(zPM!+n@pGN`!Erpba}3|XfJD$hzBiav;azzjbYJoJyE;1d~;WS?x&tY$F* zmXuh0BnnXn9;qyoj4mD0lc{EbREIG3)U+AY7`59{;7?Jej6#taCWU+&T!anjU&e*~ zap+*#3DHeoT%93NDgT)WV)OQbdIYQ&I@v$cD6e_!p@ir+Z3&SfEho`?{#YrmpOcfP zss8Ia5bY)zc3e0)QQYSzDvXa!iXNBGd?+Q-bKCmY71skFir7Y|g()Vaf4!+bmZ_zb zp*W@$$~DWeG?c7y0l7c_MBO~xqR`vC&ZjrTI~a(aiQSl^e15+ukj$tP<5A)I=Jkw# z`zP6Y0XUgZ2Gk62xQfk{u9W7cMXa;J3*-6=hO%HQRH9w2_>m%TG~xz76{UY!i~J!S z@Qr1K;E=#4q(Asfx>?tC*O^yKm&f*$qr~)B%tFJ?w12?c#hOFyiNZ%$-Nlo^6Gg3? zYe0ahmLM|c^sGiw`4NIe%FDJV*y#4@e$&QpFV0^;HWtlGgzyoyj;uJ|2V<6(3g$QR zSUb%n4cnI`h#%KWQ@pw#9X6y1D-QNmvM=QxoEcuXEh4*wb|!AC9d1teBA{DgnY}qz zEp$x`+cTSGoIZjWAAt}c?JL(}jNr}eSnxGZ553I{5`{qtg3?(o)^?Hnoz%k1E9 zAi!5vn}@sp8FXC)4`84PoXW?DhPXRytyRnw>qiBSk9N2@FzPn`&0Ss2o|CWp2D8p; zOWC$nJGFw(*#<>78t@4Fq|LvqgVEfD>=&AB-W)R}D-8%n-G2JGc8yl_34{!#*4>}n zF|+DEDRTjmL-3Ouw^#_=cZJY&(sz5T_=e-ZEsb(CO&z;Puy^pe^`XO5o-{Gu86lTH zWP-5jiV=B9p$eY(t^Gx?N<^8-`#WK=+J3Xo%+4lE#}U3)^!mMYiAE?8XhFlHVPbA0 z-(xGg&7n`ST!~r)`la+bv=wyw$GB2iA24`xKzbXzF;^t3Sapdu$nsf)NoSw@PE`k0I1d^Bdxd7{P_}(5Z(b>^(|vC7(jN0Awy0jHU*}d3Zj!H*BfHtkwUDSe5Pg& zlWC$x^h;TbWF_8GXQgWl{5biaWuWps5K$4gjVmYa4zyqdEGsJYeZzFS-aL#|qo&Gz znf$XJi=6RsqgyU{z&N^C(5=`2KC0AyjRKT7w*&Vf-ULv$iF;gpwOm%D;nylkf&zS< zZZ6wi`Tir4cbk;8NJR7vlC`oUJMnn%+Jg9n@LP;`MbspX+^I0MjF~3tH3J4d$%FAT z+?CKI!rYs&aGj=GC?XA{qXJZfnE1Y-?#j?q1U?{_b9|bSXsSe7JrpPe#qfqCc!rOM8UDPDqqUzH% z)080moT5KMUJ!21m{L~#C`^K3uT$8rs?5)xd=EwNF`yo;e3ld4JuMrKn0q4(HO_70 zMt9q@PQWO#iL36Z-WTdB`|PKjn44-23i+MYk1pW&{2yY4c2Ep;a_+N%7(8n2>h;ty zMSqf563Z-yN_8d)Vc2oBhre_k86n7w+9iNV2kh*%`hm7;5(Y zdnVH!WKmNYh2l$c7F!X^?&Xl!fjs({fm_|nJG}oUlJqE~3*3gS6mdsdK1m6{i75ru>EWek`C1Jpk*RvM5~SE-0Y-F?>{W=~ zgY(*4`_T-u3{OtVvMC+}&OyP8DGj&|6Df)x>5;T>BI?N340p^ZDKjVVs7?l=V@@T` zLCh+aDSrld>~fy%sO-B-6xDF(ebA@KTZ@eKzwoh&Cxx1H&RRb;^%NAShIgJWmuHi*n6V6xc8y zMDO{7YEuO0nR_d#AILJxE6?RO%tFuED6%DT$R^usEB$H=dY6XtV5P-ZIfG)$Gek&& z#zKL$poUTuAAkyQsPh`dBg=BXWKCpF90o{P0nI{$QykA5O|!Mmq0?QRHB_4${3 zoRZPR2SC@5mP++bG&IMmt>nzCxF=F>1qyXk(>BV1NoHuk6*U>df}QPgNsMLg?MA640*05N63C%&%T_fj9RTcn&RE`Ke@3=J%waWU`zA)*g2iJI`_wAymXCjA%{Ni;u8j;|yOg!v zo%gk&e@qk^XsjOwpYS~LO)C@cA7Gb#uYJ~fBUJ&L>DVzEMdx}_R+XHhGT1Ek0|R1G zZcZg(D@kKj;=YVF`E27*h`B|WJwiL)ngt@B-oP37<;v}v<*4c z&EHL8mx;=F*ZC^oR;s>cmF4tiSjyH;&Bk~I@Dm24vixW+@l3rd!^m4aW=>7CQqps% zN>Rn-!kQmmsIS>9ii!Bvs+$XNXgj~jx6fuymtCG_*J_#Mw5KEyrLpn^9%U=lUhSU6awQL7o^ANCIWPMhm^06I$1d0D`)H(a6kZQpy3eXD3M$nF&j z!-$bje}2aD=KMWE$JR;;b%=z$rKYEB^#E*>5mB68au}%mkXf&DF-ES<0m|S4Ir>L9@d*lJZ(WY)O%{$==b`Xeu{$0l*7&g=Wrpz%9I(t^>$LP3 zo9{a!)Qw69KyOQdyl4DsZ|3l}G$LG2Cw(6fO-Mh5abco{wXlhdi}e{}2%#CCXn=4i z1LsrIp+wcslaY~2q9At!P7e?a3Bm+j=aIvhjne3wAIEDkTqEaSBY-gh1}~RXWnrsu z{1w2eH2Cf;dx+)qV@KYv(4#Xjjjr)wc^^F#z8{xjqiHl2o(e4~rjYPY1Xn8xed=yS zhe$wbAH&OTYpX3UVPjNz&^<95ft!)>0Ysqsurtz8Bk!9!otF27tTw*1$kFZgSna{6 z+3&XChNA`3{V}@b-1eV*b7`MQ)(v%e-1EX+Tto-WuAZHAD;s|m_0@SqR_8ZMSAag8 z;YNZNtz=r<>`es|tShK1Qd?#ChCzrJqp+|_5tTc-{_P@+0LR0~livShAi=%Xm?xD1 z^3#VB*C}Ta^cPFVVAx;aT2Ya0Pi4i_=j7DbNb_XX08#ray|0(F1uXNpc9`AOAENCF zILE7tdGiO;@JTR^H3ljZz$rS7HUz3mN(8XSh>GOuHqs|@Xu@ed=iEw;Nk4pY{MFn~ zYnbx5qm2%|vB0P>$|>z8r+msSD0i>>25|Hu^1+^NiWWWHu@2b|q?}}sIVl?*v?L@F znbZgF?wNc|qz9*W@`W$Xi?WAo-n-1Tgh2W@8{pYggQj`Lk)oj+xpkSpAE+SB1dw6R zY{>T_hER4GaM8VKoWcxvHMu|YY!6qy zdDzo<8mOwT?hTd!kXL13;{SblB6aT+9g}@VIzgwww^GQ@IxlE7KMdO< z!z4!YnRaDyIK`10!L47CF$N{H*I1}j$Ti}hq~GHIY5qzs9=#?FMwnQ-w6thF)ST3x z5i_3GbvQQe@)q>DU==o(CoY<#DW{X6zAs=)2jk-^q!C%G4%yA9aa9P7Z)ZOOhoG0% zGPo+hzZA9)yhT2bzaTxa-&h3F(iWv)E*EVBp&LVM6UXcQoY{n~dMxXX_C+W7LxnCw zB|Fv$&Es*$k2EqO96oPHD>GGy5KN@Jo6}ki-IHv_t3HY8-cQow4T?i#Ge6wG==A9fw&qea}j-ly@OiD+1vXYV8v3A46| z$YufK#2H}b#4aI6gl=Hs%t|)*EhhhFQ>G>y=@OpzF5*wV{M86+-Q4uyd+PXHL&8a@ z$=%t5HMd1vm1`k@UWm^TnDz;sWCo0YsSIbX=~ZQ6*%DcjACjPR090?z-au^16c676 z8S$Y$4l=$&mLVrl#r!@ehJVd)x*y{9!s=cZa#`fc<_^iubBFo^qP%}rFVASd$8eu@ z#B+Lf{@wzdMy-^8EbLbJ%a05PG-V~SVmCz zupSr;SmxyVKwVkxEJzc@qDk|ChIUJkZQ+{l>aj4=9@@e)B+R?9wgLY79MVs7udZ(X zA)zdPpj?`gQzbq{DDhwX=hup*H<2Fioc~=C{{i%?|M(yS{{i}eAp-%?{%?2Qe_{Rp zbKfm#Y}x$myMPx1;`WGDx{Osch@f7XW&ybNzGeP6V$-)xp0p~tWX$#+f69Q3r*+e> zXHnw|g`SR&t%%Dt+v7@fCQ#-iZf=w4&g>LNWIDVPe%i@mHCYs9GX`NH_p3=IL1h-w zREjWaG$omNTiP^Y@jPd2Q1mk3q0l^33@d6F(8hVk9PNi--Yo(Xp==lz$V(aFd7%9& zRp{oG5nMrO7m9hoKR9o19#_&cJeW?%D2mloF3vq zSYKM~$LaVDrF&l6RGqg;2R`Se0(Co=F>$AOu&+rbw7*z;-DBCK{XczywVLjJ3u6SW z#{}k>1@cq749w##WXW@D`2?S*QUc07ee-3KEFk!Hv9MI`8Qy|*+n2) zYXj#&z4b;;3I9q%-UH;mjFfUt`VGDIN&tRohp-R^pvx^PymF?MzuNEkZf@`qh2N{) zNjr0Bo}NC{evHuB{lJQ}D^^HZo3}&=W{%0DIeTr_H)>orQTlON+9duaQa_=iC?&c; zeaFnxv<3}0j9TpeVOL!t1DS?_7o!MWfw<}|rL*37+0etp+faqN{HV(m80$kDGviuy ze0#YfW=<_`^Y9J{SQ;)31UtN5B)T#WS^?g~LjC(%5Bl;%!ZjPyJe6=eB{y2A9(b^B z))xft;wTrxzm+@w5`|R5f3!5_C`kESeEScZkj?OB*B5xi%yR=6Re>oU)Lu=x34z+YNPMjcARD%=1*l>ji=GB90DauD!8rI!9j zHHjSt)q(s})8|hdAl?7An*Ikp&&b)?$m>7LDGsn>q6T9soqIXG zv*pojzC!K&^0|Cppu2oh-hl_gj@ zWbOT(8OT40V<`xIoam~pJH=7^*1W00yEMPK^d$$%MNR@^XXF~FdU7gvZ_}vOhtCWZ zaazVGJsC{z)xhGTJ?TGq!T~#$$GR_&gv^6!ah;mD^gu~r+&t$o;{_P;Jzv}EY zQ9v$Pj=~M_B9a}6Xgptp4<1@0z+_+zZIt2- z<%6+RLWMvrTpkQo`^qk_xKAyNHNuC!F*)IAB8io%!QCsOeyB{0;%YEqB2 z>P9ffz{81j8M-@8!x_CMzGqO8*DL{|2NUt?Q60X@6Xg~#i^NZMxdm@3F1d%4y zkI)nld?f?6?O0_#6mYog#tk;L!Z$A|p)JkBKHop$hW>XKlW?!$S$AU{O5GD%l5SDM zsYg*xMifIVHS3K-?YAEOP|6eLe|M(97%~wp8tU-}2p}L4JRl&(|8p&w7}?qy8=2Vr zXE|MI**R=*ApN^G=xZgUGpaV^oq}m(Wg2mMsBeU}Zkd{e0HL32Mv-2mqKQ{}?75Lj z;vlD;mW4zU|BF~hMsO34_`BnPx=Ou**%ZuWpA@H1xPMk3_lajqq417!`Zc&-!;+A- zn}|5d*){NlY*={RvTqQ(5ZQT4YnW9+7ZY4r1Y1%I&vTK|9?P-cF{2qduY_%q%Lz70 zf|+$(gn%s8n@m2$^SV^0>bTR>wseFons3q=Pc zDGe18x`kxER)Ygo%`O)oF);hSiB^(jx9EFKeycjicoJDbovX^DYN56XmrMr7HwlWv z$#_M7?DJjs)c{sHK&|R82b6z>a4FCm6m#lcvk^mZHR%K=mIoY@%IV=1oH7V#ZsMGo z9U21X;S#->rZ}Bi`xux5lb@b782S6x(wR-<456-SU`n@?IG$MeUPQlr$CHJDVR7h zo08iW5KYbnjZ`L@iXRExk2Kq%(Cqe!9RK*ZvgJUy{Pg z#1rjuke>CfapYk`r@Fi3%xGulgh~ZmH(tlJXn-@RUeLX@?@BkFolCw1 z*(#-6SAp>7XeNqHU=kBUsWr9y#a|I_Rkc);_{!?eC1uXmVUFc_j^XE&#;^S66@NEn?L11FIclk&80H$GwCr zj?%p15wC3CW0pH$8Cd-9VR&eIo-@<>w__`Hgx@z5iX>Ed<6(vEI9bontzRn1v+^pNAlPJY{hz zI|Fy)vpVtLn85x4;HEvP@MPz?NnCO;n4R4@E&PwR=z#sWDp5YMlikyWu}E+t z+F+=VMX2MTc-~%@-w2nkn2m8x`AHzI$=Y9AcSAJ~zxa;GoR9VSTYQ@GMnQ)`yMzM% z=!cO(vSo*xmd^ViTLRW?@zUj0yCfH5;PF}UGU!&f5dfG~)`?!mW zVEk@Qsvaq*D?T*q!~qB8KAB4+$R9?j^k%e{biO$9P%+FCD?8|m^6(b$5Hc4#k_P-R zW!SfgQ{n~ch?E9xpFD%l{{6-2&7NOj!*Xm*sqD=i*Qkkefbne+UvKG_CCa|rag8E) zsB@LB{C_Sd&M}uaW@{%~6CL+$3i=_=F3+#&k^u$;Savi2%ts*CY?Z+E+)P0)#=g=y zx*f0#y3opA_M%!t&z99%Y|?@y0p1G8Q46uigP0iugu&5#6cEJ>r`RQ6{joxbzZ+PxmvdwU6rxwyz+;9#2pYF!Lr!Lue;*=Z+M24qvl+m>MaAx5#JcZ6O3gzS2p z{lb^+QzC28aQpOfum4m3ocW=dt#xflTfG5nwV=l{br`>*!Ncj3MiTlL;ZoD&j9jvJftz19f zsE!woR(Bqspu&e zOsCnTE$|#l2I_IE%Q?BF!{AE#r}qg|>DzqFmN*s{U9^y;sJ~pT{oo21Z6ugF8kd1) z4p45++KI-OQ3PDI;2E#$9bbl*dTLEvOlAG*GxF-B&I99q0|@eGO<0I{!nZ|1>r6*^E9!MU4Mctosz(2>LVWv7S{D|0gUEKu$^|j=Cp-T*JPb&sv z**4B@{A3m@Ej^DBrk?Wu-2I(|!yP^jkDuQg1+fwzSpT6fLCk&uM-X-j@?X8|IEui| zX5ex1FoClYVrH~}7xfXI?(2*i{`oLXLVu;rWfaPeTe`paya=+Iq6LuQ!TX1nlYNbz z#voG0gq`rPxr6rn;iZPrqw|N{GJBjA^5be3gOUjy#9~GYy3mxM?4omRa+yVv^CcY4 zscW!BybeKsT5>+>hf=6^* zCocE?iHJotq&p&pFMsTbM)HaT#*Ll~T1jy0NxXJ_d0CV|gdtj*14c*Uk2J#soA`p9 z&A`tBFS1Q+3z~x==^k8+SO%HeY7vcbSpsh~8fT}IkimH+WhdktL*hLW-vq$2yizj+ zV>vZNF|OvOh&FB@;&aAFT{wTPN34GYCfi5^567amgL-?(fEyr}1;-Q!b&U3fo&5uY zLdPM*6ad0^>~gMH1aZg-PHuxtI(P8+#D2iz2og^Us!GVl;l6t!wPL(4KM>-tx?pW{MoOcL*j=xoU!E3Dg$QHv) zX&-~YVz+!4EcZgBB7SGvqdQD0A7&DWf7ex7#hnD5PlJ!%pueMuvz7RbMB@`kL4j0dGwO=(rQkD9aSh!{fXi5~(M|@AXOqCMX%?3(Cs-^z4q!>^l zTpVY7DVa=@rs%l_F`h#>8O3ztf{n6Oh(Uu)mHS5kxYhJ%f}}qT-~Aq;*a}@oYzVC! zys}_s``TV%yC_>e-R}yRQ=tXG?tsGrS7{Qd3DBo-U+`BFBX+DW=08Hxgd{>}g5?W2ec*6f3;cfiXXsrd+ZF^&$1WezFGB0}?;D1e$(QRdp}Hq+pW{c|jso^i)D`F=AW zP~ce^h$@_agV~VE!Qk6#Nbq5^D~ajb!%3lc?Zgr_;mt@8f1g(qnn~u2sMV$Caw)RL z$8iG|&pU+NW%O#n#O~@wj>#Nz%&_XC(nS~s8+quPr}LlNVc*bfO}fzQUj6df(9>K7 z25V)*S(SJ`V~7{u=2%RPMSz;Hku_fvQ-aUECJhyOBkBG+eL>7_zD7igJ=BL8sRP?| zm9U7_6>N%RKJsJRlh(a;NZWqxiE_vVUdzgJFfcy!lrlMjfxXX*I z`mL7e{EgfcL+Um5{*x^e#r&YJvmFH!ke=Rg*!{o-q&>S8YDBFXu@Wq~@Tbn} zNBrLPscX}z9pS3!n|X2RzO(dP>|upqW@`WN;?CVCND4L&MBI=5!vQ1tj|u};+0CjXrSG3qKz_i;z;2jnt%xHvq zUklo$2l@{+$AjB~+OMM#5;;dCo1V$J)`dl$Ux8FyYS1F}V#KSNhp(CpR)@w%H-GlH z1;hSW9vJr2L)xcaN}juTc@p!FCL#i9^r8Au^yJpYEgTN1Z&8xIlR6^e?2en&_#L@I z?s`O0wAt7d=cOSB32HmEiv(;@F%;5YY`OW50S z8%LTJ(jBc$z7?>Xh(_#a1eqBSbsg$wjxS^N5d&A!{H~(`&JzFyV6IJ&nxBOm*`c7* zrXNZjxikce9azu|@!(|Mg$mkaMJZVOWU=<+Hg`d*O;E&LfG#HC_Yr3Ek|f?{)cndR zRzTF1+)!8AShpwfsa{q+=uejx9bTrG1~QV3cQYZi@NG-#((*8^*~KVMKTZ79ZJ$g_ zF)6A5=Y{u$qMmE8llrb%=EDpM@VLfA$QGzF1uRqCW!P?^HT?=|p4`6JHhKZv)N&&j z=qe-IFmc;5`rH<5r2d=(Sk^nMKBSWTRc2Y5-`BcQG|cnnbfJgv%R~+UH&5|2@om4Y zU)ZbVIf-BgW=BKb$XLi;M?CfdWO+=AChA`g9*EFATG51Ea3G&Z*Bf>KS6*HlM*k=8~t zrFWV)3mjp|rNoHsi>%`aZHuZ>o+n&?YOhe}L;PuVPHonb-st&GYre10Dbf5JyayTn zD#V`wEf%d9d7($nq^=hk$OkLN)2WH9YfTChtV7cZfFuarr8yq?QVb-HJUD^BT^`xS z$qUd=*#E6@_d?vcgrqPG<}6HKLOy-$xHFqMf`+uR0{4{A8WjXh(3`H3(7S?mc#3d%dKQ1MycOJf(BfIbZRXMG#z$KfxRu zh<*EHpyg7?t@jrno50<07!B{*@Jl5Kp~TUARLHGW`NxgM8X476VoU@( z+tpES94mwC2F6b!##XWkVS!&)FD9J85%NkOX&sEvZ>gGr4J?1M$~R1w>Gn-qIh_ zUMfYDoI2C$M3teh*>+LWhR@7SSIShue)OYFe^Zb|jhHMFR>)T#u~`+Utn)qQ8MUxV zzVd@iTrpYTbFqfO`U(wR$r(qW(97tGh=rL1Zmb8@I37b?pVr+8!HhTQe?XKh@vCT5 znx3`M#_Hy#OC@+E&da0(xO}>=cSw96^I?X-0}-FawN=^5Ay#fp3iKBC2X1(SsD8z{ z2b3H!e%VnPPm9b#iCIUb)lctBMv_V_3^O&Ch?GC^uvZo;zq$5FpD^i6LUy>)!6o;E z22&ZjUUM8g(b~14p1hp$wxqJqRGbp0LIMDx;S2x_@!5O<6~ z?>Al|3#K$~wXhGdegTW_FI!frJmiQ-6mw^JjjSenOMv&&=81!j>wLlJgXMF;7YGWV zjKRmUmXeJtTK?j#Lizi)wn;SI6=g&Uen>S?g&YvqH05eNEkTZfCgxSwXGJq60SE<~ zp@LWRLC_q^CEgATjS8K7;WK&e?#XfRol13g_p6H9tTmcL9MggbJdwQTPG!!|kmEQ= zuUgM?N*~z%VQz6!OR)cL`HG3Lp`9t`&Tdp3J`%iKxTxs6J+!tg4m;90rtj8B2l43` zM`5Qr8OB`@0-Q$V!)l}6Bu)_Rg8CB)^l#c8_y{HpeKb?pH{PDo1Ynx{+VNdn0lR4K zo?NMqm(8dB2!GMGr}C6qHT>=VVb1^-V`BHOQ2o-?==jN%9B)7Qp9FrY1h_IT8+ZMx z5-c~)w z1Wi%-aLs3`AD%CDrxc2%-A#(G3&;z(%AGS~$ta`gZbHlU^Ip`39VXFcSq4ed7~;qv zy~?P2ByKaBIULQGif8g!7T?-=lTL?M>%3~eh_IV1l6)v?aFt?(^TPLvY4T+jMP#~{ zuxkS*r@0m0$uH?1fIu)+e{>&A%V22)0H*sDea~O2OB_x@S2!_fw_>%Ih4)zC)75af z>bW6zq%k@payTia=j!sZT&S=EKCaOtg!_qQ%Ld@iaE6PVdk{cFjmc9wDuJ?h7y&QV zJ|jFh+3Bs7R9n!cGiQJ75KU$(lil%l?92Zf3BKDlxDKb+m0J!)`_QxfkL*rnU2UJN zepbTY`k{#X`ySU=(PYW|WcoxL@FSbhWchZ(JGABBIC`r~m>6T@U=+ORXKa%?qCqey3CML>W5@14-%#30g10 zux_gZ15O9p37p;(D}_ch0Wv*20t}pVLRT9{w?a%6XkQBKiD}G9G8)aV&FqZYHh2fx zDP(l&`Qq)I8QvBZQ&of|;o;vM&V{?U-XskLvoQP5y>=PX^FUO^kbHIJ7FheGaU#Jn z+goqGy`fd+wg?`zv)5k2j+*e3Hme0jL2->Z>f zA6vS7b@s$=lsrFkm#u&8DxIdE_bOFX(r>9szKmOLSf>{HX?$wI(RX_uEqbM|rzx5g zR4KCN@=mkdHnRhoVJ}nT{Tpe!rkI8sE?Y11oNz-#=u^8>qw%3RV>Fo(Mk!(@m&S*t zm%rYZn_EI(XvmmoGwI{>i60)jnZMKwY;wx|8Z!)iH9~B)g z^IF1K!Od>sq4Y0%D{rrXW5t}_GuKZP8+gd2B9MGC>_DxL599cS>nU#LwgHor zF1wFjZ02zEmGCNki3!y{2UhME{k363qM?6R%e#;_R}-_oDvUX5WORQ>y2RVE+oj>3 z94w?wwyk?h*IDvIE5}e<@3{N}L)inne4}T_ubt_9$l4<7n3{3++H?`|~g3gCi7jZc0aG&DEG# zl=oOVWc$VDpU=r>TQn)Nu1sLiioXtQuM)PJDD2x*_0&;JVOg4R#?y2WL(40z17~!o z-Whssjs6T-{~ggQ_6P^PTL104#hmfth1(re6u<4az7m=fAlEkQ-~&~cX_E0%KUPHt zZLW_=KfUdSZnt5<@sFKBqF*en=L3U# z`*r+X{O`^bGwAS}>MEbtusC$~xus)X)xBBnacJ*%_6ptbb=XK!^EZQXl2_A)5Jx{B`B z-leXIiTeXgqvppnCuDEjxXUp0L#PuuC}{nmD}D=yr)SD2KA12xFUi_&s9%Wt4&B@W zZR+&^$78zdb5*@(>z|2zOV->Ny;}Xw^6O-Ac{kS_b-A(ado+p6Gx;j5`-j2RxQ00Ci>2^)=x`}TG)V`{0dRe&F;LEFPO%`(<;y>qQ6rKr~+T1GjW8q4P^c6a{tyL56 zuYHu_7N6ZT!2QXt6Q7;;s~>WHvUBUuz@W24ce8mzv=7w37<1}LQh81763Ymqflfm; z+Wnlwwl&L@9KW_C-*iWX0lnzZwbJ8j=eHT0&ydOXl0S4^t3mcklDJ#MyyoR6Ea!YF zIPSOZ^ZAav6Rn-p^8xSOhaXm& z;9o%R?pD1!CvKP5S(%^{ovsj*OpJnm+qE<^4l1--NFlqofS=YZ@PJ z*=$4!iTR!sM>&4gap^p%(|T39kH(CVDsi$>_%GZo_ zpS@ppqO?=;y~2X`ttV~PS)W*5(QMbTN>B4`?dF$RWjo~56}!G|R~Jsvy>YNEDM#vq zLFH`c8&6)7Bu~sXS)Deq)i^@os?jx(PTx7VqNjW{zF{wXLE?_|^v=-KQvgaaipQ7j_T2x$-Pzf855(X0pYP^d+p~SG>IdowI<={-I zHLs3$RIgjR)m;2(bgpJi^N2d~@uFOP`uzniiw5qFuU$4l|82P2j|NJJf@tR3BOWbT zDbj6~AMfZk3>h*|BzU3Yfq>Y^6#KyVXOgM2Wj0TJBcghLyR^!bNeOk%R;T6$6F$6& z^%r{q`>SKUlWS}{%=YUz_+&YjJ6jIE{8B1)9(AGG26gegnsS>QXQz$~o1M+F$Cv8+ zSOOU%P@Q+tvP)JWE<6zC(D|b=R!EvEbCuIFeY+rJ9>s$Nf(! zWXPHe+hx|wd`>sN|FY`oQ=3P+{`P0or${N!J&-caWx18bNMdwX#pF(L8FT8S$nIxi z3$>579q$}D<$5x~GAFxHH!E(!XtVvhbG~Oit4Ly=u}haZlAoZcx>zK#_T~2X6^UcQ zq)V&X;sP{>HfNl!Cwn~*G2ihiOXrZ&&Ov2fdrP+}T2t*m#4~cOsJZ*joE#YbGju*wZYn7(jNW=$@I5$+y9?GaM9`|Rstp61e}v=hyuCFX&{7M_y{Jr;7l#F>tCTY#2!?jCZF|(qq{9{aaGF8@HwP)-C#7% z^?L(M6rZSBoRkfs>}ij0dHTlU;-t7sQ{U|?m_y9}b#YEbY-?S4?w;Z$j@keafKv%bOe8|NH8gm?C-3*yMHRD(lwW5sAB+Lr!)V#Q&N~jc?vfy6*1sGlYBk~(t@5LXOoy9PYLc;xhLqdluOEkZNlG+tb*lJg?$%_gRMxVwOy-SN z`gqm&+V}~(oZdwYj6Ru8(Mt8))GeB(Ue;X^m@xXa^GxT_QoHE?y>mPsKfPgihWOZ5 zO7~)JZ1I*gzf~jWb1vxNk0Q&1Qdi3UxrI9QyJl+Tv!8l{TgZfS*?hPK8tGqX?Pj;R z(Mqbc+ih!xNa5AO>ur9L>Jt-0-<8`xn&x3tZIS1i#}H1go6thm-5IktNH$J;j7{?v z-veR$&v{NbzO( zXc0b>lBQR$l{fuM(}Ty9aFa@VjmaCT4IWG%7{1ML)tVyG;)0Q@?MjW;zfxx?=u6QMWUo-Yur|HJg)Y4lOI?*9h zESLN*mxQ|H_Mb}FQuE_MLAIgs>+_3zKi$9Sb@h?MAwp4Q?jlFc#6^=2CQNh6q+eKd ztjTiw`{-*DS4JC|m|TvOnC)Qxb^4E$Bjcs)VLxKJczlu2lbAgzv5$wC#HbY#rwzU} zD`WJ}Q|qdyh+VflFl6BU08h0NE8T!MPu2Ge&>Eq@^DrspQA<@LM`lXm7mJS>*B#C`biENdcdaHrwYdpn#Ch_^eI zxZa%k^=lZdU1j^7#i>)3vyT}0C*HO3Tx9$XPVQ`6;h$33`IgmVbKNhclX3-wky1Ki; zuOF|FmcFsRX>wZI!q*i;?2e^9^H}+`{roDwkHOnxog}_De_6Y_Y=(@2_@X8$c}>w5 z6oV!D`FD-pDV+G_8gA_xQAS7bwThw}2I`ul{NANlTR%R%bW6sUFNOx% zomv}D#!@p6Z_R$42%JKO3v0r?6~K&is87nNj>yYe!U9{qHZ8?x_8 z)YJp6&R%Wa~;Qm?_+x4G{i4h;{X2hszmB%<%%gCO_a z>dEyprJ^6YooJ6!x;UD;m|mU02$$CjY^|Mhbkw3zLYLUN9q;y~_($CM_|tFO+0@pdXZ>zJS-W<1 z%QvwM)tw`4{I})8zWm6gHl%s39WH*RqBHhoR7GFTKQn5ZP^*&Bx^i`5WUjA-$4Bq+ zdPT=Fb&Eb_X6NV23a!(h@tT}iywD_fhR5-HMc&r_qgD&kQy3}$Z_nPz))-Io{I)+s*J6(-}Z-BXSFmA`cI z`{{F9B0cUnXjvVcX>xS*Hs7ioEz^UvX>r}q;xL!JqU3DC##{yEuH6!e_qQnQQrT-C zA+n?4x~r^Yw%o}1f&1o^cQN;u4L<>&8kD8}*IJu@K>tSEnD%3}c}K#M&RpK<5n>7JcyVek1a9^`7KojdhME$ zql6bq#ttg|H2Gvm+WMzT=Vbk69Q)Qu{^_#Hv5MjOth~GF?cmt8#!?dNM8^(2w_V|} z#g?gV^RKsCyc(8RxJ8^gD8%ve)Q-fopo>96M;d9Wo;<+!o^2%o8x z_%kQeC0dx(`){e0KlQ$I(Se~nsyW+o#ETm51y%}*hbJAr=YMdyiJaATNAt|f3Nd+C zq)Jkz&nr%B4Q6cm@hzm(I<3CRuhgR;uVKp1yvG4qjgk%}^A31;i3a&kOnA6#s_^Wq z`|2WBZrcBRllFw2G>uVI6D7%+Ze~Gu=%nOw5N!!c&qxGdtxeabOo?`D+gewp_x zF3YL2>g?s+iQo7BbFZ?Svdz@}V>h?Jc%;DC=n6hAIxsMHj_1f%rVV-WdGwPItt>c!|)Yn-^IN$lv8Ox6QwuMc~9h8-8Wfs4^!W635(o%LbGvV`fak)?Z(RN zmjRCp9*W7YcKwq6b+e94eZ=vpBA<8b9ijMK7m|wpU>2VrCsAGfdU)QXuM-Z|Pdw;vrbWwBDDuK&30QiD#;&oUWOM?Zda=DOIKDJzI3*UNyc?=XE{jB5+ny1S}sQve{1wVMGoC;7Yy=BeD`y7 z$*lxpO~L%ZjOw8SXAP}8=j3LveCO=2J=>;rHK?Zz(?48g`7UqngrvhuB^ES^FvX64 z7r%2~#7MuwCf8pkXzy*APM4rcFFC6O;o>ymm+xBwQa;%gj_k5k{JLd9Zii-F{Z!Q^ zd3RaW6KARl*FG5dq1wY!ealzN`m(J{ZJ(_POWEjNH2L20rwHF z=`fBC%v`0@zA{GGI(DA#QxadhsiJoWEKoEg+|i!@$+W|IAxh>u6M3DKotCp2{Zx)NJ0S|=g-x! zwzqO%1;BbSYVocRNW2r^@F`(*4?X-c85aPOr3KEGb1dxobYk7K4D)->4V-0xEfNks zp>6nrLcr1@JPfRewGCUL6FOomiN}Dm8QyV78_dO#HuzF}v^2fL14DcR$$k`en1VX# zNlV}lq=7_H7!`>=Fhp-kI9)S<9vZ?9Q$4EKe)=>5q2kA%F9G*7h{-qyyfibAH7$aMTh?U(l05iLd_Yy?T6NJstqAo)@v$RQM>FK(Oa zy4z{VvIN5GH3WimZ;P<*`GoaB@*^(C1zD!Q_w-}vSmz@4<|rli0r4l11IfOmNL-|@ z`5zTWC=dvY9qhw5V>b>GosONRrqPL1Itjmjvj(4Rdj=7C6X*o0RvXUu0p#4S0qMKc z7sEl@NhSmWNUQf@GWOoWfq@o^$G0PaA*1N*kLdCM>Gu(sgF zP}Mt2>H|h25ve`_xHt<$KNPJ2jA|GX)FXCKINQ&!_2!sX@D2f_W(%hsiix%o!1zC2+vT zKGR8C50*SZn|*K!uW<(RC!szb2l(PH`9aB`KtsV$P$f!wz!~_DO0vOWV$7Zw<#h+( zq5w{@r^oQiWGr~X4epQ6X9L8fczNm*JrRiV#sDam!tHsO46jDcfWO`1KR1hmTe4Hh9d8&VJr7AQWQN znV<&PWhp%mlQHf!Kkz;7mkk;dvem)LZBL=)y3lgX9$ffkGOFA7f$nv~IM~``y6LF!x!|VWDG&(oSMS4Qh=1gW=}n{s`e^#l5a*1w@KJbsYr#+g;V8T?RrWqi zMoKqFG#^T6D1@yV-qb+<0J!6Zj4XOESQQDqh+hEN1@DwzP;8_U#f8~tbS$>RI^F~8 z6M`+Ja8Q#SvlCYWVmQ2$5Y#);yinQkFnzjH?R@hR;M|56^#TdXT=4$J>^KedHVj;H z3TEFY5QU-*TsN5`AS_bo6iNt96P-FSF$RZQeFy@j41*U{%9Yc&;6;!%!r-H$Gzgb* z8krf;P!F*HAsAlN;j8L!1;qpVc}soTMgU6c19nH3BbZ+xmP6~SHFn~YvX8|=c0E*x zpn!~Tx*kWcpylBtY7~y}PE^kSd=XG?z^edn&%^9IrcjeS&a z(>cyy5SD}bu{#FL&Fl{j$#sSx!xNT#sBIKq=7Q&sIR+XY;FQQ4ulyi?aoUnyfb*|& z1pE`Bax#z1%vrxaLi<9r*n^=@^&BC?VTDAc`B11N9GzLbs@k6ns!WF$9c9%v&WK^$ z%q`WzHNy|!FmppA>AJUEAlY>ga|hp3CfLt~s6NY^y}zs8afFH>dDFtf!YEWaTYs_S zls&UR(OsB8cm)&6bWp>dhshWqA;QT|V}Zr*%t}azPY4m*3{At2N(?3SSx8_E^|3i6 z7Qm1`h(OTqfrnovBY%h>3}gytj{oAAi#wqQiq?P9#yvDlhLt2w4j))P(upvetssVk zllrbCu}<96?l3qG#_$s2WZfS0-<{YrmM4KP37oMH5t0`g*sfSyDb3uab+HB}4pW6c zb;TYICS#KvPl5naNSFp@#_S71tm(V1ygPLTn*IdPReNB6H$8VUPh|XT0tOo>R%v(z z!mv2VOrb&KDK(z36u8B2MIb3cgH9#kYfUZ1?#vSiA`M}!jSBqQG@kG@5;c$*61Wls z?nj}9;)*5twMTsus7xH(s-}U6<%1q?0(w*ki55U2(b*jxCK@O4NQocNb|bjQa0UWB zX*N&j-xAlrCPZL>)+GA|!N+{*X+=WvR%F2~&VkKGBC|pLKs& z@cWXl>3Ts4SSCSk6797M%r;DjV$#8H+lF8yKA#bxo(%^f=w`tqTV z#3p$5hh;i0%Z*bJCR>4JJWK{?Ut)>)-&_bYms~G_t}N`+l?jRcA;EVrM=EuMK_Vt4V-ym%&^~}hyLHT?DJISi-evshXqolTnyD5yxK1h0_JuxZh2MNA8IvDJj zHwwgc0G1sn$-Ux!klYs%D-H==v2kwrXG;>awNe-*^7-0(e=Rt_!^?W=;qup(>&;k>PyP?wB`!E@n+Wp|*Iw~4F<~~mcU-rtGeN?^~@W*Ad&~70t zRd9Q9WY6>xFX)LY^aNc!*IDw1Zcm~|P^iJK^uWHAWNa;=IAzw5ELc4=z&!PPC54Uq zMtlAk!r;vmY&`=`e;_&R_>4vAT^1v69VZjJK zzxQD>Zh8pD!g0|!&0qe}W(X!Og94+&aebL!6ycD6#_L4KhVoHA0o_ShZqj8#qUE*xrJw)|%b?Q8~l5Co0+D z_f8Dlfv;@aQCkHzat~ZPnvW2Cvn~wVoCbr?e zR~s&+$Q3~;`HVK?KD5+N0N3UNYfIQ%)B7+P&Nuo=$$#8&ePhwvMa-V#YWvB+T~I zK#pHk2!weEn{LDZ#l|&03G#fr=x#Zde z<>3{xOmLp<(+FI1fD2ttcy<(3s9iIdy;yH2Fz^Rcp&@Co( z3pD`!uKuv`oe3@hFN@UGKHRipB|s9SwA< zEc!v=Md~{eCqYiDnJ)6Mnk1ZqpiQw?iLvQw_&kK>w~}GW3@xYuYQheoFcOs>NJ2Ks zV1P|Cj5H2GNvIGRmPnPH>LCg=CC_wUCT*WRDk| zqiI=@%w_1g9u$GTejzLggVhJJt%~m~Tjbk2ti&Yp0Yw;N6(6$tuqi?MW<6tcvqa5ls zf^+mORbd>gg=5r}C_>0DpseuCgSBjoc4N1z;1u|-gtcglcH@YLz#RA%$XYT+DLxYg zrocBq)`BrgVH+qg1-|{UmWxq}IEug&_~yr2EJi8X=>k*WTOVtw7^OJAR$vNz<6|up zqZDRI0#o4I9&4Ex%0;~olaafL2L)!BoRYejS+f?2(bg|Mzz;N!L@q|bS|UbQZPLdD zqTrmo#b{Uy#3)T_F+VhXBy2Gn*7C4 zY}V2+y5~rxUI1(^$ykhqwJ?mXvmU+{fQ3T>7Gq#73mYLaIsnJ{xlT^M4TS&EU@Z!x zqv0aX4-=bjt0o8sZYdZeU@Zls!yr9P5CXhHFow@s2u4?`o%;OXbIfaF;H+g}bd2XX z@CD8_myI#77J*TpaTj)G@U}HaV6fp@0>)TaOTcI$V752EEIf187!hj$7*(fh!Tb<$ z%~N9ptmR)ccObly9|A79X$+pV_>1PIC&uvu|1Sy7VJjI%!dm)86Z;F-2}Z(4yue!c zMQ7L#oBBz{J#t_z`=W%VX@U{*8AGrZeeICxYSZ?95%NEOv6g&M>y_Ru7&W)Jg0kcW zEy1~XPmrtyUQ0v`k5mZG$JbQJTHZw~u4eZHqvSbhvKDvI>A$T`Fe)C?Cu?aJUH?vL z;6cPsVb?=zhdFCu7mX*^zvi2QLs=Jd4yQVEqArHbTGU0OBePciu{o4< zF#^_-E*ieHj1=eYbNb!@zy8(ZkQ1{T$;IX$~Ybs*e`t@p^J6Sl*Yt z*h#Wjzb_cdy1`KCL?b&%bzZP+aS$H-aO>3x-B9KAfej9*YTnZ08~$J9gGXVOC^r6? z2!Y@Zn>J7iX(L_~II#{M#hHiBnNy*dJ`eUzprPYDQ$8v1qZ>T3ZAL9wV_?hnM`iY# zRH`j_k@dtgc(~^`-48E^4h(_~$mps{%aKo9mIwxq;mcg92`R(jt`-~iy%}~Md@}rv zS@6)Q4;uF^hC%2AZAIe;)ewQud!rOQmgrNyrlg3)#%mYm~ z28%_*lU+Hyps|~3h-6=y)*Npdol5kfoB#P2u$`vw6l1Wr*vc0)hzbV>P{ZAOjDI33 zl^Dg1{N!{JsPD!0z;Iuz;k2DZ_`*$@J{R2qAeZ?4{x3EY^BQ#Y$& zyY59suxr4Q+k7)w1(HI1xv>Q77_EU_6dMREs6v0J;+th2+$G@%=Y_Zt(IzFh)&P+; zgk$KY<)Vjt6aAemxbggQ{psceo}~=-+o3A`-!r~>u-T3qiS77R*Ze`_No26lY!q>1_t-q4=`m{Nw*NR$P3QaTRr5 zfNu$*DC$DBhVhRN3g-<6QSqi5+lY*Yb)e8E;6xNk&@e&}0=Qz-7ndL}XYq$z=)oH3 z0a_p`8q14-+N;4@w$85RM4te-g$ZZQ{3SV3C=@!82%SLZfz$E=aM+QsURaQ5KGeW4 zP8zlF>*CK2&=+BFGpOH;Ruq`an!JJ%%wc3!2=5#24<<4bVpUVbFqJg~AaeYD`zr*l zkx}y?2npf9*64Gp=?5sWiyFHKTXh9s`{#2k$^L<4{NXAayyvbb4A*lQE>wi(vjyOC zi3)>>hYUxp=^Q#0_}{bPH1O0^SVN$?_tio`qTfgH@N_SD-hhp3#-K0Cy#mHIV1yER0Gkux9y1(|aa5^T zQ4+k8ET}vxgbF_%kYPG7aDk!35M1EG^M>vW06q#X0?n@|L*fWGqy9%7erKM-{rpJO zzmup7XbCPwwanQYa14vEWH9^U_#1^Mg{4mb^p#p!1n_xB%E3 z+deb)$k%V6(T7H{x7}$aU-(>79^219-;JFa-verbwtYK80BnwJuW{G)zXkYLA=!Zr zZR$Fn_(){Lk;1g`#_Ojbo)HlDp&JE`ZQ_@LBY2hM#Ih-%mTI8zXnx}1R-WKJnGcTW zwQ>v6<)G=mKsp+Tk=kYPgB}^h5jxfL)@*AK{4p3(fK zViRT2aOpqe|HebA;{6VhkvS`<|`$IB7gaxwj4GKi}1Dp8g`WGql8nRHv{5us8SG@p9nj(C} zHlBQcKivemM_=DS?3xeO%aHIQ0M(IAhs41hhCuoO^&!a}{Ii+;iDbC0&DY$eCI>;k zzJM&z#LzqVj0q3N*xN6&C_0Je0LL&n<^aUUdD>#R8tg?Ke8E@#c`V7{q4)vyo`?R{xI-Tf{6KhE*F3~eT2*{ zn&!&?DKHu5?8zPGhnr;3fVH3j=(@ddz+k>L;IGsy$#iOzBP9^d<>JY2ougFiTHule zD?+E?wPFHuxxu$OIVWu{SB;B;D=!IBK!xU177yW{${JjH2nDw<>9sn+SWyxi8Z^~8 zQc3^PSr5V-as+1(!!GhC91gK5fFfWR6u>*%OVAKs6ia5JC? z_28*?K;;fE>TaAz^n=Qt!Xdk1F+6AaHyC{tFjUkXd=Y>LoAey1pv)Xn8}7BrO9Fre zP7$&#My#ay?+sVK8NaE>U;}s@aQkdCfsm}=6x@c!xIN&N1h^Y(n7vJ5czPZtqiI6F znK;hhN_jEOaIYUB6b1;L5A2lrXInsryB%T<8N3 zmM2(`X^+t0m&tgn!9NowC`_AqO$uQ@C&z^YlN9I#>UD-{@z3V!V#Q4rUY!^A$N*L7 z?z6~#IE^RCpNzd7Rv*AR#ldu9&Q#{wKp*m9Eo_cR?Xo^k3YL^Kd%BpT!ga0*4Gsc3 zy$rjC&|wOlB{<>VoH0i>+rX1rQJ{LQli0hbXD$fazX0`5DwiYS^7w8wBFHlo@=<8_ zBy9yDMDn>D*IaX=RvS6+YNB7P<;K zHnzK2q_gqCpu|A?{Z$s^OQ-BgqKt?ym;)9a91~58i>l17m9^CZ#j61&hwk=1OvVYW zklwHmg6no6X7KmINN~h>vb$SJ6jTTMup^rFRy&N}^FOWaRUl5l7B3kE!T|goyxi2@ z7Gvrina2(Iw=NN})D|A{+KozPi_}zae}v_M8SKMDGon}Xc_RD#z0DX8yb|=ZuIgc< z?+5~cMr6-M4=Cgb{#RS^urbX@)9CQ|4h?^!$}a2EH+f=*K=j86xTv@yp%P@B0PYqo zIDM<+2^>lc!^5ppziXyeKKrRR;AVoIp}11DJaOSmz<9WrAR6nI+e8EIHSjX1Zj?Oa zh8qeuH^VZEN+SAlP-ph++8CMM2SE`-adwp<62 Date: Sat, 4 Jan 2014 00:00:57 -0800 Subject: [PATCH 1170/1571] Assign spill threshold as a fraction of maximum memory Further, divide this threshold by the number of tasks running concurrently. Note that this does not guard against the following scenario: a new task quickly fills up its share of the memory before old tasks finish spilling their contents, in which case the total memory used by such maps may exceed what was specified. Currently, spark.shuffle.safetyFraction mitigates the effect of this. --- .../scala/org/apache/spark/SparkEnv.scala | 16 ++++ .../org/apache/spark/executor/Executor.scala | 2 + .../spark/util/collection/AppendOnlyMap.scala | 5 +- .../collection/ExternalAppendOnlyMap.scala | 74 +++++++++++++------ .../ExternalAppendOnlyMapSuite.scala | 17 +++-- 5 files changed, 81 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 634a94f0a7f06..224b5c1744b52 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -59,6 +59,9 @@ class SparkEnv private[spark] ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() + // Number of tasks currently running across all threads + @volatile private var _numRunningTasks = 0 + // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() @@ -86,6 +89,19 @@ class SparkEnv private[spark] ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + /** + * Return the number of tasks currently running across all threads + */ + def numRunningTasks: Int = _numRunningTasks + + def incrementNumRunningTasks() = synchronized { + _numRunningTasks += 1 + } + + def decrementNumRunningTasks() = synchronized { + _numRunningTasks -= 1 + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e51d274d33874..bd202affa2e19 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -186,6 +186,7 @@ private[spark] class Executor( var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime + env.incrementNumRunningTasks() try { SparkEnv.set(env) @@ -279,6 +280,7 @@ private[spark] class Executor( //System.exit(1) } } finally { + env.decrementNumRunningTasks() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index d2a9574a7121b..d8fa7ed9af2cb 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -17,8 +17,7 @@ package org.apache.spark.util.collection -import java.util -import java.util.Comparator +import java.util.{Arrays, Comparator} /** * A simple open hash table optimized for the append-only use case, where keys @@ -270,7 +269,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi cmp.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) } } - util.Arrays.sort(data, 0, newIndex, rawOrdering) + Arrays.sort(data, 0, newIndex, rawOrdering) new Iterator[(K, V)] { var i = 0 diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 68a23192c0655..c348168a8bf09 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -32,17 +32,28 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * * This map takes two passes over the data: - * (1) Values are merged into combiners, which are sorted and spilled to disk in as necessary. + * + * (1) Values are merged into combiners, which are sorted and spilled to disk as necessary * (2) Combiners are read from disk and merged together * - * Two parameters control the memory threshold: `spark.shuffle.buffer.mb` specifies the maximum - * size of the in-memory map before a spill, and `spark.shuffle.buffer.fraction` specifies an - * additional margin of safety. The second parameter is important for the following reason: + * The setting of the spill threshold faces the following trade-off: If the spill threshold is + * too high, the in-memory map may occupy more memory than is available, resulting in OOM. + * However, if the spill threshold is too low, we spill frequently and incur unnecessary disk + * writes. This may lead to a performance regression compared to the normal case of using the + * non-spilling AppendOnlyMap. + * + * A few parameters control the memory threshold: + * + * `spark.shuffle.memoryFraction` specifies the collective amount of memory used for storing + * these maps as a fraction of the executor's total memory. Since each concurrently running + * task maintains one map, the actual threshold for each map is this quantity divided by the + * number of running tasks. * - * If the spill threshold is set too high, the in-memory map may occupy more memory than is - * available, resulting in OOM. However, if the spill threshold is set too low, we spill - * frequently and incur unnecessary disk writes. This may lead to a performance regression - * compared to the normal case of using the non-spilling AppendOnlyMap. + * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of + * this threshold, in case map size estimation is not sufficiently accurate. + * + * `spark.shuffle.updateThresholdInterval` controls how frequently each thread checks on + * shared executor state to update its local memory threshold. */ private[spark] class ExternalAppendOnlyMap[K, V, C]( @@ -56,35 +67,54 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( import ExternalAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, C] - private val spilledMaps = new ArrayBuffer[DiskIterator] - + private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = new SparkConf() - private val memoryThresholdMB = { - // TODO: Turn this into a fraction of memory per reducer - val bufferSize = sparkConf.getLong("spark.shuffle.buffer.mb", 1024) - val bufferPercent = sparkConf.getDouble("spark.shuffle.buffer.fraction", 0.8) - bufferSize * bufferPercent + + // Collective memory threshold shared across all running tasks + private val maxMemoryThreshold = { + val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.75) + val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } + + // Maximum size for this map before a spill is triggered + private var spillThreshold = maxMemoryThreshold + + // How often to update spillThreshold + private val updateThresholdInterval = + sparkConf.getInt("spark.shuffle.updateThresholdInterval", 100) + private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() + private var insertCount = 0 private var spillCount = 0 - def insert(key: K, value: V): Unit = { + def insert(key: K, value: V) { + insertCount += 1 val update: (Boolean, C) => C = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } currentMap.changeValue(key, update) - if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { + if (insertCount % updateThresholdInterval == 1) { + updateSpillThreshold() + } + if (currentMap.estimateSize() > spillThreshold) { spill() } } - private def spill(): Unit = { + // TODO: differentiate ShuffleMapTask's from ResultTask's + private def updateSpillThreshold() { + val numRunningTasks = math.max(SparkEnv.get.numRunningTasks, 1) + spillThreshold = maxMemoryThreshold / numRunningTasks + } + + private def spill() { spillCount += 1 - logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") - logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") + logWarning("In-memory map exceeded %s MB! Spilling to disk (%d time%s so far)" + .format(spillThreshold / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) @@ -100,7 +130,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( writer.close() } currentMap = new SizeTrackingAppendOnlyMap[K, C] - spilledMaps.append(new DiskIterator(file)) + spilledMaps.append(new DiskMapIterator(file)) } override def iterator: Iterator[(K, C)] = { @@ -228,7 +258,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } // Iterate through (K, C) pairs in sorted order from an on-disk map - private class DiskIterator(file: File) extends Iterator[(K, C)] { + private class DiskMapIterator(file: File) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) val deserializeStream = ser.deserializeStream(bufferedStream) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 6c93b1f5a0c24..ef957bb0e5d17 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -5,15 +5,13 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.SparkContext._ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { override def beforeEach() { val conf = new SparkConf(false) conf.set("spark.shuffle.externalSorting", "true") - conf.set("spark.shuffle.buffer.mb", "1024") - conf.set("spark.shuffle.buffer.fraction", "0.8") sc = new SparkContext("local", "test", conf) } @@ -27,14 +25,14 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } test("simple insert") { - var map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) // Single insert map.insert(1, 10) var it = map.iterator assert(it.hasNext) - var kv = it.next() + val kv = it.next() assert(kv._1 == 1 && kv._2 == ArrayBuffer[Int](10)) assert(!it.hasNext) @@ -59,7 +57,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local map.insert(1, 100) map.insert(2, 200) map.insert(1, 1000) - var it = map.iterator + val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) assert(result == Set[(Int, Set[Int])]( @@ -177,8 +175,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } test("spilling") { - System.setProperty("spark.shuffle.buffer.mb", "1") - System.setProperty("spark.shuffle.buffer.fraction", "0.05") + // TODO: Figure out correct memory parameters to actually induce spilling + // System.setProperty("spark.shuffle.buffer.mb", "1") + // System.setProperty("spark.shuffle.buffer.fraction", "0.05") // reduceByKey - should spill exactly 6 times val rddA = sc.parallelize(0 until 10000).map(i => (i/2, i)) @@ -226,4 +225,6 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } } } + + // TODO: Test memory allocation for multiple concurrently running tasks } From 2db7884f6f1939d2a62fb71279a3ad80706308e1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 4 Jan 2014 01:20:09 -0800 Subject: [PATCH 1171/1571] Address Mark's comments --- .../main/scala/org/apache/spark/Aggregator.scala | 8 ++++---- .../spark/util/collection/AppendOnlyMap.scala | 15 +++++---------- .../util/collection/AppendOnlyMapSuite.scala | 8 ++++---- 3 files changed, 13 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index bb488f4ad82ae..292e32e7c8547 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -50,8 +50,8 @@ case class Aggregator[K, V, C] ( val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) while (iter.hasNext) { - val kv = iter.next() - combiners.insert(kv._1, kv._2) + val (k, v) = iter.next() + combiners.insert(k, v) } combiners.iterator } @@ -72,8 +72,8 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { - val kc = iter.next() - combiners.insert(kc._1, kc._2) + val (k, c) = iter.next() + combiners.insert(k, c) } combiners.iterator } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index d8fa7ed9af2cb..6faaa3197f1bc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -49,12 +49,13 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi // Triggered by destructiveSortedIterator; the underlying data array may no longer be used private var destroyed = false + private val destructionMessage = "Map state is invalid from destructive sorting!" private val LOAD_FACTOR = 0.7 /** Get the value for a given key */ def apply(key: K): V = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { return nullValue @@ -78,7 +79,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Set the value for a key */ def update(key: K, value: V): Unit = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -113,7 +114,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi * for key, if any, or null otherwise. Returns the newly updated value. */ def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -148,7 +149,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Iterator method from Iterable */ override def iterator: Iterator[(K, V)] = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) new Iterator[(K, V)] { var pos = -1 @@ -287,10 +288,4 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } } } - - private def checkValidityOrThrowException(): Unit = { - if (destroyed) { - throw new IllegalStateException("Map state is invalid from destructive sorting!") - } - } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index 71b936b0dfb13..f44442f1a5328 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -190,9 +190,9 @@ class AppendOnlyMapSuite extends FunSuite { } // All subsequent calls to apply, update, changeValue and iterator should throw exception - intercept[IllegalStateException] { map.apply("1") } - intercept[IllegalStateException] { map.update("1", "2013") } - intercept[IllegalStateException] { map.changeValue("1", (hadValue, oldValue) => "2014") } - intercept[IllegalStateException] { map.iterator } + intercept[AssertionError] { map.apply("1") } + intercept[AssertionError] { map.update("1", "2013") } + intercept[AssertionError] { map.changeValue("1", (hadValue, oldValue) => "2014") } + intercept[AssertionError] { map.iterator } } } From 3d4474330d9cd7d7c1b1e9fc1f8678bc6ee905e9 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 4 Jan 2014 08:39:00 -0800 Subject: [PATCH 1172/1571] Removed the exponential backoff for testing. --- .../scala/org/apache/spark/deploy/worker/DriverRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e8ae2d302b888..7485b89cf41b9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -168,7 +168,7 @@ private[spark] class DriverRunner( val exitCode = process.get.waitFor() if (supervise && exitCode != 0 && !killed) { - waitSeconds = waitSeconds * 2 // exponential back-off + waitSeconds = waitSeconds * 1 // exponential back-off logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.") (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed}) } From ad35c1a5f2bbc44c077ccf1adb41910dc7ef0029 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Sat, 4 Jan 2014 11:42:17 -0600 Subject: [PATCH 1173/1571] Fix handling of empty SPARK_EXAMPLES_JAR --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 98343e9532300..7073a999a99e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -67,7 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - set("spark.jars", jars.mkString(",")) + set("spark.jars", jars.filter(_ != null).mkString(",")) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ From 4de9c9554ca6464b806496dbffe0ba99c0ae6b45 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 4 Jan 2014 11:16:30 -0800 Subject: [PATCH 1174/1571] Use AtomicInteger for numRunningTasks --- .../scala/org/apache/spark/SparkEnv.scala | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 224b5c1744b52..b581c7b0747ce 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,9 @@ package org.apache.spark -import collection.mutable -import serializer.Serializer +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable import akka.actor._ import akka.remote.RemoteActorRefProvider @@ -60,7 +61,7 @@ class SparkEnv private[spark] ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // Number of tasks currently running across all threads - @volatile private var _numRunningTasks = 0 + private val _numRunningTasks = new AtomicInteger(0) // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). @@ -93,15 +94,9 @@ class SparkEnv private[spark] ( /** * Return the number of tasks currently running across all threads */ - def numRunningTasks: Int = _numRunningTasks - - def incrementNumRunningTasks() = synchronized { - _numRunningTasks += 1 - } - - def decrementNumRunningTasks() = synchronized { - _numRunningTasks -= 1 - } + def numRunningTasks: Int = _numRunningTasks.intValue() + def incrementNumRunningTasks(): Int = _numRunningTasks.incrementAndGet() + def decrementNumRunningTasks(): Int = _numRunningTasks.decrementAndGet() } object SparkEnv extends Logging { From 0b3efbcf6273cbc11c597c368c2c123403575dae Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 1 Jan 2014 17:27:59 -0800 Subject: [PATCH 1175/1571] Adding partition level mapEdges and mapTriplets. This is necessary to support computation with random number generation. --- .../org/apache/spark/graph/EdgeRDD.scala | 16 ++-- .../scala/org/apache/spark/graph/Graph.scala | 76 ++++++++++++++++--- .../spark/graph/impl/EdgePartition.scala | 19 +++++ .../apache/spark/graph/impl/GraphImpl.scala | 49 +++++++----- 4 files changed, 125 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 9aa76c93945f9..3dda5c7c604e5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -42,32 +42,32 @@ class EdgeRDD[@specialized ED: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassManifest](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() - Iterator(Tuple2(pid, f(ep))) + Iterator(Tuple2(pid, f(pid, ep))) }, preservesPartitioning = true)) } def zipEdgePartitions[T: ClassManifest, U: ClassManifest] (other: RDD[T]) - (f: (EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { + (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => - val (_, edgePartition) = ePartIter.next() - f(edgePartition, otherIter) + val (pid, edgePartition) = ePartIter.next() + f(pid, edgePartition, otherIter) } } def zipEdgePartitions[ED2: ClassManifest, ED3: ClassManifest] (other: EdgeRDD[ED2]) - (f: (EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { + (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { (thisIter, otherIter) => val (pid, thisEPart) = thisIter.next() val (_, otherEPart) = otherIter.next() - Iterator(Tuple2(pid, f(thisEPart, otherEPart))) + Iterator(Tuple2(pid, f(pid, thisEPart, otherEPart))) }) } @@ -76,7 +76,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Manifest = classManifest[ED2] val ed3Manifest = classManifest[ED3] - zipEdgePartitions(other) { (thisEPart, otherEPart) => + zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index e8fa8e611c9cb..b725b2a15584b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -72,9 +72,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ val triplets: RDD[EdgeTriplet[VD, ED]] + /** + * Cache the vertices and edges associated with this graph. + * + * @param newLevel the level at which to cache the graph. - - def persist(newLevel: StorageLevel): Graph[VD, ED] + * @return A reference to this graph for convenience. + * + */ + def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] /** * Return a graph that is cached when first created. This is used to @@ -120,7 +126,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] /** - * Construct a new graph where each the value of each edge is + * Construct a new graph where the value of each edge is * transformed by the map operation. This function is not passed * the vertex value for the vertices adjacent to the edge. If * vertex values are desired use the mapTriplets function. @@ -137,18 +143,44 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * attributes. * */ - def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] + def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] = { + mapEdges((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapEdges[ED2: ClassManifest]( + map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): + Graph[VD, ED2] /** - * Construct a new graph where each the value of each edge is + * Construct a new graph where the value of each edge is * transformed by the map operation. This function passes vertex * values for the adjacent vertices to the map function. If * adjacent vertex values are not required, consider using the * mapEdges function instead. * - * @note This graph is not changed and that the new graph has the - * same structure. As a consequence the underlying index structures - * can be reused. + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. * * @param map the function from an edge object to a new edge value. * @@ -163,7 +195,33 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + mapTriplets((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapTriplets[ED2: ClassManifest]( + map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): + Graph[VD, ED2] /** * Construct a new graph with all the edges reversed. If this graph diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index e97522feaeaf2..bd03f14903638 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -56,6 +56,25 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(srcIds, dstIds, newData, index) } + /** + * Construct a new edge partition by using the edge attributes + * contained in the iterator. + * + * @note The input iterator should return edge attributes in the + * order of the edges returned by `EdgePartition.iterator` and + * should return attributes equal to the number of edges. + * + * @param f a function from an edge to a new attribute + * @tparam ED2 the type of the new attribute + * @return a new edge partition with the result of the function `f` + * applied to each edge + */ + def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { + val newData = iter.toArray + assert(newData.size == data.size) + new EdgePartition(srcIds, dstIds, newData, index) + } + /** * Apply the function f to all edges in this partition. * diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 16d73820f07df..79c11c780a69a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -47,8 +47,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePart, vPartIter) => + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) } @@ -149,8 +148,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println(visited) } // end of printLineage - override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), routingTable, replicatedVertexView) + override def reverse: Graph[VD, ED] = { + val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) + new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + } override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { @@ -167,25 +168,36 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), routingTable, replicatedVertexView) + override def mapEdges[ED2: ClassManifest]( + f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) + } - override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + override def mapTriplets[ED2: ClassManifest]( + f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] val newEdgePartitions = - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (edgePartition, vPartIter) => - val (pid, vPart) = vPartIter.next() + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { + (ePid, edgePartition, vTableReplicatedIter) => + val (vPid, vPart) = vTableReplicatedIter.next() + assert(!vTableReplicatedIter.hasNext) + assert(ePid == vPid) val et = new EdgeTriplet[VD, ED] - val newEdgePartition = edgePartition.map { e => + val inputIterator = edgePartition.iterator.map { e => et.set(e) et.srcAttr = vPart(e.srcId) et.dstAttr = vPart(e.dstId) - f(et) + et } - Iterator((pid, newEdgePartition)) - } + // Apply the user function to the vertex partition + val outputIter = f(ePid, inputIterator) + // Consume the iterator to update the edge attributes + val newEdgePartition = edgePartition.map(outputIter) + Iterator((ePid, newEdgePartition)) + } new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) } @@ -224,8 +236,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) - val newEdges = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newEdges, routingTable, replicatedVertexView) + val newETable = edges.mapEdgePartitions((pid, part) => part.groupEdges(merge)) + new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -253,9 +265,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vPartIter) => - val (_, vPart) = vPartIter.next() - + val preAgg = edges.zipEdgePartitions(vs) { (ePid, edgePartition, vPartIter) => + val (vPid, vPart) = vPartIter.next() + assert(!vPartIter.hasNext) + assert(ePid == vPid) // Choose scan method val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat val edgeIter = activeDirectionOpt match { From 6592be2594c50ac4a0018b59111ae23de89601b9 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 2 Jan 2014 00:00:24 -0800 Subject: [PATCH 1176/1571] slightly more efficient map operation --- .../org/apache/spark/graph/impl/EdgePartition.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index bd03f14903638..4fcf08efce382 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -70,8 +70,13 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * applied to each edge */ def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { - val newData = iter.toArray - assert(newData.size == data.size) + val newData = new Array[ED2](data.size) + var i = 0 + while (iter.hasNext) { + newData(i) = iter.next() + i += 1 + } + assert(newData.size == i) new EdgePartition(srcIds, dstIds, newData, index) } From 8d0c2f7399ebf7a38346a60cf84d7020c0b1dba1 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Sat, 4 Jan 2014 16:23:17 -0800 Subject: [PATCH 1177/1571] Added python binding for bulk recommendation --- .../mllib/api/python/PythonMLLibAPI.scala | 18 ++++++++++++++++++ .../MatrixFactorizationModel.scala | 10 +++++++++- python/pyspark/mllib/_common.py | 10 ++++++++++ python/pyspark/mllib/recommendation.py | 10 +++++++++- 4 files changed, 46 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 8247c1ebc5d2b..be2628fac5817 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -206,6 +206,24 @@ class PythonMLLibAPI extends Serializable { return new Rating(user, product, rating) } + private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { + val bb = ByteBuffer.wrap(tupleBytes) + bb.order(ByteOrder.nativeOrder()) + val v1 = bb.getInt() + val v2 = bb.getInt() + (v1, v2) + } + + private[spark] def serializeRating(rate: Rating): Array[Byte] = { + val bytes = new Array[Byte](24) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putDouble(rate.user.toDouble) + bb.putDouble(rate.product.toDouble) + bb.putDouble(rate.rating) + bytes + } + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 8caecf0fa1ba8..2c3e82830033e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -19,9 +19,11 @@ package org.apache.spark.mllib.recommendation import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.api.python.PythonMLLibAPI import org.jblas._ -import java.nio.{ByteOrder, ByteBuffer} +import org.apache.spark.api.java.JavaRDD + /** * Model representing the result of matrix factorization. @@ -65,6 +67,12 @@ class MatrixFactorizationModel( } } + def predictJavaRDD(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { + val pythonAPI = new PythonMLLibAPI() + val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) + predict(usersProducts).map(rate => pythonAPI.serializeRating(rate)) + } + // TODO: Figure out what other good bulk prediction methods would look like. // Probably want a way to get the top users for a product or vice-versa. } diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index e74ba0fabc09c..c818fc4d97180 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -213,6 +213,16 @@ def _serialize_rating(r): intpart[0], intpart[1], doublepart[0] = r return ba +def _deserialize_rating(ba): + ar = ndarray(shape=(3, ), buffer=ba, dtype="float64", order='C') + return ar.copy() + +def _serialize_tuple(t): + ba = bytearray(8) + intpart = ndarray(shape=[2], buffer=ba, dtype=int32) + intpart[0], intpart[1] = t + return ba + def _test(): import doctest globs = globals().copy() diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 14d06cba2137f..c81b482a87ef7 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -20,7 +20,10 @@ _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ _serialize_double_matrix, _deserialize_double_matrix, \ _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper + _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ + _serialize_tuple, _deserialize_rating +from pyspark.serializers import BatchedSerializer +from pyspark.rdd import RDD class MatrixFactorizationModel(object): """A matrix factorisation model trained by regularized alternating @@ -45,6 +48,11 @@ def __del__(self): def predict(self, user, product): return self._java_model.predict(user, product) + def predictAll(self, usersProducts): + usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple) + return RDD(self._java_model.predictJavaRDD(usersProductsJRDD._jrdd), + self._context, BatchedSerializer(_deserialize_rating, self._context._batchSize)) + class ALS(object): @classmethod def train(cls, sc, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): From 9e9a913c2ffe0b01f051c18018740934a6a8691e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 20:08:35 -0800 Subject: [PATCH 1178/1571] Add a script to download sbt if not present on the system --- .gitignore | 1 + project/build.properties | 2 ++ sbt | 48 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 51 insertions(+) create mode 100755 sbt diff --git a/.gitignore b/.gitignore index 399362f7d3e46..1692bde20fb2a 100644 --- a/.gitignore +++ b/.gitignore @@ -4,6 +4,7 @@ *.iml *.iws .idea/ +.sbtlib/*.jar .settings .cache /build/ diff --git a/project/build.properties b/project/build.properties index 96472771622d8..059edac7d4331 100644 --- a/project/build.properties +++ b/project/build.properties @@ -15,4 +15,6 @@ # limitations under the License. # +# Note: If you change the sbt version please also change the sbt download +# script in the root directory of the project called sbt. sbt.version=0.12.4 diff --git a/sbt b/sbt new file mode 100755 index 0000000000000..e04a20c48a964 --- /dev/null +++ b/sbt @@ -0,0 +1,48 @@ +#!/bin/bash +# This script launches sbt for this project. If present it uses the system +# version of sbt. If there is no system version of sbt it attempts to download +# sbt locally. +SBT_VERSION=0.12.4 +URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar +URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar +JAR=.sbtlib/sbt-launch-${SBT_VERSION}.jar + +printf "Checking for system sbt [" +if hash sbt 2>/dev/null; then + printf "FOUND]\n" + # Use System SBT + sbt +else + printf "NOT FOUND]\n" + # Download sbt or use allready downloaded + if [ ! -d .sbtlib ]; then + mkdir .sbtlib + fi + if [ ! -f ${JAR} ]; then + # Download + printf "Attempting to fetch sbt\n" + if hash curl 2>/dev/null; then + curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} + elif hash wget 2>/dev/null; then + wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} + else + printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" + exit + fi + fi + if [ ! -f ${JAR} ]; then + # We failed to download + printf "Our attempt to download sbt locally to {$JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit + fi + printf "Launching sbt from .sbtlib\n" + java \ + -Duser.timezone=UTC \ + -Djava.awt.headless=true \ + -Dfile.encoding=UTF-8 \ + -XX:MaxPermSize=256m \ + -Xmx1g \ + -noverify \ + -jar ${JAR} \ + "$@" +fi From 97123be1d7d1b68ec0cda09fd5894fc4af5f82c5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 20:16:56 -0800 Subject: [PATCH 1179/1571] Pass commands down to system sbt as well --- sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt b/sbt index e04a20c48a964..8cff6df69f0e1 100755 --- a/sbt +++ b/sbt @@ -11,7 +11,7 @@ printf "Checking for system sbt [" if hash sbt 2>/dev/null; then printf "FOUND]\n" # Use System SBT - sbt + sbt $@ else printf "NOT FOUND]\n" # Download sbt or use allready downloaded From b4a1ffc6c2634118bb1d07216221b862c32d6397 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 20:17:30 -0800 Subject: [PATCH 1180/1571] Switch from sbt to ./sbt in the README file --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 6daa4633ae576..db1e2c4c0aa92 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ This README file only contains basic setup instructions. Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), which can be obtained [here](http://www.scala-sbt.org). To build Spark and its example programs, run: - sbt assembly + ./sbt assembly Once you've built Spark, the easiest way to start using it is the shell: @@ -41,7 +41,7 @@ locally with one thread, or "local[N]" to run locally with N threads. Testing first requires [Building](#Building) Spark. Once Spark is built, tests can be run using: -`sbt test` +`./sbt test` ## A Note About Hadoop Versions From d2a5c75a4d3ca5123fb77ad24beb1b4474b36a4f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 21:44:04 -0800 Subject: [PATCH 1181/1571] Spelling --- sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt b/sbt index 8cff6df69f0e1..d21806ed83ae0 100755 --- a/sbt +++ b/sbt @@ -14,7 +14,7 @@ if hash sbt 2>/dev/null; then sbt $@ else printf "NOT FOUND]\n" - # Download sbt or use allready downloaded + # Download sbt or use already downloaded if [ ! -d .sbtlib ]; then mkdir .sbtlib fi From 0d6700eb5a901caad511bed3d21e43f16adcf192 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 21:44:26 -0800 Subject: [PATCH 1182/1571] Make sbt in the sbt directory --- sbt => sbt/sbt | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sbt => sbt/sbt (100%) diff --git a/sbt b/sbt/sbt similarity index 100% rename from sbt rename to sbt/sbt From d7d95a099f5f215aff465f948446a8b68113b391 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 21:45:22 -0800 Subject: [PATCH 1183/1571] And update docs to match --- .gitignore | 2 +- README.md | 4 ++-- sbt/sbt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.gitignore b/.gitignore index 1692bde20fb2a..39635d7eefbe7 100644 --- a/.gitignore +++ b/.gitignore @@ -4,7 +4,7 @@ *.iml *.iws .idea/ -.sbtlib/*.jar +sbt/*.jar .settings .cache /build/ diff --git a/README.md b/README.md index db1e2c4c0aa92..2c08a4ac638c3 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ This README file only contains basic setup instructions. Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), which can be obtained [here](http://www.scala-sbt.org). To build Spark and its example programs, run: - ./sbt assembly + ./sbt/sbt assembly Once you've built Spark, the easiest way to start using it is the shell: @@ -41,7 +41,7 @@ locally with one thread, or "local[N]" to run locally with N threads. Testing first requires [Building](#Building) Spark. Once Spark is built, tests can be run using: -`./sbt test` +`./sbt/sbt test` ## A Note About Hadoop Versions diff --git a/sbt/sbt b/sbt/sbt index d21806ed83ae0..a7146e3b052e8 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,7 +5,7 @@ SBT_VERSION=0.12.4 URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar -JAR=.sbtlib/sbt-launch-${SBT_VERSION}.jar +JAR=sbt/sbt-launch-${SBT_VERSION}.jar printf "Checking for system sbt [" if hash sbt 2>/dev/null; then From df92f1c0254dc9073c18bc7b76f8b9523ecd7cec Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 4 Jan 2014 21:48:35 -0800 Subject: [PATCH 1184/1571] reindent --- sbt/sbt | 62 ++++++++++++++++++++++++++++----------------------------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index a7146e3b052e8..6d2caca120ca9 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -9,40 +9,40 @@ JAR=sbt/sbt-launch-${SBT_VERSION}.jar printf "Checking for system sbt [" if hash sbt 2>/dev/null; then - printf "FOUND]\n" - # Use System SBT - sbt $@ + printf "FOUND]\n" + # Use System SBT + sbt $@ else - printf "NOT FOUND]\n" - # Download sbt or use already downloaded - if [ ! -d .sbtlib ]; then - mkdir .sbtlib - fi - if [ ! -f ${JAR} ]; then - # Download - printf "Attempting to fetch sbt\n" - if hash curl 2>/dev/null; then + printf "NOT FOUND]\n" + # Download sbt or use already downloaded + if [ ! -d .sbtlib ]; then + mkdir .sbtlib + fi + if [ ! -f ${JAR} ]; then + # Download + printf "Attempting to fetch sbt\n" + if hash curl 2>/dev/null; then curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} - elif hash wget 2>/dev/null; then + elif hash wget 2>/dev/null; then wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} - else + else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit - fi - fi - if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download sbt locally to {$JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" - exit - fi - printf "Launching sbt from .sbtlib\n" - java \ - -Duser.timezone=UTC \ - -Djava.awt.headless=true \ - -Dfile.encoding=UTF-8 \ - -XX:MaxPermSize=256m \ - -Xmx1g \ - -noverify \ - -jar ${JAR} \ - "$@" + fi + fi + if [ ! -f ${JAR} ]; then + # We failed to download + printf "Our attempt to download sbt locally to {$JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit + fi + printf "Launching sbt from .sbtlib\n" + java \ + -Duser.timezone=UTC \ + -Djava.awt.headless=true \ + -Dfile.encoding=UTF-8 \ + -XX:MaxPermSize=256m \ + -Xmx1g \ + -noverify \ + -jar ${JAR} \ + "$@" fi From 79f52809c836d08023aa5ca99a467d3a311a7359 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 11:43:00 -0800 Subject: [PATCH 1185/1571] Removing SPARK_EXAMPLES_JAR in the code --- .../spark/api/java/JavaSparkContext.scala | 6 +++++ .../org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../org/apache/spark/examples/JavaKMeans.java | 2 +- .../apache/spark/examples/JavaLogQuery.java | 2 +- .../apache/spark/examples/JavaPageRank.java | 3 ++- .../apache/spark/examples/JavaSparkPi.java | 2 +- .../org/apache/spark/examples/JavaTC.java | 2 +- .../apache/spark/examples/JavaWordCount.java | 2 +- .../apache/spark/mllib/examples/JavaALS.java | 2 +- .../spark/mllib/examples/JavaKMeans.java | 2 +- .../apache/spark/mllib/examples/JavaLR.java | 2 +- .../examples/JavaFlumeEventCount.java | 3 ++- .../examples/JavaKafkaWordCount.java | 3 ++- .../examples/JavaNetworkWordCount.java | 3 ++- .../streaming/examples/JavaQueueStream.java | 2 +- .../apache/spark/examples/BroadcastTest.scala | 2 +- .../examples/ExceptionHandlingTest.scala | 2 +- .../apache/spark/examples/GroupByTest.scala | 2 +- .../org/apache/spark/examples/HBaseTest.scala | 2 +- .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/examples/LogQuery.scala | 2 +- .../spark/examples/MultiBroadcastTest.scala | 2 +- .../examples/SimpleSkewedGroupByTest.scala | 2 +- .../spark/examples/SkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SparkALS.scala | 2 +- .../apache/spark/examples/SparkHdfsLR.scala | 2 +- .../apache/spark/examples/SparkKMeans.scala | 2 +- .../org/apache/spark/examples/SparkLR.scala | 2 +- .../apache/spark/examples/SparkPageRank.scala | 2 +- .../org/apache/spark/examples/SparkPi.scala | 2 +- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../streaming/examples/ActorWordCount.scala | 2 +- .../streaming/examples/FlumeEventCount.scala | 2 +- .../streaming/examples/HdfsWordCount.scala | 2 +- .../streaming/examples/KafkaWordCount.scala | 2 +- .../streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/NetworkWordCount.scala | 2 +- .../streaming/examples/QueueStream.scala | 2 +- .../streaming/examples/RawNetworkGrep.scala | 2 +- .../examples/StatefulNetworkWordCount.scala | 2 +- .../examples/TwitterAlgebirdCMS.scala | 2 +- .../examples/TwitterAlgebirdHLL.scala | 2 +- .../examples/TwitterPopularTags.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../spark/streaming/StreamingContext.scala | 6 +++++ .../api/java/JavaStreamingContext.scala | 25 +++++++++++-------- 47 files changed, 75 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 5be5317f40e7e..e93b10fd7eecb 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -431,4 +431,10 @@ object JavaSparkContext { implicit def fromSparkContext(sc: SparkContext): JavaSparkContext = new JavaSparkContext(sc) implicit def toSparkContext(jsc: JavaSparkContext): SparkContext = jsc.sc + + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c5df..12f3355bc4853 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -106,7 +106,7 @@ public static void main(String[] args) { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[2]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eaefd6..63465a3bbf33f 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -74,7 +74,7 @@ public static void main(String[] args) throws Exception { System.exit(1); } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); String path = args[1]; int K = Integer.parseInt(args[2]); double convergeDist = Double.parseDouble(args[3]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 407cd7ccfaee0..74e4d9291a57c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -104,7 +104,7 @@ public static void main(String[] args) throws Exception { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class)); JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 89aed8f279654..f774f6a04e405 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,6 +17,7 @@ package org.apache.spark.examples; +import org.apache.spark.SparkContext; import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -53,7 +54,7 @@ public static void main(String[] args) throws Exception { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class)); // Loads in input file. It should be in format of: // URL neighbor URL diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5af5..5558ab7c030e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -36,7 +36,7 @@ public static void main(String[] args) throws Exception { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class)); int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b776d1..99e6ba347cb5a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -64,7 +64,7 @@ public static void main(String[] args) { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class)); Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index bd6383e13df7a..8a071caf13f5d 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -36,7 +36,7 @@ public static void main(String[] args) throws Exception { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class)); JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 45a0d237da314..5e1a77baaa2e8 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -68,7 +68,7 @@ public static void main(String[] args) { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD ratings = lines.map(new ParseRating()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9fee..1f12f518a0fde 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -62,7 +62,7 @@ public static void main(String[] args) { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6ba6..593e4df111637 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -59,7 +59,7 @@ public static void main(String[] args) { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); double stepSize = Double.parseDouble(args[2]); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f39c..64ac72474bf39 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -50,7 +50,8 @@ public static void main(String[] args) { Duration batchInterval = new Duration(2000); JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); JavaDStream flumeStream = sc.flumeStream("localhost", port); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 75b588e4b85a9..0a56e7abdf49f 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -54,7 +54,8 @@ public static void main(String[] args) { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", - new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(2000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); int numThreads = Integer.parseInt(args[4]); Map topicMap = new HashMap(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199be57..ec6f6a8c5639f 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -48,7 +48,8 @@ public static void main(String[] args) { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1bba..4b9fd52713d11 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -40,7 +40,7 @@ public static void main(String[] args) throws InterruptedException { // Create the context JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class)); // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index a1199809923bc..0097dade190f6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -33,7 +33,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 92eb96bd8e0c2..b3eb611dd228f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 42c2e0e8e19c4..39752fdd0eec4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -34,7 +34,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index efe2e93b0dc91..65d67356be2f6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormat object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index d6a88d3032c49..c3597d94a224e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 17ff3ce76497f..bddb54b39cdd5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -45,7 +45,7 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index e1afc29f9aca9..4aef04fc060b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 37ddfb5db7635..73b0e216cac98 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 9c954b2b5baa9..31c6d108f34ae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -34,7 +34,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 814944ba1c6bf..30c86d83e688c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 86dd9ca1b3e58..ff72532db1f17 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -54,7 +54,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)))) val lines = sc.textFile(inputPath) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index bc2db39c1231b..8c99025eaa6da 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -55,7 +55,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 9ed9fe4d761d5..c54a55bdb4a11 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -49,7 +49,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index a508c0df577cd..d203f4d20e15f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -38,7 +38,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index a689e5a360b6b..e5a09ecec006f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 8543ce0e3285e..24e8afa26bc5f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -46,7 +46,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 3641517934b74..546495357f677 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -151,7 +151,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 9f6e163454a64..5ef19282949fe 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -48,7 +48,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a flume stream val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 61be1ce4b1726..1486d77d8a807 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -40,7 +40,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 8dc8a3531a014..172091be2eed9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -48,7 +48,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ea138f55e872b..2d02ef77c03e1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -96,7 +96,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + StreamingContext.jarOfClass(this.getClass)) val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index ce8df8c5023ee..74d76ec26c268 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -41,7 +41,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index fad512eebad12..9d640e716bca9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -33,7 +33,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index 0b45c30d20dc3..c0706d0724982 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -49,7 +49,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index e55d71edfcdc9..f43c8ab61d9d2 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -49,7 +49,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 35b6329ab3152..9d21d3178f346 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -60,7 +60,7 @@ object TwitterAlgebirdCMS { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 8bfde2a8297c1..5111e6f62ad3d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -49,7 +49,7 @@ object TwitterAlgebirdHLL { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 27aa6b14bf221..7a3df687b7e37 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -38,7 +38,7 @@ object TwitterPopularTags { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 2948aa7cc4944..89d304212324e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -78,7 +78,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 968b578487a5e..0569846f189a7 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -42,7 +42,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 304986f187936..b3a7cf08b92ee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -595,6 +595,12 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls) + protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second batch intervals. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b79173c6aa9db..7dec4b3ad794b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,29 +17,27 @@ package org.apache.spark.streaming.api.java -import java.lang.{Integer => JInt} import java.io.InputStream -import java.util.{Map => JMap, List => JList} +import java.lang.{Integer => JInt} +import java.util.{List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import akka.actor.{Props, SupervisorStrategy} +import akka.util.ByteString +import akka.zeromq.Subscribe import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.zeromq.Subscribe -import akka.util.ByteString - import twitter4j.auth.Authorization +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.SparkConf import org.apache.spark.streaming.scheduler.StreamingListener /** @@ -716,5 +714,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Sstops the execution of the streams. */ def stop() = ssc.stop() +} +object JavaStreamingContext { + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } From aaaa6731845495743aff4cc9bd64a54b9aa36c27 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 13:57:42 -0800 Subject: [PATCH 1186/1571] Quite akka when remote lifecycle logging is disabled. I noticed when connecting to a standalone cluster Spark gives a bunch of Akka ERROR logs that make it seem like something is failing. This patch does two things: 1. Akka dead letter logging is turned on/off according to the existing lifecycle spark property. 2. We explicitly silence akka's EndpointWriter log in log4j. This is necessary because for some reason that log doesn't pick up on the lifecycle logging settings. After a few hours of debugging this was the only solution I found that worked. --- .../scala/org/apache/spark/util/AkkaUtils.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 362cea5e3e462..5729334e248e1 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -21,6 +21,8 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory +import org.apache.log4j.{Level, Logger} + import org.apache.spark.SparkConf /** @@ -47,8 +49,13 @@ private[spark] object AkkaUtils { val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt - val lifecycleEvents = - if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean + val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" + if (!akkaLogLifecycleEvents) { + Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) + } + + val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off" val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = @@ -73,7 +80,10 @@ private[spark] object AkkaUtils { |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize + |akka.log-config-on-start = $logAkkaConfig |akka.remote.log-remote-lifecycle-events = $lifecycleEvents + |akka.log-dead-letters = $lifecycleEvents + |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin) val actorSystem = if (indestructible) { From 94fdcda89638498f127abf3bb5231064182b4945 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 15:10:05 -0800 Subject: [PATCH 1187/1571] Provide logging when attempts to connect to the master fail. Without these it's a bit less clear what's going on for the user. One thing I realize when doing this is that akka itself actually retries the initial association. So the retry we currently have is redundant with akka's. --- .../org/apache/spark/deploy/client/Client.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 9bbd635ab9bb0..481026eaa2106 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -24,7 +24,8 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} + import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ @@ -110,6 +111,12 @@ private[spark] class Client( } } + private def isPossibleMaster(remoteUrl: Address) = { + masterUrls.map(s => Master.toAkkaUrl(s)) + .map(u => AddressFromURIString(u).hostPort) + .contains(remoteUrl.hostPort) + } + override def receive = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ @@ -145,6 +152,9 @@ private[spark] class Client( logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() + case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) => + logWarning(s"Could not connect to $address: $cause") + case StopClient => markDead() sender ! true From 63f906322d173c0e5e74c815d638db1bea338340 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jan 2014 15:52:43 -0800 Subject: [PATCH 1188/1571] Fall back to zero-arg constructor for Serializer initialization if there is no constructor that accepts SparkConf. This maintains backward compatibility with older serializers implemented by users. --- .../org/apache/spark/serializer/Serializer.scala | 3 +++ .../spark/serializer/SerializerManager.scala | 15 +++++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 160cca4d6c54a..9a5e3cb77e1d5 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -29,6 +29,9 @@ import org.apache.spark.util.{NextIterator, ByteBufferInputStream} * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are * guaranteed to only be called from one thread at a time. + * + * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a + * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence. */ trait Serializer { def newInstance(): SerializerInstance diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 22465272f3722..36a37af4f821d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkConf * creating a new one. */ private[spark] class SerializerManager { + // TODO: Consider moving this into SparkConf itself to remove the global singleton. private val serializers = new ConcurrentHashMap[String, Serializer] private var _default: Serializer = _ @@ -53,8 +54,18 @@ private[spark] class SerializerManager { if (serializer == null) { val clsLoader = Thread.currentThread.getContextClassLoader val cls = Class.forName(clsName, true, clsLoader) - val constructor = cls.getConstructor(classOf[SparkConf]) - serializer = constructor.newInstance(conf).asInstanceOf[Serializer] + + // First try with the constructor that takes SparkConf. If we can't find one, + // use a no-arg constructor instead. + try { + val constructor = cls.getConstructor(classOf[SparkConf]) + serializer = constructor.newInstance(conf).asInstanceOf[Serializer] + } catch { + case _: NoSuchMethodException => + val constructor = cls.getConstructor() + serializer = constructor.newInstance().asInstanceOf[Serializer] + } + serializers.put(clsName, serializer) } serializer From a4048ff31e6f8d3e1451d8ae2d5b9edee42cfbbe Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 09:18:17 +0800 Subject: [PATCH 1189/1571] Get rid of `Either[ActorRef, ActorSelection]' Although we can send messages via an ActorSelection, it would be better to identify the actor and obtain an ActorRef first, so that we can get informed earlier if the remote actor doesn't exist, and get rid of the annoying Either wrapper. --- .../org/apache/spark/MapOutputTracker.scala | 14 ++------------ .../main/scala/org/apache/spark/SparkEnv.scala | 16 ++++++++-------- .../spark/storage/BlockManagerMaster.scala | 8 ++------ .../org/apache/spark/storage/ThreadingTest.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 5 +++++ .../spark/storage/DiskBlockManagerSuite.scala | 4 +--- 6 files changed, 19 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index cdae167aef829..77b8ca1cce80b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -55,7 +55,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) // Set to the MapOutputTrackerActor living on the driver - var trackerActor: Either[ActorRef, ActorSelection] = _ + var trackerActor: ActorRef = _ protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -71,17 +71,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { - /* - The difference between ActorRef and ActorSelection is well explained here: - http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor - In spark a map output tracker can be either started on Driver where it is created which - is an ActorRef or it can be on executor from where it is looked up which is an - actorSelection. - */ - val future = trackerActor match { - case Left(a: ActorRef) => a.ask(message)(timeout) - case Right(b: ActorSelection) => b.ask(message)(timeout) - } + val future = trackerActor.ask(message)(timeout) Await.result(future, timeout) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 634a94f0a7f06..2e36ccb9a0f07 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,11 +17,10 @@ package org.apache.spark -import collection.mutable -import serializer.Serializer +import scala.collection.mutable +import scala.concurrent.Await import akka.actor._ -import akka.remote.RemoteActorRefProvider import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -157,17 +156,18 @@ object SparkEnv extends Logging { conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) - def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { logInfo("Registering " + name) - Left(actorSystem.actorOf(Props(newActor), name = name)) + actorSystem.actorOf(Props(newActor), name = name) } else { val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.get("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") - val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) - logInfo("Connecting to " + name + ": " + url) - Right(actorSystem.actorSelection(url)) + val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" + val timeout = AkkaUtils.lookupTimeout(conf) + logInfo(s"Connecting to $name: $url") + Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index b5afe8cd23ca8..51a29ed8ef81a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,8 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], - conf: SparkConf) extends Logging { +class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt @@ -159,10 +158,7 @@ class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = driverActor match { - case Left(a: ActorRef) => a.ask(message)(timeout) - case Right(b: ActorSelection) => b.ask(message)(timeout) - } + val future = driverActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { throw new SparkException("BlockManagerMaster returned null") diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index dca98c6c05988..729ba2c550a20 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -95,7 +95,7 @@ private[spark] object ThreadingTest { val conf = new SparkConf() val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 362cea5e3e462..b4c4e1dbbc7f0 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -91,4 +91,9 @@ private[spark] object AkkaUtils { def askTimeout(conf: SparkConf): FiniteDuration = { Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds") } + + /** Returns the default Spark timeout to use for Akka remote actor lookup. */ + def lookupTimeout(conf: SparkConf): FiniteDuration = { + Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds") + } } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index af4b31d53c26c..829f389460f3b 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -23,9 +23,7 @@ import scala.collection.mutable import com.google.common.io.Files import org.apache.spark.SparkConf -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import scala.util.Try -import akka.actor.{Props, ActorSelection, ActorSystem} +import org.scalatest.{BeforeAndAfterEach, FunSuite} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { private val testConf = new SparkConf(false) From 8e88db3ca56e6a56668b029e39c8e96b86d4dd5e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 6 Jan 2014 02:21:56 +0000 Subject: [PATCH 1190/1571] Bug fixes to the DriverRunner and minor changes here and there. --- conf/slaves | 7 +++++-- .../org/apache/spark/deploy/worker/DriverRunner.scala | 8 ++++---- .../apache/spark/streaming/dstream/FileInputDStream.scala | 2 +- .../apache/spark/streaming/scheduler/JobGenerator.scala | 8 ++++---- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/conf/slaves b/conf/slaves index da0a01343d20a..30ea300e07c4d 100644 --- a/conf/slaves +++ b/conf/slaves @@ -1,2 +1,5 @@ -# A Spark Worker will be started on each of the machines listed below. -localhost \ No newline at end of file +ec2-54-221-59-252.compute-1.amazonaws.com +ec2-67-202-26-243.compute-1.amazonaws.com +ec2-23-22-220-97.compute-1.amazonaws.com +ec2-50-16-98-100.compute-1.amazonaws.com +ec2-54-234-164-206.compute-1.amazonaws.com diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 7485b89cf41b9..2d567b7a411b6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -119,15 +119,15 @@ private[spark] class DriverRunner( val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path val jarFileSystem = jarPath.getFileSystem(emptyConf) - val destPath = new Path(driverDir.getAbsolutePath()) - val destFileSystem = destPath.getFileSystem(emptyConf) + val destPath = new File(driverDir.getAbsolutePath(), jarPath.getName()) + // val destFileSystem = destPath.getFileSystem(emptyConf) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) val localJarFilename = localJarFile.getAbsolutePath if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destFileSystem, destPath, false, false, emptyConf) + FileUtil.copy(jarFileSystem, jarPath, destPath, false, emptyConf) } if (!localJarFile.exists()) { // Verify copy succeeded @@ -161,7 +161,7 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) - Files.write(header, stderr, Charsets.UTF_8) + Files.append(header, stderr, Charsets.UTF_8) CommandUtils.redirectStream(process.get.getErrorStream, stderr) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 39e25239bf59c..a5a5f2e751058 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -175,7 +175,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas override def cleanup() { } override def restore() { - hadoopFiles.foreach { + hadoopFiles.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, f) => { // Restore the metadata in both files and generatedRDDs logInfo("Restoring files for time " + t + " - " + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 1cd0b9b0a4ab7..6c1df4f9c931d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -85,14 +85,14 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val checkpointTime = ssc.initialCheckpoint.checkpointTime val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds)) val downTimes = checkpointTime.until(restartTime, batchDuration) - logInfo("Batches during down time: " + downTimes.mkString(", ")) + logInfo("Batches during down time (" + downTimes.size + " batches): " + downTimes.mkString(", ")) // Batches that were unprocessed before failure - val pendingTimes = ssc.initialCheckpoint.pendingTimes - logInfo("Batches pending processing: " + pendingTimes.mkString(", ")) + val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering) + logInfo("Batches pending processing (" + pendingTimes.size + " batches): " + pendingTimes.mkString(", ")) // Reschedule jobs for these times val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) - logInfo("Batches to reschedule: " + timesToReschedule.mkString(", ")) + logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => jobScheduler.runJobs(time, graph.generateJobs(time)) ) From 5c152e3e219a44f97d9df38ba00cdc4adbf4d873 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 10:39:05 +0800 Subject: [PATCH 1191/1571] Fixed several compilation errors in test suites --- .../org/apache/spark/MapOutputTrackerSuite.scala | 11 +++++++---- .../org/apache/spark/storage/BlockManagerSuite.scala | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 10b8b441fde18..82dc30ecc47be 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -23,6 +23,7 @@ import akka.actor._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils +import scala.concurrent.Await class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { private val conf = new SparkConf @@ -101,13 +102,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = Left(actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) val slaveTracker = new MapOutputTracker(conf) - slaveTracker.trackerActor = Right(slaveSystem.actorSelection( - "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index a0fc3445be422..032c2f2f69e84 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -58,7 +58,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT conf.set("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") From eb24684748da5dc2495fc4afe6da58edb463294b Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 11:21:35 +0800 Subject: [PATCH 1192/1571] Fixed test suite compilation errors --- .../test/scala/org/apache/spark/MapOutputTrackerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 82dc30ecc47be..afc1beff989c4 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -50,14 +50,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -76,7 +76,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) From a72107284ae4d8b6c7c47ded31c6784732028603 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 6 Jan 2014 12:30:17 +0800 Subject: [PATCH 1193/1571] fix logistic loss bug --- .../scala/org/apache/spark/mllib/optimization/Gradient.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 749e7364f4de1..c590492e7abd6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -50,8 +50,8 @@ class LogisticGradient extends Gradient { val gradient = data.mul(gradientMultiplier) val loss = - if (margin > 0) { - math.log(1 + math.exp(0 - margin)) + if (label > 0) { + math.log(1 + math.exp(margin)) } else { math.log(1 + math.exp(margin)) - margin } From 675d7eb4f064129d275a45df4c5c43f558638422 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 21:23:14 -0800 Subject: [PATCH 1194/1571] Responding to Aaron's review --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 5729334e248e1..7df7e3d8e561c 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -52,6 +52,8 @@ private[spark] object AkkaUtils { val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" if (!akkaLogLifecycleEvents) { + // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent. + // See: https://www.assembla.com/spaces/akka/tickets/3787#/ Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) } From d86dc74d796121b61ff43c632791c52dd49ff8ad Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 5 Jan 2014 22:05:30 -0800 Subject: [PATCH 1195/1571] Code review feedback --- README.md | 14 ++++++++------ docs/README.md | 4 ++-- docs/_plugins/copy_api_dirs.rb | 4 ++-- docs/api.md | 2 +- docs/hadoop-third-party-distributions.md | 2 +- docs/index.md | 6 +++--- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 8 ++++---- docs/running-on-yarn.md | 6 +++--- docs/scala-programming-guide.md | 2 +- make-distribution.sh | 7 +++++-- sbt/sbt | 13 ++++--------- 12 files changed, 35 insertions(+), 35 deletions(-) diff --git a/README.md b/README.md index 2c08a4ac638c3..b91e4cf86713b 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,9 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained [here](http://www.scala-sbt.org). To build Spark and its example programs, run: +which can be obtained [here](http://www.scala-sbt.org). If SBT is installed we +will use the system version of sbt otherwise we will attempt to download it +automatically. To build Spark and its example programs, run: ./sbt/sbt assembly @@ -55,22 +57,22 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 - $ SPARK_HADOOP_VERSION=1.2.1 sbt assembly + $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly # Cloudera CDH 4.2.0 with MapReduce v1 - $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt assembly + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly # Cloudera CDH 4.2.0 with MapReduce v2 - $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt assembly + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly # Apache Hadoop 2.2.X and newer - $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt assembly + $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're diff --git a/docs/README.md b/docs/README.md index e3d6c9a5bc211..dfcf7535538f0 100644 --- a/docs/README.md +++ b/docs/README.md @@ -27,10 +27,10 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri ## API Docs (Scaladoc and Epydoc) -You can build just the Spark scaladoc by running `sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. -When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index ef9912c808259..431de909cbf4b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -26,8 +26,8 @@ curr_dir = pwd cd("..") - puts "Running sbt doc from " + pwd + "; this may take a few minutes..." - puts `sbt doc` + puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt doc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/api.md b/docs/api.md index 11e2c15324ef0..e86d07770a80b 100644 --- a/docs/api.md +++ b/docs/api.md @@ -3,7 +3,7 @@ layout: global title: Spark API documentation (Scaladoc) --- -Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt doc` from the Spark project home directory. +Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory. - [Spark](api/core/index.html) - [Spark Examples](api/examples/index.html) diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 141d475ba6610..de6a2b0a43bd5 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -12,7 +12,7 @@ with these distributions: When compiling Spark, you'll need to [set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): - SPARK_HADOOP_VERSION=1.0.4 sbt assembly + SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark diff --git a/docs/index.md b/docs/index.md index bf8d1c3375e85..86d574daaab4a 100644 --- a/docs/index.md +++ b/docs/index.md @@ -17,7 +17,7 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you n Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run - sbt assembly + sbt/sbt assembly For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you will need to use this same version of Scala in your own program -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). @@ -56,12 +56,12 @@ Hadoop, you must build Spark against the same version that your cluster uses. By default, Spark links to Hadoop 1.0.4. You can change this by setting the `SPARK_HADOOP_VERSION` variable when compiling: - SPARK_HADOOP_VERSION=2.2.0 sbt assembly + SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set `SPARK_YARN` to `true`: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 5d48cb676a142..dc187b3efec9b 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -69,7 +69,7 @@ The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: {% highlight bash %} -$ sbt assembly +$ sbt/sbt assembly $ ./bin/pyspark {% endhighlight %} diff --git a/docs/quick-start.md b/docs/quick-start.md index 9b9261cfff8ea..153081bdaa286 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -12,7 +12,7 @@ See the [programming guide](scala-programming-guide.html) for a more complete re To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: {% highlight bash %} -$ sbt assembly +$ sbt/sbt assembly {% endhighlight %} # Interactive Analysis with the Spark Shell @@ -146,7 +146,7 @@ If you also wish to read data from Hadoop's HDFS, you will also need to add a de libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" {% endhighlight %} -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt run` to execute our program. +Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program. {% highlight bash %} $ find . @@ -157,8 +157,8 @@ $ find . ./src/main/scala ./src/main/scala/SimpleApp.scala -$ sbt package -$ sbt run +$ sbt/sbt package +$ sbt/sbt run ... Lines with a: 46, Lines with b: 23 {% endhighlight %} diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index a35e003cdc1ee..717071d72c9b9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -12,7 +12,7 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly The assembled JAR will be something like this: `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. @@ -25,7 +25,7 @@ The build process now also supports new YARN versions (2.2.x). See below. - The assembled jar can be installed into HDFS or used locally. - Your application code must be packaged into a separate JAR file. -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. # Configuration @@ -72,7 +72,7 @@ The command to launch the YARN Client is as follows: For example: # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly # Configure logging $ cp conf/log4j.properties.template conf/log4j.properties diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 3d0e8923d5427..c1ef46a1cded7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -31,7 +31,7 @@ In addition, if you wish to access an HDFS cluster, you need to add a dependency artifactId = hadoop-client version = -For other build systems, you can run `sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). +For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: diff --git a/make-distribution.sh b/make-distribution.sh index 6c466c8a06a6d..61e6654dcb70b 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -44,13 +44,16 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -if ! test `which sbt` ;then +VERSIONSTRING=$FWDIR/sbt/sbt "show version" + +if [ $? == -1 ] ;then echo -e "You need sbt installed and available on your path." echo -e "Download sbt from http://www.scala-sbt.org/" exit -1; fi -VERSION=$(sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') +VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') +echo "Version is ${VERSION}" # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 diff --git a/sbt/sbt b/sbt/sbt index 6d2caca120ca9..09cc5a0b4ac48 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -27,22 +27,17 @@ else wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" - exit + exit -1 fi fi if [ ! -f ${JAR} ]; then # We failed to download - printf "Our attempt to download sbt locally to {$JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" - exit + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 fi printf "Launching sbt from .sbtlib\n" java \ - -Duser.timezone=UTC \ - -Djava.awt.headless=true \ - -Dfile.encoding=UTF-8 \ - -XX:MaxPermSize=256m \ - -Xmx1g \ - -noverify \ + -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ -jar ${JAR} \ "$@" fi From 5a598b2d7b72db7f732a63e6e92d54786f68ee1e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 5 Jan 2014 22:07:32 -0800 Subject: [PATCH 1196/1571] Fix indentatation --- sbt/sbt | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index 09cc5a0b4ac48..0951e9daa68e6 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -16,28 +16,28 @@ else printf "NOT FOUND]\n" # Download sbt or use already downloaded if [ ! -d .sbtlib ]; then - mkdir .sbtlib + mkdir .sbtlib fi if [ ! -f ${JAR} ]; then - # Download - printf "Attempting to fetch sbt\n" - if hash curl 2>/dev/null; then - curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} - elif hash wget 2>/dev/null; then - wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} - else - printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 - fi + # Download + printf "Attempting to fetch sbt\n" + if hash curl 2>/dev/null; then + curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} + elif hash wget 2>/dev/null; then + wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} + else + printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 + fi fi if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 + # We failed to download + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 fi printf "Launching sbt from .sbtlib\n" java \ -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ - -jar ${JAR} \ - "$@" + -jar ${JAR} \ + "$@" fi From 7d0094bb562926af1d9b84a19d85329674aee003 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 5 Jan 2014 22:12:47 -0800 Subject: [PATCH 1197/1571] Finish documentation changes --- bin/pyspark | 2 +- bin/spark-class | 2 +- make-distribution.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index f97dfa7e2ff46..d6810f4686bf5 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -31,7 +31,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to build Spark with sbt assembly before running this program" >&2 + echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi fi diff --git a/bin/spark-class b/bin/spark-class index 49b0bef0bd3da..c4225a392d6da 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -104,7 +104,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark with 'sbt assembly' before running this program." >&2 + echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 exit 1 fi if [ "$num_jars" -gt "1" ]; then diff --git a/make-distribution.sh b/make-distribution.sh index 61e6654dcb70b..1a3a5d0209ccf 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -95,7 +95,7 @@ export SPARK_HADOOP_VERSION export SPARK_YARN cd $FWDIR -"sbt" "assembly/assembly" +"sbt/sbt" "assembly/assembly" # Make directories rm -rf "$DISTDIR" From 2dc83de72e7803fbf69f55fd1edf1969bc48e074 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 5 Jan 2014 23:29:26 -0800 Subject: [PATCH 1198/1571] CR feedback (sbt -> sbt/sbt and correct JAR path in script) :) --- bin/run-example | 2 +- project/build.properties | 2 +- sbt/sbt | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/run-example b/bin/run-example index dfb4bf7baf0c2..6c5d4a6a8f364 100755 --- a/bin/run-example +++ b/bin/run-example @@ -55,7 +55,7 @@ if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 - echo "You need to build Spark with sbt assembly before running this program" >&2 + echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi diff --git a/project/build.properties b/project/build.properties index 059edac7d4331..03438f6558abe 100644 --- a/project/build.properties +++ b/project/build.properties @@ -16,5 +16,5 @@ # # Note: If you change the sbt version please also change the sbt download -# script in the root directory of the project called sbt. +# script sbt/sbt sbt.version=0.12.4 diff --git a/sbt/sbt b/sbt/sbt index 0951e9daa68e6..af422d9e891bd 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -35,7 +35,7 @@ else printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" exit -1 fi - printf "Launching sbt from .sbtlib\n" + printf "Launching sbt from ${JAR}\n" java \ -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ -jar ${JAR} \ From ac1f4b06c12dae922172b6fa907eec0ae0bd0170 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 5 Jan 2014 23:42:53 -0800 Subject: [PATCH 1199/1571] Added a hashmap to cache file mod times. --- .../spark/util/TimeStampedHashMap.scala | 8 +++-- .../streaming/dstream/FileInputDStream.scala | 30 +++++++++++++++---- 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 181ae2fd45baf..9ce4ef744ec78 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -30,12 +30,16 @@ import org.apache.spark.Logging * threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in * replacement of scala.collection.mutable.HashMap. */ -class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging { +class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends Map[A, B]() with Logging { val internalMap = new ConcurrentHashMap[A, (B, Long)]() def get(key: A): Option[B] = { val value = internalMap.get(key) - if (value != null) Some(value._1) else None + if (value != null && updateTimeStampOnGet) { + internalMap.replace(key, value, (value._1, currentTime)) + } + Option(value).map(_._1) } def iterator: Iterator[(A, B)] = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index b4743013b1829..0028422db9a1c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -23,10 +23,10 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time} +import org.apache.spark.util.TimeStampedHashMap private[streaming] @@ -46,6 +46,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null @transient private[streaming] var files = new HashMap[Time, Array[String]] + @transient private var fileModTimes = new TimeStampedHashMap[String, Long](true) + @transient private var lastNewFileFindingTime = 0L override def start() { if (newFilesOnly) { @@ -96,6 +98,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) logDebug("Cleared files are:\n" + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) + // Delete file times that weren't accessed in the last round of getting new files + fileModTimes.clearOldValues(lastNewFileFindingTime - 1) } /** @@ -104,8 +108,18 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas */ private def findNewFiles(currentTime: Long): (Seq[String], Long, Seq[String]) = { logDebug("Trying to get new files for time " + currentTime) + lastNewFileFindingTime = System.currentTimeMillis val filter = new CustomPathFilter(currentTime) - val newFiles = fs.listStatus(path, filter).map(_.getPath.toString) + val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val timeTaken = System.currentTimeMillis - lastNewFileFindingTime + logInfo("Finding new files took " + timeTaken + " ms") + if (timeTaken > slideDuration.milliseconds) { + logWarning( + "Time taken to find new files exceeds the batch size. " + + "Consider increasing the batch size or reduceing the number of " + + "files in the monitored directory." + ) + } (newFiles, filter.latestModTime, filter.latestModTimeFiles.toSeq) } @@ -122,16 +136,20 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas new UnionRDD(context.sparkContext, fileRDDs) } - private def path: Path = { + private def directoryPath: Path = { if (path_ == null) path_ = new Path(directory) path_ } private def fs: FileSystem = { - if (fs_ == null) fs_ = path.getFileSystem(new Configuration()) + if (fs_ == null) fs_ = directoryPath.getFileSystem(new Configuration()) fs_ } + private def getFileModTime(path: Path) = { + fileModTimes.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) + } + private def reset() { fs_ = null } @@ -142,6 +160,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[(K,V)]] () files = new HashMap[Time, Array[String]] + fileModTimes = new TimeStampedHashMap[String, Long](true) } /** @@ -187,14 +206,13 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Latest file mod time seen in this round of fetching files and its corresponding files var latestModTime = 0L val latestModTimeFiles = new HashSet[String]() - def accept(path: Path): Boolean = { try { if (!filter(path)) { // Reject file if it does not satisfy filter logDebug("Rejected by filter " + path) return false } - val modTime = fs.getFileStatus(path).getModificationTime() + val modTime = getFileModTime(path) logDebug("Mod time for " + path + " is " + modTime) if (modTime < prevModTime) { logDebug("Mod time less than last mod time") From 05e6d5b454b74b222b4131af24c8f750c30a05fb Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 6 Jan 2014 16:54:00 +0800 Subject: [PATCH 1200/1571] Added GradientDescentSuite --- .../optimization/GradientDescentSuite.scala | 116 ++++++++++++++++++ 1 file changed, 116 insertions(+) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala new file mode 100644 index 0000000000000..a6028a1e981dd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -0,0 +1,116 @@ +/* + * 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.mllib.optimization + +import scala.util.Random +import scala.collection.JavaConversions._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.regression._ + +object GradientDescentSuite { + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateGDInput(offset, scale, nPoints, seed)) + } + + // Generate input of the form Y = logistic(offset + scale * X) + def generateGDInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val unifRand = new scala.util.Random(45) + val rLogis = (0 until nPoints).map { i => + val u = unifRand.nextDouble() + math.log(u) - math.log(1.0-u) + } + + val y: Seq[Int] = (0 until nPoints).map { i => + val yVal = offset + scale * x1(i) + rLogis(i) + if (yVal > 0) 1 else 0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) + testData + } +} + +class GradientDescentSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("Assert the loss is decreasing.") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val gradient = new LogisticGradient() + val updater = new SimpleUpdater() + val stepSize = 1.0 + val numIterations = 10 + val regParam = 0 + val miniBatchFrac = 1.0 + + // Add a extra variable consisting of all 1.0's for the intercept. + val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42) + val data = testData.map { case LabeledPoint(label, features) => + label -> Array(1.0, features: _*) + } + + val dataRDD = sc.parallelize(data, 2).cache() + val initialWeightsWithIntercept = Array(1.0, initialWeights: _*) + + val (_, loss) = GradientDescent.runMiniBatchSGD( + dataRDD, + gradient, + updater, + stepSize, + numIterations, + regParam, + miniBatchFrac, + initialWeightsWithIntercept) + + assert(loss.last - loss.head < 0, "loss isn't decreasing.") + + val lossDiff = loss.init.zip(loss.tail).map { case (lhs, rhs) => lhs - rhs } + assert(lossDiff.count(_ > 0).toDouble / lossDiff.size > 0.8) + } +} From 355a0338937342622ec5db85c853573d79980311 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 14:38:27 +0530 Subject: [PATCH 1201/1571] SPARK-1005 Ning upgrade --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index aa2f076aacc24..4f1396b179847 100644 --- a/pom.xml +++ b/pom.xml @@ -188,7 +188,7 @@ com.ning compress-lzf - 0.8.4 + 1.0.0 org.xerial.snappy diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6c560d5c6bae..4d2aa456645ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -230,7 +230,7 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 - "com.ning" % "compress-lzf" % "0.8.4", + "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", "org.ow2.asm" % "asm" % "4.0", "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), From d0fd3b9ad238294346eb3465c489eabd41fb2380 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 6 Jan 2014 01:47:53 -0800 Subject: [PATCH 1202/1571] Changed JavaStreamingContextWith*** to ***Function in streaming.api.java.*** package. Also fixed packages of Flume and MQTT tests. --- .../examples/JavaFlumeEventCount.java | 6 +++--- .../examples/JavaKafkaWordCount.java | 6 +++--- .../java/flume/FlumeFunctions.scala} | 10 +++++----- .../flume}/JavaFlumeStreamSuite.java | 13 +++++-------- .../java/kafka/KafkaFunctions.scala} | 12 ++++++------ .../streaming/kafka/JavaKafkaStreamSuite.java | 15 +++++++-------- .../api/java/mqtt/MQTTFunctions.scala} | 10 +++++----- .../streaming/mqtt/MQTTFunctions.scala | 0 .../streaming/mqtt/MQTTInputDStream.scala | 0 .../{spark => }/streaming/mqtt/package.scala | 0 .../streaming/mqtt/JavaMQTTStreamSuite.java | 10 ++++------ .../java/twitter/TwitterFunctions.scala} | 18 +++++++++--------- .../twitter/JavaTwitterStreamSuite.java | 19 +++++++++---------- .../java/zeromq/ZeroMQFunctions.scala} | 12 ++++++------ .../zeromq/JavaZeroMQStreamSuite.java | 12 +++++------- 15 files changed, 67 insertions(+), 76 deletions(-) rename external/flume/src/main/scala/org/apache/spark/streaming/{flume/JavaStreamingContextWithFlume.scala => api/java/flume/FlumeFunctions.scala} (86%) rename external/flume/src/test/java/{ => org/apache/spark/streaming/flume}/JavaFlumeStreamSuite.java (71%) rename external/kafka/src/main/scala/org/apache/spark/streaming/{kafka/JavaStreamingContextWithKafka.scala => api/java/kafka/KafkaFunctions.scala} (89%) rename external/mqtt/src/main/scala/org/apache/spark/{spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala => streaming/api/java/mqtt/MQTTFunctions.scala} (87%) rename external/mqtt/src/main/scala/org/apache/spark/{spark => }/streaming/mqtt/MQTTFunctions.scala (100%) rename external/mqtt/src/main/scala/org/apache/spark/{spark => }/streaming/mqtt/MQTTInputDStream.scala (100%) rename external/mqtt/src/main/scala/org/apache/spark/{spark => }/streaming/mqtt/package.scala (100%) rename external/twitter/src/main/scala/org/apache/spark/streaming/{twitter/JavaStreamingContextWithTwitter.scala => api/java/twitter/TwitterFunctions.scala} (86%) rename external/zeromq/src/main/scala/org/apache/spark/streaming/{zeromq/JavaStreamingContextWithZeroMQ.scala => api/java/zeromq/ZeroMQFunctions.scala} (91%) diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index e53c4f9e83215..64832a97219f3 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -20,7 +20,7 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; +import org.apache.spark.streaming.api.java.flume.FlumeFunctions; import org.apache.spark.streaming.flume.SparkFlumeEvent; /** @@ -52,8 +52,8 @@ public static void main(String[] args) { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); - JavaDStream flumeStream = sscWithFlume.flumeStream("localhost", port); + FlumeFunctions flumeFunc = new FlumeFunctions(ssc); + JavaDStream flumeStream = flumeFunc.flumeStream("localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index de0420ca83a7c..207ce8cd4f3a3 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,7 +29,7 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.kafka.JavaStreamingContextWithKafka; +import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import scala.Tuple2; /** @@ -64,8 +64,8 @@ public static void main(String[] args) { topicMap.put(topic, numThreads); } - JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); - JavaPairDStream messages = sscWithKafka.kafkaStream(args[1], args[2], topicMap); + KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); + JavaPairDStream messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala similarity index 86% rename from external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala rename to external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala index 4e66ae35355b6..3347d19796a68 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala @@ -15,24 +15,24 @@ * limitations under the License. */ -package org.apache.spark.streaming.flume +package org.apache.spark.streaming.api.java.flume import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.flume._ import org.apache.spark.storage.StorageLevel /** * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra * functions for creating Flume input streams. */ -class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { +class FlumeFunctions(javaStreamingContext: JavaStreamingContext) { /** * Creates a input stream from a Flume source. * @param hostname Hostname of the slave machine to which the flume data will be sent * @param port Port of the slave machine to which the flume data will be sent */ def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port) + javaStreamingContext.ssc.flumeStream(hostname, port) } /** @@ -43,6 +43,6 @@ class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext) */ def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port, storageLevel) + javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel) } } diff --git a/external/flume/src/test/java/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java similarity index 71% rename from external/flume/src/test/java/JavaFlumeStreamSuite.java rename to external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index deffc78c4c557..5930fee925d2d 100644 --- a/external/flume/src/test/java/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -1,4 +1,4 @@ -/* +package org.apache.spark.streaming.flume;/* * 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. @@ -18,21 +18,18 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; +import org.apache.spark.streaming.api.java.flume.FlumeFunctions; import org.apache.spark.streaming.flume.SparkFlumeEvent; import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { - JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); + FlumeFunctions flumeFunc = new FlumeFunctions(ssc); // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithFlume.flumeStream("localhost", 12345); - JavaDStream test2 = sscWithFlume.flumeStream("localhost", 12345, + JavaDStream test1 = flumeFunc.flumeStream("localhost", 12345); + JavaDStream test2 = flumeFunc.flumeStream("localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithFlume.socketTextStream("localhost", 9999); } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala similarity index 89% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala rename to external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala index ab0e8a6c8df16..491331bb37390 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.api.java.kafka import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -27,13 +27,13 @@ import kafka.serializer.Decoder import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} +import org.apache.spark.streaming.kafka._ /** * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra * functions for creating Kafka input streams. */ -class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { +class KafkaFunctions(javaStreamingContext: JavaStreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. @@ -49,7 +49,7 @@ class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) ): JavaPairDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } /** @@ -69,7 +69,7 @@ class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) ): JavaPairDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } /** @@ -101,7 +101,7 @@ class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] - ssc.kafkaStream[K, V, U, T]( + javaStreamingContext.ssc.kafkaStream[K, V, U, T]( kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 66236df662a67..fdea96e506223 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,6 +18,8 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; + +import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; @@ -31,21 +33,18 @@ public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); + KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); // tests the API, does not actually test data receiving - JavaPairDStream test1 = sscWithKafka.kafkaStream("localhost:12345", "group", topics); - JavaPairDStream test2 = sscWithKafka.kafkaStream("localhost:12345", "group", topics, + JavaPairDStream test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics); + JavaPairDStream test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); - kafkaParams.put("zookeeper.connect","localhost:12345"); + kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = sscWithKafka.kafkaStream( + JavaPairDStream test3 = kafkaFunc.kafkaStream( String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithKafka.socketTextStream("localhost", 9999); } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala similarity index 87% rename from external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala rename to external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala index d814da0f0d3d6..72124956fcc57 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.spark.streaming.mqtt +package org.apache.spark.streaming.api.java.mqtt import scala.reflect.ClassTag import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.mqtt._ /** * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra * functions for creating MQTT input streams. */ -class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { +class MQTTFunctions(javaStreamingContext: JavaStreamingContext) { /** * Create an input stream that receives messages pushed by a MQTT publisher. @@ -39,7 +39,7 @@ class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext) topic: String ): JavaDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.mqttStream(brokerUrl, topic) + javaStreamingContext.ssc.mqttStream(brokerUrl, topic) } /** @@ -54,6 +54,6 @@ class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext) storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): JavaDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.mqttStream(brokerUrl, topic, storageLevel) + javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala similarity index 100% rename from external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala rename to external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala similarity index 100% rename from external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala rename to external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala similarity index 100% rename from external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala rename to external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index c1f41640dcc43..3ddb4d084fb5e 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -19,6 +19,7 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -28,14 +29,11 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { public void testMQTTStream() { String brokerUrl = "abc"; String topic = "def"; - JavaStreamingContextWithMQTT sscWithMQTT = new JavaStreamingContextWithMQTT(ssc); + MQTTFunctions mqttFunc = new MQTTFunctions(ssc); // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithMQTT.mqttStream(brokerUrl, topic); - JavaDStream test2 = sscWithMQTT.mqttStream(brokerUrl, topic, + JavaDStream test1 = mqttFunc.mqttStream(brokerUrl, topic); + JavaDStream test2 = mqttFunc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithMQTT.socketTextStream("localhost", 9999); } } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala similarity index 86% rename from external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala rename to external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala index 02503643313d0..22e297a03af21 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.spark.streaming.twitter +package org.apache.spark.streaming.api.java.twitter import twitter4j.Status import twitter4j.auth.Authorization import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.twitter._ /** * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra * functions for creating Twitter input streams. */ -class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { +class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { /** * Create a input stream that returns tweets received from Twitter using Twitter4J's default @@ -37,7 +37,7 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext * twitter4j.oauth.accessTokenSecret. */ def twitterStream(): JavaDStream[Status] = { - ssc.twitterStream(None) + javaStreamingContext.ssc.twitterStream(None) } /** @@ -48,7 +48,7 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream(filters: Array[String]): JavaDStream[Status] = { - ssc.twitterStream(None, filters) + javaStreamingContext.ssc.twitterStream(None, filters) } /** @@ -60,7 +60,7 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext * @param storageLevel Storage level to use for storing the received objects */ def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { - ssc.twitterStream(None, filters, storageLevel) + javaStreamingContext.ssc.twitterStream(None, filters, storageLevel) } /** @@ -68,7 +68,7 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext * @param twitterAuth Twitter4J Authorization */ def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth)) + javaStreamingContext.ssc.twitterStream(Some(twitterAuth)) } /** @@ -80,7 +80,7 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext twitterAuth: Authorization, filters: Array[String] ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters) + javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters) } /** @@ -94,6 +94,6 @@ class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext filters: Array[String], storageLevel: StorageLevel ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel) } } diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java index 34e4fbdd85a13..4564d6cd334c5 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java @@ -18,6 +18,8 @@ package org.apache.spark.streaming.twitter; import java.util.Arrays; + +import org.apache.spark.streaming.api.java.twitter.TwitterFunctions; import org.junit.Test; import twitter4j.Status; @@ -31,21 +33,18 @@ public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { @Test public void testTwitterStream() { - JavaStreamingContextWithTwitter sscWithTwitter = new JavaStreamingContextWithTwitter(ssc); + TwitterFunctions twitterFunc = new TwitterFunctions(ssc); String[] filters = (String[])Arrays.asList("filter1", "filter2").toArray(); Authorization auth = NullAuthorization.getInstance(); // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithTwitter.twitterStream(); - JavaDStream test2 = sscWithTwitter.twitterStream(filters); + JavaDStream test1 = twitterFunc.twitterStream(); + JavaDStream test2 = twitterFunc.twitterStream(filters); JavaDStream test3 = - sscWithTwitter.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = sscWithTwitter.twitterStream(auth); - JavaDStream test5 = sscWithTwitter.twitterStream(auth, filters); + twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test4 = twitterFunc.twitterStream(auth); + JavaDStream test5 = twitterFunc.twitterStream(auth, filters); JavaDStream test6 = - sscWithTwitter.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithTwitter.socketTextStream("localhost", 9999); + twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala similarity index 91% rename from external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala rename to external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala index dc5d1f05beafa..a9bbce71f5ff1 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.zeromq +package org.apache.spark.streaming.api.java.zeromq import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -27,13 +27,13 @@ import akka.zeromq.Subscribe import org.apache.spark.storage.StorageLevel import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.zeromq._ /** * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra * functions for creating ZeroMQ input streams. */ -class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { +class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { /** * Create an input stream that receives messages pushed by a zeromq publisher. @@ -55,7 +55,7 @@ class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) } /** @@ -77,7 +77,7 @@ class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) } /** @@ -97,6 +97,6 @@ class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn) + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn) } } diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index 96af7d737d346..b020ae4ceff93 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming.zeromq; +import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions; import org.junit.Test; import akka.actor.SupervisorStrategy; @@ -32,7 +33,7 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @Test // tests the API, does not actually test data receiving public void testZeroMQStream() { - JavaStreamingContextWithZeroMQ sscWithZeroMQ = new JavaStreamingContextWithZeroMQ(ssc); + ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc); String publishUrl = "abc"; Subscribe subscribe = new Subscribe((ByteString)null); Function> bytesToObjects = new Function>() { @@ -42,14 +43,11 @@ public Iterable call(byte[][] bytes) throws Exception { } }; - JavaDStream test1 = sscWithZeroMQ.zeroMQStream( + JavaDStream test1 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = sscWithZeroMQ.zeroMQStream( + JavaDStream test2 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = sscWithZeroMQ.zeroMQStream( + JavaDStream test3 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithZeroMQ.socketTextStream("localhost", 9999); } } From 2d0825e9f4f4059e0008c5753840b73893667701 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 16:03:31 +0530 Subject: [PATCH 1203/1571] Made java options to be applied during tests so that they become self explanatory. --- project/SparkBuild.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6c560d5c6bae..eda925227ea67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -70,9 +70,7 @@ object SparkBuild extends Build { lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") val sparkHome = System.getProperty("user.dir") - System.setProperty("spark.home", sparkHome) - System.setProperty("spark.testing", "1") - + // Allows build configuration to be set through environment variables lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { @@ -115,8 +113,8 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions += "-Dspark.home=" + sparkHome, - javaOptions += "-Dspark.testing=1", + javaOptions in Test += "-Dspark.home=" + sparkHome, + javaOptions in Test += "-Dspark.testing=1", javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), From 25446dd931cce5916de5dddf4689b41ee6fd3148 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Jan 2014 07:58:59 -0600 Subject: [PATCH 1204/1571] Add warning to null setJars check --- core/src/main/scala/org/apache/spark/SparkConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 7073a999a99e4..55f27033b5579 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -24,7 +24,7 @@ import com.typesafe.config.ConfigFactory * * @param loadDefaults whether to load values from the system properties and classpath */ -class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { +class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with Logging { /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) @@ -67,6 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { + for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") set("spark.jars", jars.filter(_ != null).mkString(",")) } From 2dd4fb5698220bc33acb878254d41704221573bd Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 6 Jan 2014 09:01:46 -0800 Subject: [PATCH 1205/1571] Clarify spark.cores.max It controls the count of cores across the cluster, not on a per-machine basis. --- docs/configuration.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 567aba07f0e91..09342fedfc1a0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -81,7 +81,8 @@ there are at least five properties that you will commonly want to control: When running on a standalone deploy cluster or a Mesos cluster in "coarse-grained" - sharing mode, how many CPU cores to request at most. The default will use all available cores + sharing mode, the maximum amount of CPU cores to request for the application from + across the cluster (not from each machine). The default will use all available cores offered by the cluster manager. From 6ab1db8071101a097a3508861aaa12550a5bf9d5 Mon Sep 17 00:00:00 2001 From: walker Date: Tue, 7 Jan 2014 01:21:25 +0800 Subject: [PATCH 1206/1571] add inline comments --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 52a424db22678..a028783757471 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -348,6 +348,10 @@ object BlockManagerMasterActor { if (storageLevel.isValid) { // isValid means it is either stored in-memory or on-disk. + // But the memSize here indicates the data size in or dropped from memory, + // and the diskSize here indicates the data size in or dropped to disk. + // They can be both large than 0, when a block is dropped from memory to disk. + // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. if (storageLevel.useMemory) { _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) _remainingMem -= memSize From 2ad315e80fb297f1e6cf77ec6fcdcf25750b4a3d Mon Sep 17 00:00:00 2001 From: walker Date: Tue, 7 Jan 2014 01:27:57 +0800 Subject: [PATCH 1207/1571] add inline comments --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index a028783757471..2c1a4e2f5d3a1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -350,7 +350,7 @@ object BlockManagerMasterActor { // isValid means it is either stored in-memory or on-disk. // But the memSize here indicates the data size in or dropped from memory, // and the diskSize here indicates the data size in or dropped to disk. - // They can be both large than 0, when a block is dropped from memory to disk. + // They can be both larger than 0, when a block is dropped from memory to disk. // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. if (storageLevel.useMemory) { _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) From 1f7c090e4b5750987c300fda6fcf397bb518cd48 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Jan 2014 12:04:22 -0600 Subject: [PATCH 1208/1571] Change protobuf version for yarn alpha back to 2.4.1 --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index aa2f076aacc24..72a9244d17d34 100644 --- a/pom.xml +++ b/pom.xml @@ -727,7 +727,6 @@ 2 0.23.7 - 2.5.0 From 11a93fb5a8fafa940db27b652e4c21f6713ed8d1 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 6 Jan 2014 12:18:03 -0800 Subject: [PATCH 1209/1571] Added serializing method for Rating object --- .../mllib/api/python/PythonMLLibAPI.scala | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index be2628fac5817..2d8623392eb4e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -197,6 +197,7 @@ class PythonMLLibAPI extends Serializable { return ret } + /** Unpack a Rating object from an array of bytes */ private def unpackRating(ratingBytes: Array[Byte]): Rating = { val bb = ByteBuffer.wrap(ratingBytes) bb.order(ByteOrder.nativeOrder()) @@ -206,6 +207,7 @@ class PythonMLLibAPI extends Serializable { return new Rating(user, product, rating) } + /** Unpack a tuple of Ints from an array of bytes */ private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { val bb = ByteBuffer.wrap(tupleBytes) bb.order(ByteOrder.nativeOrder()) @@ -214,13 +216,23 @@ class PythonMLLibAPI extends Serializable { (v1, v2) } + /** + * Serialize a Rating object into an array of bytes. + * It can be deserialized using RatingDeserializer(). + * + * @param rate + * @return + */ private[spark] def serializeRating(rate: Rating): Array[Byte] = { - val bytes = new Array[Byte](24) + val len = 3 + val bytes = new Array[Byte](4 + 8 * len) val bb = ByteBuffer.wrap(bytes) bb.order(ByteOrder.nativeOrder()) - bb.putDouble(rate.user.toDouble) - bb.putDouble(rate.product.toDouble) - bb.putDouble(rate.rating) + bb.putInt(len) + val db = bb.asDoubleBuffer() + db.put(rate.user.toDouble) + db.put(rate.product.toDouble) + db.put(rate.rating) bytes } From 04132ea9b20a95cd68482605d4022f692bb556e5 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 6 Jan 2014 12:19:08 -0800 Subject: [PATCH 1210/1571] Added Rating deserializer --- .../MatrixFactorizationModel.scala | 9 +++++++- python/pyspark/mllib/_common.py | 21 ++++++++++++++++--- 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 2c3e82830033e..443fc5de5bf04 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -67,7 +67,14 @@ class MatrixFactorizationModel( } } - def predictJavaRDD(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { + /** + * Predict the rating of many users for many products. + * This is a Java stub for python predictAll() + * + * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product) + * @return JavaRDD of serialized Rating objects. + */ + def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { val pythonAPI = new PythonMLLibAPI() val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) predict(usersProducts).map(rate => pythonAPI.serializeRating(rate)) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index c818fc4d97180..769d88dfb9b56 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -18,6 +18,9 @@ from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape from pyspark import SparkContext +from pyspark.serializers import Serializer +import struct + # Double vector format: # # [8-byte 1] [8-byte length] [length*8 bytes of data] @@ -213,9 +216,21 @@ def _serialize_rating(r): intpart[0], intpart[1], doublepart[0] = r return ba -def _deserialize_rating(ba): - ar = ndarray(shape=(3, ), buffer=ba, dtype="float64", order='C') - return ar.copy() +class RatingDeserializer(Serializer): + def loads(self, stream): + length = struct.unpack("!i", stream.read(4))[0] + ba = stream.read(length) + res = ndarray(shape=(3, ), buffer=ba, dtype="float64", offset=4) + return int(res[0]), int(res[1]), res[2] + + def load_stream(self, stream): + while True: + try: + yield self.loads(stream) + except struct.error: + return + except EOFError: + return def _serialize_tuple(t): ba = bytearray(8) From 754f5300a1e0a214b62cbd6db2398dea4dfbceb4 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 6 Jan 2014 12:19:43 -0800 Subject: [PATCH 1211/1571] Added predictAll python function to MatrixFactorizationModel --- python/pyspark/mllib/recommendation.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index c81b482a87ef7..0eeb5bb66b9d4 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -21,8 +21,7 @@ _serialize_double_matrix, _deserialize_double_matrix, \ _serialize_double_vector, _deserialize_double_vector, \ _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _serialize_tuple, _deserialize_rating -from pyspark.serializers import BatchedSerializer + _serialize_tuple, RatingDeserializer from pyspark.rdd import RDD class MatrixFactorizationModel(object): @@ -36,6 +35,9 @@ class MatrixFactorizationModel(object): >>> model = ALS.trainImplicit(sc, ratings, 1) >>> model.predict(2,2) is not None True + >>> testset = sc.parallelize([(1, 2), (1, 1)]) + >>> model.predictAll(testset).count == 2 + True """ def __init__(self, sc, java_model): @@ -50,8 +52,8 @@ def predict(self, user, product): def predictAll(self, usersProducts): usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple) - return RDD(self._java_model.predictJavaRDD(usersProductsJRDD._jrdd), - self._context, BatchedSerializer(_deserialize_rating, self._context._batchSize)) + return RDD(self._java_model.predict(usersProductsJRDD._jrdd), + self._context, RatingDeserializer()) class ALS(object): @classmethod From 150089dae12bbba693db4edbfcea360b443637df Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 6 Jan 2014 12:43:17 -0800 Subject: [PATCH 1212/1571] Added proper evaluation example for collaborative filtering and fixed typo --- docs/mllib-guide.md | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 3fd3c91e2a110..5f3b6761265dc 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,7 +3,7 @@ layout: global title: Machine Learning Library (MLlib) --- -* Table of contests +* Table of contents {:toc} MLlib is a Spark implementation of some common machine learning (ML) @@ -403,8 +403,8 @@ Errors. ## Collaborative Filtering In the following example we load rating data. Each row consists of a user, a product and a rating. -We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation -on one example. +We use the default ALS.train() method which assumes ratings are explicit. We evaluate the +recommendation by measuring the Mean Squared Error of rating prediction. {% highlight python %} from pyspark.mllib.recommendation import ALS @@ -418,7 +418,11 @@ ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) model = ALS.train(sc, ratings, 1, 20) # Evaluate the model on training data -print("predicted rating of user {0} for item {1} is {2:.6}".format(1, 2, model.predict(1, 2))) +testdata = ratings.map(lambda p: (int(p[0]), int(p[1]))) +predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2])) +ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions) +MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).reduce(lambda x, y: x + y)/ratesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) {% endhighlight %} If the rating matrix is derived from other source of information (i.e., it is inferred from other From 9272a004af7dd464809d9effc4ddc248a53919df Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 6 Jan 2014 13:03:19 -0800 Subject: [PATCH 1213/1571] Fix test breaking downstream builds --- .../org/apache/spark/deploy/worker/ExecutorRunnerTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index f58b1ee05a64b..7e5aaa3f98af5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.env.get("spark.home")).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" From fa8ce3fdd7f91add0e8ead6e48f4d69e67c604b9 Mon Sep 17 00:00:00 2001 From: Adam Novak Date: Mon, 6 Jan 2014 14:45:00 -0800 Subject: [PATCH 1214/1571] Changing org.apache.spark.util.collection.PrimitiveKeyOpenHashMap to have a real no-argument constructor, instead of a one-argument constructor with a default value. The lack of a real no-argument constructor was causing "sbt/sbt publish-local" to fail thusly: ``` [error] /pod/home/anovak/build/graphx/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala:172: not enough arguments for constructor PrimitiveKeyOpenHashMap: (initialCapacity: Int)(implicit evidence$3: ClassManifest[Int], implicit evidence$4: ClassManifest[Int])org.apache.spark.util.collection.PrimitiveKeyOpenHashMap[Int,Int] [error] private val mapIdToIndex = new PrimitiveKeyOpenHashMap[Int, Int]() [error] ^ [info] No documentation generated with unsucessful compiler run [error] one error found [error] (core/compile:doc) Scaladoc generation failed [error] Total time: 67 s, completed Jan 6, 2014 2:20:51 PM ``` In theory a no-argument constructor ought not to differ from one with a single argument that has a default value, but in practice there seems to be an issue. --- .../spark/util/collection/PrimitiveKeyOpenHashMap.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index ee1b168028b4a..d6a3cdb405cbc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -35,8 +35,15 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, /** * Allocate an OpenHashMap with a fixed initial capacity */ - def this(initialCapacity: Int = 64) = + def this(initialCapacity: Int) = this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + + /** + * Allocate an OpenHashMap with a default initial capacity, providing a true + * no-argument constructor. + */ + def this() = this(64) + /** * Allocate an OpenHashMap with a fixed initial capacity From f236ddd1a245a587d5ee331fb67cf41456ed383c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 2 Jan 2014 18:10:37 -0800 Subject: [PATCH 1215/1571] Changes based on review feedback. --- .../spark/deploy/client/DriverClient.scala | 2 +- .../deploy/client/DriverClientArguments.scala | 26 ++++++++++++------- .../spark/deploy/worker/CommandUtils.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 15 ++++++++--- .../spark/deploy/worker/DriverWrapper.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 7 +++-- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- 7 files changed, 34 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala index 8a4cdf07bbbaf..e319e75bae0ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala @@ -62,7 +62,7 @@ object DriverClient extends Logging { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + val (actorSystem, _) = AkkaUtils.createActorSystem( "driverClient", Utils.localHostName(), 0) val master = driverArgs.master val response = promise[(Boolean, String)] diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala index 6a15422c6cd6d..d9e1c8a1b0b32 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala @@ -23,6 +23,9 @@ import scala.collection.mutable.ListBuffer * Command-line parser for the driver client. */ private[spark] class DriverClientArguments(args: Array[String]) { + val defaultCores = 1 + val defaultMemory = 512 + var cmd: String = "" // 'launch' or 'kill' // launch parameters @@ -30,8 +33,8 @@ private[spark] class DriverClientArguments(args: Array[String]) { var jarUrl: String = "" var mainClass: String = "" var supervise: Boolean = false - var memory: Int = 512 - var cores: Int = 1 + var memory: Int = defaultMemory + var cores: Int = defaultCores private var _driverOptions = ListBuffer[String]() def driverOptions = _driverOptions.toSeq @@ -78,14 +81,17 @@ private[spark] class DriverClientArguments(args: Array[String]) { def printUsageAndExit(exitCode: Int) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. - System.err.println( - "usage: DriverClient [options] launch " + - "[driver options]\n" + - "usage: DriverClient kill \n\n" + - "Options:\n" + - " -c CORES, --cores CORES Number of cores to request \n" + - " -m MEMORY, --memory MEMORY Megabytes of memory to request\n" + - " -s, --supervise Whether to restart the driver on failure\n") + val usage = + s""" + |Usage: DriverClient [options] launch [driver options] + |Usage: DriverClient kill + | + |Options: + | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) + | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) + | -s, --supervise Whether to restart the driver on failure + """.stripMargin + System.err.println(usage) System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 785aecf1fe471..7507bf8ad0e6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -21,7 +21,7 @@ object CommandUtils extends Logging { } private def getEnv(key: String, command: Command): Option[String] = - command.environment.get(key).orElse(Option(getenv(key))) + command.environment.get(key).orElse(Option(System.getenv(key))) /** * Attention: this must always be aligned with the environment variables in the run scripts and diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e8ae2d302b888..f726089faaabe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -119,15 +119,14 @@ private[spark] class DriverRunner( val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path val jarFileSystem = jarPath.getFileSystem(emptyConf) - val destPath = new Path(driverDir.getAbsolutePath()) - val destFileSystem = destPath.getFileSystem(emptyConf) + val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) val localJarFilename = localJarFile.getAbsolutePath if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destFileSystem, destPath, false, false, emptyConf) + FileUtil.copy(jarFileSystem, jarPath, destPath, false, emptyConf) } if (!localJarFile.exists()) { // Verify copy succeeded @@ -140,8 +139,12 @@ private[spark] class DriverRunner( /** Launch the supplied command. */ private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File, supervise: Boolean) { + // Time to wait between submission retries. var waitSeconds = 1 + // A run of this many seconds resets the exponential back-off. + val successfulRunDuration = 1 + var keepTrying = !killed while (keepTrying) { @@ -161,11 +164,15 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) - Files.write(header, stderr, Charsets.UTF_8) + Files.append(header, stderr, Charsets.UTF_8) CommandUtils.redirectStream(process.get.getErrorStream, stderr) } + val processStart = System.currentTimeMillis() val exitCode = process.get.waitFor() + if (System.currentTimeMillis() - processStart > successfulRunDuration * 1000) { + waitSeconds = 1 + } if (supervise && exitCode != 0 && !killed) { waitSeconds = waitSeconds * 2 // exponential back-off diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 8c13b10c5152e..2deb21aac65d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -11,7 +11,7 @@ object DriverWrapper { def main(args: Array[String]) { args.toList match { case workerUrl :: mainClass :: extraArgs => - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("Driver", + val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", Utils.localHostName(), 0) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 4e23e0d1eb567..2947ed16924c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -260,8 +260,8 @@ private[spark] class Worker( case KillDriver(driverId) => { logInfo(s"Asked to kill driver $driverId") - drivers.find(_._1 == driverId) match { - case Some((id, runner)) => + drivers.get(driverId) match { + case Some(runner) => runner.kill() case None => logError(s"Asked to kill unknown driver $driverId") @@ -280,8 +280,7 @@ private[spark] class Worker( masterLock.synchronized { master ! DriverStateChanged(driverId, state, exception) } - val driver = drivers(driverId) - drivers -= driverId + val driver = drivers.remove(driverId).get finishedDrivers(driverId) = driver memoryUsed -= driver.driverDesc.mem coresUsed -= driver.driverDesc.cores diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 921b887a8978a..0615f7b565e40 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -47,7 +47,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{WORKER_URL}}", "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) From da4694a0d85433832fda1dd917fc698a08e65907 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 6 Jan 2014 16:11:39 +0800 Subject: [PATCH 1216/1571] Minor typo fix for yarn client --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 6abb4d501779f..23781ea35c670 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -102,7 +102,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", (args.userJar == null) -> "Error: You must specify a user jar!", (args.userClass == null) -> "Error: You must specify a user class!", - (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", + (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 440ad5cde54eb..be323d77835a8 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -122,7 +122,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", (args.userJar == null) -> "Error: You must specify a user jar!", (args.userClass == null) -> "Error: You must specify a user class!", - (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", + (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + From 67af8031368c4a3558aeb46c290b7b7a09516539 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 6 Jan 2014 16:12:25 +0800 Subject: [PATCH 1217/1571] Export --file for YarnClient mode to support sending extra files to worker on yarn cluster --- docs/running-on-yarn.md | 2 ++ .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index a35e003cdc1ee..da5a502fd5ffc 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -114,6 +114,8 @@ For example: SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./bin/spark-shell +You can also send extra files to yarn cluster for worker to use by exporting SPARK_YARN_DIST_FILES=file1,file2... etc. + # Building Spark for Hadoop/YARN 2.2.x See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 324ef4616fe26..4b1b5da048df4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -39,6 +39,7 @@ private[spark] class YarnClientSchedulerBackend( val defaultWorkerNumber = "1" val userJar = System.getenv("SPARK_YARN_APP_JAR") + val distFiles = System.getenv("SPARK_YARN_DIST_FILES") var workerCores = System.getenv("SPARK_WORKER_CORES") var workerMemory = System.getenv("SPARK_WORKER_MEMORY") var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") @@ -64,7 +65,8 @@ private[spark] class YarnClientSchedulerBackend( "--worker-memory", workerMemory, "--worker-cores", workerCores, "--num-workers", workerNumber, - "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher", + "--files", distFiles ) val args = new ClientArguments(argsArray, conf) From dea4ba9d80f96d2240fdd99b65ba0ac697135823 Mon Sep 17 00:00:00 2001 From: sproblvem Date: Tue, 7 Jan 2014 11:11:59 +0800 Subject: [PATCH 1218/1571] Update stop-slaves.sh The most recently version has changed the directory structure, but this script "sbin/stop-all.sh" doesn't change with it accordingly. This mistake makes "sbin/stop-all.sh" can't stop the slave node. --- sbin/stop-slaves.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index c6b0b6ab6695c..eb803b4900347 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -bin=`dirname "$0"` -bin=`cd "$sbin"; pwd` +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` . "$sbin/spark-config.sh" From 6a3daead2d5c82136fefa2de9bced036d1ccb759 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 6 Jan 2014 20:12:45 -0800 Subject: [PATCH 1219/1571] Fixes after merge --- .../org/apache/spark/deploy/client/DriverClient.scala | 7 ++++--- .../org/apache/spark/deploy/worker/DriverWrapper.scala | 5 +++-- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala index e319e75bae0ab..1cd5d99bcf779 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala @@ -23,7 +23,7 @@ import scala.concurrent._ import akka.actor._ -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -59,11 +59,12 @@ object DriverClient extends Logging { def main(args: Array[String]) { val driverArgs = new DriverClientArguments(args) + val conf = new SparkConf() // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0) + "driverClient", Utils.localHostName(), 0, false, conf) val master = driverArgs.master val response = promise[(Boolean, String)] val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response))) @@ -95,7 +96,7 @@ object DriverClient extends Logging { val (success, message) = try { - Await.result(response.future, AkkaUtils.askTimeout) + Await.result(response.future, AkkaUtils.askTimeout(conf)) } catch { case e: TimeoutException => (false, s"Master $master failed to respond in time") } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 2deb21aac65d2..1640d5fee0f77 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -2,6 +2,7 @@ package org.apache.spark.deploy.worker import akka.actor._ +import org.apache.spark.SparkConf import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -12,7 +13,7 @@ object DriverWrapper { args.toList match { case workerUrl :: mainClass :: extraArgs => val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0) + Utils.localHostName(), 0, false, new SparkConf()) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class @@ -20,7 +21,7 @@ object DriverWrapper { val mainMethod = clazz.getMethod("main", classOf[Array[String]]) mainMethod.invoke(null, extraArgs.toArray[String]) - actorSystem.awaitTermination() + actorSystem.shutdown() case _ => System.err.println("Usage: DriverWrapper [options]") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 2072f00fae311..4546e3892fbc5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -335,7 +335,7 @@ private[spark] object Worker { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir, conf), name = actorName) + masterUrls, systemName, actorName, workDir, conf), name = actorName) (actorSystem, boundPort) } From b72cceba2727586c1e1f89c58b66417628e1afa7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 6 Jan 2014 22:05:53 -0800 Subject: [PATCH 1220/1571] Some doc fixes --- docs/spark-standalone.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index f426db014188a..7da64749b7837 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -157,8 +157,7 @@ You may also run your application entirely inside of the cluster by submitting y [application-options] cluster-url: The URL of the master node. - application-jar-url: Path to a bundled jar including your application and all dependencies. - Accepts hdfs://, file://, and http:// paths. + application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be visible from inside of your cluster, for instance, in an HDFS directory. main-class: The entry point for your application. Client Options: @@ -170,7 +169,7 @@ Keep in mind that your driver program will be executed on a remote worker machin * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program. * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client. - * _Dependencies_: You'll still need to call `sc.addJar` inside of your driver program to add your application jar and any dependencies. If you submit a local application jar to the client (e.g one with a `file://` URL), it will be uploaded into the working directory of your driver program. Then, you can add it using `sc.addJar("jar-name.jar")`. + * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes. Once you submit a driver program, it will appear in the cluster management UI at port 8080 and be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using From a91f14cfdccd57cb80138d4905930c5b1b82ce1d Mon Sep 17 00:00:00 2001 From: prabeesh Date: Tue, 7 Jan 2014 12:21:20 +0530 Subject: [PATCH 1221/1571] spark -> org.apache.spark --- .../org/apache/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../org/apache/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../org/apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../apache/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../spark/streaming/examples/clickstream/PageViewStream.scala | 4 ++-- 8 files changed, 12 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 546495357f677..4e0058cd70777 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -134,9 +134,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 1486d77d8a807..ea6ea674196a1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 172091be2eed9..197461655e837 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index 74d76ec26c268..6a32c75373a7e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index f43c8ab61d9d2..002db57d592b2 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 89d304212324e..beda73a71b8a2 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -62,9 +62,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index 1a40fdb9a3153..4fe57de4a4058 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -39,8 +39,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 0569846f189a7..807af199f4fd0 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ object PageViewStream { def main(args: Array[String]) { From b590adb2ad06fbb5f38c55aa16369c878d84839e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 6 Jan 2014 23:31:39 -0800 Subject: [PATCH 1222/1571] Put quote arround arguments passed down to system sbt --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index af422d9e891bd..2c19987b92f2e 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -11,7 +11,7 @@ printf "Checking for system sbt [" if hash sbt 2>/dev/null; then printf "FOUND]\n" # Use System SBT - sbt $@ + sbt "$@" else printf "NOT FOUND]\n" # Download sbt or use already downloaded From b3018811e106e6414816380a35c07a8564945d37 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 15:47:40 +0530 Subject: [PATCH 1223/1571] Allow users to set arbitrary akka configurations via spark conf. --- core/src/main/scala/org/apache/spark/SparkConf.scala | 7 +++++++ core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 8 +++++--- docs/configuration.md | 8 ++++++++ 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 55f27033b5579..2d437f1b21363 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -172,6 +172,13 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with .map{case (k, v) => (k.substring(prefix.length), v)} } + /** Get all akka conf variables set on this SparkConf */ + def getAkkaConf: Seq[(String, String)] = { + getAll.filter { + case (k, v) => k.startsWith("akka.") + } + } + /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 7df7e3d8e561c..2ee37815dee21 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,12 +17,13 @@ package org.apache.spark.util +import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory -import org.apache.log4j.{Level, Logger} +import org.apache.log4j.{Level, Logger} import org.apache.spark.SparkConf /** @@ -64,7 +65,8 @@ private[spark] object AkkaUtils { conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt - val akkaConf = ConfigFactory.parseString( + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( + ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] @@ -86,7 +88,7 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents |akka.log-dead-letters = $lifecycleEvents |akka.log-dead-letters-during-shutdown = $lifecycleEvents - """.stripMargin) + """.stripMargin)) val actorSystem = if (indestructible) { IndestructibleActorSystem(name, akkaConf) diff --git a/docs/configuration.md b/docs/configuration.md index 09342fedfc1a0..8a8857bb3b84e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -360,6 +360,14 @@ Apart from these, the following properties are also available, and may be useful Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. + + akka.x.y.... + value + + An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that spark context and its assigned executors as well. + + + spark.shuffle.consolidateFiles false From b84dc780d3ad3fc2f0f1e9ba3fd1676447c65e96 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 15:51:32 +0530 Subject: [PATCH 1224/1571] Allow configuration to be printed in logs for diagnosis. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++++ docs/configuration.md | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e80e43af6d4e7..99dcced7d759b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -116,6 +116,10 @@ class SparkContext( throw new SparkException("An application must be set in your configuration") } + if (conf.get("spark.log-conf", "false").toBoolean) { + logInfo("Spark configuration:\n" + conf.toDebugString) + } + // Set Spark driver host and port system properties conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") diff --git a/docs/configuration.md b/docs/configuration.md index 8a8857bb3b84e..f00fe05476e43 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -403,6 +403,13 @@ Apart from these, the following properties are also available, and may be useful How many times slower a task is than the median to be considered for speculation. + + spark.log-conf + false + + Log the supplied SparkConf as INFO at start of spark context. + + ## Viewing Spark Properties From c729fa7c8ed733a778a7201ed17bf74f3e132845 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 7 Jan 2014 13:08:16 +0530 Subject: [PATCH 1225/1571] formatting related fixes suggested by Patrick. --- core/src/main/scala/org/apache/spark/SparkConf.scala | 6 +----- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 2d437f1b21363..b1665276146f4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -173,11 +173,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with } /** Get all akka conf variables set on this SparkConf */ - def getAkkaConf: Seq[(String, String)] = { - getAll.filter { - case (k, v) => k.startsWith("akka.") - } - } + def getAkkaConf: Seq[(String, String)] = getAll.filter {case (k, v) => k.startsWith("akka.")} /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) diff --git a/docs/configuration.md b/docs/configuration.md index f00fe05476e43..1d36ecb9c1630 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -364,7 +364,7 @@ Apart from these, the following properties are also available, and may be useful akka.x.y.... value - An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that spark context and its assigned executors as well. + An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that SparkContext and its assigned executors as well. From 60a7a6b31a441f5f716091316f90819c62b22e70 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 6 Jan 2014 23:45:27 -0800 Subject: [PATCH 1226/1571] Use awk to extract the version --- project/build.properties | 3 --- sbt/sbt | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/project/build.properties b/project/build.properties index 03438f6558abe..839f5fbb0c347 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,7 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -# Note: If you change the sbt version please also change the sbt download -# script sbt/sbt sbt.version=0.12.4 diff --git a/sbt/sbt b/sbt/sbt index 2c19987b92f2e..22672f2346461 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -2,7 +2,7 @@ # This script launches sbt for this project. If present it uses the system # version of sbt. If there is no system version of sbt it attempts to download # sbt locally. -SBT_VERSION=0.12.4 +SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties` URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar JAR=sbt/sbt-launch-${SBT_VERSION}.jar From 4689ce29fd506d001d15c863ab4fe29bfac90326 Mon Sep 17 00:00:00 2001 From: Luca Rosellini Date: Tue, 7 Jan 2014 09:44:24 +0100 Subject: [PATCH 1227/1571] Added license header and removed @author tag --- .../apache/spark/repl/SparkCommandLine.scala | 19 +++++++++++++++++-- .../spark/repl/SparkRunnerSettings.scala | 19 +++++++++++++++++-- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala index acb1e4ceda8fd..05816941b54b3 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala @@ -1,3 +1,20 @@ +/* + * 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.repl import scala.tools.nsc.{Settings, CompilerCommand} @@ -6,8 +23,6 @@ import scala.Predef._ /** * Command class enabling Spark-specific command line options (provided by * org.apache.spark.repl.SparkRunnerSettings). - * - * @author Luca Rosellini */ class SparkCommandLine(args: List[String], override val settings: Settings) extends CompilerCommand(args, settings) { diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala index 235a688334c37..7fd5fbb42468c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala @@ -1,3 +1,20 @@ +/* + * 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.repl import scala.tools.nsc.Settings @@ -5,8 +22,6 @@ import scala.tools.nsc.Settings /** * scala.tools.nsc.Settings implementation adding Spark-specific REPL * command line options. - * - * @author Luca Rosellini */ class SparkRunnerSettings(error: String => Unit) extends Settings(error){ From 4b92a20232bc24fd858ed4eb7c45462241e36829 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 7 Jan 2014 09:38:45 +0000 Subject: [PATCH 1228/1571] Issue #318 : minor style updates per review from Reynold Xin --- .../src/main/java/org/apache/spark/examples/JavaHdfsLR.java | 5 +---- .../src/main/java/org/apache/spark/examples/JavaKMeans.java | 3 --- .../main/java/org/apache/spark/examples/JavaLogQuery.java | 3 --- .../main/java/org/apache/spark/examples/JavaPageRank.java | 4 ---- .../src/main/java/org/apache/spark/examples/JavaSparkPi.java | 3 --- examples/src/main/java/org/apache/spark/examples/JavaTC.java | 3 --- .../main/java/org/apache/spark/examples/JavaWordCount.java | 3 --- .../main/java/org/apache/spark/mllib/examples/JavaALS.java | 5 +---- .../java/org/apache/spark/mllib/examples/JavaKMeans.java | 3 --- .../main/java/org/apache/spark/mllib/examples/JavaLR.java | 3 --- 10 files changed, 2 insertions(+), 33 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 71bd3b48214f8..d552c47b22231 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -35,9 +35,6 @@ public final class JavaHdfsLR { private static final int D = 10; // Number of dimensions private static final Random rand = new Random(42); - private JavaHdfsLR() { - } - static class DataPoint implements Serializable { DataPoint(double[] x, double y) { this.x = x; @@ -57,7 +54,7 @@ public DataPoint call(String line) { double y = Double.parseDouble(tok[0]); double[] x = new double[D]; for (int i = 0; i < D; i++) { - x[i] = Double.parseDouble(tok[i+1]); + x[i] = Double.parseDouble(tok[i + 1]); } return new DataPoint(x, y); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 0808f33e6a1d8..0dc879275a22a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -36,9 +36,6 @@ public final class JavaKMeans { private static final Pattern SPACE = Pattern.compile(" "); - private JavaKMeans() { - } - /** Parses numbers split by whitespace to a vector */ static Vector parseVector(String line) { String[] splits = SPACE.split(line); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index d45d96d8045d9..9eb1cadd71d22 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -54,9 +54,6 @@ public final class JavaLogQuery { public static final Pattern apacheLogRegex = Pattern.compile( "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); - private JavaLogQuery() { - } - /** Tracks the total query count and number of aggregate bytes for a particular group. */ public static class Stats implements Serializable { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 12d2cce1a7b81..a84245b0c7449 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,6 @@ package org.apache.spark.examples; -import org.apache.spark.SparkContext; import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -43,9 +42,6 @@ public final class JavaPageRank { private static final Pattern SPACES = Pattern.compile("\\s+"); - private JavaPageRank() { - } - private static class Sum extends Function2 { @Override public Double call(Double a, Double b) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index f6ed510e05827..3ec4a58d48ed6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -28,9 +28,6 @@ /** Computes an approximation to pi */ public final class JavaSparkPi { - private JavaSparkPi() { - } - public static void main(String[] args) throws Exception { if (args.length == 0) { System.err.println("Usage: JavaLogQuery [slices]"); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 12b564d1efdcf..2ceb0fd94ba65 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -37,9 +37,6 @@ public final class JavaTC { private static final int numVertices = 100; private static final Random rand = new Random(42); - private JavaTC() { - } - static List> generateGraph() { Set> edges = new HashSet>(numEdges); while (edges.size() < numEdges) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index fc9beb8fe5383..6651f98d56711 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -32,9 +32,6 @@ public final class JavaWordCount { private static final Pattern SPACE = Pattern.compile(" "); - private JavaWordCount() { - } - public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaWordCount "); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index c42d9cb7882da..435a86e62abc5 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -33,10 +33,7 @@ /** * Example using MLLib ALS from Java. */ -public final class JavaALS { - - private JavaALS() { - } +public final class JavaALS { static class ParseRating extends Function { private static final Pattern COMMA = Pattern.compile(","); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 9d10473aed739..4b2658f257b3c 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -32,9 +32,6 @@ */ public final class JavaKMeans { - private JavaKMeans() { - } - static class ParsePoint extends Function { private static final Pattern SPACE = Pattern.compile(" "); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index b057f71e083b8..21586ce817d09 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -34,9 +34,6 @@ */ public final class JavaLR { - private JavaLR() { - } - static class ParsePoint extends Function { private static final Pattern COMMA = Pattern.compile(","); private static final Pattern SPACE = Pattern.compile(" "); From aa99f226a691ddcb4442d60f4cd4908f434cc4ce Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 7 Jan 2014 01:56:15 -0800 Subject: [PATCH 1229/1571] Removed XYZFunctions and added XYZUtils as a common Scala and Java interface for creating XYZ streams. --- .../examples/JavaFlumeEventCount.java | 5 +- .../examples/JavaKafkaWordCount.java | 9 +- .../streaming/examples/FlumeEventCount.scala | 2 +- .../streaming/examples/KafkaWordCount.scala | 2 +- .../streaming/examples/MQTTWordCount.scala | 2 +- .../examples/TwitterAlgebirdCMS.scala | 4 +- .../examples/TwitterAlgebirdHLL.scala | 2 +- .../examples/TwitterPopularTags.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 3 +- .../api/java/flume/FlumeFunctions.scala | 48 ------ ...{FlumeFunctions.scala => FlumeUtils.scala} | 41 ++++- .../spark/streaming/flume/package.scala | 23 --- .../streaming/flume/JavaFlumeStreamSuite.java | 13 +- .../streaming/flume/FlumeStreamSuite.scala | 2 +- .../api/java/kafka/KafkaFunctions.scala | 107 ------------ .../streaming/kafka/KafkaFunctions.scala | 73 --------- .../spark/streaming/kafka/KafkaUtils.scala | 153 ++++++++++++++++++ .../spark/streaming/kafka/package.scala | 23 --- .../streaming/kafka/JavaKafkaStreamSuite.java | 11 +- .../streaming/kafka/KafkaStreamSuite.scala | 8 +- .../spark/streaming/mqtt/MQTTFunctions.scala | 43 ----- .../MQTTUtils.scala} | 54 ++++--- .../apache/spark/streaming/mqtt/package.scala | 24 --- .../streaming/mqtt/JavaMQTTStreamSuite.java | 6 +- .../streaming/mqtt/MQTTStreamSuite.scala | 4 +- .../streaming/twitter/TwitterFunctions.scala | 49 ------ .../TwitterUtils.scala} | 75 ++++++--- .../spark/streaming/twitter/package.scala | 22 --- .../twitter/JavaTwitterStreamSuite.java | 20 +-- .../twitter/TwitterStreamSuite.scala | 13 +- .../streaming/zeromq/ZeroMQFunctions.scala | 57 ------- .../ZeroMQUtils.scala} | 80 +++++---- .../spark/streaming/zeromq/package.scala | 24 --- .../zeromq/JavaZeroMQStreamSuite.java | 17 +- .../streaming/zeromq/ZeroMQStreamSuite.scala | 8 +- 35 files changed, 383 insertions(+), 646 deletions(-) delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala rename external/flume/src/main/scala/org/apache/spark/streaming/flume/{FlumeFunctions.scala => FlumeUtils.scala} (55%) delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala rename external/mqtt/src/main/scala/org/apache/spark/streaming/{api/java/mqtt/MQTTFunctions.scala => mqtt/MQTTUtils.scala} (52%) delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala rename external/twitter/src/main/scala/org/apache/spark/streaming/{api/java/twitter/TwitterFunctions.scala => twitter/TwitterUtils.scala} (53%) delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala rename external/zeromq/src/main/scala/org/apache/spark/streaming/{api/java/zeromq/ZeroMQFunctions.scala => zeromq/ZeroMQUtils.scala} (55%) delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 83900a18df327..0a2b3def183fc 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -20,7 +20,7 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.api.java.flume.FlumeFunctions; +import org.apache.spark.streaming.flume.FlumeUtils; import org.apache.spark.streaming.flume.SparkFlumeEvent; /** @@ -53,8 +53,7 @@ public static void main(String[] args) { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); - FlumeFunctions flumeFunc = new FlumeFunctions(ssc); - JavaDStream flumeStream = flumeFunc.flumeStream("localhost", port); + JavaDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 51de4054cc93e..3bd7a3a90ef26 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,7 +29,7 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; +import org.apache.spark.streaming.kafka.KafkaUtils; import scala.Tuple2; /** @@ -54,7 +54,7 @@ public static void main(String[] args) { } // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", + JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount", new Duration(2000), System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); @@ -65,8 +65,7 @@ public static void main(String[] args) { topicMap.put(topic, numThreads); } - KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); - JavaPairDStream messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap); + JavaPairDStream messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override @@ -96,6 +95,6 @@ public Integer call(Integer i1, Integer i2) throws Exception { }); wordCounts.print(); - ssc.start(); + jssc.start(); } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 149640e0d18cd..ae3709b3d97f5 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -52,7 +52,7 @@ object FlumeEventCount { System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a flume stream - val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) + val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) // Print out the count of events received from this server in each batch stream.count().map(cnt => "Received " + cnt + " flume events." ).print() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 633712e816b27..022c8c5cb90a0 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -53,7 +53,7 @@ object KafkaWordCount { ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2) + val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1l)) .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index f65c3f8b91dba..325290b66f4de 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -97,7 +97,7 @@ object MQTTWordCount { val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY) + val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index a60570f884f87..3ccdc908e23c4 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.twitter._ *

      *

      * - * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure + * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a data structure * for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc), * that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the * estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold @@ -63,7 +63,7 @@ object TwitterAlgebirdCMS { val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 1382fa4d1d0b1..c7e83e76b0057 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -50,7 +50,7 @@ object TwitterAlgebirdHLL { val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 84842b3d65bde..e2b0418d55d2b 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -40,7 +40,7 @@ object TwitterPopularTags { val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters) + val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 789c5f2d0809a..5a7673756e53b 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -85,11 +85,10 @@ object ZeroMQWordCount { def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator //For this stream, a zeroMQ publisher should be running. - val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator) + val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() } - } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala deleted file mode 100644 index 3347d19796a68..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala +++ /dev/null @@ -1,48 +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.streaming.api.java.flume - -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.flume._ -import org.apache.spark.storage.StorageLevel - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Flume input streams. - */ -class FlumeFunctions(javaStreamingContext: JavaStreamingContext) { - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { - javaStreamingContext.ssc.flumeStream(hostname, port) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): - JavaDStream[SparkFlumeEvent] = { - javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel) - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala similarity index 55% rename from external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala rename to external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 35e7a01abc030..834b775d4fd2b 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -18,20 +18,19 @@ package org.apache.spark.streaming.flume import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -/** - * Extra Flume input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversion. Import org.apache.spark.streaming.flume._ to use these functions. - */ -class FlumeFunctions(ssc: StreamingContext) { +object FlumeUtils { /** * Create a input stream from a Flume source. + * @param ssc StreamingContext object * @param hostname Hostname of the slave machine to which the flume data will be sent * @param port Port of the slave machine to which the flume data will be sent * @param storageLevel Storage level to use for storing the received objects */ - def flumeStream ( + def createStream ( + ssc: StreamingContext, hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 @@ -40,4 +39,32 @@ class FlumeFunctions(ssc: StreamingContext) { ssc.registerInputStream(inputStream) inputStream } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port, storageLevel) + } } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala deleted file mode 100644 index c087a39d1cd78..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala +++ /dev/null @@ -1,23 +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.streaming - -package object flume { - implicit def sscToFlumeFunctions(ssc: StreamingContext) = new FlumeFunctions(ssc) -} - diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 5930fee925d2d..733389b98d22d 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -1,4 +1,4 @@ -package org.apache.spark.streaming.flume;/* +/* * 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. @@ -15,21 +15,20 @@ * limitations under the License. */ +package org.apache.spark.streaming.flume; + import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.flume.FlumeFunctions; -import org.apache.spark.streaming.flume.SparkFlumeEvent; + import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { - FlumeFunctions flumeFunc = new FlumeFunctions(ssc); - // tests the API, does not actually test data receiving - JavaDStream test1 = flumeFunc.flumeStream("localhost", 12345); - JavaDStream test2 = flumeFunc.flumeStream("localhost", 12345, + JavaDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); + JavaDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 74840f6499425..2e8e9fac45553 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -39,7 +39,7 @@ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala deleted file mode 100644 index 491331bb37390..0000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala +++ /dev/null @@ -1,107 +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.streaming.api.java.kafka - -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - -import java.lang.{Integer => JInt} -import java.util.{Map => JMap} - -import kafka.serializer.Decoder - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.kafka._ - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Kafka input streams. - */ -class KafkaFunctions(javaStreamingContext: JavaStreamingContext) { - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - * - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. Defaults to memory-only - */ - def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] - - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] - - javaStreamingContext.ssc.kafkaStream[K, V, U, T]( - kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala deleted file mode 100644 index 2135634a69b8b..0000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala +++ /dev/null @@ -1,73 +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.streaming.kafka - -import scala.reflect.ClassTag - -import kafka.serializer.{Decoder, StringDecoder} - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra Kafka input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversion. Import org.apache.spark.streaming.kafka._ to use these functions. - */ -class KafkaFunctions(ssc: StreamingContext) { - /** - * Create an input stream that pulls messages from a Kafka Broker. - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - kafkaStream[String, String, StringDecoder, StringDecoder]( - kafkaParams, - topics, - storageLevel) - } - - /** - * Create an input stream that pulls messages from a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def kafkaStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): DStream[(K, V)] = { - val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 0000000000000..c2d851f94311d --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,153 @@ +/* + * 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.streaming.kafka + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import java.lang.{Integer => JInt} +import java.util.{Map => JMap} + +import kafka.serializer.{Decoder, StringDecoder} + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} + + +object KafkaUtils { + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ + def createStream( + ssc: StreamingContext, + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, storageLevel) + } + + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): DStream[(K, V)] = { + val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). + * @param groupId The group id for this consumer. + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + * + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of RDD + * @param valueTypeClass value type of RDD + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. Defaults to MEMORY_AND_DISK_2. + */ + def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val valueCmt: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + + implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] + implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + + createStream[K, V, U, T]( + jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala deleted file mode 100644 index 44e7ce6e1bd37..0000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala +++ /dev/null @@ -1,23 +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.streaming - -package object kafka { - implicit def sscToKafkaFunctions(ssc: StreamingContext) = new KafkaFunctions(ssc) -} - diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index fdea96e506223..7b4999447ee69 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,32 +18,27 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; - -import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test public void testKafkaStream() { - HashMap topics = Maps.newHashMap(); - KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); // tests the API, does not actually test data receiving - JavaPairDStream test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics); - JavaPairDStream test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics, + JavaPairDStream test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); + JavaPairDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = kafkaFunc.kafkaStream( + JavaPairDStream test3 = KafkaUtils.createStream(ssc, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 2ef3e99c558d2..9c81f23c19118 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -28,11 +28,11 @@ class KafkaStreamSuite extends TestSuiteBase { val topics = Map("my-topic" -> 1) // tests the API, does not actually test data receiving - val test1 = ssc.kafkaStream("localhost:12345", "group", topics) - val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) + val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = ssc.kafkaStream[String, String, StringDecoder, StringDecoder]( - kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala deleted file mode 100644 index 86f4e9c724301..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala +++ /dev/null @@ -1,43 +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.streaming.mqtt - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions. - */ -class MQTTFunctions(ssc: StreamingContext) { - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala similarity index 52% rename from external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala rename to external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index 72124956fcc57..0e6c25dbee8fb 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -15,45 +15,61 @@ * limitations under the License. */ -package org.apache.spark.streaming.api.java.mqtt - -import scala.reflect.ClassTag +package org.apache.spark.streaming.mqtt import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.mqtt._ +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} +import scala.reflect.ClassTag -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating MQTT input streams. - */ -class MQTTFunctions(javaStreamingContext: JavaStreamingContext) { +object MQTTUtils { + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param ssc StreamingContext object + * @param brokerUrl Url of remote MQTT publisher + * @param topic Topic name to subscribe to + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def createStream( + ssc: StreamingContext, + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[String] = { + val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } /** * Create an input stream that receives messages pushed by a MQTT publisher. + * @param jssc JavaStreamingContext object * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to + * @param topic Topic name to subscribe to */ - def mqttStream( + def createStream( + jssc: JavaStreamingContext, brokerUrl: String, topic: String ): JavaDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.mqttStream(brokerUrl, topic) + createStream(jssc.ssc, brokerUrl, topic) } /** * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. + * @param jssc JavaStreamingContext object + * @param brokerUrl Url of remote MQTT publisher + * @param topic Topic name to subscribe to + * @param storageLevel RDD storage level. */ - def mqttStream( + def createStream( + jssc: JavaStreamingContext, brokerUrl: String, topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + storageLevel: StorageLevel ): JavaDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel) + createStream(jssc.ssc, brokerUrl, topic, storageLevel) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala deleted file mode 100644 index 28a944f57e3c3..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala +++ /dev/null @@ -1,24 +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.streaming - -package object mqtt { - implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc) -} - - diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index 3ddb4d084fb5e..44743aaecf986 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -19,7 +19,6 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -29,11 +28,10 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { public void testMQTTStream() { String brokerUrl = "abc"; String topic = "def"; - MQTTFunctions mqttFunc = new MQTTFunctions(ssc); // tests the API, does not actually test data receiving - JavaDStream test1 = mqttFunc.mqttStream(brokerUrl, topic); - JavaDStream test2 = mqttFunc.mqttStream(brokerUrl, topic, + JavaDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); + JavaDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index ab6542918b579..fcc159e85a85b 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -28,8 +28,8 @@ class MQTTStreamSuite extends TestSuiteBase { val topic = "def" // tests the API, does not actually test data receiving - val test1 = ssc.mqttStream(brokerUrl, topic) - val test2 = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic) + val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala deleted file mode 100644 index e91049d9b163c..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala +++ /dev/null @@ -1,49 +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.streaming.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra Twitter input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.twitter._ to use these functions. - */ -class TwitterFunctions(ssc: StreamingContext) { - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth - * authorization; this uses the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Option[Authorization], - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { - val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala similarity index 53% rename from external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala rename to external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala index 22e297a03af21..5e506ffabcfc4 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala @@ -15,29 +15,45 @@ * limitations under the License. */ -package org.apache.spark.streaming.api.java.twitter +package org.apache.spark.streaming.twitter import twitter4j.Status import twitter4j.auth.Authorization - import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.twitter._ -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Twitter input streams. - */ -class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { +object TwitterUtils { + /** + * Create a input stream that returns tweets received from Twitter. + * @param ssc StreamingContext object + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + ssc: StreamingContext, + twitterAuth: Option[Authorization], + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = { + val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } /** * Create a input stream that returns tweets received from Twitter using Twitter4J's default * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and * twitter4j.oauth.accessTokenSecret. + * @param jssc JavaStreamingContext object */ - def twitterStream(): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None) + def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = { + createStream(jssc.ssc, None) } /** @@ -45,10 +61,11 @@ class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and * twitter4j.oauth.accessTokenSecret. + * @param jssc JavaStreamingContext object * @param filters Set of filter strings to get only those tweets that match them */ - def twitterStream(filters: Array[String]): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None, filters) + def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = { + createStream(jssc.ssc, None, filters) } /** @@ -56,44 +73,54 @@ class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them + * @param jssc JavaStreamingContext object + * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ - def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None, filters, storageLevel) + def createStream( + jssc: JavaStreamingContext, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + createStream(jssc.ssc, None, filters, storageLevel) } /** * Create a input stream that returns tweets received from Twitter. + * @param jssc JavaStreamingContext object * @param twitterAuth Twitter4J Authorization */ - def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth)) + def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = { + createStream(jssc.ssc, Some(twitterAuth)) } /** * Create a input stream that returns tweets received from Twitter. + * @param jssc JavaStreamingContext object * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them + * @param filters Set of filter strings to get only those tweets that match them */ - def twitterStream( + def createStream( + jssc: JavaStreamingContext, twitterAuth: Authorization, filters: Array[String] ): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters) + createStream(jssc.ssc, Some(twitterAuth), filters) } /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them + * @param jssc JavaStreamingContext object + * @param twitterAuth Twitter4J Authorization object + * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ - def twitterStream( + def createStream( + jssc: JavaStreamingContext, twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel ): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) } } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala deleted file mode 100644 index 23f82c58859b8..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala +++ /dev/null @@ -1,22 +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.streaming - -package object twitter { - implicit def sscToTwitterFunctions(ssc: StreamingContext) = new TwitterFunctions(ssc) -} diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java index 4564d6cd334c5..e46b4e5c7531d 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java @@ -19,13 +19,10 @@ import java.util.Arrays; -import org.apache.spark.streaming.api.java.twitter.TwitterFunctions; import org.junit.Test; - import twitter4j.Status; import twitter4j.auth.Authorization; import twitter4j.auth.NullAuthorization; - import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; @@ -33,18 +30,17 @@ public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { @Test public void testTwitterStream() { - TwitterFunctions twitterFunc = new TwitterFunctions(ssc); String[] filters = (String[])Arrays.asList("filter1", "filter2").toArray(); Authorization auth = NullAuthorization.getInstance(); // tests the API, does not actually test data receiving - JavaDStream test1 = twitterFunc.twitterStream(); - JavaDStream test2 = twitterFunc.twitterStream(filters); - JavaDStream test3 = - twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = twitterFunc.twitterStream(auth); - JavaDStream test5 = twitterFunc.twitterStream(auth, filters); - JavaDStream test6 = - twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test1 = TwitterUtils.createStream(ssc); + JavaDStream test2 = TwitterUtils.createStream(ssc, filters); + JavaDStream test3 = TwitterUtils.createStream( + ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test4 = TwitterUtils.createStream(ssc, auth); + JavaDStream test5 = TwitterUtils.createStream(ssc, auth, filters); + JavaDStream test6 = TwitterUtils.createStream(ssc, + auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala index d7f6d35e07463..a0a8fe617b134 100644 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -29,12 +29,13 @@ class TwitterStreamSuite extends TestSuiteBase { val authorization: Authorization = NullAuthorization.getInstance() // tests the API, does not actually test data receiving - val test1 = ssc.twitterStream(None) - val test2 = ssc.twitterStream(None, filters) - val test3 = ssc.twitterStream(None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4 = ssc.twitterStream(Some(authorization)) - val test5 = ssc.twitterStream(Some(authorization), filters) - val test6 = ssc.twitterStream(Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = TwitterUtils.createStream(ssc, None) + val test2 = TwitterUtils.createStream(ssc, None, filters) + val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4 = TwitterUtils.createStream(ssc, Some(authorization)) + val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters) + val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters, + StorageLevel.MEMORY_AND_DISK_SER_2) // Note that actually testing the data receiving is hard as authentication keys are // necessary for accessing Twitter live stream diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala deleted file mode 100644 index f4c75ab7c9fc5..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala +++ /dev/null @@ -1,57 +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.streaming.zeromq - -import scala.reflect.ClassTag - -import akka.actor.{Props, SupervisorStrategy} -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ -import org.apache.spark.streaming.receivers._ - -/** - * Extra ZeroMQ input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.zeromq._ to use these functions. - */ -class ZeroMQFunctions(ssc: StreamingContext) { - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic - * and each frame has sequence of byte thus it needs the converter - * (which might be deserializer of bytes) to translate from sequence - * of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to memory-only. - */ - def zeroMQStream[T: ClassTag]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] ⇒ Iterator[T], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { - ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), - "ZeroMQReceiver", storageLevel, supervisorStrategy) - } -} - diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala similarity index 55% rename from external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala rename to external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index a9bbce71f5ff1..546d9df3b5df9 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -15,37 +15,57 @@ * limitations under the License. */ -package org.apache.spark.streaming.api.java.zeromq +package org.apache.spark.streaming.zeromq import scala.reflect.ClassTag import scala.collection.JavaConversions._ - -import akka.actor.SupervisorStrategy +import akka.actor.{Props, SupervisorStrategy} import akka.util.ByteString import akka.zeromq.Subscribe - -import org.apache.spark.storage.StorageLevel import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.zeromq._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating ZeroMQ input streams. - */ -class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { +object ZeroMQUtils { + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param ssc StreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic + * and each frame has sequence of byte thus it needs the converter + * (which might be deserializer of bytes) to translate from sequence + * of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def createStream[T: ClassTag]( + ssc: StreamingContext, + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] ⇒ Iterator[T], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { + ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), + "ZeroMQReceiver", storageLevel, supervisorStrategy) + } /** * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote ZeroMQ publisher - * @param subscribe topic to subscribe to + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote ZeroMQ publisher + * @param subscribe Topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence * of byte thus it needs the converter(which might be deserializer of bytes) * to translate from sequence of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. * @param storageLevel Storage level to use for storing the received objects */ - def zeroMQStream[T]( + def createStream[T]( + jssc: JavaStreamingContext, publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], @@ -54,21 +74,23 @@ class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) } /** * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence * of byte thus it needs the converter(which might be deserializer of bytes) * to translate from sequence of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. + * @param storageLevel RDD storage level. */ - def zeroMQStream[T]( + def createStream[T]( + jssc: JavaStreamingContext, publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], @@ -76,27 +98,29 @@ class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel) } /** * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence * of byte thus it needs the converter(which might be deserializer of bytes) * to translate from sequence of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. */ - def zeroMQStream[T]( + def createStream[T]( + jssc: JavaStreamingContext, publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn) + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn) } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala deleted file mode 100644 index dc2717814925b..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala +++ /dev/null @@ -1,24 +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.streaming - -package object zeromq { - implicit def sscToZeroMQFunctions(ssc: StreamingContext) = new ZeroMQFunctions(ssc) -} - - diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index b020ae4ceff93..d2361e14b898a 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,13 +17,10 @@ package org.apache.spark.streaming.zeromq; -import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions; import org.junit.Test; - import akka.actor.SupervisorStrategy; import akka.util.ByteString; import akka.zeromq.Subscribe; - import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -33,7 +30,6 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @Test // tests the API, does not actually test data receiving public void testZeroMQStream() { - ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc); String publishUrl = "abc"; Subscribe subscribe = new Subscribe((ByteString)null); Function> bytesToObjects = new Function>() { @@ -43,11 +39,12 @@ public Iterable call(byte[][] bytes) throws Exception { } }; - JavaDStream test1 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); + JavaDStream test1 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects); + JavaDStream test2 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test3 = ZeroMQUtils.createStream( + ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), + SupervisorStrategy.defaultStrategy()); } } diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala index 5adcdb821f369..4193b8a02f14a 100644 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -33,10 +33,10 @@ class ZeroMQStreamSuite extends TestSuiteBase { val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] // tests the API, does not actually test data receiving - val test1 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects) - val test2 = ssc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects, + val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) + val test2 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) + val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) // TODO: Actually test data receiving From 3633172e30ab118d2bfd33ce0fdb50a1697904d6 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 7 Jan 2014 12:53:04 -0500 Subject: [PATCH 1230/1571] add the comments about SPARK_WORKER_DIR MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit this env variable seems to be forgotten … --- conf/spark-env.sh.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 0a35ee7c79b12..e2071e2ade8cd 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -18,4 +18,4 @@ # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node - +# - SPARK_WORKER_DIR, to set the working directory of worker processes From 8f02f1c3d45ee553ed6bec2dc81fbae4435274fc Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 7 Jan 2014 11:02:29 -0800 Subject: [PATCH 1231/1571] Fixed examples/pom.xml and run-example based on Patrick's suggestions. --- bin/run-example | 7 +------ examples/pom.xml | 7 +------ 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/bin/run-example b/bin/run-example index dfb4bf7baf0c2..a94913ddefc3e 100755 --- a/bin/run-example +++ b/bin/run-example @@ -45,20 +45,15 @@ fi EXAMPLES_DIR="$FWDIR"/examples SPARK_EXAMPLES_JAR="" if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` fi -if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar` -fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 echo "You need to build Spark with sbt assembly before running this program" >&2 exit 1 fi + # Since the examples JAR ideally shouldn't include spark-core (that dependency should be # "provided"), also add our standard Spark classpath, built using compute-classpath.sh. CLASSPATH=`$FWDIR/bin/compute-classpath.sh` diff --git a/examples/pom.xml b/examples/pom.xml index 1839667feae71..9c776401000c2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -75,31 +75,26 @@ org.apache.spark spark-streaming-twitter_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-kafka_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-flume_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-zeromq_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-mqtt_${scala.binary.version} ${project.version} - provided org.apache.hbase @@ -189,7 +184,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar *:* From d8bcc8e9a095c1b20dd7a17b6535800d39bff80e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 14:35:52 -0500 Subject: [PATCH 1232/1571] Add way to limit default # of cores used by applications on standalone mode Also documents the spark.deploy.spreadOut option. --- .../scala/org/apache/spark/SparkConf.scala | 7 +++- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/master/ApplicationInfo.scala | 7 ++-- .../apache/spark/deploy/master/Master.scala | 8 +++-- docs/configuration.md | 33 ++++++++++++++++--- docs/css/bootstrap.min.css | 2 +- docs/job-scheduling.md | 5 ++- docs/spark-standalone.md | 10 ++++++ 8 files changed, 60 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b1665276146f4..2de32231e8714 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -67,7 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") + for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") set("spark.jars", jars.filter(_ != null).mkString(",")) } @@ -165,6 +165,11 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with getOption(key).map(_.toDouble).getOrElse(defaultValue) } + /** Get a parameter as a boolean, falling back to a default if not set */ + def getBoolean(key: String, defaultValue: Boolean): Boolean = { + getOption(key).map(_.toBoolean).getOrElse(defaultValue) + } + /** Get all executor environment variables set on this SparkConf */ def getExecutorEnv: Seq[(String, String)] = { val prefix = "spark.executorEnv." diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 99dcced7d759b..0e47f4e442927 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -116,7 +116,7 @@ class SparkContext( throw new SparkException("An application must be set in your configuration") } - if (conf.get("spark.log-conf", "false").toBoolean) { + if (conf.get("spark.logConf", "false").toBoolean) { logInfo("Spark configuration:\n" + conf.toDebugString) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 5150b7c7dec6a..1321d9200be2c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,8 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUiUrl: String) + val appUiUrl: String, + defaultCores: Int) extends Serializable { @transient var state: ApplicationState.Value = _ @@ -81,7 +82,9 @@ private[spark] class ApplicationInfo( } } - def coresLeft: Int = desc.maxCores - coresGranted + private val myMaxCores = if (desc.maxCores == Int.MaxValue) defaultCores else desc.maxCores + + def coresLeft: Int = myMaxCores - coresGranted private var _retryCount = 0 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7b696cfcca66e..ee01fb11df5a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -88,7 +88,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.get("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + + // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) + val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -426,7 +429,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) + new ApplicationInfo( + now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores) } def registerApplication(app: ApplicationInfo): Unit = { diff --git a/docs/configuration.md b/docs/configuration.md index 1d36ecb9c1630..52ed59be30e46 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -77,13 +77,14 @@ there are at least five properties that you will commonly want to control: spark.cores.max - (infinite) + (not set) When running on a standalone deploy cluster or a Mesos cluster in "coarse-grained" sharing mode, the maximum amount of CPU cores to request for the application from - across the cluster (not from each machine). The default will use all available cores - offered by the cluster manager. + across the cluster (not from each machine). If not set, the default will be + spark.deploy.defaultCores on Spark's standalone cluster manager, or + infinite (all available cores) on Mesos. @@ -404,12 +405,36 @@ Apart from these, the following properties are also available, and may be useful - spark.log-conf + spark.logConf false Log the supplied SparkConf as INFO at start of spark context. + + spark.deploy.spreadOut + true + + Whether the standalone cluster manager should spread applications out across nodes or try + to consolidate them onto as few nodes as possible. Spreading out is usually better for + data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      + Note: this setting needs to be configured in the cluster master, not in individual + applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + + + + spark.deploy.defaultCores + (infinite) + + Default number of cores to give to applications in Spark's standalone mode if they don't + set spark.cores.max. If not set, applications always get all available + cores unless they configure spark.cores.max themselves. + Set this lower on a shared cluster to prevent users from grabbing + the whole cluster by default.
      + Note: this setting needs to be configured in the cluster master, not in individual + applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + + ## Viewing Spark Properties diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index 119bb5dba7a81..30ea440c008e3 100644 --- a/docs/css/bootstrap.min.css +++ b/docs/css/bootstrap.min.css @@ -6,4 +6,4 @@ * http://www.apache.org/licenses/LICENSE-2.0 * * Designed and built with all the love in the world @twitter by @mdo and @fat. - */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#feecda);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feecda));background-image:-webkit-linear-gradient(top,#fff,#feecda);background-image:-o-linear-gradient(top,#fff,#feecda);background-image:linear-gradient(to bottom,#fff,#feecda);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fffeecda',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} + */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#feead8);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feead8));background-image:-webkit-linear-gradient(top,#fff,#feead8);background-image:-o-linear-gradient(top,#fff,#feead8);background-image:linear-gradient(to bottom,#fff,#feead8);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fffeead8',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 5951155fe3c0e..df2faa5e41b18 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,9 +32,8 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit - the number of nodes an application uses by setting the `spark.cores.max` configuration property in it. This - will allow multiple users/applications to run concurrently. For example, you might launch a long-running - server that uses 10 cores, and allow users to launch shells that use 20 cores each. + the number of nodes an application uses by setting the `spark.cores.max` configuration property in it, + or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c851833a18dd9..f47d41f966e3b 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -167,6 +167,16 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} +In addition, you can configure `spark.deploy.defaultCores` on the cluster master process to change the +default for applications that don't set `spark.cores.max` to something less than infinite. +Do this by adding the following to `conf/spark-env.sh`: + +{% highlight bash %} +export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=" +{% endhighlight %} + +This is useful on shared clusters where users might not have configured a maximum number of cores +individually. # Monitoring and Logging From 80ba9f8ba06e623600469ddb3e59dffcbedee1d0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 7 Jan 2014 12:44:22 -0800 Subject: [PATCH 1233/1571] Get SparkConf from SparkEnv, rather than creating new ones --- core/src/main/scala/org/apache/spark/Aggregator.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 6 +++--- .../spark/util/collection/ExternalAppendOnlyMap.scala | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 292e32e7c8547..08a96b0c34040 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -31,7 +31,7 @@ case class Aggregator[K, V, C] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf private val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 7dc7094aac3ca..b7c7773e5815c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -22,7 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} -import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency, SparkConf} +import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,7 +66,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf private var serializerClass: String = null def setSerializer(cls: String): CoGroupedRDD[K] = { @@ -122,7 +122,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) + val ser = SparkEnv.get.serializerManager.get(serializerClass, sparkConf) val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) rddIterators += ((it, depNum)) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index c348168a8bf09..a5897e8066c4c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,7 +24,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import org.apache.spark.{SparkConf, Logging, SparkEnv} +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} @@ -68,7 +68,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf // Collective memory threshold shared across all running tasks private val maxMemoryThreshold = { From e688e11206401850a13a87d7db52941cc716f88a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 12:42:19 -0800 Subject: [PATCH 1234/1571] Add log4j exclusion rule to maven. To make this work I had to rename the defaults file. Otherwise maven's pattern matching rules included it when trying to match other log4j.properties files. I also fixed a bug in the existing maven build where two tags were present in assembly/pom.xml such that one overwrote the other. --- assembly/pom.xml | 6 +++--- ...ault-log4j.properties => log4j-defaults.properties} | 0 core/src/main/scala/org/apache/spark/Logging.scala | 10 ++++++---- examples/pom.xml | 3 +++ 4 files changed, 12 insertions(+), 7 deletions(-) rename core/src/main/resources/org/apache/spark/{default-log4j.properties => log4j-defaults.properties} (100%) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9b70812c64b1e..54a25910ced7d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -108,12 +108,12 @@ META-INF/services/org.apache.hadoop.fs.FileSystem - - - reference.conf + + log4j.properties +
      diff --git a/core/src/main/resources/org/apache/spark/default-log4j.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties similarity index 100% rename from core/src/main/resources/org/apache/spark/default-log4j.properties rename to core/src/main/resources/org/apache/spark/log4j-defaults.properties diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index d519fc5a29419..4a34989e50e57 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -104,13 +104,15 @@ trait Logging { // If Log4j doesn't seem initialized, load a default properties file val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { - val defaultLogProps = "org/apache/spark/default-log4j.properties" + val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => PropertyConfigurator.configure(url) - case None => System.err.println(s"Spark was unable to load $defaultLogProps") + case Some(url) => + PropertyConfigurator.configure(url) + log.info(s"Using Spark's default log4j profile: $defaultLogProps") + case None => + System.err.println(s"Spark was unable to load $defaultLogProps") } - log.info(s"Using Spark's default log4j profile: $defaultLogProps") } Logging.initialized = true diff --git a/examples/pom.xml b/examples/pom.xml index 7a7032c319b8f..7e41bef2529f8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -203,6 +203,9 @@ reference.conf + + log4j.properties + From 044c8ad3a47d245198a16a68e36a417e80e8c37e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 16:12:20 -0500 Subject: [PATCH 1235/1571] Fix unit test compilation --- .../test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 0b38e239f9b02..810ebf4140338 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -73,7 +73,8 @@ class JsonProtocolSuite extends FunSuite { new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl") } def createAppInfo() : ApplicationInfo = { - new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr") + new ApplicationInfo( + 3, "id", createAppDesc(), new Date(123456789), null, "appUriStr", Int.MaxValue) } def createWorkerInfo() : WorkerInfo = { new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress") From e21a707a13b437327cef25d44ef08ddb2e3931af Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 00:21:43 -0800 Subject: [PATCH 1236/1571] Adding unit tests and some refactoring to promote testability. --- core/pom.xml | 5 + .../spark/deploy/worker/DriverRunner.scala | 88 ++++++++---- .../spark/deploy/worker/ExecutorRunner.scala | 10 +- .../spark/deploy/worker/WorkerWatcher.scala | 14 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../deploy/worker/DriverRunnerTest.scala | 131 ++++++++++++++++++ .../deploy/worker/ExecutorRunnerTest.scala | 4 +- .../deploy/worker/WorkerWatcherSuite.scala | 32 +++++ pom.xml | 12 ++ project/SparkBuild.scala | 1 + 10 files changed, 264 insertions(+), 35 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala diff --git a/core/pom.xml b/core/pom.xml index aac0a9d11e12d..1c52b334d0c6c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -98,6 +98,11 @@ ${akka.group} akka-slf4j_${scala.binary.version}
      + + ${akka.group} + akka-testkit_${scala.binary.version} + test + org.scala-lang scala-library diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index f726089faaabe..d13d7eff0907f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.worker import java.io._ +import scala.collection.JavaConversions._ import scala.collection.mutable.Map import akka.actor.ActorRef @@ -47,6 +48,16 @@ private[spark] class DriverRunner( @volatile var process: Option[Process] = None @volatile var killed = false + // Decoupled for testing + private[deploy] def setClock(_clock: Clock) = clock = _clock + private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + private var clock = new Clock { + def currentTimeMillis(): Long = System.currentTimeMillis() + } + private var sleeper = new Sleeper { + def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) + } + /** Starts a thread to run and manage the driver. */ def start() = { new Thread("DriverRunner for " + driverId) { @@ -63,10 +74,9 @@ private[spark] class DriverRunner( env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" val newCommand = Command(driverDesc.command.mainClass, driverDesc.command.arguments.map(substituteVariables), env) - val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - runCommand(command, env, driverDir, driverDesc.supervise) + launchDriver(command, env, driverDir, driverDesc.supervise) } catch { case e: Exception => exn = Some(e) @@ -116,7 +126,7 @@ private[spark] class DriverRunner( val jarPath = new Path(driverDesc.jarUrl) - val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path + val emptyConf = new Configuration() val jarFileSystem = jarPath.getFileSystem(emptyConf) val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) @@ -136,51 +146,77 @@ private[spark] class DriverRunner( localJarFilename } - /** Launch the supplied command. */ - private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File, - supervise: Boolean) { + private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File, + supervise: Boolean) { + val builder = new ProcessBuilder(command: _*).directory(baseDir) + envVars.map{ case(k,v) => builder.environment().put(k, v) } + + def initialize(process: Process) = { + // Redirect stdout and stderr to files + val stdout = new File(baseDir, "stdout") + CommandUtils.redirectStream(process.getInputStream, stdout) + + val stderr = new File(baseDir, "stderr") + val header = "Launch Command: %s\n%s\n\n".format( + command.mkString("\"", "\" \"", "\""), "=" * 40) + Files.append(header, stderr, Charsets.UTF_8) + CommandUtils.redirectStream(process.getErrorStream, stderr) + } + runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) + } + private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, + supervise: Boolean) { // Time to wait between submission retries. var waitSeconds = 1 // A run of this many seconds resets the exponential back-off. - val successfulRunDuration = 1 + val successfulRunDuration = 5 var keepTrying = !killed while (keepTrying) { - logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\"")) - val builder = new ProcessBuilder(command: _*).directory(baseDir) - envVars.map{ case(k,v) => builder.environment().put(k, v) } + logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\"")) synchronized { if (killed) { return } - - process = Some(builder.start()) - - // Redirect stdout and stderr to files - val stdout = new File(baseDir, "stdout") - CommandUtils.redirectStream(process.get.getInputStream, stdout) - - val stderr = new File(baseDir, "stderr") - val header = "Launch Command: %s\n%s\n\n".format( - command.mkString("\"", "\" \"", "\""), "=" * 40) - Files.append(header, stderr, Charsets.UTF_8) - CommandUtils.redirectStream(process.get.getErrorStream, stderr) + process = Some(command.start()) + initialize(process.get) } - val processStart = System.currentTimeMillis() + val processStart = clock.currentTimeMillis() val exitCode = process.get.waitFor() - if (System.currentTimeMillis() - processStart > successfulRunDuration * 1000) { + if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) { waitSeconds = 1 } if (supervise && exitCode != 0 && !killed) { - waitSeconds = waitSeconds * 2 // exponential back-off logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.") - (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed}) + sleeper.sleep(waitSeconds) + waitSeconds = waitSeconds * 2 // exponential back-off } keepTrying = supervise && exitCode != 0 && !killed } } } + +private[deploy] trait Clock { + def currentTimeMillis(): Long +} + +private[deploy] trait Sleeper { + def sleep(seconds: Int) +} + +// Needed because ProcessBuilder is a final class and cannot be mocked +private[deploy] trait ProcessBuilderLike { + def start(): Process + def command: Seq[String] +} + +private[deploy] object ProcessBuilderLike { + def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike { + def start() = processBuilder.start() + def command = processBuilder.command() + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index fdc9a34886eb8..a9cb998cc2ff4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -98,6 +98,12 @@ private[spark] class ExecutorRunner( case other => other } + def getCommandSeq = { + val command = Command(appDesc.command.mainClass, + appDesc.command.arguments.map(substituteVariables), appDesc.command.environment) + CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) + } + /** * Download and run the executor described in our ApplicationDescription */ @@ -110,9 +116,7 @@ private[spark] class ExecutorRunner( } // Launch the process - val fullCommand = new Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables), appDesc.command.environment) - val command = CommandUtils.buildCommandSeq(fullCommand, memory, sparkHome.getAbsolutePath) + val command = getCommandSeq logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index f4184bc5db17b..0e0d0cd6264cf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -10,7 +10,8 @@ import org.apache.spark.deploy.DeployMessages.SendHeartbeat * Actor which connects to a worker process and terminates the JVM if the connection is severed. * Provides fate sharing between a worker and its associated child processes. */ -private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging { +private[spark] class WorkerWatcher(workerUrl: String) extends Actor + with Logging { override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -19,10 +20,17 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging worker ! SendHeartbeat // need to send a message here to initiate connection } + // Used to avoid shutting down JVM during tests + private[deploy] var isShutDown = false + private[deploy] def setTesting(testing: Boolean) = isTesting = testing + private var isTesting = false + // Lets us filter events only from the worker's actor system private val expectedHostPort = AddressFromURIString(workerUrl).hostPort private def isWorker(address: Address) = address.hostPort == expectedHostPort + def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) + override def receive = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") @@ -32,12 +40,12 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging // These logs may not be seen if the worker (and associated pipe) has died logError(s"Could not initialize connection to worker $workerUrl. Exiting.") logError(s"Error was: $cause") - System.exit(-1) + exitNonZero() case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => // This log message will never be seen logError(s"Lost connection to worker actor $workerUrl. Exiting.") - System.exit(-1) + exitNonZero() case e: AssociationEvent => // pass through association events relating to other remote actor systems diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 372c9f4378988..028196fe8665e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -86,7 +86,7 @@ class JsonProtocolSuite extends FunSuite { ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, - createDriverCommand()) + false, createDriverCommand()) def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date()) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala new file mode 100644 index 0000000000000..45dbcaffae94f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -0,0 +1,131 @@ +package org.apache.spark.deploy.worker + +import java.io.File + +import scala.collection.JavaConversions._ + +import org.mockito.Mockito._ +import org.mockito.Matchers._ +import org.scalatest.FunSuite + +import org.apache.spark.deploy.{Command, DriverDescription} +import org.mockito.stubbing.Answer +import org.mockito.invocation.InvocationOnMock + +class DriverRunnerTest extends FunSuite { + private def createDriverRunner() = { + val command = new Command("mainClass", Seq(), Map()) + val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) + new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, + null, "akka://1.2.3.4/worker/") + } + + private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { + val processBuilder = mock(classOf[ProcessBuilderLike]) + when(processBuilder.command).thenReturn(Seq("mocked", "command")) + val process = mock(classOf[Process]) + when(processBuilder.start()).thenReturn(process) + (processBuilder, process) + } + + test("Process succeeds instantly") { + val runner = createDriverRunner() + + val sleeper = mock(classOf[Sleeper]) + runner.setSleeper(sleeper) + + val (processBuilder, process) = createProcessBuilderAndProcess() + // One failure then a successful run + when(process.waitFor()).thenReturn(0) + runner.runCommandWithRetry(processBuilder, p => (), supervise = true) + + verify(process, times(1)).waitFor() + verify(sleeper, times(0)).sleep(anyInt()) + } + + test("Process failing several times and then succeeding") { + val runner = createDriverRunner() + + val sleeper = mock(classOf[Sleeper]) + runner.setSleeper(sleeper) + + val (processBuilder, process) = createProcessBuilderAndProcess() + // fail, fail, fail, success + when(process.waitFor()).thenReturn(-1).thenReturn(-1).thenReturn(-1).thenReturn(0) + runner.runCommandWithRetry(processBuilder, p => (), supervise = true) + + verify(process, times(4)).waitFor() + verify(sleeper, times(3)).sleep(anyInt()) + verify(sleeper, times(1)).sleep(1) + verify(sleeper, times(1)).sleep(2) + verify(sleeper, times(1)).sleep(4) + } + + test("Process doesn't restart if not supervised") { + val runner = createDriverRunner() + + val sleeper = mock(classOf[Sleeper]) + runner.setSleeper(sleeper) + + val (processBuilder, process) = createProcessBuilderAndProcess() + when(process.waitFor()).thenReturn(-1) + + runner.runCommandWithRetry(processBuilder, p => (), supervise = false) + + verify(process, times(1)).waitFor() + verify(sleeper, times(0)).sleep(anyInt()) + } + + test("Process doesn't restart if killed") { + val runner = createDriverRunner() + + val sleeper = mock(classOf[Sleeper]) + runner.setSleeper(sleeper) + + val (processBuilder, process) = createProcessBuilderAndProcess() + when(process.waitFor()).thenAnswer(new Answer[Int] { + def answer(invocation: InvocationOnMock): Int = { + runner.kill() + -1 + } + }) + + runner.runCommandWithRetry(processBuilder, p => (), supervise = true) + + verify(process, times(1)).waitFor() + verify(sleeper, times(0)).sleep(anyInt()) + } + + test("Reset of backoff counter") { + val runner = createDriverRunner() + + val sleeper = mock(classOf[Sleeper]) + runner.setSleeper(sleeper) + + val clock = mock(classOf[Clock]) + runner.setClock(clock) + + val (processBuilder, process) = createProcessBuilderAndProcess() + + when(process.waitFor()) + .thenReturn(-1) // fail 1 + .thenReturn(-1) // fail 2 + .thenReturn(-1) // fail 3 + .thenReturn(-1) // fail 4 + .thenReturn(0) // success + when(clock.currentTimeMillis()) + .thenReturn(0).thenReturn(1000) // fail 1 (short) + .thenReturn(1000).thenReturn(2000) // fail 2 (short) + .thenReturn(2000).thenReturn(10000) // fail 3 (long) + .thenReturn(10000).thenReturn(11000) // fail 4 (short) + .thenReturn(11000).thenReturn(21000) // success (long) + + runner.runCommandWithRetry(processBuilder, p => (), supervise = true) + + verify(sleeper, times(4)).sleep(anyInt()) + // Expected sequence of sleeps is 1,2,1,2 + verify(sleeper, times(2)).sleep(1) + verify(sleeper, times(2)).sleep(2) + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 7e5aaa3f98af5..bdb2c86d89041 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -31,8 +31,8 @@ class ExecutorRunnerTest extends FunSuite { sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), - f("ooga"), ExecutorState.RUNNING) + f("ooga"), "blah", ExecutorState.RUNNING) - assert(er.buildCommandSeq().last === appId) + assert(er.getCommandSeq.last === appId) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala new file mode 100644 index 0000000000000..94d88d307a163 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -0,0 +1,32 @@ +package org.apache.spark.deploy.worker + + +import akka.testkit.TestActorRef +import org.scalatest.FunSuite +import akka.remote.DisassociatedEvent +import akka.actor.{ActorSystem, AddressFromURIString, Props} + +class WorkerWatcherSuite extends FunSuite { + test("WorkerWatcher shuts down on valid disassociation") { + val actorSystem = ActorSystem("test") + val targetWorkerUrl = "akka://1.2.3.4/user/Worker" + val targetWorkerAddress = AddressFromURIString(targetWorkerUrl) + val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) + val workerWatcher = actorRef.underlyingActor + workerWatcher.setTesting(testing = true) + actorRef.underlyingActor.receive(new DisassociatedEvent(null, targetWorkerAddress, false)) + assert(actorRef.underlyingActor.isShutDown) + } + + test("WorkerWatcher stays alive on invalid disassociation") { + val actorSystem = ActorSystem("test") + val targetWorkerUrl = "akka://1.2.3.4/user/Worker" + val otherAkkaURL = "akka://4.3.2.1/user/OtherActor" + val otherAkkaAddress = AddressFromURIString(otherAkkaURL) + val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) + val workerWatcher = actorRef.underlyingActor + workerWatcher.setTesting(testing = true) + actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) + assert(!actorRef.underlyingActor.isShutDown) + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 78d2f162b5d02..7b734c5371f3f 100644 --- a/pom.xml +++ b/pom.xml @@ -269,6 +269,18 @@ + + ${akka.group} + akka-testkit_${scala.binary.version} + ${akka.version} + test + + + org.jboss.netty + netty + + + it.unimi.dsi fastutil diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 051e5105f3909..bd5f3f79c9020 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -233,6 +233,7 @@ object SparkBuild extends Build { "org.ow2.asm" % "asm" % "4.0", "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", From 2c421749eae1e3945ca34ce006addd98a0c1a00b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 19:30:23 -0500 Subject: [PATCH 1237/1571] Address review comments --- .../org/apache/spark/deploy/ApplicationDescription.scala | 2 +- .../scala/org/apache/spark/deploy/client/TestClient.scala | 2 +- .../org/apache/spark/deploy/master/ApplicationInfo.scala | 2 +- .../main/scala/org/apache/spark/deploy/master/Master.scala | 3 +++ .../spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- .../scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 2 +- .../org/apache/spark/deploy/worker/ExecutorRunnerTest.scala | 2 +- docs/configuration.md | 4 ++-- 8 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 19d393a0dbce7..e38459b883b66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy private[spark] class ApplicationDescription( val name: String, - val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */ + val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, val sparkHome: String, diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index ef649fd80ca52..28ebbdc66bbb8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -48,7 +48,7 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = new SparkConf) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 1321d9200be2c..3e26379166885 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -82,7 +82,7 @@ private[spark] class ApplicationInfo( } } - private val myMaxCores = if (desc.maxCores == Int.MaxValue) defaultCores else desc.maxCores + private val myMaxCores = desc.maxCores.getOrElse(defaultCores) def coresLeft: Int = myMaxCores - coresGranted diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index ee01fb11df5a9..6617b7100f44b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -92,6 +92,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + if (defaultCores < 1) { + throw new SparkException("spark.deploy.defaultCores must be positive") + } override def preStart() { logInfo("Starting Spark master at " + masterUrl) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9858717d134a5..73fc37444e18f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -38,7 +38,7 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOption("spark.cores.max").map(_.toInt) override def start() { super.start() diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 810ebf4140338..331fa3a642967 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -70,7 +70,7 @@ class JsonProtocolSuite extends FunSuite { def createAppDesc() : ApplicationDescription = { val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) - new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl") + new ApplicationDescription("name", Some(4), 1234, cmd, "sparkHome", "appUiUrl") } def createAppInfo() : ApplicationInfo = { new ApplicationInfo( diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 7e5aaa3f98af5..be93074b7b3b0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,7 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get - val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), diff --git a/docs/configuration.md b/docs/configuration.md index 52ed59be30e46..1d6c3d16333c5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -418,7 +418,7 @@ Apart from these, the following properties are also available, and may be useful Whether the standalone cluster manager should spread applications out across nodes or try to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      - Note: this setting needs to be configured in the cluster master, not in individual + Note: this setting needs to be configured in the standalone cluster master, not in individual applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. @@ -431,7 +431,7 @@ Apart from these, the following properties are also available, and may be useful cores unless they configure spark.cores.max themselves. Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
      - Note: this setting needs to be configured in the cluster master, not in individual + Note: this setting needs to be configured in the standalone cluster master, not in individual applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. From 86ed1ad2520662f4a16e535cc05bf2296e6053df Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 7 Jan 2014 16:39:37 -0800 Subject: [PATCH 1238/1571] Fix BlockManagerSuite#after --- .../org/apache/spark/storage/BlockManagerSuite.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index a0fc3445be422..fded582640a6e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -40,8 +40,6 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var actorSystem: ActorSystem = null var master: BlockManagerMaster = null var oldArch: String = null - var oldOops: String = null - var oldHeartBeat: String = null // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -61,7 +59,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - System.setProperty("os.arch", "amd64") + oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") @@ -94,11 +92,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.clearProperty("os.arch") } - if (oldOops != null) { - conf.set("spark.test.useCompressedOops", oldOops) - } else { - System.clearProperty("spark.test.useCompressedOops") - } + System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { From 82a1d38aea3b10930a2659b9c0e7ad2fb2c2ab4a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 17:46:02 -0800 Subject: [PATCH 1239/1571] Simplify and fix pyspark script. This patch removes compatibility for IPython < 1.0 but fixes the launch script and makes it much simpler. I tested this using the three commands in the PySpark documentation page: 1. IPYTHON=1 ./pyspark 2. IPYTHON_OPTS="notebook" ./pyspark 3. IPYTHON_OPTS="notebook --pylab inline" ./pyspark There are two changes: - We rely on PYTHONSTARTUP env var to start PySpark - Removed the quotes around $IPYTHON_OPTS... having quotes gloms them together as a single argument passed to `exec` which seemed to cause ipython to fail (it instead expects them as multiple arguments). --- bin/pyspark | 8 +------- docs/python-programming-guide.md | 5 +++-- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index d6810f4686bf5..a70da5674e1c7 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -51,7 +51,6 @@ export PYSPARK_PYTHON export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: -export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py if [ -n "$IPYTHON_OPTS" ]; then @@ -59,12 +58,7 @@ if [ -n "$IPYTHON_OPTS" ]; then fi if [[ "$IPYTHON" = "1" ]] ; then - # IPython <1.0.0 doesn't honor PYTHONSTARTUP, while 1.0.0+ does. - # Hence we clear PYTHONSTARTUP and use the -c "%run $IPYTHONSTARTUP" command which works on all versions - # We also force interactive mode with "-i" - IPYTHONSTARTUP=$PYTHONSTARTUP - PYTHONSTARTUP= - exec ipython "$IPYTHON_OPTS" -i -c "%run $IPYTHONSTARTUP" + exec ipython $IPYTHON_OPTS else exec "$PYSPARK_PYTHON" "$@" fi diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index dc187b3efec9b..c4236f83124b2 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -99,8 +99,9 @@ $ MASTER=local[4] ./bin/pyspark ## IPython -It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. -To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`: +It is also possible to launch PySpark in [IPython](http://ipython.org), the +enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To +use IPython, set the `IPYTHON` variable to `1` when running `bin/pyspark`: {% highlight bash %} $ IPYTHON=1 ./bin/pyspark From 226b58ada2a3a658420ad90f743d8873f1c30445 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 7 Jan 2014 21:07:27 -0800 Subject: [PATCH 1240/1571] Add ASF header to the new sbt script. --- sbt/sbt | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sbt/sbt b/sbt/sbt index 22672f2346461..7f47d90cf11bb 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,4 +1,22 @@ #!/bin/bash + +# +# 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. +# + # This script launches sbt for this project. If present it uses the system # version of sbt. If there is no system version of sbt it attempts to download # sbt locally. From d1f2805712b4006c1f66f95ba14430f642ed8037 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 8 Jan 2014 11:36:26 +0530 Subject: [PATCH 1241/1571] Update README.md The link does not work otherwise. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b91e4cf86713b..c840a68f76b17 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ locally with one thread, or "local[N]" to run locally with N threads. ## Running tests -Testing first requires [Building](#Building) Spark. Once Spark is built, tests +Testing first requires [Building](#building) Spark. Once Spark is built, tests can be run using: `./sbt/sbt test` From 4517326ec68d15f7d9bedade5f9bba33d760e6b4 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 7 Jan 2014 22:55:56 -0800 Subject: [PATCH 1242/1571] Remove calls to deprecated mapred's OutputCommitter.cleanupJob because since Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job. In fact the implementation of mapred OutputCommitter.commitJob looks like this: public void commitJob(JobContext jobContext) throws IOException { cleanupJob(jobContext); } (The jobContext input argument is type of org.apache.hadoop.mapred.JobContext) --- core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala | 4 ---- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 -- 2 files changed, 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 103a1c205185f..618d95015f747 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -127,10 +127,6 @@ class SparkHadoopWriter(@transient jobConf: JobConf) cmtr.commitJob(getJobContext()) } - def cleanup() { - getOutputCommitter().cleanupJob(getJobContext()) - } - // ********* Private Functions ********* private def getOutputFormat(): OutputFormat[AnyRef,AnyRef] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 04a8d05988f16..629fb390af536 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -638,7 +638,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) jobCommitter.setupJob(jobTaskContext) val count = self.context.runJob(self, writeShard _).sum jobCommitter.commitJob(jobTaskContext) - jobCommitter.cleanupJob(jobTaskContext) } /** @@ -728,7 +727,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) self.context.runJob(self, writeToFile _) writer.commitJob() - writer.cleanup() } /** From f6b6f88367351f99d02a7de0dbd5c1980cc97bbf Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 7 Jan 2014 23:23:17 -0800 Subject: [PATCH 1243/1571] Set boolean param name for two files call to SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set. --- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2662d48c84a2e..73d15b90822ad 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -76,7 +76,7 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0) + val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance if (format.isInstanceOf[Configurable]) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 04a8d05988f16..c8446fd0fc9ca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -613,7 +613,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.partitionId, attemptNumber) + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance val committer = format.getOutputCommitter(hadoopContext) @@ -632,7 +632,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * however we're only going to use this local OutputCommitter for * setupJob/commitJob, so we just use a dummy "map" task. */ - val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0) + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) From 3d939e5fe8930368b46eb49659a9cca8f41c8768 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 23:27:18 -0800 Subject: [PATCH 1244/1571] Adding --verbose option to DriverClient --- .../spark/deploy/client/DriverClient.scala | 18 ++++++++++++++---- .../deploy/client/DriverClientArguments.scala | 8 ++++++++ 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala index 1cd5d99bcf779..8b066ba1a5832 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala @@ -22,12 +22,15 @@ import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ +import akka.actor.Actor -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.log4j.{Logger, Level} +import akka.remote.RemotingLifecycleEvent /** * Actor that sends a single message to the standalone master and returns the response in the @@ -55,12 +58,18 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends /** * Executable utility for starting and terminating drivers inside of a standalone cluster. */ -object DriverClient extends Logging { +object DriverClient { def main(args: Array[String]) { val driverArgs = new DriverClientArguments(args) val conf = new SparkConf() + if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { + conf.set("spark.akka.logLifecycleEvents", "true") + } + conf.set("spark.akka.askTimeout", "5") + Logger.getRootLogger.setLevel(driverArgs.logLevel) + // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( @@ -69,6 +78,7 @@ object DriverClient extends Logging { val response = promise[(Boolean, String)] val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response))) + println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}") driverArgs.cmd match { case "launch" => // TODO: We could add an env variable here and intercept it in `sc.addJar` that would @@ -98,9 +108,9 @@ object DriverClient extends Logging { try { Await.result(response.future, AkkaUtils.askTimeout(conf)) } catch { - case e: TimeoutException => (false, s"Master $master failed to respond in time") + case e: TimeoutException => (false, s"Error: Timed out sending message to $master") } - if (success) logInfo(message) else logError(message) + println(message) actorSystem.shutdown() actorSystem.awaitTermination() } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala index d9e1c8a1b0b32..7774a5615cee5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.client import scala.collection.mutable.ListBuffer +import org.apache.log4j.Level + /** * Command-line parser for the driver client. */ @@ -27,6 +29,7 @@ private[spark] class DriverClientArguments(args: Array[String]) { val defaultMemory = 512 var cmd: String = "" // 'launch' or 'kill' + var logLevel = Level.WARN // launch parameters var master: String = "" @@ -59,6 +62,10 @@ private[spark] class DriverClientArguments(args: Array[String]) { case ("--help" | "-h") :: tail => printUsageAndExit(0) + case ("--verbose" | "-v") :: tail => + logLevel = Level.INFO + parse(tail) + case "launch" :: _master :: _jarUrl :: _mainClass :: tail => cmd = "launch" master = _master @@ -90,6 +97,7 @@ private[spark] class DriverClientArguments(args: Array[String]) { | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) | -s, --supervise Whether to restart the driver on failure + | -v, --verbose Print more debugging output """.stripMargin System.err.println(usage) System.exit(exitCode) From d0533f704681adccc8fe2b814dc9e5082646057a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 23:38:46 -0800 Subject: [PATCH 1245/1571] Rename to Client --- .../deploy/{client/DriverClient.scala => Client.scala} | 9 +++------ ...DriverClientArguments.scala => ClientArguments.scala} | 4 ++-- 2 files changed, 5 insertions(+), 8 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/{client/DriverClient.scala => Client.scala} (94%) rename core/src/main/scala/org/apache/spark/deploy/{client/DriverClientArguments.scala => ClientArguments.scala} (96%) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala rename to core/src/main/scala/org/apache/spark/deploy/Client.scala index 8b066ba1a5832..0475bb17c010c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -15,22 +15,19 @@ * limitations under the License. */ -package org.apache.spark.deploy.client +package org.apache.spark.deploy import scala.collection.JavaConversions._ import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ -import akka.actor.Actor +import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.log4j.{Logger, Level} -import akka.remote.RemotingLifecycleEvent /** * Actor that sends a single message to the standalone master and returns the response in the @@ -61,7 +58,7 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends object DriverClient { def main(args: Array[String]) { - val driverArgs = new DriverClientArguments(args) + val driverArgs = new ClientArguments(args) val conf = new SparkConf() if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala rename to core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 7774a5615cee5..50b92e1eabeca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.client +package org.apache.spark.deploy import scala.collection.mutable.ListBuffer @@ -24,7 +24,7 @@ import org.apache.log4j.Level /** * Command-line parser for the driver client. */ -private[spark] class DriverClientArguments(args: Array[String]) { +private[spark] class ClientArguments(args: Array[String]) { val defaultCores = 1 val defaultMemory = 512 From c78b381e91c9902a1510a2ed4ec5c898b51adfe8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 23:56:04 -0800 Subject: [PATCH 1246/1571] Fixes --- core/src/main/scala/org/apache/spark/deploy/Client.scala | 2 +- .../org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +- docs/spark-standalone.md | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 0475bb17c010c..43b9b1cff9b9f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -55,7 +55,7 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends /** * Executable utility for starting and terminating drivers inside of a standalone cluster. */ -object DriverClient { +object Client { def main(args: Array[String]) { val driverArgs = new ClientArguments(args) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index a9cb998cc2ff4..18885d7ca6daa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -100,7 +100,7 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7da64749b7837..ecd642cc60378 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -151,19 +151,20 @@ You can also pass an option `-c ` to control the number of cores that You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows: - ./spark-class org.apache.spark.deploy.client.DriverClient launch + ./spark-class org.apache.spark.deploy.Client launch [client-options] \ \ [application-options] cluster-url: The URL of the master node. - application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be visible from inside of your cluster, for instance, in an HDFS directory. + application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. main-class: The entry point for your application. Client Options: --memory (amount of memory, in MB, allocated for your driver program) --cores (number of cores allocated for your driver program) --supervise (whether to automatically restart your driver on application or node failure) + --verbose (prints increased logging output) Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways: From 3ec21f2eee0a322bcb5cce9678a816b012227a61 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 00:30:10 -0800 Subject: [PATCH 1247/1571] Show more helpful information in UI --- .../spark/deploy/master/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 16 ++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 1 + .../spark/deploy/worker/ui/IndexPage.scala | 8 ++++++-- 4 files changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index a72d76be521ca..db1dde59d6c9d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -162,7 +162,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {driver.id} {driver.submitDate} - {driver.worker.map(w => w.id.toString).getOrElse("None")} + {driver.worker.map(w => {w.id.toString}).getOrElse("None")} {driver.state} {driver.desc.cores} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index d13d7eff0907f..ad70345a7f599 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -32,6 +32,7 @@ import org.apache.spark.Logging import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState +import org.apache.spark.deploy.master.DriverState.DriverState /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -48,6 +49,10 @@ private[spark] class DriverRunner( @volatile var process: Option[Process] = None @volatile var killed = false + // Populated once finished + var finalState: Option[DriverState] = None + var finalException: Option[Exception] = None + // Decoupled for testing private[deploy] def setClock(_clock: Clock) = clock = _clock private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper @@ -62,8 +67,6 @@ private[spark] class DriverRunner( def start() = { new Thread("DriverRunner for " + driverId) { override def run() { - var exn: Option[Exception] = None - try { val driverDir = createWorkingDirectory() val localJarFilename = downloadUserJar(driverDir) @@ -79,15 +82,16 @@ private[spark] class DriverRunner( launchDriver(command, env, driverDir, driverDesc.supervise) } catch { - case e: Exception => exn = Some(e) + case e: Exception => finalException = Some(e) } - val finalState = + val state = if (killed) { DriverState.KILLED } - else if (exn.isDefined) { DriverState.FAILED } + else if (finalException.isDefined) { DriverState.FAILED } else { DriverState.FINISHED } + finalState = Some(state) - worker ! DriverStateChanged(driverId, finalState, exn) + worker ! DriverStateChanged(driverId, state, finalException) } }.start() } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 4546e3892fbc5..2a2b7a38810b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -31,6 +31,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} +import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 93c6ad49d7a12..0a7f56e2d310a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.worker.ui import scala.concurrent.Await -import scala.concurrent.duration._ import scala.xml.Node import akka.pattern.ask @@ -27,6 +26,7 @@ import net.liftweb.json.JsonAST.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} +import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils @@ -52,7 +52,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val finishedExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors) - val driverHeaders = Seq("DriverID", "Main Class", "Cores", "Memory", "Logs") + val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes") val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers) val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse @@ -134,6 +134,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {driver.driverId} {driver.driverDesc.command.mainClass} + {driver.finalState.getOrElse(DriverState.RUNNING)} {driver.driverDesc.cores.toString} @@ -144,6 +145,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { stdout stderr + + {driver.finalException.getOrElse("")} + } } From aa56585d2148b3ced506d2fff89da0858300928c Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Wed, 8 Jan 2014 00:38:29 -0800 Subject: [PATCH 1248/1571] Resolve PR review over 100 chars --- .../src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index c8446fd0fc9ca..4fe3bc552beef 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -613,7 +613,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance val committer = format.getOutputCommitter(hadoopContext) From 62b08faac5278d289bdaefb42ff6f65b62ce48aa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 00:45:41 -0800 Subject: [PATCH 1249/1571] Adding mockito to maven build --- core/pom.xml | 5 +++++ pom.xml | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 1c52b334d0c6c..9e5a450d57a47 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -170,6 +170,11 @@ scalatest_${scala.binary.version} test
      + + org.mockito + mockito-all + test + org.scalacheck scalacheck_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 9f08e8ad4e714..c2b1a7795a256 100644 --- a/pom.xml +++ b/pom.xml @@ -351,6 +351,12 @@ 1.9.1 test + + org.mockito + mockito-all + test + 1.8.5 + commons-io commons-io From 3209a86f3955e5f94834c44dfea8c7df947b029f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 01:18:19 -0800 Subject: [PATCH 1250/1571] Add CDH Repository to Maven Build --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index c8e373772eb0e..b338a957f3877 100644 --- a/pom.xml +++ b/pom.xml @@ -145,6 +145,11 @@ false + + cloudera-repo + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos + From 277b4a36c580e88d9ba60c5efe63753350197874 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 8 Jan 2014 16:32:55 +0530 Subject: [PATCH 1251/1571] we clone hadoop key and values by default and reuse if specified. --- .../scala/org/apache/spark/SparkContext.scala | 68 +++++++++++-------- .../org/apache/spark/rdd/HadoopRDD.scala | 22 ++++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 15 +++- .../scala/org/apache/spark/util/Utils.scala | 23 ++++++- 4 files changed, 87 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0e47f4e442927..97fec7f737dd7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -341,25 +341,27 @@ class SparkContext( * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, * etc). */ - def hadoopRDD[K, V]( + def hadoopRDD[K: ClassTag, V: ClassTag]( conf: JobConf, inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int = defaultMinSplits + minSplits: Int = defaultMinSplits, + cloneKeyValues: Boolean = true ): RDD[(K, V)] = { // Add necessary security credentials to the JobConf before broadcasting it. SparkHadoopUtil.get.addCredentials(conf) - new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) + new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits, cloneKeyValues) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat */ - def hadoopFile[K, V]( + def hadoopFile[K: ClassTag, V: ClassTag]( path: String, inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int = defaultMinSplits + minSplits: Int = defaultMinSplits, + cloneKeyValues: Boolean = true ): RDD[(K, V)] = { // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration)) @@ -371,7 +373,8 @@ class SparkContext( inputFormatClass, keyClass, valueClass, - minSplits) + minSplits, + cloneKeyValues) } /** @@ -382,14 +385,15 @@ class SparkContext( * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits) * }}} */ - def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int) - (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) - : RDD[(K, V)] = { + def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int, + cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F] + ): RDD[(K, V)] = { hadoopFile(path, fm.runtimeClass.asInstanceOf[Class[F]], km.runtimeClass.asInstanceOf[Class[K]], vm.runtimeClass.asInstanceOf[Class[V]], - minSplits) + minSplits, + cloneKeyValues = cloneKeyValues) } /** @@ -400,61 +404,67 @@ class SparkContext( * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path) * }}} */ - def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) + def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = - hadoopFile[K, V, F](path, defaultMinSplits) + hadoopFile[K, V, F](path, defaultMinSplits, cloneKeyValues) /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */ - def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String) - (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { + def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String, + cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F] + ): RDD[(K, V)] = { newAPIHadoopFile( path, fm.runtimeClass.asInstanceOf[Class[F]], km.runtimeClass.asInstanceOf[Class[K]], - vm.runtimeClass.asInstanceOf[Class[V]]) + vm.runtimeClass.asInstanceOf[Class[V]], + cloneKeyValues = cloneKeyValues) } /** * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. */ - def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]( + def newAPIHadoopFile[K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]]( path: String, fClass: Class[F], kClass: Class[K], vClass: Class[V], - conf: Configuration = hadoopConfiguration): RDD[(K, V)] = { + conf: Configuration = hadoopConfiguration, + cloneKeyValues: Boolean = true): RDD[(K, V)] = { val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) val updatedConf = job.getConfiguration - new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf) + new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf, cloneKeyValues) } /** * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. */ - def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]]( + def newAPIHadoopRDD[K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]]( conf: Configuration = hadoopConfiguration, fClass: Class[F], kClass: Class[K], - vClass: Class[V]): RDD[(K, V)] = { - new NewHadoopRDD(this, fClass, kClass, vClass, conf) + vClass: Class[V], + cloneKeyValues: Boolean = true): RDD[(K, V)] = { + new NewHadoopRDD(this, fClass, kClass, vClass, conf, cloneKeyValues) } /** Get an RDD for a Hadoop SequenceFile with given key and value types. */ - def sequenceFile[K, V](path: String, + def sequenceFile[K: ClassTag, V: ClassTag](path: String, keyClass: Class[K], valueClass: Class[V], - minSplits: Int + minSplits: Int, + cloneKeyValues: Boolean = true ): RDD[(K, V)] = { val inputFormatClass = classOf[SequenceFileInputFormat[K, V]] - hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits) + hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits, cloneKeyValues) } /** Get an RDD for a Hadoop SequenceFile with given key and value types. */ - def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] = - sequenceFile(path, keyClass, valueClass, defaultMinSplits) + def sequenceFile[K: ClassTag, V: ClassTag](path: String, keyClass: Class[K], valueClass: Class[V], + cloneKeyValues: Boolean = true): RDD[(K, V)] = + sequenceFile(path, keyClass, valueClass, defaultMinSplits, cloneKeyValues) /** * Version of sequenceFile() for types implicitly convertible to Writables through a @@ -472,8 +482,8 @@ class SparkContext( * for the appropriate type. In addition, we pass the converter a ClassTag of its type to * allow it to figure out the Writable class to use in the subclass case. */ - def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits) - (implicit km: ClassTag[K], vm: ClassTag[V], + def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits, + cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]) : RDD[(K, V)] = { val kc = kcf() @@ -481,7 +491,7 @@ class SparkContext( val format = classOf[SequenceFileInputFormat[Writable, Writable]] val writables = hadoopFile(path, format, kc.writableClass(km).asInstanceOf[Class[Writable]], - vc.writableClass(vm).asInstanceOf[Class[Writable]], minSplits) + vc.writableClass(vm).asInstanceOf[Class[Writable]], minSplits, cloneKeyValues) writables.map{case (k,v) => (kc.convert(k), vc.convert(v))} } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 53f77a38f55f6..13949a1bdb464 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -19,7 +19,9 @@ package org.apache.spark.rdd import java.io.EOFException -import org.apache.hadoop.mapred.FileInputFormat +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.{Configuration, Configurable} import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf @@ -31,7 +33,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator -import org.apache.hadoop.conf.{Configuration, Configurable} +import org.apache.spark.util.Utils.cloneWritables /** @@ -62,14 +64,15 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ -class HadoopRDD[K, V]( +class HadoopRDD[K: ClassTag, V: ClassTag]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], initLocalJobConfFuncOpt: Option[JobConf => Unit], inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int) + minSplits: Int, + cloneKeyValues: Boolean) extends RDD[(K, V)](sc, Nil) with Logging { def this( @@ -78,7 +81,8 @@ class HadoopRDD[K, V]( inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int) = { + minSplits: Int, + cloneKeyValues: Boolean) = { this( sc, sc.broadcast(new SerializableWritable(conf)) @@ -87,7 +91,7 @@ class HadoopRDD[K, V]( inputFormatClass, keyClass, valueClass, - minSplits) + minSplits, cloneKeyValues) } protected val jobConfCacheKey = "rdd_%d_job_conf".format(id) @@ -169,7 +173,11 @@ class HadoopRDD[K, V]( case eof: EOFException => finished = true } - (key, value) + if (cloneKeyValues) { + (cloneWritables(key, getConf), cloneWritables(value, getConf)) + } else { + (key, value) + } } override def close() { diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2662d48c84a2e..5428fc5691d6e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -20,11 +20,14 @@ package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date +import scala.reflect.ClassTag + import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.util.Utils.cloneWritables private[spark] @@ -36,12 +39,13 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS override def hashCode(): Int = (41 * (41 + rddId) + index) } -class NewHadoopRDD[K, V]( +class NewHadoopRDD[K: ClassTag, V: ClassTag]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - @transient conf: Configuration) + @transient conf: Configuration, + cloneKeyValues: Boolean) extends RDD[(K, V)](sc, Nil) with SparkHadoopMapReduceUtil with Logging { @@ -105,7 +109,12 @@ class NewHadoopRDD[K, V]( throw new java.util.NoSuchElementException("End of stream") } havePair = false - (reader.getCurrentKey, reader.getCurrentValue) + val key = reader.getCurrentKey + val value = reader.getCurrentValue + if (cloneKeyValues) { + (cloneWritables(key, conf), cloneWritables(value, conf)) + } else + (key, value) } private def close() { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 5f1253100b338..192806e1782c1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,23 +26,42 @@ import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} +import org.apache.hadoop.io._ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer -import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging} +import org.apache.spark.{SparkConf, SparkException, Logging} /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { + + /** + * We try to clone for most common types of writables and we call WritableUtils.clone otherwise + * intention is to optimize, for example for NullWritable there is no need and for Long, int and + * String creating a new object with value set would be faster. + */ + def cloneWritables[T: ClassTag](obj: T, conf: Configuration): T = { + val cloned = classTag[T] match { + case ClassTag(_: Text) => new Text(obj.asInstanceOf[Text].getBytes) + case ClassTag(_: LongWritable) => new LongWritable(obj.asInstanceOf[LongWritable].get) + case ClassTag(_: IntWritable) => new IntWritable(obj.asInstanceOf[IntWritable].get) + case ClassTag(_: NullWritable) => obj // TODO: should we clone this ? + case _ => WritableUtils.clone(obj.asInstanceOf[Writable], conf) // slower way of cloning. + } + cloned.asInstanceOf[T] + } + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() From 0b7a132d03d5a0106d85a8cca1ab28d6af9c8b55 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 8 Jan 2014 03:22:06 -0800 Subject: [PATCH 1252/1571] Modified checkpoing file clearing policy. --- .../apache/spark/streaming/Checkpoint.scala | 7 ++- .../org/apache/spark/streaming/DStream.scala | 15 +++-- .../streaming/DStreamCheckpointData.scala | 63 +++++++++++++------ .../apache/spark/streaming/DStreamGraph.scala | 30 +++++---- .../streaming/dstream/FileInputDStream.scala | 8 +-- .../streaming/scheduler/JobGenerator.scala | 23 +++++-- .../spark/streaming/CheckpointSuite.scala | 10 +-- 7 files changed, 104 insertions(+), 52 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 09b184b9cfb65..155d5bc02eca9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -28,6 +28,7 @@ import org.apache.spark.{SparkException, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.MetadataCleaner import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.streaming.scheduler.JobGenerator private[streaming] @@ -58,7 +59,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) * Convenience class to handle the writing of graph checkpoint to file */ private[streaming] -class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends Logging { +class CheckpointWriter(jobGenerator: JobGenerator, checkpointDir: String, hadoopConf: Configuration) extends Logging { val file = new Path(checkpointDir, "graph") val MAX_ATTEMPTS = 3 val executor = Executors.newFixedThreadPool(1) @@ -80,7 +81,7 @@ class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") + logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") // This is inherently thread unsafe, so alleviating it by writing to '.new' and // then moving it to the final file val fos = fs.create(writeFile) @@ -96,6 +97,7 @@ class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds") + jobGenerator.onCheckpointCompletion(checkpointTime) return } catch { case ioe: IOException => @@ -116,6 +118,7 @@ class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends bos.close() try { executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + logInfo("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index a78d3965ee94e..20074249d74a9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -331,13 +331,12 @@ abstract class DStream[T: ClassTag] ( * implementation clears the old generated RDDs. Subclasses of DStream may override * this to clear their own metadata along with the generated RDDs. */ - protected[streaming] def clearOldMetadata(time: Time) { - var numForgotten = 0 + protected[streaming] def clearMetadata(time: Time) { val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) generatedRDDs --= oldRDDs.keys logInfo("Cleared " + oldRDDs.size + " RDDs that were older than " + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) - dependencies.foreach(_.clearOldMetadata(time)) + dependencies.foreach(_.clearMetadata(time)) } /* Adds metadata to the Stream while it is running. @@ -358,12 +357,18 @@ abstract class DStream[T: ClassTag] ( */ protected[streaming] def updateCheckpointData(currentTime: Time) { logInfo("Updating checkpoint data for time " + currentTime) - checkpointData.update() + checkpointData.update(currentTime) dependencies.foreach(_.updateCheckpointData(currentTime)) - checkpointData.cleanup() logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData) } + protected[streaming] def clearCheckpointData(time: Time) { + logInfo("Clearing checkpoint data") + checkpointData.cleanup(time) + dependencies.foreach(_.clearCheckpointData(time)) + logInfo("Cleared checkpoint data") + } + /** * Restore the RDDs in generatedRDDs from the checkpointData. This is an internal method * that should not be called directly. This is a default implementation that recreates RDDs diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala index 3fd5d52403c14..cc2f08a7d186c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala @@ -17,15 +17,16 @@ package org.apache.spark.streaming +import scala.collection.mutable.{HashMap, HashSet} +import scala.reflect.ClassTag + import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.conf.Configuration -import collection.mutable.HashMap import org.apache.spark.Logging -import scala.collection.mutable.HashMap -import scala.reflect.ClassTag +import java.io.{ObjectInputStream, IOException} private[streaming] @@ -33,35 +34,35 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) extends Serializable with Logging { protected val data = new HashMap[Time, AnyRef]() + @transient private var allCheckpointFiles = new HashMap[Time, String] + @transient private var timeToLastCheckpointFileTime = new HashMap[Time, Time] @transient private var fileSystem : FileSystem = null - @transient private var lastCheckpointFiles: HashMap[Time, String] = null - protected[streaming] def checkpointFiles = data.asInstanceOf[HashMap[Time, String]] + //@transient private var lastCheckpointFiles: HashMap[Time, String] = null + + protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** * Updates the checkpoint data of the DStream. This gets called every time * the graph checkpoint is initiated. Default implementation records the * checkpoint files to which the generate RDDs of the DStream has been saved. */ - def update() { + def update(time: Time) { // Get the checkpointed RDDs from the generated RDDs - val newCheckpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined) + val checkpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined) .map(x => (x._1, x._2.getCheckpointFile.get)) // Make a copy of the existing checkpoint data (checkpointed RDDs) - lastCheckpointFiles = checkpointFiles.clone() + //lastCheckpointFiles = checkpointFiles.clone() // If the new checkpoint data has checkpoints then replace existing with the new one - if (newCheckpointFiles.size > 0) { - checkpointFiles.clear() - checkpointFiles ++= newCheckpointFiles - } - - // TODO: remove this, this is just for debugging - newCheckpointFiles.foreach { - case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") } + if (currentCheckpointFiles.size > 0) { + currentCheckpointFiles.clear() + currentCheckpointFiles ++= checkpointFiles } + allCheckpointFiles ++= currentCheckpointFiles + timeToLastCheckpointFileTime(time) = currentCheckpointFiles.keys.min(Time.ordering) } /** @@ -69,7 +70,8 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) * checkpoint is initiated, but after `update` is called. Default * implementation, cleans up old checkpoint files. */ - def cleanup() { + def cleanup(time: Time) { + /* // If there is at least on checkpoint file in the current checkpoint files, // then delete the old checkpoint files. if (checkpointFiles.size > 0 && lastCheckpointFiles != null) { @@ -89,6 +91,23 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } } } + */ + val lastCheckpointFileTime = timeToLastCheckpointFileTime.remove(time).get + allCheckpointFiles.filter(_._1 < lastCheckpointFileTime).foreach { + case (time, file) => + try { + val path = new Path(file) + if (fileSystem == null) { + fileSystem = path.getFileSystem(dstream.ssc.sparkContext.hadoopConfiguration) + } + fileSystem.delete(path, true) + allCheckpointFiles -= time + logInfo("Deleted checkpoint file '" + file + "' for time " + time) + } catch { + case e: Exception => + logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) + } + } } /** @@ -98,7 +117,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) */ def restore() { // Create RDDs from the checkpoint data - checkpointFiles.foreach { + currentCheckpointFiles.foreach { case(time, file) => { logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'") dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file))) @@ -107,6 +126,12 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } override def toString() = { - "[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]" + "[\n" + currentCheckpointFiles.size + " checkpoint files \n" + currentCheckpointFiles.mkString("\n") + "\n]" + } + + @throws(classOf[IOException]) + private def readObject(ois: ObjectInputStream) { + timeToLastCheckpointFileTime = new HashMap[Time, Time] + allCheckpointFiles = new HashMap[Time, String] } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index daed7ff7c3f13..bfedef2e4ed51 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -105,36 +105,44 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def getOutputStreams() = this.synchronized { outputStreams.toArray } def generateJobs(time: Time): Seq[Job] = { + logInfo("Generating jobs for time " + time) this.synchronized { - logInfo("Generating jobs for time " + time) val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time)) logInfo("Generated " + jobs.length + " jobs for time " + time) jobs } } - def clearOldMetadata(time: Time) { + def clearMetadata(time: Time) { + logInfo("Clearing metadata for time " + time) this.synchronized { - logInfo("Clearing old metadata for time " + time) - outputStreams.foreach(_.clearOldMetadata(time)) - logInfo("Cleared old metadata for time " + time) + outputStreams.foreach(_.clearMetadata(time)) } + logInfo("Cleared old metadata for time " + time) } def updateCheckpointData(time: Time) { + logInfo("Updating checkpoint data for time " + time) this.synchronized { - logInfo("Updating checkpoint data for time " + time) outputStreams.foreach(_.updateCheckpointData(time)) - logInfo("Updated checkpoint data for time " + time) } + logInfo("Updated checkpoint data for time " + time) + } + + def clearCheckpointData(time: Time) { + logInfo("Restoring checkpoint data") + this.synchronized { + outputStreams.foreach(_.clearCheckpointData(time)) + } + logInfo("Restored checkpoint data") } def restoreCheckpointData() { + logInfo("Restoring checkpoint data") this.synchronized { - logInfo("Restoring checkpoint data") outputStreams.foreach(_.restoreCheckpointData()) - logInfo("Restored checkpoint data") } + logInfo("Restored checkpoint data") } def validate() { @@ -147,8 +155,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { + logDebug("DStreamGraph.writeObject used") this.synchronized { - logDebug("DStreamGraph.writeObject used") checkpointInProgress = true oos.defaultWriteObject() checkpointInProgress = false @@ -157,8 +165,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { @throws(classOf[IOException]) private def readObject(ois: ObjectInputStream) { + logDebug("DStreamGraph.readObject used") this.synchronized { - logDebug("DStreamGraph.readObject used") checkpointInProgress = true ois.defaultReadObject() checkpointInProgress = false diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 0028422db9a1c..4585e3f6bd5b2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -90,8 +90,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } /** Clear the old time-to-files mappings along with old RDDs */ - protected[streaming] override def clearOldMetadata(time: Time) { - super.clearOldMetadata(time) + protected[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) val oldFiles = files.filter(_._1 <= (time - rememberDuration)) files --= oldFiles.keys logInfo("Cleared " + oldFiles.size + " old files that were older than " + @@ -172,12 +172,12 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] - override def update() { + override def update(time: Time) { hadoopFiles.clear() hadoopFiles ++= files } - override def cleanup() { } + override def cleanup(time: Time) { } override def restore() { hadoopFiles.toSeq.sortBy(_._1)(Time.ordering).foreach { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 5f48692df8732..6fbe6da921792 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -26,8 +26,9 @@ import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent -private[scheduler] case class ClearOldMetadata(time: Time) extends JobGeneratorEvent +private[scheduler] case class ClearMetadata(time: Time) extends JobGeneratorEvent private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent +private[scheduler] case class ClearCheckpointData(time: Time) extends JobGeneratorEvent /** * This class generates jobs from DStreams as well as drives checkpointing and cleaning @@ -55,7 +56,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, longTime => eventProcessorActor ! GenerateJobs(new Time(longTime))) lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) + new CheckpointWriter(this, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) } else { null } @@ -79,15 +80,20 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { * On batch completion, clear old metadata and checkpoint computation. */ private[scheduler] def onBatchCompletion(time: Time) { - eventProcessorActor ! ClearOldMetadata(time) + eventProcessorActor ! ClearMetadata(time) + } + + private[streaming] def onCheckpointCompletion(time: Time) { + eventProcessorActor ! ClearCheckpointData(time) } /** Processes all events */ private def processEvent(event: JobGeneratorEvent) { event match { case GenerateJobs(time) => generateJobs(time) - case ClearOldMetadata(time) => clearOldMetadata(time) + case ClearMetadata(time) => clearMetadata(time) case DoCheckpoint(time) => doCheckpoint(time) + case ClearCheckpointData(time) => clearCheckpointData(time) } } @@ -143,11 +149,16 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } /** Clear DStream metadata for the given `time`. */ - private def clearOldMetadata(time: Time) { - ssc.graph.clearOldMetadata(time) + private def clearMetadata(time: Time) { + ssc.graph.clearMetadata(time) eventProcessorActor ! DoCheckpoint(time) } + /** Clear DStream checkpoint data for the given `time`. */ + private def clearCheckpointData(time: Time) { + ssc.graph.clearCheckpointData(time) + } + /** Perform checkpoint for the give `time`. */ private def doCheckpoint(time: Time) = synchronized { if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 4e25c9566c063..53bc24ff7ac22 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -84,9 +84,9 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() advanceTimeWithRealDelay(ssc, firstNumBatches) logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData) - assert(!stateStream.checkpointData.checkpointFiles.isEmpty, + assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") - stateStream.checkpointData.checkpointFiles.foreach { + stateStream.checkpointData.currentCheckpointFiles.foreach { case (time, file) => { assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist") @@ -95,7 +95,7 @@ class CheckpointSuite extends TestSuiteBase { // Run till a further time such that previous checkpoint files in the stream would be deleted // and check whether the earlier checkpoint files are deleted - val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2)) + val checkpointFiles = stateStream.checkpointData.currentCheckpointFiles.map(x => new File(x._2)) advanceTimeWithRealDelay(ssc, secondNumBatches) checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) @@ -114,9 +114,9 @@ class CheckpointSuite extends TestSuiteBase { // is present in the checkpoint data or not ssc.start() advanceTimeWithRealDelay(ssc, 1) - assert(!stateStream.checkpointData.checkpointFiles.isEmpty, + assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") - stateStream.checkpointData.checkpointFiles.foreach { + stateStream.checkpointData.currentCheckpointFiles.foreach { case (time, file) => { assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist") From a17cc602ac79b22457ed457023493fe82e9d39df Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 8 Jan 2014 04:12:05 -0800 Subject: [PATCH 1253/1571] More bug fixes. --- .../streaming/DStreamCheckpointData.scala | 45 +++++++++++-------- 1 file changed, 26 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala index cc2f08a7d186c..e0567a1c19731 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala @@ -22,7 +22,6 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.conf.Configuration import org.apache.spark.Logging @@ -53,16 +52,17 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) val checkpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined) .map(x => (x._1, x._2.getCheckpointFile.get)) + logInfo("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) // Make a copy of the existing checkpoint data (checkpointed RDDs) - //lastCheckpointFiles = checkpointFiles.clone() + // lastCheckpointFiles = checkpointFiles.clone() // If the new checkpoint data has checkpoints then replace existing with the new one - if (currentCheckpointFiles.size > 0) { + if (!currentCheckpointFiles.isEmpty) { currentCheckpointFiles.clear() currentCheckpointFiles ++= checkpointFiles + allCheckpointFiles ++= currentCheckpointFiles + timeToLastCheckpointFileTime(time) = currentCheckpointFiles.keys.min(Time.ordering) } - allCheckpointFiles ++= currentCheckpointFiles - timeToLastCheckpointFileTime(time) = currentCheckpointFiles.keys.min(Time.ordering) } /** @@ -92,21 +92,28 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } } */ - val lastCheckpointFileTime = timeToLastCheckpointFileTime.remove(time).get - allCheckpointFiles.filter(_._1 < lastCheckpointFileTime).foreach { - case (time, file) => - try { - val path = new Path(file) - if (fileSystem == null) { - fileSystem = path.getFileSystem(dstream.ssc.sparkContext.hadoopConfiguration) - } - fileSystem.delete(path, true) - allCheckpointFiles -= time - logInfo("Deleted checkpoint file '" + file + "' for time " + time) - } catch { - case e: Exception => - logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) + timeToLastCheckpointFileTime.remove(time) match { + case Some(lastCheckpointFileTime) => + logInfo("Deleting all files before " + time) + val filesToDelete = allCheckpointFiles.filter(_._1 < lastCheckpointFileTime) + logInfo("Files to delete:\n" + filesToDelete.mkString(",")) + filesToDelete.foreach { + case (time, file) => + try { + val path = new Path(file) + if (fileSystem == null) { + fileSystem = path.getFileSystem(dstream.ssc.sparkContext.hadoopConfiguration) + } + fileSystem.delete(path, true) + allCheckpointFiles -= time + logInfo("Deleted checkpoint file '" + file + "' for time " + time) + } catch { + case e: Exception => + logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) + } } + case None => + logInfo("Nothing to delete") } } From cf4aaf92d672e03f5d53078cdfe6ea60ba0c1c0c Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Thu, 9 Jan 2014 00:34:53 +0800 Subject: [PATCH 1254/1571] fix make-distribution.sh show version: command not found --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 1a3a5d0209ccf..e6b5956d1e7e2 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -44,7 +44,7 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -VERSIONSTRING=$FWDIR/sbt/sbt "show version" +VERSIONSTRING=$($FWDIR/sbt/sbt "show version") if [ $? == -1 ] ;then echo -e "You need sbt installed and available on your path." From 46f6a3b6aa23aa6dada66423d2851de5a4e8bee5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Jan 2014 14:55:04 -0800 Subject: [PATCH 1255/1571] Minor style cleanup. Mostly on indenting & line width changes. --- .../scala/org/apache/spark/Partitioner.scala | 7 +-- .../scala/org/apache/spark/SparkContext.scala | 30 ++++++----- .../apache/spark/rdd/PairRDDFunctions.scala | 52 ++++++++++--------- .../main/scala/org/apache/spark/rdd/RDD.scala | 32 ++++++------ 4 files changed, 67 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 31b0773bfe06c..9b043f06dd734 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -61,7 +61,8 @@ object Partitioner { } /** - * A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`. + * A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using + * Java's `Object.hashCode`. * * Java arrays have hashCodes that are based on the arrays' identities rather than their contents, * so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will @@ -84,8 +85,8 @@ class HashPartitioner(partitions: Int) extends Partitioner { } /** - * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges. - * Determines the ranges by sampling the RDD passed in. + * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly + * equal ranges. The ranges are determined by sampling the content of the RDD passed in. */ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( partitions: Int, diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0e47f4e442927..47574ab34cdac 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -31,9 +31,9 @@ import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, -FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} + FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, -TextInputFormat} + TextInputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary @@ -49,7 +49,7 @@ import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType, -ClosureCleaner} + ClosureCleaner} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -517,15 +517,15 @@ class SparkContext( // Methods for creating shared variables /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values - * to using the `+=` method. Only the driver can access the accumulator's `value`. + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) /** - * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values with `+=`. - * Only the driver can access the accumuable's `value`. + * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values + * with `+=`. Only the driver can access the accumuable's `value`. * @tparam T accumulator type * @tparam R type that can be added to the accumulator */ @@ -538,13 +538,15 @@ class SparkContext( * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ - def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T](initialValue: R) = { + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] + (initialValue: R) = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) } /** - * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.broadcast.Broadcast]] object for + * Broadcast a read-only variable to the cluster, returning a + * [[org.apache.spark.broadcast.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. */ def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) @@ -1010,7 +1012,8 @@ object SparkContext { implicit def stringToText(s: String) = new Text(s) - private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = { + private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) + : ArrayWritable = { def anyToWritable[U <% Writable](u: U): Writable = u new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]], @@ -1033,7 +1036,9 @@ object SparkContext { implicit def booleanWritableConverter() = simpleWritableConverter[Boolean, BooleanWritable](_.get) - implicit def bytesWritableConverter() = simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) + implicit def bytesWritableConverter() = { + simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) + } implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString) @@ -1049,7 +1054,8 @@ object SparkContext { if (uri != null) { val uriStr = uri.toString if (uriStr.startsWith("jar:file:")) { - // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", so pull out the /path/foo.jar + // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", + // so pull out the /path/foo.jar List(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) } else { Nil diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index a0809cd43a6fe..c118ddfc0138f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -18,35 +18,34 @@ package org.apache.spark.rdd import java.nio.ByteBuffer -import java.util.Date import java.text.SimpleDateFormat +import java.util.Date import java.util.{HashMap => JHashMap} -import scala.collection.{mutable, Map} +import scala.collection.Map +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.{ClassTag, classTag} -import org.apache.hadoop.mapred._ -import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.SequenceFile.CompressionType -import org.apache.hadoop.mapred.FileOutputFormat -import org.apache.hadoop.mapred.OutputFormat +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import com.clearspring.analytics.stream.cardinality.HyperLogLog +// SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark. +import org.apache.hadoop.mapred.SparkHadoopWriter +import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} -import org.apache.spark.Aggregator -import org.apache.spark.Partitioner import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.util.SerializableHyperLogLog @@ -120,9 +119,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key @@ -138,18 +137,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) } /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { foldByKey(zeroValue, defaultPartitioner(self))(func) @@ -226,7 +225,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } /** - * Return approximate number of distinct values for each key in this RDD. + * Return approximate number of distinct values for each key in this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in * more accurate counts but increase the memory footprint and vise versa. HashPartitions the @@ -579,7 +578,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec) + val runtimeClass = fm.runtimeClass + saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec) } /** @@ -599,7 +599,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: NewOutputFormat[_, _]], - conf: Configuration = self.context.hadoopConfiguration) { + conf: Configuration = self.context.hadoopConfiguration) + { val job = new NewAPIHadoopJob(conf) job.setOutputKeyClass(keyClass) job.setOutputValueClass(valueClass) @@ -668,7 +669,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) codec: Option[Class[_ <: CompressionCodec]] = None) { conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) - // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug + // Doesn't work in Scala 2.9 due to what may be a generics bug + // TODO: Should we uncomment this for Scala 2.10? + // conf.setOutputFormat(outputFormatClass) conf.set("mapred.output.format.class", outputFormatClass.getName) for (c <- codec) { conf.setCompressMapOutput(true) @@ -702,7 +705,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) throw new SparkException("Output value class not set") } - logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")") + logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + + valueClass.getSimpleName+ ")") val writer = new SparkHadoopWriter(conf) writer.preSetup() diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3f41b66279987..f9dc12eee3291 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -23,7 +23,6 @@ import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import scala.reflect.{classTag, ClassTag} import org.apache.hadoop.io.BytesWritable @@ -52,11 +51,13 @@ import org.apache.spark._ * partitioned collection of elements that can be operated on in parallel. This class contains the * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, * [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs of key-value - * pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] contains - * operations available only on RDDs of Doubles; and [[org.apache.spark.rdd.SequenceFileRDDFunctions]] - * contains operations available on RDDs that can be saved as SequenceFiles. These operations are - * automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit - * conversions when you `import org.apache.spark.SparkContext._`. + * pairs, such as `groupByKey` and `join`; + * [[org.apache.spark.rdd.DoubleRDDFunctions]] contains operations available only on RDDs of + * Doubles; and + * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that + * can be saved as SequenceFiles. + * These operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] + * through implicit conversions when you `import org.apache.spark.SparkContext._`. * * Internally, each RDD is characterized by five main properties: * @@ -235,12 +236,9 @@ abstract class RDD[T: ClassTag]( /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ - private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { - if (isCheckpointed) { - firstParent[T].iterator(split, context) - } else { - compute(split, context) - } + private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = + { + if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context) } // Transformations (return a new RDD) @@ -268,6 +266,9 @@ abstract class RDD[T: ClassTag]( def distinct(numPartitions: Int): RDD[T] = map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) + /** + * Return a new RDD containing the distinct elements in this RDD. + */ def distinct(): RDD[T] = distinct(partitions.size) /** @@ -280,7 +281,7 @@ abstract class RDD[T: ClassTag]( * which can avoid performing a shuffle. */ def repartition(numPartitions: Int): RDD[T] = { - coalesce(numPartitions, true) + coalesce(numPartitions, shuffle = true) } /** @@ -646,7 +647,8 @@ abstract class RDD[T: ClassTag]( } /** - * Reduces the elements of this RDD using the specified commutative and associative binary operator. + * Reduces the elements of this RDD using the specified commutative and + * associative binary operator. */ def reduce(f: (T, T) => T): T = { val cleanF = sc.clean(f) @@ -953,7 +955,7 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - @transient private[spark] val origin = sc.getCallSite + @transient private[spark] val origin = sc.getCallSite() private[spark] def elementClassTag: ClassTag[T] = classTag[T] From 0f9d2ace6baefeacb1abf9d51a457644b67f2f8d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 16:53:04 -0800 Subject: [PATCH 1256/1571] Adding polling to driver submission client. --- .../org/apache/spark/deploy/Client.scala | 141 +++++++++++------- .../apache/spark/deploy/DeployMessage.scala | 11 +- .../spark/deploy/master/DriverState.scala | 5 +- .../apache/spark/deploy/master/Master.scala | 29 ++-- .../spark/deploy/worker/DriverRunner.scala | 12 +- .../apache/spark/deploy/worker/Worker.scala | 2 +- 6 files changed, 132 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 43b9b1cff9b9f..e133893f6ca5b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -22,60 +22,30 @@ import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ +import akka.pattern.ask import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.DeployMessages._ -import org.apache.spark.deploy.master.Master +import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.util.{AkkaUtils, Utils} +import akka.actor.Actor.emptyBehavior +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} /** - * Actor that sends a single message to the standalone master and returns the response in the - * given promise. + * Proxy that relays messages to the driver. */ -class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging { - override def receive = { - case SubmitDriverResponse(success, message) => { - response.success((success, message)) - } - - case KillDriverResponse(success, message) => { - response.success((success, message)) - } +class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging { + var masterActor: ActorSelection = _ + val timeout = AkkaUtils.askTimeout(conf) - // Relay all other messages to the master. - case message => { - logInfo(s"Sending message to master $master...") - val masterActor = context.actorSelection(Master.toAkkaUrl(master)) - masterActor ! message - } - } -} + override def preStart() = { + masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master)) -/** - * Executable utility for starting and terminating drivers inside of a standalone cluster. - */ -object Client { - - def main(args: Array[String]) { - val driverArgs = new ClientArguments(args) - val conf = new SparkConf() - - if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { - conf.set("spark.akka.logLifecycleEvents", "true") - } - conf.set("spark.akka.askTimeout", "5") - Logger.getRootLogger.setLevel(driverArgs.logLevel) - - // TODO: See if we can initialize akka so return messages are sent back using the same TCP - // flow. Else, this (sadly) requires the DriverClient be routable from the Master. - val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf) - val master = driverArgs.master - val response = promise[(Boolean, String)] - val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response))) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}") + driverArgs.cmd match { case "launch" => // TODO: We could add an env variable here and intercept it in `sc.addJar` that would @@ -94,21 +64,88 @@ object Client { driverArgs.cores, driverArgs.supervise, command) - driver ! RequestSubmitDriver(driverDescription) + + masterActor ! RequestSubmitDriver(driverDescription) case "kill" => val driverId = driverArgs.driverId - driver ! RequestKillDriver(driverId) + val killFuture = masterActor ! RequestKillDriver(driverId) + } + } + + /* Find out driver status then exit the JVM */ + def pollAndReportStatus(driverId: String) { + println(s"... waiting before polling master for driver state") + Thread.sleep(5000) + println("... polling master for driver state") + val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout) + .mapTo[DriverStatusResponse] + val statusResponse = Await.result(statusFuture, timeout) + + statusResponse.found match { + case false => + println(s"ERROR: Cluster master did not recognize $driverId") + System.exit(-1) + case true => + println(s"State of $driverId is ${statusResponse.state.get}") + // Worker node, if present + (statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match { + case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) => + println(s"Driver running on $hostPort ($id)") + case _ => + } + // Exception, if present + statusResponse.exception.map { e => + println(s"Exception from cluster was: $e") + System.exit(-1) + } + System.exit(0) } + } + + override def receive = { + + case SubmitDriverResponse(success, driverId, message) => + println(message) + if (success) pollAndReportStatus(driverId.get) else System.exit(-1) + + case KillDriverResponse(driverId, success, message) => + println(message) + if (success) pollAndReportStatus(driverId) else System.exit(-1) + + case DisassociatedEvent(_, remoteAddress, _) => + println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") + System.exit(-1) + + case AssociationErrorEvent(cause, _, remoteAddress, _) => + println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") + println(s"Cause was: $cause") + System.exit(-1) + } +} + +/** + * Executable utility for starting and terminating drivers inside of a standalone cluster. + */ +object Client { + def main(args: Array[String]) { + val conf = new SparkConf() + val driverArgs = new ClientArguments(args) + + if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { + conf.set("spark.akka.logLifecycleEvents", "true") + } + conf.set("spark.akka.askTimeout", "10") + conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING")) + Logger.getRootLogger.setLevel(driverArgs.logLevel) + + // TODO: See if we can initialize akka so return messages are sent back using the same TCP + // flow. Else, this (sadly) requires the DriverClient be routable from the Master. + val (actorSystem, _) = AkkaUtils.createActorSystem( + "driverClient", Utils.localHostName(), 0, false, conf) + + actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) - val (success, message) = - try { - Await.result(response.future, AkkaUtils.askTimeout(conf)) - } catch { - case e: TimeoutException => (false, s"Error: Timed out sending message to $master") - } - println(message) - actorSystem.shutdown() actorSystem.awaitTermination() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 34460d359d2e8..5e824e1a678b6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -112,11 +112,18 @@ private[deploy] object DeployMessages { case class RequestSubmitDriver(driverDescription: DriverDescription) extends DeployMessage - case class SubmitDriverResponse(success: Boolean, message: String) extends DeployMessage + case class SubmitDriverResponse(success: Boolean, driverId: Option[String], message: String) + extends DeployMessage case class RequestKillDriver(driverId: String) extends DeployMessage - case class KillDriverResponse(success: Boolean, message: String) extends DeployMessage + case class KillDriverResponse(driverId: String, success: Boolean, message: String) + extends DeployMessage + + case class RequestDriverStatus(driverId: String) extends DeployMessage + + case class DriverStatusResponse(found: Boolean, state: Option[DriverState], + workerId: Option[String], workerHostPort: Option[String], exception: Option[Exception]) // Internal message in AppClient diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 93b260740e818..26a68bade3c60 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -27,6 +27,7 @@ private[spark] object DriverState extends Enumeration { // RELAUNCHING: Exited non-zero or due to worker failure, but has not yet started running again // UNKNOWN: The state of the driver is temporarily not known due to master failure recovery // KILLED: A user manually killed this driver - // FAILED: Unable to run due to an unrecoverable error (e.g. missing jar file) - val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED = Value + // FAILED: The driver exited non-zero and was not supervised + // ERROR: Unable to run or restart due to an unrecoverable error (e.g. missing jar file) + val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED, ERROR = Value } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f62601fa6c39d..cd3f3ebefc139 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -186,7 +186,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case RequestSubmitDriver(description) => { if (state != RecoveryState.ALIVE) { val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state." - sender ! SubmitDriverResponse(false, msg) + sender ! SubmitDriverResponse(false, None, msg) } else { logInfo("Driver submitted " + description.command.mainClass) val driver = createDriver(description) @@ -198,14 +198,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // TODO: It might be good to instead have the submission client poll the master to determine // the current status of the driver. For now it's simply "fire and forget". - sender ! SubmitDriverResponse(true, s"Driver successfully submitted as ${driver.id}") + sender ! SubmitDriverResponse(true, Some(driver.id), + s"Driver successfully submitted as ${driver.id}") } } case RequestKillDriver(driverId) => { if (state != RecoveryState.ALIVE) { val msg = s"Can only kill drivers in ALIVE state. Current state: $state." - sender ! KillDriverResponse(false, msg) + sender ! KillDriverResponse(driverId, success = false, msg) } else { logInfo("Asked to kill driver " + driverId) val driver = drivers.find(_.id == driverId) @@ -226,15 +227,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // TODO: It would be nice for this to be a synchronous response val msg = s"Kill request for $driverId submitted" logInfo(msg) - sender ! KillDriverResponse(true, msg) + sender ! KillDriverResponse(driverId, success = true, msg) case None => - val msg = s"Could not find running driver $driverId" + val msg = s"Driver $driverId has already finished or does not exist" logWarning(msg) - sender ! KillDriverResponse(false, msg) + sender ! KillDriverResponse(driverId, success = false, msg) } } } + case RequestDriverStatus(driverId) => { + (drivers ++ completedDrivers).find(_.id == driverId) match { + case Some(driver) => + sender ! DriverStatusResponse(found = true, Some(driver.state), + driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception) + case None => + sender ! DriverStatusResponse(found = false, None, None, None, None) + } + } + case RegisterApplication(description) => { if (state == RecoveryState.STANDBY) { // ignore, don't send response @@ -279,7 +290,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case DriverStateChanged(driverId, state, exception) => { state match { - case DriverState.FAILED | DriverState.FINISHED | DriverState.KILLED => + case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => removeDriver(driverId, state, exception) case _ => throw new Exception(s"Received unexpected state update for driver $driverId: $state") @@ -410,7 +421,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logWarning(s"Re-launching ${d.id}") relaunchDriver(d) } else { - removeDriver(d.id, DriverState.FAILED, None) + removeDriver(d.id, DriverState.ERROR, None) logWarning(s"Did not re-launch ${d.id} because it was not supervised") } } @@ -539,7 +550,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act relaunchDriver(driver) } else { logInfo(s"Not re-launching ${driver.id} because it was not supervised") - removeDriver(driver.id, DriverState.FAILED, None) + removeDriver(driver.id, DriverState.ERROR, None) } } persistenceEngine.removeWorker(worker) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index ad70345a7f599..b4df1a0dd4718 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -52,6 +52,7 @@ private[spark] class DriverRunner( // Populated once finished var finalState: Option[DriverState] = None var finalException: Option[Exception] = None + var finalExitCode: Option[Int] = None // Decoupled for testing private[deploy] def setClock(_clock: Clock) = clock = _clock @@ -87,8 +88,14 @@ private[spark] class DriverRunner( val state = if (killed) { DriverState.KILLED } - else if (finalException.isDefined) { DriverState.FAILED } - else { DriverState.FINISHED } + else if (finalException.isDefined) { DriverState.ERROR } + else { + finalExitCode match { + case Some(0) => DriverState.FINISHED + case _ => DriverState.FAILED + } + } + finalState = Some(state) worker ! DriverStateChanged(driverId, state, finalException) @@ -200,6 +207,7 @@ private[spark] class DriverRunner( } keepTrying = supervise && exitCode != 0 && !killed + finalExitCode = Some(exitCode) } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 2a2b7a38810b0..273bacded6fb4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -272,7 +272,7 @@ private[spark] class Worker( case DriverStateChanged(driverId, state, exception) => { state match { - case DriverState.FAILED => + case DriverState.ERROR => logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") case DriverState.FINISHED => logInfo(s"Driver $driverId exited successfully") From 112c0a1776bbc866a1026a9579c6f72f293414c4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 21:16:16 -0800 Subject: [PATCH 1257/1571] Fixing config option "retained_stages" => "retainedStages". This is a very esoteric option and it's out of sync with the style we use. So it seems fitting to fix it for 0.9.0. --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 2 +- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index b7b87250b98ee..d6d9f0cedf923 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.get("spark.ui.retainedStages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/docs/configuration.md b/docs/configuration.md index 1d6c3d16333c5..6717757781974 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -130,7 +130,7 @@ Apart from these, the following properties are also available, and may be useful - spark.ui.retained_stages + spark.ui.retainedStages 1000 How many stages the Spark UI remembers before garbage collecting. From 78d6b13ac88d4f0e52cf430d0bc3c1eb5369e4dc Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:33:23 -0700 Subject: [PATCH 1258/1571] Fix mis-merge in 44fd30d3fbcf830deecbe8ea3e8ea165e74e6edd --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3f41b66279987..2142ae730e9ff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -547,6 +547,11 @@ abstract class RDD[T: ClassTag]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ + def zipPartitions[B: ClassTag, V: ClassTag] + (rdd2: RDD[B], preservesPartitioning: Boolean) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) + def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B]) (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = From ac536345f86e467ac83cb9c0dccbb34150335e26 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:50:56 -0700 Subject: [PATCH 1259/1571] ClassManifest -> ClassTag --- .../org/apache/spark/graph/Analytics.scala | 2 +- .../org/apache/spark/graph/EdgeRDD.scala | 16 ++++---- .../org/apache/spark/graph/EdgeTriplet.scala | 4 +- .../scala/org/apache/spark/graph/Graph.scala | 28 +++++++------ .../org/apache/spark/graph/GraphLab.scala | 4 +- .../org/apache/spark/graph/GraphLoader.scala | 4 +- .../org/apache/spark/graph/GraphOps.scala | 10 +++-- .../scala/org/apache/spark/graph/Pregel.scala | 4 +- .../org/apache/spark/graph/VertexRDD.scala | 28 +++++++------ .../graph/algorithms/TriangleCount.scala | 4 +- .../spark/graph/impl/EdgePartition.scala | 10 +++-- .../graph/impl/EdgePartitionBuilder.scala | 3 +- .../graph/impl/EdgeTripletIterator.scala | 4 +- .../apache/spark/graph/impl/GraphImpl.scala | 40 ++++++++++--------- .../spark/graph/impl/MessageToPartition.scala | 24 ++++++----- .../graph/impl/ReplicatedVertexView.scala | 10 +++-- .../spark/graph/impl/VertexPartition.scala | 22 +++++----- .../spark/graph/util/GraphGenerators.scala | 20 ++-------- .../spark/graph/impl/EdgePartitionSuite.scala | 3 +- 19 files changed, 129 insertions(+), 111 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 2012dadb2f305..14b9be73f1651 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -335,7 +335,7 @@ object Analytics extends Logging { // /** // * // */ - // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], + // def alternatingLeastSquares[VD: ClassTag, ED: ClassTag](graph: Graph[VD, Double], // latentK: Int, lambda: Double, numIter: Int) = { // val vertices = graph.vertices.mapPartitions( _.map { // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 3dda5c7c604e5..1c21967c9cab3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -1,12 +1,14 @@ package org.apache.spark.graph +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -class EdgeRDD[@specialized ED: ClassManifest]( +class EdgeRDD[@specialized ED: ClassTag]( val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { @@ -42,7 +44,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassManifest](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassTag](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => @@ -51,7 +53,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( }, preservesPartitioning = true)) } - def zipEdgePartitions[T: ClassManifest, U: ClassManifest] + def zipEdgePartitions[T: ClassTag, U: ClassTag] (other: RDD[T]) (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => @@ -60,7 +62,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( } } - def zipEdgePartitions[ED2: ClassManifest, ED3: ClassManifest] + def zipEdgePartitions[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { @@ -71,11 +73,11 @@ class EdgeRDD[@specialized ED: ClassManifest]( }) } - def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { - val ed2Manifest = classManifest[ED2] - val ed3Manifest = classManifest[ED3] + val ed2Manifest = classTag[ED2] + val ed3Manifest = classTag[ED3] zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 76768489eed37..5a384a5f84dd9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -14,8 +14,8 @@ import org.apache.spark.graph.impl.VertexPartition * that is not a trait. */ class EdgeTriplet[VD, ED] extends Edge[ED] { -// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, -// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { +// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassTag, +// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag] extends Edge[ED] { /** diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index b725b2a15584b..9dd26f7679a2d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.graph.impl._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -23,7 +25,7 @@ import org.apache.spark.storage.StorageLevel * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -abstract class Graph[VD: ClassManifest, ED: ClassManifest] { +abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * Get the vertices and their data. @@ -123,7 +125,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassTag](map: (Vid, VD) => VD2): Graph[VD2, ED] /** * Construct a new graph where the value of each edge is @@ -143,7 +145,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * attributes. * */ - def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] = { + def mapEdges[ED2: ClassTag](map: Edge[ED] => ED2): Graph[VD, ED2] = { mapEdges((pid, iter) => iter.map(map)) } @@ -167,7 +169,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassManifest]( + def mapEdges[ED2: ClassTag]( map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] @@ -195,7 +197,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + def mapTriplets[ED2: ClassTag](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { mapTriplets((pid, iter) => iter.map(map)) } @@ -219,7 +221,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @tparam ED2 the new edge data type * */ - def mapTriplets[ED2: ClassManifest]( + def mapTriplets[ED2: ClassTag]( map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] @@ -261,7 +263,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @return a graph with vertices and edges that exists in both the current graph and other, * with vertex and edge data from the current graph. */ - def mask[VD2: ClassManifest, ED2: ClassManifest](other: Graph[VD2, ED2]): Graph[VD, ED] + def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] /** * This function merges multiple edges between two vertices into a single Edge. For correct @@ -313,7 +315,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * predicate or implement PageRank. * */ - def mapReduceTriplets[A: ClassManifest]( + def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) @@ -348,7 +350,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def outerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) + def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(Vid, U)]) (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] @@ -376,7 +378,7 @@ object Graph { * @return a graph with edge attributes containing either the count of duplicate edges or 1 * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. */ - def fromEdgeTuples[VD: ClassManifest]( + def fromEdgeTuples[VD: ClassTag]( rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { @@ -397,7 +399,7 @@ object Graph { * @return a graph with edge attributes described by `edges` and vertices * given by all vertices in `edges` with value `defaultValue` */ - def fromEdges[VD: ClassManifest, ED: ClassManifest]( + def fromEdges[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], defaultValue: VD): Graph[VD, ED] = { GraphImpl(edges, defaultValue) @@ -418,7 +420,7 @@ object Graph { * @param partitionStrategy the partition strategy to use when * partitioning the edges. */ - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { @@ -432,5 +434,5 @@ object Graph { * convenience operations are defined in the GraphOps class which may be shared across multiple * graph implementations. */ - implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops } // end of Graph object diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 5d2f0f4bda6f1..c1ce5cd9ccada 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.Logging import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD @@ -36,7 +38,7 @@ object GraphLab extends Logging { * @tparam A The type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], numIter: Int, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index a69bfde5322df..7daac4fcc56c3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,6 +1,8 @@ package org.apache.spark.graph import java.util.{Arrays => JArrays} +import scala.reflect.ClassTag + import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} @@ -22,7 +24,7 @@ object GraphLoader extends Logging { * the Edge RDD * */ - def textFile[ED: ClassManifest]( + def textFile[ED: ClassTag]( sc: SparkContext, path: String, edgeParser: Array[String] => ED, diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 091c7782757f2..11c6120beb414 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.SparkException @@ -15,7 +17,7 @@ import org.apache.spark.SparkException * @tparam ED the edge attribute type * */ -class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { +class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { /** * Compute the number of edges in the graph. @@ -109,7 +111,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @todo Should this return a graph with the new vertex values? * */ - def aggregateNeighbors[A: ClassManifest]( + def aggregateNeighbors[A: ClassTag]( mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) @@ -226,7 +228,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * }}} * */ - def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + def joinVertices[U: ClassTag](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) : Graph[VD, ED] = { val uf = (id: Vid, data: VD, o: Option[U]) => { o match { @@ -262,7 +264,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * }}} * */ - def filter[VD2: ClassManifest, ED2: ClassManifest]( + def filter[VD2: ClassTag, ED2: ClassTag]( preprocess: Graph[VD, ED] => Graph[VD2, ED2], epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, vpred: (Vid, VD2) => Boolean = (v:Vid, d:VD2) => true): Graph[VD, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 285e857b693fc..4664091b5714b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + /** * This object implements a Pregel-like bulk-synchronous @@ -84,7 +86,7 @@ object Pregel { * @return the resulting graph at the end of the computation * */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( vprog: (Vid, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index c274e342c7566..8e5e319928274 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ @@ -50,7 +52,7 @@ import org.apache.spark.graph.impl.VertexPartition * }}} * */ -class VertexRDD[@specialized VD: ClassManifest]( +class VertexRDD[@specialized VD: ClassTag]( val partitionsRDD: RDD[VertexPartition[VD]]) extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { @@ -111,7 +113,7 @@ class VertexRDD[@specialized VD: ClassManifest]( /** * Return a new VertexRDD by applying a function to each VertexPartition of this RDD. */ - def mapVertexPartitions[VD2: ClassManifest](f: VertexPartition[VD] => VertexPartition[VD2]) + def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -121,7 +123,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Return a new VertexRDD by applying a function to corresponding * VertexPartitions of this VertexRDD and another one. */ - def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] + def zipVertexPartitions[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2]) (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { val newPartitionsRDD = partitionsRDD.zipPartitions( @@ -160,7 +162,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: VD => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map((vid, attr) => f(attr))) /** @@ -174,7 +176,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: (Vid, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** @@ -205,7 +207,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * other VertexSet. * */ - def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftZipJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.leftJoin(otherPart)(f) @@ -231,7 +233,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * @return a VertexRDD containing all the vertices in this * VertexRDD with the attribute emitted by f. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: RDD[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3) : VertexRDD[VD3] = @@ -257,7 +259,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` * must have the same index. */ - def innerZipJoin[U: ClassManifest, VD2: ClassManifest](other: VertexRDD[U]) + def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.innerJoin(otherPart)(f) @@ -268,7 +270,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Replace vertices with corresponding vertices in `other`, and drop vertices without a * corresponding vertex in `other`. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest](other: RDD[(Vid, U)]) + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(Vid, U)]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient innerZipJoin @@ -291,7 +293,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Aggregate messages with the same ids using `reduceFunc`, returning a VertexRDD that is * co-indexed with this one. */ - def aggregateUsingIndex[VD2: ClassManifest]( + def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) @@ -318,7 +320,7 @@ object VertexRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { + def apply[VD: ClassTag](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) @@ -338,7 +340,7 @@ object VertexRDD { * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = + def apply[VD: ClassTag](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd @@ -350,7 +352,7 @@ object VertexRDD { new VertexRDD(vertexPartitions) } - def apply[VD: ClassManifest](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) + def apply[VD: ClassTag](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) : VertexRDD[VD] = { VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala index b1cd3c47d0cf1..a6384320bab6b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graph._ @@ -21,7 +23,7 @@ object TriangleCount { * * @return */ - def run[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD,ED]): Graph[Int, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { // Remove redundant edges val g = graph.groupEdges((a, b) => a).cache diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 4fcf08efce382..7367269f67bfa 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap @@ -13,7 +15,7 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * @param index a clustered index on source vertex id * @tparam ED the edge attribute type. */ -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( val srcIds: Array[Vid], val dstIds: Array[Vid], val data: Array[ED], @@ -41,7 +43,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassManifest](f: Edge[ED] => ED2): EdgePartition[ED2] = { + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() val size = data.size @@ -69,7 +71,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) var i = 0 while (iter.hasNext) { @@ -132,7 +134,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * If there are multiple edges with the same src and dst in `other`, `f` will only be invoked * once. */ - def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgePartition[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgePartition[ED3] = { val builder = new EdgePartitionBuilder[ED3] diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index 38762733692cc..ae3f3a6d03145 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag import scala.util.Sorting import org.apache.spark.graph._ @@ -7,7 +8,7 @@ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVecto //private[graph] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassManifest](size: Int = 64) { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala index c9e1e081534cb..4d5eb240a91b7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap @@ -10,7 +12,7 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * debug / profile. */ private[impl] -class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( +class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( val vidToIndex: VertexIdToIndexMap, val vertexArray: Array[VD], val edgePartition: EdgePartition[ED]) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 79c11c780a69a..1dfd9cf316efb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.util.collection.PrimitiveVector import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.SparkContext._ @@ -23,7 +25,7 @@ import org.apache.spark.util.ClosureCleaner * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created * using the routing table. */ -class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( +class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], @transient val routingTable: RoutingTable, @@ -45,8 +47,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdManifest = classManifest[VD] - val edManifest = classManifest[ED] + val vdManifest = classTag[VD] + val edManifest = classTag[ED] edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) @@ -63,7 +65,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size - val edManifest = classManifest[ED] + val edManifest = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) @@ -153,8 +155,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { - if (classManifest[VD] equals classManifest[VD2]) { + override def mapVertices[VD2: ClassTag](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) @@ -168,17 +170,17 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - override def mapEdges[ED2: ClassManifest]( + override def mapEdges[ED2: ClassTag]( f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) } - override def mapTriplets[ED2: ClassManifest]( + override def mapTriplets[ED2: ClassTag]( f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). - val vdManifest = classManifest[VD] + val vdManifest = classTag[VD] val newEdgePartitions = edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePid, edgePartition, vTableReplicatedIter) => @@ -208,7 +210,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) // Filter the edges - val edManifest = classManifest[ED] + val edManifest = classTag[ED] val newEdges = new EdgeRDD[ED](triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.mapPartitionsWithIndex( { (pid, iter) => @@ -224,7 +226,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) } // end of subgraph - override def mask[VD2: ClassManifest, ED2: ClassManifest] ( + override def mask[VD2: ClassTag, ED2: ClassTag] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } @@ -244,7 +246,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// - override def mapReduceTriplets[A: ClassManifest]( + override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { @@ -311,9 +313,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( vertices.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets - override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] + override def outerJoinVertices[U: ClassTag, VD2: ClassTag] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { - if (classManifest[VD] equals classManifest[VD2]) { + if (classTag[VD] equals classTag[VD2]) { // updateF preserves type, so we can use incremental replication val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) @@ -340,20 +342,20 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } - def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( + def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( edgePartitions: RDD[(Pid, EdgePartition[ED])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = @@ -381,7 +383,7 @@ object GraphImpl { * pair: the key is the partition id, and the value is an EdgePartition object containing all the * edges in a partition. */ - private def createEdgeRDD[ED: ClassManifest]( + private def createEdgeRDD[ED: ClassTag]( edges: RDD[Edge[ED]]): EdgeRDD[ED] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => val builder = new EdgePartitionBuilder[ED] @@ -393,7 +395,7 @@ object GraphImpl { new EdgeRDD(edgePartitions) } - private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( edges: EdgeRDD[ED], defaultVertexAttr: VD): GraphImpl[VD, ED] = { edges.cache() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index c2e452cc72ab5..66fe796d2eae5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.Partitioner import org.apache.spark.graph.{Pid, Vid} import org.apache.spark.rdd.{ShuffledRDD, RDD} @@ -37,16 +39,16 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef } -class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcastMsg[T]]) { +class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { val rdd = new ShuffledRDD[Pid, (Vid, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. - if (classManifest[T] == ClassManifest.Int) { + if (classTag[T] == ClassTag.Int) { rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Long) { + } else if (classTag[T] == ClassTag.Long) { rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Double) { + } else if (classTag[T] == ClassTag.Double) { rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) } rdd @@ -54,7 +56,7 @@ class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcast } -class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { +class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { /** * Return a copy of the RDD partitioned using the specified partitioner. @@ -67,23 +69,23 @@ class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { object MsgRDDFunctions { - implicit def rdd2PartitionRDDFunctions[T: ClassManifest](rdd: RDD[MessageToPartition[T]]) = { + implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { new MsgRDDFunctions(rdd) } - implicit def rdd2vertexMessageRDDFunctions[T: ClassManifest](rdd: RDD[VertexBroadcastMsg[T]]) = { + implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { new VertexBroadcastMsgRDDFunctions(rdd) } - def partitionForAggregation[T: ClassManifest](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { + def partitionForAggregation[T: ClassTag](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { val rdd = new ShuffledRDD[Vid, T, (Vid, T)](msgs, partitioner) // Set a custom serializer if the data is of int or double type. - if (classManifest[T] == ClassManifest.Int) { + if (classTag[T] == ClassTag.Int) { rdd.setSerializer(classOf[IntAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Long) { + } else if (classTag[T] == ClassTag.Long) { rdd.setSerializer(classOf[LongAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Double) { + } else if (classTag[T] == ClassTag.Double) { rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) } rdd diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 175586b87eb16..2124144df792e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} @@ -17,7 +19,7 @@ import org.apache.spark.graph._ * example. */ private[impl] -class ReplicatedVertexView[VD: ClassManifest]( +class ReplicatedVertexView[VD: ClassTag]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], routingTable: RoutingTable, @@ -80,7 +82,7 @@ class ReplicatedVertexView[VD: ClassManifest]( private def create(includeSrc: Boolean, includeDst: Boolean) : RDD[(Pid, VertexPartition[VD])] = { - val vdManifest = classManifest[VD] + val vdManifest = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD @@ -125,7 +127,7 @@ class ReplicatedVertexView[VD: ClassManifest]( } object ReplicatedVertexView { - protected def buildBuffer[VD: ClassManifest]( + protected def buildBuffer[VD: ClassTag]( pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { val pid2vid: Array[Array[Vid]] = pid2vidIter.next() @@ -173,6 +175,6 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { +class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) { def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 9b2d66999cc5a..7048a40f42364 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} import org.apache.spark.Logging @@ -8,7 +10,7 @@ import org.apache.spark.graph._ private[graph] object VertexPartition { - def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { + def apply[VD: ClassTag](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[Vid, VD] iter.foreach { case (k, v) => map(k) = v @@ -16,7 +18,7 @@ private[graph] object VertexPartition { new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) } - def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) + def apply[VD: ClassTag](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) : VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[Vid, VD] @@ -29,7 +31,7 @@ private[graph] object VertexPartition { private[graph] -class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( +class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( val index: VertexIdToIndexMap, val values: Array[VD], val mask: BitSet, @@ -70,7 +72,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexPartition retains the same index. */ - def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { + def map[VD2: ClassTag](f: (Vid, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation val newValues = new Array[VD2](capacity) var i = mask.nextSetBit(0) @@ -126,7 +128,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } /** Left outer join another VertexPartition. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexPartition[VD2]) (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { if (index != other.index) { @@ -146,14 +148,14 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } /** Left outer join another iterator of messages. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: Iterator[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { leftJoin(createUsingIndex(other))(f) } /** Inner join another VertexPartition. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest](other: VertexPartition[U]) + def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") @@ -173,7 +175,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** * Inner join an iterator of messages. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest] + def innerJoin[U: ClassTag, VD2: ClassTag] (iter: Iterator[Product2[Vid, U]]) (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { innerJoin(createUsingIndex(iter))(f) @@ -182,7 +184,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** * Similar effect as aggregateUsingIndex((a, b) => a) */ - def createUsingIndex[VD2: ClassManifest](iter: Iterator[Product2[Vid, VD2]]) + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[Vid, VD2]]) : VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) @@ -214,7 +216,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, newValues, newMask) } - def aggregateUsingIndex[VD2: ClassManifest]( + def aggregateUsingIndex[VD2: ClassTag]( iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { val newMask = new BitSet(capacity) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index a1e285816b9b7..d61f358bb08bd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -1,10 +1,9 @@ package org.apache.spark.graph.util -import util._ -import math._ import scala.annotation.tailrec -//import scala.collection.mutable - +import scala.math._ +import scala.reflect.ClassTag +import scala.util._ import org.apache.spark._ import org.apache.spark.serializer._ @@ -155,7 +154,7 @@ object GraphGenerators { } - def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) @@ -281,14 +280,3 @@ object GraphGenerators { } // end of Graph Generators - - - - - - - - - - - diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index a52a5653e2cf6..f951fd7a82741 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag import scala.util.Random import org.scalatest.FunSuite @@ -59,7 +60,7 @@ class EdgePartitionSuite extends FunSuite { } test("innerJoin") { - def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { val builder = new EdgePartitionBuilder[A] for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } builder.toEdgePartition From 0ad75cdfb0093a0b525c598c5af4b9745581a6d7 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:55:05 -0700 Subject: [PATCH 1260/1571] Manifest -> Tag in variable names --- .../scala/org/apache/spark/graph/EdgeRDD.scala | 6 +++--- .../org/apache/spark/graph/impl/GraphImpl.scala | 16 ++++++++-------- .../spark/graph/impl/ReplicatedVertexView.scala | 8 ++++---- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 1c21967c9cab3..6f1d790325ce1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -76,10 +76,10 @@ class EdgeRDD[@specialized ED: ClassTag]( def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { - val ed2Manifest = classTag[ED2] - val ed3Manifest = classTag[ED3] + val ed2Tag = classTag[ED2] + val ed3Tag = classTag[ED3] zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => - thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) + thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 1dfd9cf316efb..826c1074a85f6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -47,11 +47,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdManifest = classTag[VD] - val edManifest = classTag[ED] + val vdTag = classTag[VD] + val edTag = classTag[ED] edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() - new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) + new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) } } @@ -65,7 +65,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size - val edManifest = classTag[ED] + val edTag = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) @@ -74,7 +74,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edManifest) + val builder = new EdgePartitionBuilder[ED]()(edTag) iter.foreach { message => val data = message.data builder.add(data._1, data._2, data._3) @@ -180,7 +180,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). - val vdManifest = classTag[VD] + val vdTag = classTag[VD] val newEdgePartitions = edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePid, edgePartition, vTableReplicatedIter) => @@ -210,11 +210,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) // Filter the edges - val edManifest = classTag[ED] + val edTag = classTag[ED] val newEdges = new EdgeRDD[ED](triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edManifest) + val builder = new EdgePartitionBuilder[ED]()(edTag) iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 2124144df792e..033971c1af3c3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -82,12 +82,12 @@ class ReplicatedVertexView[VD: ClassTag]( private def create(includeSrc: Boolean, includeDst: Boolean) : RDD[(Pid, VertexPartition[VD])] = { - val vdManifest = classTag[VD] + val vdTag = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD val shippedVerts = routingTable.get(includeSrc, includeDst) - .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdManifest)) + .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdTag)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. @@ -109,7 +109,7 @@ class ReplicatedVertexView[VD: ClassTag]( val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map - val vertexArray = vdManifest.newArray(vidToIndex.capacity) + val vertexArray = vdTag.newArray(vidToIndex.capacity) for ((_, block) <- shippedVertsIter) { for (i <- 0 until block.vids.size) { val vid = block.vids(i) @@ -119,7 +119,7 @@ class ReplicatedVertexView[VD: ClassTag]( } } val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) + vidToIndex, vertexArray, vidToIndex.getBitSet)(vdTag) Iterator((pid, newVPart)) }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) } From ab861d8450140cdb0a3d9f9b830ec076d8af746d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 13:25:03 -0700 Subject: [PATCH 1261/1571] Take SparkConf in constructor of Serializer subclasses --- .../apache/spark/graph/impl/Serializers.scala | 15 +++++----- .../apache/spark/graph/SerializerSuite.scala | 30 +++++++++++-------- 2 files changed, 26 insertions(+), 19 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index e4fa4a4421786..dcf619fa85e8a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -3,10 +3,11 @@ package org.apache.spark.graph.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import org.apache.spark.SparkConf import org.apache.spark.graph._ import org.apache.spark.serializer._ -class VidMsgSerializer extends Serializer { +class VidMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -26,7 +27,7 @@ class VidMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ -class IntVertexBroadcastMsgSerializer extends Serializer { +class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -49,7 +50,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Long]. */ -class LongVertexBroadcastMsgSerializer extends Serializer { +class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -72,7 +73,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ -class DoubleVertexBroadcastMsgSerializer extends Serializer { +class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -95,7 +96,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Int]. */ -class IntAggMsgSerializer extends Serializer { +class IntAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -118,7 +119,7 @@ class IntAggMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Long]. */ -class LongAggMsgSerializer extends Serializer { +class LongAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -141,7 +142,7 @@ class LongAggMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Double]. */ -class DoubleAggMsgSerializer extends Serializer { +class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 80075f3437fb8..4014cbe440d8d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -15,14 +15,15 @@ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { test("IntVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -36,14 +37,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -57,14 +59,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -78,14 +81,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Int) = inStrm.readObject() val inMsg2: (Vid, Int) = inStrm.readObject() assert(outMsg === inMsg1) @@ -97,14 +101,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 1L << 32) val bout = new ByteArrayOutputStream - val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Long) = inStrm.readObject() val inMsg2: (Vid, Long) = inStrm.readObject() assert(outMsg === inMsg1) @@ -116,14 +121,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Double) = inStrm.readObject() val inMsg2: (Vid, Double) = inStrm.readObject() assert(outMsg === inMsg1) From 74fdfac11266652ca87e05ae9b6510b75318728d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 8 Jan 2014 11:45:31 -0800 Subject: [PATCH 1262/1571] Fix AbstractMethodError by inlining zip{Edge,Vertex}Partitions The zip{Edge,Vertex}Partitions methods created doubly-nested closures and passed them to zipPartitions. For some reason this caused an AbstractMethodError when zipPartitions tried to invoke the closure. This commit works around the problem by inlining these methods wherever they are called, eliminating the doubly-nested closure. --- .../org/apache/spark/graph/EdgeRDD.scala | 29 +++---------- .../org/apache/spark/graph/VertexRDD.scala | 43 +++++++++---------- .../apache/spark/graph/impl/GraphImpl.scala | 12 ++++-- 3 files changed, 35 insertions(+), 49 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 6f1d790325ce1..230202d6b0a6f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -53,34 +53,17 @@ class EdgeRDD[@specialized ED: ClassTag]( }, preservesPartitioning = true)) } - def zipEdgePartitions[T: ClassTag, U: ClassTag] - (other: RDD[T]) - (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { - partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => - val (pid, edgePartition) = ePartIter.next() - f(pid, edgePartition, otherIter) - } - } - - def zipEdgePartitions[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2]) - (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { - new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { - (thisIter, otherIter) => - val (pid, thisEPart) = thisIter.next() - val (_, otherEPart) = otherIter.next() - Iterator(Tuple2(pid, f(pid, thisEPart, otherEPart))) - }) - } - def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => - thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag) - } + new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + (thisIter, otherIter) => + val (pid, thisEPart) = thisIter.next() + val (_, otherEPart) = otherIter.next() + Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) + }) } def collectVids(): RDD[Vid] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 8e5e319928274..c5fb4aeca73a0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -119,22 +119,6 @@ class VertexRDD[@specialized VD: ClassTag]( new VertexRDD(newPartitionsRDD) } - /** - * Return a new VertexRDD by applying a function to corresponding - * VertexPartitions of this VertexRDD and another one. - */ - def zipVertexPartitions[VD2: ClassTag, VD3: ClassTag] - (other: VertexRDD[VD2]) - (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { - val newPartitionsRDD = partitionsRDD.zipPartitions( - other.partitionsRDD, preservesPartitioning = true - ) { (thisIter, otherIter) => - val thisPart = thisIter.next() - val otherPart = otherIter.next() - Iterator(f(thisPart, otherPart)) - } - new VertexRDD(newPartitionsRDD) - } /** * Restrict the vertex set to the set of vertices satisfying the @@ -184,9 +168,14 @@ class VertexRDD[@specialized VD: ClassTag]( * the values from `other`. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.diff(otherPart) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.diff(otherPart)) } + new VertexRDD(newPartitionsRDD) } /** @@ -209,9 +198,14 @@ class VertexRDD[@specialized VD: ClassTag]( */ def leftZipJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.leftJoin(otherPart)(f) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.leftJoin(otherPart)(f)) } + new VertexRDD(newPartitionsRDD) } /** @@ -261,9 +255,14 @@ class VertexRDD[@specialized VD: ClassTag]( */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.innerJoin(otherPart)(f) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.innerJoin(otherPart)(f)) } + new VertexRDD(newPartitionsRDD) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 826c1074a85f6..4d35755e7e5cc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -49,7 +49,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdTag = classTag[VD] val edTag = classTag[ED] - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => + edges.partitionsRDD.zipPartitions( + replicatedVertexView.get(true, true), true) { (ePartIter, vPartIter) => + val (pid, ePart) = ePartIter.next() val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) } @@ -182,8 +184,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( // manifest from GraphImpl (which would require serializing GraphImpl). val vdTag = classTag[VD] val newEdgePartitions = - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { - (ePid, edgePartition, vTableReplicatedIter) => + edges.partitionsRDD.zipPartitions(replicatedVertexView.get(true, true), true) { + (ePartIter, vTableReplicatedIter) => + val (ePid, edgePartition) = ePartIter.next() val (vPid, vPart) = vTableReplicatedIter.next() assert(!vTableReplicatedIter.hasNext) assert(ePid == vPid) @@ -267,7 +270,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.zipEdgePartitions(vs) { (ePid, edgePartition, vPartIter) => + val preAgg = edges.partitionsRDD.zipPartitions(vs, true) { (ePartIter, vPartIter) => + val (ePid, edgePartition) = ePartIter.next() val (vPid, vPart) = vPartIter.next() assert(!vPartIter.hasNext) assert(ePid == vPid) From 22374559a23adbcb5c286e0aadc7cd40c228726f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 8 Jan 2014 22:48:54 -0800 Subject: [PATCH 1263/1571] Remove GraphX README --- README.md | 184 ++++++++++++++++-------------------------------------- 1 file changed, 53 insertions(+), 131 deletions(-) diff --git a/README.md b/README.md index 5b06d82225229..c840a68f76b17 100644 --- a/README.md +++ b/README.md @@ -1,143 +1,57 @@ -# GraphX: Unifying Graphs and Tables +# Apache Spark - -GraphX extends the distributed fault-tolerant collections API and -interactive console of [Spark](http://spark.incubator.apache.org) with -a new graph API which leverages recent advances in graph systems -(e.g., [GraphLab](http://graphlab.org)) to enable users to easily and -interactively build, transform, and reason about graph structured data -at scale. - - -## Motivation - -From social networks and targeted advertising to protein modeling and -astrophysics, big graphs capture the structure in data and are central -to the recent advances in machine learning and data mining. Directly -applying existing *data-parallel* tools (e.g., -[Hadoop](http://hadoop.apache.org) and -[Spark](http://spark.incubator.apache.org)) to graph computation tasks -can be cumbersome and inefficient. The need for intuitive, scalable -tools for graph computation has lead to the development of new -*graph-parallel* systems (e.g., -[Pregel](http://http://giraph.apache.org) and -[GraphLab](http://graphlab.org)) which are designed to efficiently -execute graph algorithms. Unfortunately, these systems do not address -the challenges of graph construction and transformation and provide -limited fault-tolerance and support for interactive analysis. - -

      - -

      - - - -## Solution - -The GraphX project combines the advantages of both data-parallel and -graph-parallel systems by efficiently expressing graph computation -within the [Spark](http://spark.incubator.apache.org) framework. We -leverage new ideas in distributed graph representation to efficiently -distribute graphs as tabular data-structures. Similarly, we leverage -advances in data-flow systems to exploit in-memory computation and -fault-tolerance. We provide powerful new operations to simplify graph -construction and transformation. Using these primitives we implement -the PowerGraph and Pregel abstractions in less than 20 lines of code. -Finally, by exploiting the Scala foundation of Spark, we enable users -to interactively load, transform, and compute on massive graphs. - -

      - -

      - -## Examples - -Suppose I want to build a graph from some text files, restrict the graph -to important relationships and users, run page-rank on the sub-graph, and -then finally return attributes associated with the top users. I can do -all of this in just a few lines with GraphX: - -```scala -// Connect to the Spark cluster -val sc = new SparkContext("spark://master.amplab.org", "research") - -// Load my user data and prase into tuples of user id and attribute list -val users = sc.textFile("hdfs://user_attributes.tsv") - .map(line => line.split).map( parts => (parts.head, parts.tail) ) - -// Parse the edge data which is already in userId -> userId format -val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") - -// Attach the user attributes -val graph = followerGraph.outerJoinVertices(users){ - case (uid, deg, Some(attrList)) => attrList - // Some users may not have attributes so we set them as empty - case (uid, deg, None) => Array.empty[String] - } - -// Restrict the graph to users which have exactly two attributes -val subgraph = graph.subgraph((vid, attr) => attr.size == 2) - -// Compute the PageRank -val pagerankGraph = Analytics.pagerank(subgraph) - -// Get the attributes of the top pagerank users -val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices){ - case (uid, attrList, Some(pr)) => (pr, attrList) - case (uid, attrList, None) => (pr, attrList) - } - -println(userInfoWithPageRank.top(5)) - -``` +Lightning-Fast Cluster Computing - ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at -. This README -file only contains basic setup instructions. +guide, on the project webpage at . +This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The -project is built using Simple Build Tool (SBT), which is packaged with -it. To build Spark and its example programs, run: +Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), +which can be obtained [here](http://www.scala-sbt.org). If SBT is installed we +will use the system version of sbt otherwise we will attempt to download it +automatically. To build Spark and its example programs, run: - sbt/sbt assembly + ./sbt/sbt assembly -Once you've built Spark, the easiest way to start using it is the -shell: +Once you've built Spark, the easiest way to start using it is the shell: - ./spark-shell + ./bin/spark-shell -Or, for the Python API, the Python shell (`./pyspark`). +Or, for the Python API, the Python shell (`./bin/pyspark`). -Spark also comes with several sample programs in the `examples` -directory. To run one of them, use `./run-example -`. For example: +Spark also comes with several sample programs in the `examples` directory. +To run one of them, use `./bin/run-example `. For example: - ./run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example org.apache.spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. Each of the example programs prints usage help if no params are given. -All of the Spark samples take a `` parameter that is the -cluster URL to connect to. This can be a mesos:// or spark:// URL, or -"local" to run locally with one thread, or "local[N]" to run locally -with N threads. +All of the Spark samples take a `` parameter that is the cluster URL +to connect to. This can be a mesos:// or spark:// URL, or "local" to run +locally with one thread, or "local[N]" to run locally with N threads. + +## Running tests +Testing first requires [Building](#building) Spark. Once Spark is built, tests +can be run using: +`./sbt/sbt test` + ## A Note About Hadoop Versions -Spark uses the Hadoop core library to talk to HDFS and other -Hadoop-supported storage systems. Because the protocols have changed -in different versions of Hadoop, you must build Spark against the same -version that your cluster runs. You can change the version by setting -the `SPARK_HADOOP_VERSION` environment when building Spark. +Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported +storage systems. Because the protocols have changed in different versions of +Hadoop, you must build Spark against the same version that your cluster runs. +You can change the version by setting the `SPARK_HADOOP_VERSION` environment +when building Spark. For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: @@ -148,7 +62,7 @@ versions without YARN, use: # Cloudera CDH 4.2.0 with MapReduce v1 $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha @@ -157,8 +71,8 @@ with YARN, also set `SPARK_YARN=true`: # Cloudera CDH 4.2.0 with MapReduce v2 $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly -For convenience, these variables may also be set through the -`conf/spark-env.sh` file described below. + # Apache Hadoop 2.2.X and newer + $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're @@ -167,8 +81,7 @@ using Hadoop 1.2.1 and build your application using SBT, add this entry to "org.apache.hadoop" % "hadoop-client" % "1.2.1" -If your project is built with Maven, add this to your POM file's -`` section: +If your project is built with Maven, add this to your POM file's `` section: org.apache.hadoop @@ -179,19 +92,28 @@ If your project is built with Maven, add this to your POM file's ## Configuration -Please refer to the [Configuration -guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. -## Contributing to GraphX +## Apache Incubator Notice + +Apache Spark is an effort undergoing incubation at The Apache Software +Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of +all newly accepted projects until a further review indicates that the +infrastructure, communications, and decision making process have stabilized in +a manner consistent with other successful ASF projects. While incubation status +is not necessarily a reflection of the completeness or stability of the code, +it does indicate that the project has yet to be fully endorsed by the ASF. + + +## Contributing to Spark -Contributions via GitHub pull requests are gladly accepted from their -original author. Along with any pull requests, please state that the -contribution is your original work and that you license the work to -the project under the project's open source license. Whether or not -you state this explicitly, by submitting any copyrighted material via -pull request, email, or other means you agree to license the material -under the project's open source license and warrant that you have the -legal authority to do so. +Contributions via GitHub pull requests are gladly accepted from their original +author. Along with any pull requests, please state that the contribution is +your original work and that you license the work to the project under the +project's open source license. Whether or not you state this explicitly, by +submitting any copyrighted material via pull request, email, or other means +you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. From 59b03e015d581bbab74f1fe33a3ec1fd7840c3db Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 9 Jan 2014 11:58:18 +0530 Subject: [PATCH 1264/1571] Fixes corresponding to Reynolds feedback comments --- .../scala/org/apache/spark/SparkContext.scala | 30 +++++++++---------- .../org/apache/spark/rdd/HadoopRDD.scala | 11 ++++--- .../org/apache/spark/rdd/NewHadoopRDD.scala | 11 ++++--- .../scala/org/apache/spark/util/Utils.scala | 23 ++++++++------ 4 files changed, 43 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 97fec7f737dd7..bceeaa04482b7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -385,14 +385,14 @@ class SparkContext( * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits) * }}} */ - def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int, - cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F] - ): RDD[(K, V)] = { + def hadoopFile[K, V, F <: InputFormat[K, V]] + (path: String, minSplits: Int, cloneKeyValues: Boolean = true) + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { hadoopFile(path, - fm.runtimeClass.asInstanceOf[Class[F]], - km.runtimeClass.asInstanceOf[Class[K]], - vm.runtimeClass.asInstanceOf[Class[V]], - minSplits, + fm.runtimeClass.asInstanceOf[Class[F]], + km.runtimeClass.asInstanceOf[Class[K]], + vm.runtimeClass.asInstanceOf[Class[V]], + minSplits, cloneKeyValues = cloneKeyValues) } @@ -409,15 +409,15 @@ class SparkContext( hadoopFile[K, V, F](path, defaultMinSplits, cloneKeyValues) /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */ - def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String, - cloneKeyValues: Boolean = true) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F] - ): RDD[(K, V)] = { + def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]] + (path: String, cloneKeyValues: Boolean = true) + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { newAPIHadoopFile( - path, - fm.runtimeClass.asInstanceOf[Class[F]], - km.runtimeClass.asInstanceOf[Class[K]], - vm.runtimeClass.asInstanceOf[Class[V]], - cloneKeyValues = cloneKeyValues) + path, + fm.runtimeClass.asInstanceOf[Class[F]], + km.runtimeClass.asInstanceOf[Class[K]], + vm.runtimeClass.asInstanceOf[Class[V]], + cloneKeyValues = cloneKeyValues) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 13949a1bdb464..2da4611b9c0ff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -22,6 +22,7 @@ import java.io.EOFException import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configuration, Configurable} +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf @@ -91,7 +92,8 @@ class HadoopRDD[K: ClassTag, V: ClassTag]( inputFormatClass, keyClass, valueClass, - minSplits, cloneKeyValues) + minSplits, + cloneKeyValues) } protected val jobConfCacheKey = "rdd_%d_job_conf".format(id) @@ -162,10 +164,10 @@ class HadoopRDD[K: ClassTag, V: ClassTag]( // Register an on-task-completion callback to close the input stream. context.addOnCompleteCallback{ () => closeIfNeeded() } - val key: K = reader.createKey() + val keyCloneFunc = cloneWritables[K](getConf) val value: V = reader.createValue() - + val valueCloneFunc = cloneWritables[V](getConf) override def getNext() = { try { finished = !reader.next(key, value) @@ -174,7 +176,8 @@ class HadoopRDD[K: ClassTag, V: ClassTag]( finished = true } if (cloneKeyValues) { - (cloneWritables(key, getConf), cloneWritables(value, getConf)) + (keyCloneFunc(key.asInstanceOf[Writable]), + valueCloneFunc(value.asInstanceOf[Writable])) } else { (key, value) } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 5428fc5691d6e..e1f9995a9a12a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -92,7 +92,8 @@ class NewHadoopRDD[K: ClassTag, V: ClassTag]( // Register an on-task-completion callback to close the input stream. context.addOnCompleteCallback(() => close()) - + val keyCloneFunc = cloneWritables[K](conf) + val valueCloneFunc = cloneWritables[V](conf) var havePair = false var finished = false @@ -112,9 +113,11 @@ class NewHadoopRDD[K: ClassTag, V: ClassTag]( val key = reader.getCurrentKey val value = reader.getCurrentValue if (cloneKeyValues) { - (cloneWritables(key, conf), cloneWritables(value, conf)) - } else - (key, value) + (keyCloneFunc(key.asInstanceOf[Writable]), + valueCloneFunc(value.asInstanceOf[Writable])) + } else { + (key, value) + } } private def close() { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 192806e1782c1..23b72701c2c14 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -51,15 +51,20 @@ private[spark] object Utils extends Logging { * intention is to optimize, for example for NullWritable there is no need and for Long, int and * String creating a new object with value set would be faster. */ - def cloneWritables[T: ClassTag](obj: T, conf: Configuration): T = { - val cloned = classTag[T] match { - case ClassTag(_: Text) => new Text(obj.asInstanceOf[Text].getBytes) - case ClassTag(_: LongWritable) => new LongWritable(obj.asInstanceOf[LongWritable].get) - case ClassTag(_: IntWritable) => new IntWritable(obj.asInstanceOf[IntWritable].get) - case ClassTag(_: NullWritable) => obj // TODO: should we clone this ? - case _ => WritableUtils.clone(obj.asInstanceOf[Writable], conf) // slower way of cloning. - } - cloned.asInstanceOf[T] + def cloneWritables[T: ClassTag](conf: Configuration): Writable => T = { + val cloneFunc = classTag[T] match { + case ClassTag(_: Text) => + (w: Writable) => new Text(w.asInstanceOf[Text].getBytes).asInstanceOf[T] + case ClassTag(_: LongWritable) => + (w: Writable) => new LongWritable(w.asInstanceOf[LongWritable].get).asInstanceOf[T] + case ClassTag(_: IntWritable) => + (w: Writable) => new IntWritable(w.asInstanceOf[IntWritable].get).asInstanceOf[T] + case ClassTag(_: NullWritable) => + (w: Writable) => w.asInstanceOf[T] // TODO: should we clone this ? + case _ => + (w: Writable) => WritableUtils.clone(w, conf).asInstanceOf[T] // slower way of cloning. + } + cloneFunc } /** Serialize an object using Java serialization */ From 4d2e388e6a2df3a8e24f63d7a61074ad8f8c9ee4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 23:56:53 -0800 Subject: [PATCH 1265/1571] Don't delegate to users `sbt`. This changes our `sbt/sbt` script to not delegate to the user's `sbt` even if it is present. If users already have sbt installed and they want to use their own sbt, we'd expect them to just call sbt directly from within Spark. We no longer set any enironment variables or anything from this script, so they should just launch sbt directly on their own. There are a number of hard-to-debug issues which can come from the current appraoch. One is if the user is unaware of an existing sbt installation and now without explanation their build breaks because they haven't configured options correctly (such as permgen size) within their sbt. Another is if the user has a much older version of sbt hanging around, in which case some of the older versions don't acutally work well when newer verisons of sbt are specified in the build file (reported by @marmbrus). A third is if the user has done some other modification to their sbt script, such as setting it to delegate to sbt/sbt in Spark, and this causes that to break (also reported by @marmbrus). So to keep things simple let's just avoid this path and remove it. Any user who already has sbt and wants to build spark with it should be able to understand easily how to do it. --- sbt/sbt | 51 ++++++++++++++++++++------------------------------- 1 file changed, 20 insertions(+), 31 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index 7f47d90cf11bb..98c86db702451 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,37 +25,26 @@ URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/s URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar JAR=sbt/sbt-launch-${SBT_VERSION}.jar -printf "Checking for system sbt [" -if hash sbt 2>/dev/null; then - printf "FOUND]\n" - # Use System SBT - sbt "$@" -else - printf "NOT FOUND]\n" - # Download sbt or use already downloaded - if [ ! -d .sbtlib ]; then - mkdir .sbtlib - fi - if [ ! -f ${JAR} ]; then - # Download - printf "Attempting to fetch sbt\n" - if hash curl 2>/dev/null; then - curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} - elif hash wget 2>/dev/null; then - wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} - else - printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 - fi - fi - if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" +# Download sbt or use already downloaded +if [ ! -f ${JAR} ]; then + # Download + printf "Attempting to fetch sbt\n" + if hash curl 2>/dev/null; then + curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR} + elif hash wget 2>/dev/null; then + wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR} + else + printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit -1 fi - printf "Launching sbt from ${JAR}\n" - java \ - -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ - -jar ${JAR} \ - "$@" fi +if [ ! -f ${JAR} ]; then + # We failed to download + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 +fi +printf "Launching sbt from ${JAR}\n" +java \ + -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ + -jar ${JAR} \ + "$@" From a01f3401e32ca4324884d13c9fad53c6c87bb5f0 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 8 Jan 2014 17:32:15 -0500 Subject: [PATCH 1266/1571] Use typed getters for configuration settings --- .../scala/org/apache/spark/SparkContext.scala | 4 ++-- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../apache/spark/api/python/PythonRDD.scala | 4 ++-- .../spark/broadcast/HttpBroadcast.scala | 4 ++-- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 6 ++--- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../apache/spark/io/CompressionCodec.scala | 2 +- .../spark/network/ConnectionManager.scala | 18 +++++++-------- .../spark/network/netty/ShuffleCopier.scala | 2 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 4 ++-- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 8 +++---- .../spark/scheduler/TaskSetManager.scala | 8 +++---- .../CoarseGrainedSchedulerBackend.scala | 5 ++--- .../cluster/SimrSchedulerBackend.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/serializer/KryoSerializer.scala | 4 ++-- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 22 +++++++++---------- .../spark/storage/BlockManagerMaster.scala | 4 ++-- .../spark/storage/DiskBlockManager.scala | 2 +- .../spark/storage/ShuffleBlockManager.scala | 4 ++-- .../spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/util/AkkaUtils.scala | 20 ++++++++--------- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../apache/spark/util/XORShiftRandom.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../spark/util/XORShiftRandomSuite.scala | 2 +- .../dstream/NetworkInputDStream.scala | 4 ++-- .../spark/streaming/scheduler/Job.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 2 +- .../streaming/scheduler/JobScheduler.scala | 2 +- 34 files changed, 78 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0e47f4e442927..9d52544371cce 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -116,7 +116,7 @@ class SparkContext( throw new SparkException("An application must be set in your configuration") } - if (conf.get("spark.logConf", "false").toBoolean) { + if (conf.getBoolean("spark.logConf", false)) { logInfo("Spark configuration:\n" + conf.toDebugString) } @@ -1203,7 +1203,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new TaskSchedulerImpl(sc) - val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean + val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2e36ccb9a0f07..e093e2f162022 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -162,7 +162,7 @@ object SparkEnv extends Logging { actorSystem.actorOf(Props(newActor), name = name) } else { val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.get("spark.driver.port", "7077").toInt + val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 32cc70e8c9dda..40c519b5bd970 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = conf.get("spark.buffer.size", "65536").toInt + val bufferSize = conf.getInt("spark.buffer.size", 65536) override def getPartitions = parent.partitions @@ -250,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt + val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index db596d5fcc054..0eacda3d7dc2b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -92,8 +92,8 @@ private object HttpBroadcast extends Logging { def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = conf.get("spark.buffer.size", "65536").toInt - compress = conf.get("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getInt("spark.buffer.size", 65536) + compress = conf.getBoolean("spark.broadcast.compress", true) if (isDriver) { createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 95309382786a9..fdf92eca4f25e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -180,7 +180,7 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 6617b7100f44b..066d1107d4f95 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -43,9 +43,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val conf = new SparkConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt + val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 + val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index fcaf4e92b1858..0538e521f5e24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -55,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index a1e98845f6a84..59801773205bd 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -71,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768) new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 46c40d0a2a029..e6e01783c8895 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - conf.get("spark.core.connection.handler.threads.min", "20").toInt, - conf.get("spark.core.connection.handler.threads.max", "60").toInt, - conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.getInt("spark.core.connection.handler.threads.min", 20), + conf.getInt("spark.core.connection.handler.threads.max", 60), + conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - conf.get("spark.core.connection.io.threads.min", "4").toInt, - conf.get("spark.core.connection.io.threads.max", "32").toInt, - conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.getInt("spark.core.connection.io.threads.min", 4), + conf.getInt("spark.core.connection.io.threads.max", 32), + conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - conf.get("spark.core.connection.connect.threads.min", "1").toInt, - conf.get("spark.core.connection.connect.threads.max", "8").toInt, - conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.getInt("spark.core.connection.connect.threads.min", 1), + conf.getInt("spark.core.connection.connect.threads.max", 8), + conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index b729eb11c5142..d87157e12c487 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000) val fc = new FileClient(handler, connectTimeout) try { diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 6d4f46125f1a6..83109d1a6f853 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -97,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getInt("spark.buffer.size", 65536) val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -131,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging { ): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) - val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getInt("spark.buffer.size", 65536) val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index e22b1e53e8048..c52d6175d28fa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends Logging { - private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt + private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0c8ed6275991a..d4f74d3e18543 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -51,15 +51,15 @@ private[spark] class TaskSchedulerImpl( isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt) + def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -125,7 +125,7 @@ private[spark] class TaskSchedulerImpl( override def start() { backend.start() - if (!isLocal && conf.get("spark.speculation", "false").toBoolean) { + if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 6dd1469d8f801..a10e5397ad4a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -57,11 +57,11 @@ private[spark] class TaskSetManager( val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) + val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5) // Serializer for closures and tasks. val env = SparkEnv.get @@ -116,7 +116,7 @@ private[spark] class TaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - conf.get("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getLong("spark.logging.exceptionPrintInterval", 10000) // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 2f5bcafe40394..8d596a76c224c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -63,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -209,8 +209,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } override def defaultParallelism(): Int = { - conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse( - math.max(totalCoreCount.get(), 2)) + conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2)) } // Called by subclasses when notified of a lost worker diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index b44d1e43c85c7..d99c76117c168 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -33,7 +33,7 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = conf.get("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getInt("spark.simr.executor.cores", 1) override def start() { super.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d46fceba8918a..e16d60c54cc1c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -77,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) var nextMesosTaskId = 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index ae8d527352f73..b428c82a48f59 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index a24a3b04b87cc..c14cd47556987 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -36,7 +36,7 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = { - conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +48,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getBoolean("spark.kryo.referenceTracking", true)) for (cls <- KryoSerializer.toRegister) kryo.register(cls) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 47478631a11f0..4fa2ab96d9725 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6d2cda97b04eb..c56e2ca2df08c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -58,8 +58,8 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) + val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0) if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } @@ -72,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) // Whether to compress shuffle output that are stored - val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getBoolean("spark.rdd.compress", false) val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) @@ -443,7 +443,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (conf.get("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getBoolean("spark.shuffle.use.netty", false)) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -469,7 +469,7 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean + val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } @@ -864,15 +864,15 @@ private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator def getMaxMemory(conf: SparkConf): Long = { - val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.66) (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = - conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) / 4 def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = - conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.getBoolean("spark.test.disableBlockManagerHeartBeat", false) /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 51a29ed8ef81a..c54e4f2664753 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -30,8 +30,8 @@ import org.apache.spark.util.AkkaUtils private[spark] class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) + val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 55dcb3742c967..edc1133172d2a 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64) // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 39dc7bb19afee..e2b24298a55e8 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -64,9 +64,9 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.get("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getBoolean("spark.shuffle.consolidateFiles", false) - private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index d6d9f0cedf923..bcd282445050d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.get("spark.ui.retainedStages", "1000").toInt + val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 3f009a8998cbd..761d378c7fd8b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -44,13 +44,13 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = conf.get("spark.akka.threads", "4").toInt - val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getInt("spark.akka.threads", 4) + val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getInt("spark.akka.timeout", 100) - val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt - val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean + val akkaFrameSize = conf.getInt("spark.akka.frameSize", 10) + val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" if (!akkaLogLifecycleEvents) { // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent. @@ -58,12 +58,12 @@ private[spark] object AkkaUtils { Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) } - val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off" + val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 600) val akkaFailureDetector = - conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt + conf.getDouble("spark.akka.failure-detector.threshold", 300.0) + val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( ConfigFactory.parseString( @@ -103,7 +103,7 @@ private[spark] object AkkaUtils { /** Returns the default Spark timeout to use for Akka ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds") + Duration.create(conf.getLong("spark.akka.askTimeout", 30), "seconds") } /** Returns the default Spark timeout to use for Akka remote actor lookup. */ diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index aa7f52cafbf37..3d1e90a3522a4 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -74,7 +74,7 @@ object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { def getDelaySeconds(conf: SparkConf) = { - conf.get("spark.cleaner.ttl", "3500").toInt + conf.getInt("spark.cleaner.ttl", 3500) } def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = diff --git a/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala index e9907e6c855ae..08b31ac64f290 100644 --- a/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/XORShiftRandom.scala @@ -91,4 +91,4 @@ private[spark] object XORShiftRandom { } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 1eec6726f48bc..c9f6cc5d079b5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -83,7 +83,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong + val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) val MAX_TASK_FAILURES = 4 test("TaskSet with no preferences") { diff --git a/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala index b78367b6cac02..f1d7b61b31e63 100644 --- a/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/XORShiftRandomSuite.scala @@ -73,4 +73,4 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 27d474c0a0459..d41f726f8322c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -175,7 +175,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.get("spark.driver.port", "7077").toInt + val port = env.conf.getInt("spark.driver.port", 7077) val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -212,7 +212,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala index 7341bfbc99399..c8ee93bf5bde7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -38,4 +38,4 @@ class Job(val time: Time, func: () => _) { } override def toString = id -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 5f8be93a98518..3c624e8199a7e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -104,7 +104,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getLong("spark.streaming.manualClock.jump", 0) clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 9304fc1a9338d..30c070c274d85 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -31,7 +31,7 @@ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { val jobSets = new ConcurrentHashMap[Time, JobSet] - val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt + val numConcurrentJobs = ssc.conf.getInt("spark.streaming.concurrentJobs", 1) val executor = Executors.newFixedThreadPool(numConcurrentJobs) val generator = new JobGenerator(this) val listenerBus = new StreamingListenerBus() From 49cbf48bcc4a3866984c8370da1917dd1c142115 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 00:12:34 -0800 Subject: [PATCH 1267/1571] Small typo fix --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 98c86db702451..62ead8a69dbf6 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,7 +25,7 @@ URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/s URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar JAR=sbt/sbt-launch-${SBT_VERSION}.jar -# Download sbt or use already downloaded +# Download sbt launch jar if it hasn't been downloaded yet if [ ! -f ${JAR} ]; then # Download printf "Attempting to fetch sbt\n" From 7309a29c755ae833f35523546e851be4c2f328be Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 00:13:23 -0800 Subject: [PATCH 1268/1571] Removed Kryo dependency and graphx-shell --- .../apache/spark/util/collection/BitSet.scala | 2 +- .../spark/util/collection/OpenHashSet.scala | 2 +- .../scala/org/apache/spark/graph/Edge.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 2 +- .../spark/graph/impl/MessageToPartition.scala | 4 +- .../graph/impl/ReplicatedVertexView.scala | 3 +- graphx-shell | 124 ------------------ 7 files changed, 8 insertions(+), 131 deletions(-) delete mode 100755 graphx-shell diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 0e12779152bda..f6e03bc0e190d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -22,7 +22,7 @@ package org.apache.spark.util.collection * A simple, fixed-size bit set implementation. This implementation is fast because it avoids * safety/bound checking. */ -class BitSet(numBits: Int) { +class BitSet(numBits: Int) extends Serializable { private val words = new Array[Long](bit2words(numBits)) private val numWords = words.length diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index c908512b0f6db..895ccb9be0ca9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -279,7 +279,7 @@ object OpenHashSet { * A set of specialized hash function implementation to avoid boxing hash code computation * in the specialized implementation of OpenHashSet. */ - sealed class Hasher[@specialized(Long, Int) T] { + sealed class Hasher[@specialized(Long, Int) T] extends Serializable { def hash(o: T): Int = o.hashCode() } diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 7e8ae7c790b84..5ac77839eb866 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -19,7 +19,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** * The attribute associated with the edge. */ - var attr: ED = nullValue[ED]) { + var attr: ED = nullValue[ED]) extends Serializable { /** * Given one vertex in the edge return the other vertex. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 7367269f67bfa..7ae4d7df43a2f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -19,7 +19,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) val srcIds: Array[Vid], val dstIds: Array[Vid], val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[Vid, Int]) { + val index: PrimitiveKeyOpenHashMap[Vid, Int]) extends Serializable { /** * Reverse all the edges in this partition. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 66fe796d2eae5..bf033945dee24 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -11,7 +11,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: Pid, var vid: Vid, var data: T) - extends Product2[Pid, (Vid, T)] { + extends Product2[Pid, (Vid, T)] with Serializable { override def _1 = partition @@ -29,7 +29,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( @transient var partition: Pid, var data: T) - extends Product2[Pid, T] { + extends Product2[Pid, T] with Serializable { override def _1 = partition diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 033971c1af3c3..970acfed27425 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -175,6 +175,7 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) { +class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) + extends Serializable { def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graphx-shell b/graphx-shell deleted file mode 100755 index 4dd6c68ace888..0000000000000 --- a/graphx-shell +++ /dev/null @@ -1,124 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. -# Options: -# -c Set the number of cores for REPL to use -# - -# Enter posix mode for bash -set -o posix - - -# Update the the banner logo -export SPARK_BANNER_TEXT="Welcome to - ______ __ _ __ - / ____/________ _____ / /_ | |/ / - / / __/ ___/ __ \`/ __ \/ __ \| / - / /_/ / / / /_/ / /_/ / / / / | - \____/_/ \__,_/ .___/_/ /_/_/|_| - /_/ Alpha Release - -Powered by: - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ \`/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ - /_/ version 0.9.0 - -Example: - - scala> val graph = GraphLoader.textFile(sc, \"hdfs://links\") - scala> graph.numVertices - scala> graph.numEdges - scala> val pageRankGraph = Analytics.pagerank(graph, 10) // 10 iterations - scala> val maxPr = pageRankGraph.vertices.map{ case (vid, pr) => pr }.max - scala> println(maxPr) - -" - -export SPARK_SHELL_INIT_BLOCK="import org.apache.spark.graph._;" - -# Set the serializer to use Kryo for graphx objects -SPARK_JAVA_OPTS+=" -Dspark.serializer=org.apache.spark.serializer.KryoSerializer " -SPARK_JAVA_OPTS+="-Dspark.kryo.registrator=org.apache.spark.graph.GraphKryoRegistrator " -SPARK_JAVA_OPTS+="-Dspark.kryoserializer.buffer.mb=10 " - - - -FWDIR="`dirname $0`" - -for o in "$@"; do - if [ "$1" = "-c" -o "$1" = "--cores" ]; then - shift - if [ -n "$1" ]; then - OPTIONS="-Dspark.cores.max=$1" - shift - fi - fi -done - -# Set MASTER from spark-env if possible -if [ -z "$MASTER" ]; then - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi - if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then - MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - export MASTER - fi -fi - -# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in -# binary distribution of Spark where Scala is not installed -exit_status=127 -saved_stty="" - -# restore stty settings (echo in particular) -function restoreSttySettings() { - stty $saved_stty - saved_stty="" -} - -function onExit() { - if [[ "$saved_stty" != "" ]]; then - restoreSttySettings - fi - exit $exit_status -} - -# to reenable echo if we are interrupted before completing. -trap onExit INT - -# save terminal settings -saved_stty=$(stty -g 2>/dev/null) -# clear on error so we don't later try to restore them -if [[ ! $? ]]; then - saved_stty="" -fi - -$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@" - -# record the exit status lest it be overwritten: -# then reenable echo and propagate the code. -exit_status=$? -onExit From 295d82583a8e162a41a7cc59a6612490580ffe09 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 9 Jan 2014 00:30:22 -0800 Subject: [PATCH 1269/1571] Minor update on SparkContext.broadcast's JavaDoc. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 47574ab34cdac..fce8f2d48cf2b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -546,8 +546,8 @@ class SparkContext( /** * Broadcast a read-only variable to the cluster, returning a - * [[org.apache.spark.broadcast.Broadcast]] object for - * reading it in distributed functions. The variable will be sent to each cluster only once. + * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. + * The variable will be sent to each cluster only once. */ def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) From 181471906ed590347cbbe3422bd92e9b82f9e1bf Mon Sep 17 00:00:00 2001 From: Pillis Date: Thu, 9 Jan 2014 10:16:19 +0100 Subject: [PATCH 1270/1571] SPARK-961 Add a Vector.random() method --- .../scala/org/apache/spark/util/Vector.scala | 4 ++ .../org/apache/spark/util/VectorSuite.scala | 46 +++++++++++++++++++ 2 files changed, 50 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/util/VectorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index fe710c58acc44..f9c6cdf2be8a5 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import scala.util.Random + class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length @@ -124,6 +126,8 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) + def random(length: Int, random: Random = new Random()) = Vector(length, _ => random.nextDouble()); + class Multiplier(num: Double) { def * (vec: Vector) = vec * num } diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala new file mode 100644 index 0000000000000..23d1bdb193372 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -0,0 +1,46 @@ +/* + * 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.util + +import scala.util.Random + +import org.scalatest.FunSuite + +/** + * Tests org.apache.spark.util.Vector functionality + */ +class VectorSuite extends FunSuite { + + def verifyVector(vector: Vector, expectedLength: Int) = { + assert(vector.length == expectedLength); // Array must be of expected length + assert(vector.length == vector.elements.distinct.length); // Values should not repeat + assert(vector.sum > 0); // All values must not be 0 + assert(vector.sum < vector.length); // All values must not be 1 + assert(vector.elements.product > 0); // No value is 0 + } + + test("random with default random number generator") { + val vector100 = Vector.random(100); + verifyVector(vector100, 100); + } + + test("random with given random number generator") { + val vector100 = Vector.random(100, new Random(100)); + verifyVector(vector100, 100); + } +} From c617083e478e3cfbddc4232060aa7b7a0c5812d4 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 Jan 2014 09:53:51 -0600 Subject: [PATCH 1271/1571] yarn-client addJar fix and misc other --- .../scala/org/apache/spark/SparkContext.scala | 8 +-- docs/running-on-yarn.md | 15 +++++- .../spark/deploy/yarn/WorkerLauncher.scala | 29 +++++++++-- .../cluster/YarnClientSchedulerBackend.scala | 50 ++++++++++--------- .../spark/deploy/yarn/WorkerLauncher.scala | 29 +++++++++-- 5 files changed, 94 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fce8f2d48cf2b..f1695c936d72d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -669,10 +669,10 @@ class SparkContext( key = uri.getScheme match { // A JAR file which exists only on the driver node case null | "file" => - if (SparkHadoopUtil.get.isYarnMode()) { - // In order for this to work on yarn the user must specify the --addjars option to - // the client to upload the file into the distributed cache to make it show up in the - // current working directory. + if (SparkHadoopUtil.get.isYarnMode() && master == "yarn-standalone") { + // In order for this to work in yarn standalone mode the user must specify the + // --addjars option to the client to upload the file into the distributed cache + // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { env.httpFileServer.addJar(new File(fileName)) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b20627010798a..3bd62646bab06 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -101,7 +101,19 @@ With this mode, your application is actually run on the remote machine where the With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR -In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh +Configuration in yarn-client mode: + +In order to tune worker core/number/memory etc. You need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. + +* `SPARK_YARN_APP_JAR`, Path to your application's JAR file (required) +* `SPARK_WORKER_INSTANCES`, Number of workers to start (Default: 2) +* `SPARK_WORKER_CORES`, Number of cores for the workers (Default: 1). +* `SPARK_WORKER_MEMORY`, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +* `SPARK_MASTER_MEMORY`, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +* `SPARK_YARN_APP_NAME`, The name of your application (Default: Spark) +* `SPARK_YARN_QUEUE`, The hadoop queue to use for allocation requests (Default: 'default') +* `SPARK_YARN_DIST_FILES`, Comma separated list of files to be distributed with the job. +* `SPARK_YARN_DIST_ARCHIVES`, Comma separated list of archives to be distributed with the job. For example: @@ -114,7 +126,6 @@ For example: SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./bin/spark-shell -You can also send extra files to yarn cluster for worker to use by exporting SPARK_YARN_DIST_FILES=file1,file2... etc. # Building Spark for Hadoop/YARN 2.2.x diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index ddfec1a4ac672..66e38ee84051e 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -76,6 +76,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar def run() { + // Setup the directories so things go to yarn approved directories rather + // then user specified and /tmp. + System.setProperty("spark.local.dir", getLocalDirs()) + appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() @@ -103,10 +107,12 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - // must be <= timeoutInterval/ 2. - // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. - // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. - val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + // we want to be reasonably responsive without causing too many requests to RM. + val schedulerInterval = + System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + // must be <= timeoutInterval / 2. + val interval = math.min(timeoutInterval / 2, schedulerInterval) + reporterThread = launchReporterThread(interval) // Wait for the reporter thread to Finish. @@ -119,6 +125,21 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar System.exit(0) } + /** Get the Yarn approved local directories. */ + private def getLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs + } + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 4b1b5da048df4..22e55e0c60647 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -22,6 +22,8 @@ import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl +import scala.collection.mutable.ArrayBuffer + private[spark] class YarnClientSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) @@ -31,45 +33,47 @@ private[spark] class YarnClientSchedulerBackend( var client: Client = null var appId: ApplicationId = null + private[spark] def addArg(optionName: String, optionalParam: String, arrayBuf: ArrayBuffer[String]) { + Option(System.getenv(optionalParam)) foreach { + optParam => { + arrayBuf += (optionName, optParam) + } + } + } + override def start() { super.start() - val defalutWorkerCores = "2" - val defalutWorkerMemory = "512m" - val defaultWorkerNumber = "1" - val userJar = System.getenv("SPARK_YARN_APP_JAR") - val distFiles = System.getenv("SPARK_YARN_DIST_FILES") - var workerCores = System.getenv("SPARK_WORKER_CORES") - var workerMemory = System.getenv("SPARK_WORKER_MEMORY") - var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") - if (userJar == null) throw new SparkException("env SPARK_YARN_APP_JAR is not set") - if (workerCores == null) - workerCores = defalutWorkerCores - if (workerMemory == null) - workerMemory = defalutWorkerMemory - if (workerNumber == null) - workerNumber = defaultWorkerNumber - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort - val argsArray = Array[String]( + val argsArrayBuf = new ArrayBuffer[String]() + argsArrayBuf += ( "--class", "notused", "--jar", userJar, "--args", hostport, - "--worker-memory", workerMemory, - "--worker-cores", workerCores, - "--num-workers", workerNumber, - "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher", - "--files", distFiles + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" ) - val args = new ClientArguments(argsArray, conf) + // process any optional arguments, use the defaults already defined in ClientArguments + // if things aren't specified + Map("--master-memory" -> "SPARK_MASTER_MEMORY", + "--num-workers" -> "SPARK_WORKER_INSTANCES", + "--worker-memory" -> "SPARK_WORKER_MEMORY", + "--worker-cores" -> "SPARK_WORKER_CORES", + "--queue" -> "SPARK_YARN_QUEUE", + "--name" -> "SPARK_YARN_APP_NAME", + "--files" -> "SPARK_YARN_DIST_FILES", + "--archives" -> "SPARK_YARN_DIST_ARCHIVES") + .foreach { case (optName, optParam) => addArg(optName, optParam, argsArrayBuf) } + + logDebug("ClientArguments called with: " + argsArrayBuf) + val args = new ClientArguments(argsArrayBuf.toArray, conf) client = new Client(args, conf) appId = client.runApp() waitForApp() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 49248a8516b9c..3e3a4672b4c61 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -78,6 +78,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar def run() { + // Setup the directories so things go to yarn approved directories rather + // then user specified and /tmp. + System.setProperty("spark.local.dir", getLocalDirs()) + amClient = AMRMClient.createAMRMClient() amClient.init(yarnConf) amClient.start() @@ -94,10 +98,12 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - // must be <= timeoutInterval/ 2. - // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. - // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. - val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L)) + // we want to be reasonably responsive without causing too many requests to RM. + val schedulerInterval = + System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + // must be <= timeoutInterval / 2. + val interval = math.min(timeoutInterval / 2, schedulerInterval) + reporterThread = launchReporterThread(interval) // Wait for the reporter thread to Finish. @@ -110,6 +116,21 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar System.exit(0) } + /** Get the Yarn approved local directories. */ + private def getLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs + } + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) From 5fccd7051d40ca022f4d3bab661c8f53a748aa8d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 Jan 2014 10:32:35 -0600 Subject: [PATCH 1272/1571] add application type --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index be323d77835a8..952e963389c0a 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -99,6 +99,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) appContext.setApplicationName(args.appName) appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) + appContext.setApplicationType("SPARK") // Memory for the ApplicationMaster. val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] From e4483582fc59330af8a43e8a152959f927103c79 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 10:23:35 -0800 Subject: [PATCH 1273/1571] Add docs/graphx-programming-guide.md from 7210257ba3038d5e22d4b60fe9c3113dc45c3dff:README.md --- docs/graphx-programming-guide.md | 197 +++++++++++++++++++++++++++++++ 1 file changed, 197 insertions(+) create mode 100644 docs/graphx-programming-guide.md diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md new file mode 100644 index 0000000000000..5b06d82225229 --- /dev/null +++ b/docs/graphx-programming-guide.md @@ -0,0 +1,197 @@ +# GraphX: Unifying Graphs and Tables + + +GraphX extends the distributed fault-tolerant collections API and +interactive console of [Spark](http://spark.incubator.apache.org) with +a new graph API which leverages recent advances in graph systems +(e.g., [GraphLab](http://graphlab.org)) to enable users to easily and +interactively build, transform, and reason about graph structured data +at scale. + + +## Motivation + +From social networks and targeted advertising to protein modeling and +astrophysics, big graphs capture the structure in data and are central +to the recent advances in machine learning and data mining. Directly +applying existing *data-parallel* tools (e.g., +[Hadoop](http://hadoop.apache.org) and +[Spark](http://spark.incubator.apache.org)) to graph computation tasks +can be cumbersome and inefficient. The need for intuitive, scalable +tools for graph computation has lead to the development of new +*graph-parallel* systems (e.g., +[Pregel](http://http://giraph.apache.org) and +[GraphLab](http://graphlab.org)) which are designed to efficiently +execute graph algorithms. Unfortunately, these systems do not address +the challenges of graph construction and transformation and provide +limited fault-tolerance and support for interactive analysis. + +

      + +

      + + + +## Solution + +The GraphX project combines the advantages of both data-parallel and +graph-parallel systems by efficiently expressing graph computation +within the [Spark](http://spark.incubator.apache.org) framework. We +leverage new ideas in distributed graph representation to efficiently +distribute graphs as tabular data-structures. Similarly, we leverage +advances in data-flow systems to exploit in-memory computation and +fault-tolerance. We provide powerful new operations to simplify graph +construction and transformation. Using these primitives we implement +the PowerGraph and Pregel abstractions in less than 20 lines of code. +Finally, by exploiting the Scala foundation of Spark, we enable users +to interactively load, transform, and compute on massive graphs. + +

      + +

      + +## Examples + +Suppose I want to build a graph from some text files, restrict the graph +to important relationships and users, run page-rank on the sub-graph, and +then finally return attributes associated with the top users. I can do +all of this in just a few lines with GraphX: + +```scala +// Connect to the Spark cluster +val sc = new SparkContext("spark://master.amplab.org", "research") + +// Load my user data and prase into tuples of user id and attribute list +val users = sc.textFile("hdfs://user_attributes.tsv") + .map(line => line.split).map( parts => (parts.head, parts.tail) ) + +// Parse the edge data which is already in userId -> userId format +val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") + +// Attach the user attributes +val graph = followerGraph.outerJoinVertices(users){ + case (uid, deg, Some(attrList)) => attrList + // Some users may not have attributes so we set them as empty + case (uid, deg, None) => Array.empty[String] + } + +// Restrict the graph to users which have exactly two attributes +val subgraph = graph.subgraph((vid, attr) => attr.size == 2) + +// Compute the PageRank +val pagerankGraph = Analytics.pagerank(subgraph) + +// Get the attributes of the top pagerank users +val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices){ + case (uid, attrList, Some(pr)) => (pr, attrList) + case (uid, attrList, None) => (pr, attrList) + } + +println(userInfoWithPageRank.top(5)) + +``` + + +## Online Documentation + +You can find the latest Spark documentation, including a programming +guide, on the project webpage at +. This README +file only contains basic setup instructions. + + +## Building + +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The +project is built using Simple Build Tool (SBT), which is packaged with +it. To build Spark and its example programs, run: + + sbt/sbt assembly + +Once you've built Spark, the easiest way to start using it is the +shell: + + ./spark-shell + +Or, for the Python API, the Python shell (`./pyspark`). + +Spark also comes with several sample programs in the `examples` +directory. To run one of them, use `./run-example +`. For example: + + ./run-example org.apache.spark.examples.SparkLR local[2] + +will run the Logistic Regression example locally on 2 CPUs. + +Each of the example programs prints usage help if no params are given. + +All of the Spark samples take a `` parameter that is the +cluster URL to connect to. This can be a mesos:// or spark:// URL, or +"local" to run locally with one thread, or "local[N]" to run locally +with N threads. + + +## A Note About Hadoop Versions + +Spark uses the Hadoop core library to talk to HDFS and other +Hadoop-supported storage systems. Because the protocols have changed +in different versions of Hadoop, you must build Spark against the same +version that your cluster runs. You can change the version by setting +the `SPARK_HADOOP_VERSION` environment when building Spark. + +For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop +versions without YARN, use: + + # Apache Hadoop 1.2.1 + $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + + # Cloudera CDH 4.2.0 with MapReduce v1 + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +with YARN, also set `SPARK_YARN=true`: + + # Apache Hadoop 2.0.5-alpha + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly + + # Cloudera CDH 4.2.0 with MapReduce v2 + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly + +For convenience, these variables may also be set through the +`conf/spark-env.sh` file described below. + +When developing a Spark application, specify the Hadoop version by adding the +"hadoop-client" artifact to your project's dependencies. For example, if you're +using Hadoop 1.2.1 and build your application using SBT, add this entry to +`libraryDependencies`: + + "org.apache.hadoop" % "hadoop-client" % "1.2.1" + +If your project is built with Maven, add this to your POM file's +`` section: + + + org.apache.hadoop + hadoop-client + 1.2.1 + + + +## Configuration + +Please refer to the [Configuration +guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +in the online documentation for an overview on how to configure Spark. + + +## Contributing to GraphX + +Contributions via GitHub pull requests are gladly accepted from their +original author. Along with any pull requests, please state that the +contribution is your original work and that you license the work to +the project under the project's open source license. Whether or not +you state this explicitly, by submitting any copyrighted material via +pull request, email, or other means you agree to license the material +under the project's open source license and warrant that you have the +legal authority to do so. + From 35f80da21aaea8c6fde089754ef3a86bc78e0428 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Jan 2014 20:54:24 -0800 Subject: [PATCH 1274/1571] Set default logging to WARN for Spark streaming examples. This programatically sets the log level to WARN by default for streaming tests. If the user has already specified a log4j.properties file, the user's file will take precedence over this default. --- .../examples/JavaFlumeEventCount.java | 2 ++ .../examples/JavaKafkaWordCount.java | 2 ++ .../examples/JavaNetworkWordCount.java | 2 ++ .../streaming/examples/JavaQueueStream.java | 2 ++ .../streaming/examples/ActorWordCount.scala | 12 +++++------ .../streaming/examples/FlumeEventCount.scala | 4 +++- .../streaming/examples/HdfsWordCount.scala | 3 ++- .../streaming/examples/KafkaWordCount.scala | 5 +++-- .../streaming/examples/MQTTWordCount.scala | 8 +++---- .../streaming/examples/NetworkWordCount.scala | 2 ++ .../streaming/examples/QueueStream.scala | 8 ++++--- .../streaming/examples/RawNetworkGrep.scala | 5 +++-- .../examples/StatefulNetworkWordCount.scala | 2 ++ .../examples/StreamingExamples.scala | 21 +++++++++++++++++++ .../examples/TwitterAlgebirdCMS.scala | 10 +++++---- .../examples/TwitterAlgebirdHLL.scala | 9 +++++--- .../examples/TwitterPopularTags.scala | 2 ++ .../streaming/examples/ZeroMQWordCount.scala | 1 + .../examples/clickstream/PageViewStream.scala | 4 +++- .../org/apache/spark/streaming/DStream.scala | 2 +- .../apache/spark/streaming/DStreamGraph.scala | 8 +++---- 21 files changed, 80 insertions(+), 34 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index b11cfa667eb92..7b5a243e26414 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -47,6 +47,8 @@ public static void main(String[] args) { System.exit(1); } + StreamingExamples.setStreamingLogLevels(); + String master = args[0]; String host = args[1]; int port = Integer.parseInt(args[2]); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 16b8a948e6154..04f62ee204145 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -59,6 +59,8 @@ public static void main(String[] args) { System.exit(1); } + StreamingExamples.setStreamingLogLevels(); + // Create the context with a 1 second batch size JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount", new Duration(2000), System.getenv("SPARK_HOME"), diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 1e2efd359cff2..c37b0cacc9a51 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -53,6 +53,8 @@ public static void main(String[] args) { System.exit(1); } + StreamingExamples.setStreamingLogLevels(); + // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", new Duration(1000), System.getenv("SPARK_HOME"), diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index e05551ab83301..7ef9c6c8f4aaf 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -41,6 +41,8 @@ public static void main(String[] args) throws Exception { System.exit(1); } + StreamingExamples.setStreamingLogLevels(); + // Create the context JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class)); diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 4e0058cd70777..57e1b1f806e82 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -18,17 +18,13 @@ package org.apache.spark.streaming.examples import scala.collection.mutable.LinkedList -import scala.util.Random import scala.reflect.ClassTag +import scala.util.Random -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Props -import akka.actor.actorRef2Scala +import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} import org.apache.spark.SparkConf -import org.apache.spark.streaming.Seconds -import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.util.AkkaUtils @@ -147,6 +143,8 @@ object ActorWordCount { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val Seq(master, host, port) = args.toSeq // Create the context and set the batch size diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index ae3709b3d97f5..a59be7899dd37 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -17,10 +17,10 @@ package org.apache.spark.streaming.examples -import org.apache.spark.util.IntParam import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam /** * Produces a count of events received from Flume. @@ -44,6 +44,8 @@ object FlumeEventCount { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val Array(master, host, IntParam(port)) = args val batchInterval = Milliseconds(2000) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index ea6ea674196a1..704b315ef8b22 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.examples import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ - /** * Counts words in new text files created in the given directory * Usage: HdfsWordCount @@ -38,6 +37,8 @@ object HdfsWordCount { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 31a94bd224a45..4a3d81c09a122 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -23,8 +23,8 @@ import kafka.producer._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.streaming.util.RawTextHelper._ import org.apache.spark.streaming.kafka._ +import org.apache.spark.streaming.util.RawTextHelper._ /** * Consumes messages from one or more topics in Kafka and does wordcount. @@ -40,12 +40,13 @@ import org.apache.spark.streaming.kafka._ */ object KafkaWordCount { def main(args: Array[String]) { - if (args.length < 5) { System.err.println("Usage: KafkaWordCount ") System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 325290b66f4de..78b49fdcf1eb3 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -17,12 +17,8 @@ package org.apache.spark.streaming.examples -import org.eclipse.paho.client.mqttv3.MqttClient -import org.eclipse.paho.client.mqttv3.MqttClientPersistence +import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic} import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence -import org.eclipse.paho.client.mqttv3.MqttException -import org.eclipse.paho.client.mqttv3.MqttMessage -import org.eclipse.paho.client.mqttv3.MqttTopic import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -43,6 +39,8 @@ object MQTTPublisher { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val Seq(brokerUrl, topic) = args.toSeq try { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index 6a32c75373a7e..c12139b3ec863 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -39,6 +39,8 @@ object NetworkWordCount { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 9d640e716bca9..4d4968ba6ae3e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.examples +import scala.collection.mutable.SynchronizedQueue + import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ -import scala.collection.mutable.SynchronizedQueue - object QueueStream { def main(args: Array[String]) { @@ -30,7 +30,9 @@ object QueueStream { System.err.println("Usage: QueueStream ") System.exit(1) } - + + StreamingExamples.setStreamingLogLevels() + // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index c0706d0724982..3d08d86567a9a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -17,11 +17,10 @@ package org.apache.spark.streaming.examples -import org.apache.spark.util.IntParam import org.apache.spark.storage.StorageLevel - import org.apache.spark.streaming._ import org.apache.spark.streaming.util.RawTextHelper +import org.apache.spark.util.IntParam /** * Receives text from multiple rawNetworkStreams and counts how many '\n' delimited @@ -45,6 +44,8 @@ object RawNetworkGrep { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args // Create the context diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index 002db57d592b2..1183eba84686b 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -39,6 +39,8 @@ object StatefulNetworkWordCount { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val updateFunc = (values: Seq[Int], state: Option[Int]) => { val currentCount = values.foldLeft(0)(_ + _) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala new file mode 100644 index 0000000000000..d41d84a980dc7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala @@ -0,0 +1,21 @@ +package org.apache.spark.streaming.examples + +import org.apache.spark.Logging + +import org.apache.log4j.{Level, Logger} + +/** Utility functions for Spark Streaming examples. */ +object StreamingExamples extends Logging { + + /** Set reasonable logging levels for streaming if the user has not configured log4j. */ + def setStreamingLogLevels() { + val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized) { + // We first log something to initialize Spark's default logging, then we override the + // logging level. + logInfo("Setting log level to [WARN] for streaming example." + + " To override add a custom log4j.properties to the classpath.") + Logger.getRootLogger.setLevel(Level.WARN) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 3ccdc908e23c4..80b5a98b142c1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.examples -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.storage.StorageLevel import com.twitter.algebird._ -import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.SparkContext._ +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.twitter._ /** @@ -51,6 +51,8 @@ object TwitterAlgebirdCMS { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + // CMS parameters val DELTA = 1E-3 val EPS = 0.01 diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index c7e83e76b0057..cb2f2c51a0cd6 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.examples -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.storage.StorageLevel -import com.twitter.algebird.HyperLogLog._ import com.twitter.algebird.HyperLogLogMonoid +import com.twitter.algebird.HyperLogLog._ + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.twitter._ /** @@ -44,6 +45,8 @@ object TwitterAlgebirdHLL { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 val (master, filters) = (args.head, args.tail) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index e2b0418d55d2b..16c10feaba2c1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -36,6 +36,8 @@ object TwitterPopularTags { System.exit(1) } + StreamingExamples.setStreamingLogLevels() + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 03902ec353bab..12d2a1084f900 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -76,6 +76,7 @@ object ZeroMQWordCount { "In local mode, should be 'local[n]' with n > 1") System.exit(1) } + StreamingExamples.setStreamingLogLevels() val Seq(master, url, topic) = args.toSeq // Create the context and set the batch size diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 807af199f4fd0..da6b67bcceefe 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -17,9 +17,10 @@ package org.apache.spark.streaming.examples.clickstream +import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.SparkContext._ +import org.apache.spark.streaming.examples.StreamingExamples /** Analyses a streaming dataset of web page views. This class demonstrates several types of * operators available in Spark streaming. @@ -36,6 +37,7 @@ object PageViewStream { " errorRatePerZipCode, activeUserCount, popularUsersSeen") System.exit(1) } + StreamingExamples.setStreamingLogLevels() val metric = args(0) val host = args(1) val port = args(2).toInt diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 00671ba5206f9..837f1ea1d8962 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -333,7 +333,7 @@ abstract class DStream[T: ClassTag] ( var numForgotten = 0 val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) generatedRDDs --= oldRDDs.keys - logInfo("Cleared " + oldRDDs.size + " RDDs that were older than " + + logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) dependencies.foreach(_.clearOldMetadata(time)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index a09b891956efe..62d07b22c6d5c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -105,18 +105,18 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def generateJobs(time: Time): Seq[Job] = { this.synchronized { - logInfo("Generating jobs for time " + time) + logDebug("Generating jobs for time " + time) val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time)) - logInfo("Generated " + jobs.length + " jobs for time " + time) + logDebug("Generated " + jobs.length + " jobs for time " + time) jobs } } def clearOldMetadata(time: Time) { this.synchronized { - logInfo("Clearing old metadata for time " + time) + logDebug("Clearing old metadata for time " + time) outputStreams.foreach(_.clearOldMetadata(time)) - logInfo("Cleared old metadata for time " + time) + logDebug("Cleared old metadata for time " + time) } } From 67b9a33628b9934804c36620d8cbc73ef70106ce Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 12:42:37 -0800 Subject: [PATCH 1275/1571] Some usability improvements --- .../apache/spark/deploy/ClientArguments.scala | 14 ++++++- .../spark/deploy/master/ui/IndexPage.scala | 35 +++++++++++------ .../spark/deploy/worker/ui/IndexPage.scala | 38 ++++++++++++------- 3 files changed, 62 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 50b92e1eabeca..db67c6d1bb55c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy +import java.net.URL + import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -68,8 +70,18 @@ private[spark] class ClientArguments(args: Array[String]) { case "launch" :: _master :: _jarUrl :: _mainClass :: tail => cmd = "launch" - master = _master + + try { + new URL(_jarUrl) + } catch { + case e: Exception => + println(s"Jar url '${_jarUrl}' is not a valid URL.") + println(s"Jar must be in URL format (e.g. hdfs://XX, file://XX)") + printUsageAndExit(-1) + } + jarUrl = _jarUrl + master = _master mainClass = _mainClass _driverOptions ++= tail diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index db1dde59d6c9d..a9af8df5525d6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -63,6 +63,10 @@ private[spark] class IndexPage(parent: MasterWebUI) { val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers) + // For now we only show driver information if the user has submitted drivers to the cluster. + // This is until we integrate the notion of drivers and applications in the UI. + def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0 + val content =
      @@ -98,26 +102,35 @@ private[spark] class IndexPage(parent: MasterWebUI) {
      -
      -
      -

      Completed Applications

      - {completedAppsTable} +
      + {if (hasDrivers) +
      +
      +

      Running Drivers

      + {activeDriversTable} +
      + }
      -

      Active Drivers

      - {activeDriversTable} +

      Completed Applications

      + {completedAppsTable}
      -
      -
      -

      Completed Drivers

      - {completedDriversTable} +
      + {if (hasDrivers) +
      +
      +

      Completed Drivers

      + {completedDriversTable} +
      + }
      ; + UIUtils.basicSparkPage(content, "Spark Master at " + state.uri) } @@ -170,7 +183,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {Utils.megabytesToString(driver.desc.mem.toLong)} - {driver.desc.command.mainClass} + {driver.desc.command.arguments(1)} } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 0a7f56e2d310a..925c6fb1832d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -58,6 +58,10 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) + // For now we only show driver information if the user has submitted drivers to the cluster. + // This is until we integrate the notion of drivers and applications in the UI. + def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0 + val content =
      @@ -81,6 +85,17 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
      +
      + {if (hasDrivers) +
      +
      +

      Running Drivers {workerState.drivers.size}

      + {runningDriverTable} +
      +
      + } +
      +

      Finished Executors

      @@ -88,18 +103,15 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
      -
      -
      -

      Running Drivers {workerState.drivers.size}

      - {runningDriverTable} -
      -
      - -
      -
      -

      Finished Drivers

      - {finishedDriverTable} -
      +
      + {if (hasDrivers) +
      +
      +

      Finished Drivers

      + {finishedDriverTable} +
      +
      + }
      ; UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format( @@ -133,7 +145,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { def driverRow(driver: DriverRunner): Seq[Node] = { {driver.driverId} - {driver.driverDesc.command.mainClass} + {driver.driverDesc.command.arguments(1)} {driver.finalState.getOrElse(DriverState.RUNNING)} {driver.driverDesc.cores.toString} From b5b0de2de53563c43e1c5844a52b4eeeb2542ea5 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:24:25 -0800 Subject: [PATCH 1276/1571] Start fixing formatting of graphx-programming-guide --- docs/graphx-programming-guide.md | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 5b06d82225229..ebc47f5d1c43c 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -1,4 +1,7 @@ -# GraphX: Unifying Graphs and Tables +--- +layout: global +title: "GraphX: Unifying Graphs and Tables" +--- GraphX extends the distributed fault-tolerant collections API and @@ -26,11 +29,8 @@ execute graph algorithms. Unfortunately, these systems do not address the challenges of graph construction and transformation and provide limited fault-tolerance and support for interactive analysis. -

      - -

      - - +{:.pagination-centered} +![Data-parallel vs. graph-parallel]({{ site.url }}/img/data_parallel_vs_graph_parallel.png) ## Solution @@ -194,4 +194,3 @@ you state this explicitly, by submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. - From 4aa9be16333b61139fbe81a079f46ef32df0b1fd Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:26:17 -0800 Subject: [PATCH 1277/1571] Make GraphImpl serializable to work around capture --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 4d35755e7e5cc..2ce5404e94744 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -30,7 +30,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val edges: EdgeRDD[ED], @transient val routingTable: RoutingTable, @transient val replicatedVertexView: ReplicatedVertexView[VD]) - extends Graph[VD, ED] { + extends Graph[VD, ED] with Serializable { def this( vertices: VertexRDD[VD], From 3b2e22e2c343ce4615f31c3d94f9af568ea0ea42 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:27:40 -0800 Subject: [PATCH 1278/1571] Revert changes to examples/.../PageRankUtils.scala Reverts to 04d83fc37f9eef89c20331c85291a0a169f75e6d:examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala. --- .../org/apache/spark/examples/bagel/PageRankUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index 8dd7fb40e86aa..cfafbaf23e4c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -31,16 +31,16 @@ import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} import com.esotericsoftware.kryo._ class PageRankUtils extends Serializable { - def computeWithCombiner(numVertices: Long, epsilon: Double, terminateSteps: Int = 10)( + def computeWithCombiner(numVertices: Long, epsilon: Double)( self: PRVertex, messageSum: Option[Double], superstep: Int ): (PRVertex, Array[PRMessage]) = { val newValue = messageSum match { case Some(msgSum) if msgSum != 0 => - 0.15 + 0.85 * msgSum + 0.15 / numVertices + 0.85 * msgSum case _ => self.value } - val terminate = superstep >= terminateSteps + val terminate = superstep >= 10 val outbox: Array[PRMessage] = if (!terminate) From ec12c63409c2db85c27a87813a8d0505ea8f6c21 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:29:20 -0800 Subject: [PATCH 1279/1571] Unwrap Graph.mapEdges signature --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 9dd26f7679a2d..420d01b426180 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -169,9 +169,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassTag]( - map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): - Graph[VD, ED2] + def mapEdges[ED2: ClassTag](map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] /** * Construct a new graph where the value of each edge is From 6f713e2a3e56185368b66fb087637dec112a1f5d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 9 Jan 2014 13:42:04 -0800 Subject: [PATCH 1280/1571] Changed the way StreamingContext finds and reads checkpoint files, and added JavaStreamingContext.getOrCreate. --- conf/slaves | 6 +- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../examples/JavaNetworkWordCount.java | 7 +- .../streaming/examples/NetworkWordCount.scala | 2 +- .../RecoverableNetworkWordCount.scala | 43 +++++++- .../apache/spark/streaming/Checkpoint.scala | 98 ++++++++++--------- .../streaming/DStreamCheckpointData.scala | 57 +++-------- .../apache/spark/streaming/DStreamGraph.scala | 4 +- .../spark/streaming/StreamingContext.scala | 64 +++++++++--- .../api/java/JavaStreamingContext.scala | 96 +++++++++++++++++- 10 files changed, 254 insertions(+), 125 deletions(-) diff --git a/conf/slaves b/conf/slaves index 30ea300e07c4d..2fbb50c4a8dc7 100644 --- a/conf/slaves +++ b/conf/slaves @@ -1,5 +1 @@ -ec2-54-221-59-252.compute-1.amazonaws.com -ec2-67-202-26-243.compute-1.amazonaws.com -ec2-23-22-220-97.compute-1.amazonaws.com -ec2-50-16-98-100.compute-1.amazonaws.com -ec2-54-234-164-206.compute-1.amazonaws.com +localhost diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7514ce58fba4a..304e85f1c0354 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -29,7 +29,7 @@ import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.hadoop.io.ArrayWritable import org.apache.hadoop.io.BooleanWritable import org.apache.hadoop.io.BytesWritable diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199be57..d8d6046914f72 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -41,17 +41,17 @@ public class JavaNetworkWordCount { public static void main(String[] args) { if (args.length < 3) { - System.err.println("Usage: NetworkWordCount \n" + + System.err.println("Usage: JavaNetworkWordCount \n" + "In local mode, should be 'local[n]' with n > 1"); System.exit(1); } // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') + // words in input stream of \n delimited text (eg. generated by 'nc') JavaDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override @@ -74,6 +74,5 @@ public Integer call(Integer i1, Integer i2) throws Exception { wordCounts.print(); ssc.start(); - } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index e2487dca5f2cc..5ad4875980d1d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -44,7 +44,7 @@ object NetworkWordCount { System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') + // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.socketTextStream(args(1), args(2).toInt) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala index 0e5f39f7722ab..739f805e87c2a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import org.apache.spark.streaming.{Time, Seconds, StreamingContext} @@ -8,20 +25,37 @@ import org.apache.spark.rdd.RDD import com.google.common.io.Files import java.nio.charset.Charset +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Usage: NetworkWordCount + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the TCP server that Spark Streaming would connect to receive data. + * directory in a Hadoop compatible file system to which checkpoint + * data will be saved to; this must be a fault-tolerant file system + * like HDFS for the system to recover from driver failures + * (x, 1)).reduceByKey(_ + _) @@ -39,10 +73,10 @@ object RecoverableNetworkWordCount { System.err.println("You arguments were " + args.mkString("[", ", ", "]")) System.err.println( """ - |Usage: RecoverableNetworkWordCount + |Usage: RecoverableNetworkWordCount | |In local mode, should be 'local[n]' with n > 1 - |Both and should be full paths + |Both and should be full paths """.stripMargin ) System.exit(1) @@ -53,6 +87,5 @@ object RecoverableNetworkWordCount { createContext(master, ip, port, outputPath) }) ssc.start() - } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 155d5bc02eca9..a32e4852c501b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -82,22 +82,28 @@ class CheckpointWriter(jobGenerator: JobGenerator, checkpointDir: String, hadoop attempts += 1 try { logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") - // This is inherently thread unsafe, so alleviating it by writing to '.new' and + // This is inherently thread unsafe, so alleviating it by writing to '.next' and // then moving it to the final file val fos = fs.create(writeFile) fos.write(bytes) fos.close() + + // Back up existing checkpoint if it exists if (fs.exists(file) && fs.rename(file, bakFile)) { logDebug("Moved existing checkpoint file to " + bakFile) } - // paranoia - fs.delete(file, false) - fs.rename(writeFile, file) - - val finishTime = System.currentTimeMillis() - logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + - "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds") - jobGenerator.onCheckpointCompletion(checkpointTime) + fs.delete(file, false) // paranoia + + // Rename temp written file to the right location + if (fs.rename(writeFile, file)) { + val finishTime = System.currentTimeMillis() + logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") + jobGenerator.onCheckpointCompletion(checkpointTime) + } else { + throw new SparkException("Failed to rename checkpoint file from " + + writeFile + " to " + file) + } return } catch { case ioe: IOException => @@ -154,47 +160,47 @@ class CheckpointWriter(jobGenerator: JobGenerator, checkpointDir: String, hadoop private[streaming] object CheckpointReader extends Logging { - def doesCheckpointExist(path: String): Boolean = { - val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk")) - val fs = new Path(path).getFileSystem(new Configuration()) - (attempts.count(p => fs.exists(p)) > 1) - } + private val graphFileNames = Seq("graph", "graph.bk") + + def read(checkpointDir: String, hadoopConf: Configuration): Option[Checkpoint] = { + val checkpointPath = new Path(checkpointDir) + def fs = checkpointPath.getFileSystem(hadoopConf) + val existingFiles = graphFileNames.map(new Path(checkpointPath, _)).filter(fs.exists) + + // Log the file listing if graph checkpoint file was not found + if (existingFiles.isEmpty) { + logInfo("Could not find graph file in " + checkpointDir + ", which contains the files:\n" + + fs.listStatus(checkpointPath).mkString("\n")) + return None + } + logInfo("Checkpoint files found: " + existingFiles.mkString(",")) - def read(path: String): Checkpoint = { - val fs = new Path(path).getFileSystem(new Configuration()) - val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk")) val compressionCodec = CompressionCodec.createCodec() - - attempts.foreach(file => { - if (fs.exists(file)) { - logInfo("Attempting to load checkpoint from file '" + file + "'") - try { - val fis = fs.open(file) - // ObjectInputStream uses the last defined user-defined class loader in the stack - // to find classes, which maybe the wrong class loader. Hence, a inherited version - // of ObjectInputStream is used to explicitly use the current thread's default class - // loader to find and load classes. This is a well know Java issue and has popped up - // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val zis = compressionCodec.compressedInputStream(fis) - val ois = new ObjectInputStreamWithLoader(zis, - Thread.currentThread().getContextClassLoader) - val cp = ois.readObject.asInstanceOf[Checkpoint] - ois.close() - fs.close() - cp.validate() - logInfo("Checkpoint successfully loaded from file '" + file + "'") - logInfo("Checkpoint was generated at time " + cp.checkpointTime) - return cp - } catch { - case e: Exception => - logError("Error loading checkpoint from file '" + file + "'", e) - } - } else { - logWarning("Could not read checkpoint from file '" + file + "' as it does not exist") + existingFiles.foreach(file => { + logInfo("Attempting to load checkpoint from file '" + file + "'") + try { + val fis = fs.open(file) + // ObjectInputStream uses the last defined user-defined class loader in the stack + // to find classes, which maybe the wrong class loader. Hence, a inherited version + // of ObjectInputStream is used to explicitly use the current thread's default class + // loader to find and load classes. This is a well know Java issue and has popped up + // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) + val zis = compressionCodec.compressedInputStream(fis) + val ois = new ObjectInputStreamWithLoader(zis, + Thread.currentThread().getContextClassLoader) + val cp = ois.readObject.asInstanceOf[Checkpoint] + ois.close() + fs.close() + cp.validate() + logInfo("Checkpoint successfully loaded from file '" + file + "'") + logInfo("Checkpoint was generated at time " + cp.checkpointTime) + return Some(cp) + } catch { + case e: Exception => + logWarning("Error reading checkpoint from file '" + file + "'", e) } - }) - throw new SparkException("Could not read checkpoint from path '" + path + "'") + throw new SparkException("Failed to read checkpoint from directory '" + checkpointDir + "'") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala index e0567a1c19731..1081d3c807ef0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala @@ -27,18 +27,16 @@ import org.apache.spark.Logging import java.io.{ObjectInputStream, IOException} - private[streaming] class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) extends Serializable with Logging { protected val data = new HashMap[Time, AnyRef]() - @transient private var allCheckpointFiles = new HashMap[Time, String] - @transient private var timeToLastCheckpointFileTime = new HashMap[Time, Time] + // Mapping of the batch time to the checkpointed RDD file of that time + @transient private var timeToCheckpointFile = new HashMap[Time, String] + // Mapping of the batch time to the time of the oldest checkpointed RDD in that batch's checkpoint data + @transient private var timeToOldestCheckpointFileTime = new HashMap[Time, Time] @transient private var fileSystem : FileSystem = null - - //@transient private var lastCheckpointFiles: HashMap[Time, String] = null - protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** @@ -51,17 +49,14 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) // Get the checkpointed RDDs from the generated RDDs val checkpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined) .map(x => (x._1, x._2.getCheckpointFile.get)) + logDebug("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) - logInfo("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) - // Make a copy of the existing checkpoint data (checkpointed RDDs) - // lastCheckpointFiles = checkpointFiles.clone() - - // If the new checkpoint data has checkpoints then replace existing with the new one + // Add the checkpoint files to the data to be serialized if (!currentCheckpointFiles.isEmpty) { currentCheckpointFiles.clear() currentCheckpointFiles ++= checkpointFiles - allCheckpointFiles ++= currentCheckpointFiles - timeToLastCheckpointFileTime(time) = currentCheckpointFiles.keys.min(Time.ordering) + timeToCheckpointFile ++= currentCheckpointFiles + timeToOldestCheckpointFileTime(time) = currentCheckpointFiles.keys.min(Time.ordering) } } @@ -71,32 +66,10 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) * implementation, cleans up old checkpoint files. */ def cleanup(time: Time) { - /* - // If there is at least on checkpoint file in the current checkpoint files, - // then delete the old checkpoint files. - if (checkpointFiles.size > 0 && lastCheckpointFiles != null) { - (lastCheckpointFiles -- checkpointFiles.keySet).foreach { - case (time, file) => { - try { - val path = new Path(file) - if (fileSystem == null) { - fileSystem = path.getFileSystem(new Configuration()) - } - fileSystem.delete(path, true) - logInfo("Deleted checkpoint file '" + file + "' for time " + time) - } catch { - case e: Exception => - logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) - } - } - } - } - */ - timeToLastCheckpointFileTime.remove(time) match { + timeToOldestCheckpointFileTime.remove(time) match { case Some(lastCheckpointFileTime) => - logInfo("Deleting all files before " + time) - val filesToDelete = allCheckpointFiles.filter(_._1 < lastCheckpointFileTime) - logInfo("Files to delete:\n" + filesToDelete.mkString(",")) + val filesToDelete = timeToCheckpointFile.filter(_._1 < lastCheckpointFileTime) + logDebug("Files to delete:\n" + filesToDelete.mkString(",")) filesToDelete.foreach { case (time, file) => try { @@ -105,11 +78,12 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) fileSystem = path.getFileSystem(dstream.ssc.sparkContext.hadoopConfiguration) } fileSystem.delete(path, true) - allCheckpointFiles -= time + timeToCheckpointFile -= time logInfo("Deleted checkpoint file '" + file + "' for time " + time) } catch { case e: Exception => logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) + fileSystem = null } } case None => @@ -138,7 +112,8 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) @throws(classOf[IOException]) private def readObject(ois: ObjectInputStream) { - timeToLastCheckpointFileTime = new HashMap[Time, Time] - allCheckpointFiles = new HashMap[Time, String] + ois.defaultReadObject() + timeToOldestCheckpointFileTime = new HashMap[Time, Time] + timeToCheckpointFile = new HashMap[Time, String] } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index bfedef2e4ed51..34919d315ccca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -130,11 +130,11 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } def clearCheckpointData(time: Time) { - logInfo("Restoring checkpoint data") + logInfo("Clearing checkpoint data for time " + time) this.synchronized { outputStreams.foreach(_.clearCheckpointData(time)) } - logInfo("Restored checkpoint data") + logInfo("Cleared checkpoint data for time " + time) } def restoreCheckpointData() { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 59d2d546e6107..30deba417e0dd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -45,10 +45,11 @@ import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{LocalFileSystem, Path} import twitter4j.Status import twitter4j.auth.Authorization +import org.apache.hadoop.conf.Configuration /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -89,10 +90,12 @@ class StreamingContext private ( /** * Re-create a StreamingContext from a checkpoint file. - * @param path Path either to the directory that was specified as the checkpoint directory, or - * to the checkpoint file 'graph' or 'graph.bk'. + * @param path Path to the directory that was specified as the checkpoint directory + * @param hadoopConf Optional, configuration object if necessary for reading from + * HDFS compatible filesystems */ - def this(path: String) = this(null, CheckpointReader.read(path), null) + def this(path: String, hadoopConf: Configuration = new Configuration) = + this(null, CheckpointReader.read(path, hadoopConf).get, null) initLogging() @@ -170,8 +173,9 @@ class StreamingContext private ( /** * Set the context to periodically checkpoint the DStream operations for master - * fault-tolerance. The graph will be checkpointed every batch interval. - * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored + * fault-tolerance. + * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored. + * Note that this must be a fault-tolerant file system like HDFS for */ def checkpoint(directory: String) { if (directory != null) { @@ -577,6 +581,10 @@ class StreamingContext private ( } } +/** + * StreamingContext object contains a number of utility functions related to the + * StreamingContext class. + */ object StreamingContext extends Logging { @@ -584,19 +592,45 @@ object StreamingContext extends Logging { new PairDStreamFunctions[K, V](stream) } + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc`. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new StreamingContext + * @param hadoopConf Optional Hadoop configuration if necessary for reading from the + * file system + * @param createOnError Optional, whether to create a new StreamingContext if there is an + * error in reading checkpoint data. By default, an exception will be + * thrown on error. + */ def getOrCreate( checkpointPath: String, creatingFunc: () => StreamingContext, - createOnCheckpointError: Boolean = false + hadoopConf: Configuration = new Configuration(), + createOnError: Boolean = false ): StreamingContext = { - if (CheckpointReader.doesCheckpointExist(checkpointPath)) { - logInfo("Creating streaming context from checkpoint file") - new StreamingContext(checkpointPath) - } else { - logInfo("Creating new streaming context") - val ssc = creatingFunc() - ssc.checkpoint(checkpointPath) - ssc + + try { + CheckpointReader.read(checkpointPath, hadoopConf) match { + case Some(checkpoint) => + return new StreamingContext(null, checkpoint, null) + case None => + logInfo("Creating new StreamingContext") + return creatingFunc() + } + } catch { + case e: Exception => + if (createOnError) { + logWarning("Error reading checkpoint", e) + logInfo("Creating new StreamingContext") + return creatingFunc() + } else { + logError("Error reading checkpoint", e) + throw e + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index aad0d931e7bde..f38d1453175bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -40,6 +40,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.scheduler.StreamingListener +import org.apache.hadoop.conf.Configuration /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -125,10 +126,16 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Re-creates a StreamingContext from a checkpoint file. - * @param path Path either to the directory that was specified as the checkpoint directory, or - * to the checkpoint file 'graph' or 'graph.bk'. + * @param path Path to the directory that was specified as the checkpoint directory */ - def this(path: String) = this (new StreamingContext(path)) + def this(path: String) = this(new StreamingContext(path)) + + /** + * Re-creates a StreamingContext from a checkpoint file. + * @param path Path to the directory that was specified as the checkpoint directory + * + */ + def this(path: String, hadoopConf: Configuration) = this(new StreamingContext(path, hadoopConf)) /** The underlying SparkContext */ val sc: JavaSparkContext = new JavaSparkContext(ssc.sc) @@ -699,13 +706,92 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Starts the execution of the streams. + * Start the execution of the streams. */ def start() = ssc.start() /** - * Sstops the execution of the streams. + * Stop the execution of the streams. */ def stop() = ssc.stop() +} + +/** + * JavaStreamingContext object contains a number of static utility functions. + */ +object JavaStreamingContext { + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc`. + * + * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program + * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext + */ + def getOrCreate( + checkpointPath: String, + factory: JavaStreamingContextFactory + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + factory.create.ssc + }) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc`. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + */ + def getOrCreate( + checkpointPath: String, + hadoopConf: Configuration, + factory: JavaStreamingContextFactory + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + factory.create.ssc + }, hadoopConf) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc`. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + * @param createOnError Whether to create a new JavaStreamingContext if there is an + * error in reading checkpoint data. + */ + def getOrCreate( + checkpointPath: String, + hadoopConf: Configuration, + factory: JavaStreamingContextFactory, + createOnError: Boolean + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + factory.create.ssc + }, hadoopConf, createOnError) + new JavaStreamingContext(ssc) + } +} + +/** + * Factory interface for creating a new JavaStreamingContext + */ +trait JavaStreamingContextFactory { + def create(): JavaStreamingContext } From 9bdfbc0492f2d7408c250ae165763719cf290eeb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 Jan 2014 15:45:33 -0600 Subject: [PATCH 1281/1571] Update based on review comments --- .../spark/deploy/yarn/ApplicationMaster.scala | 13 ++++++------- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 11 +++++------ .../spark/deploy/yarn/ApplicationMaster.scala | 13 ++++++------- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 13 ++++++------- 4 files changed, 23 insertions(+), 27 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 2bb11e54c549a..2e46d750c4a38 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -127,14 +127,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) - - if (localDirs.isEmpty()) { - throw new Exception("Yarn Local dirs can't be empty") + .orElse(Option(System.getenv("LOCAL_DIRS"))) + + localDirs match { + case None => throw new Exception("Yarn Local dirs can't be empty") + case Some(l) => l } - localDirs - } + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 66e38ee84051e..62b20b8fbaf19 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -131,13 +131,12 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) - - if (localDirs.isEmpty()) { - throw new Exception("Yarn Local dirs can't be empty") + .orElse(Option(System.getenv("LOCAL_DIRS"))) + + localDirs match { + case None => throw new Exception("Yarn Local dirs can't be empty") + case Some(l) => l } - localDirs } private def getApplicationAttemptId(): ApplicationAttemptId = { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 69ae14ce8385c..4b777d5fa7a28 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -116,14 +116,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) - - if (localDirs.isEmpty()) { - throw new Exception("Yarn Local dirs can't be empty") + .orElse(Option(System.getenv("LOCAL_DIRS"))) + + localDirs match { + case None => throw new Exception("Yarn Local dirs can't be empty") + case Some(l) => l } - localDirs - } + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 3e3a4672b4c61..78353224fa4b8 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -117,18 +117,17 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { + private def getLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) + .orElse(Option(System.getenv("LOCAL_DIRS"))) - if (localDirs.isEmpty()) { - throw new Exception("Yarn Local dirs can't be empty") - } - localDirs + localDirs match { + case None => throw new Exception("Yarn Local dirs can't be empty") + case Some(l) => l + } } private def getApplicationAttemptId(): ApplicationAttemptId = { From da83038234de1a16de38a24633c73fd950d4a85f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:52:07 -0800 Subject: [PATCH 1282/1571] Vid -> VertexID --- .../scala/org/apache/spark/graph/Edge.scala | 10 ++-- .../org/apache/spark/graph/EdgeRDD.scala | 4 +- .../org/apache/spark/graph/EdgeTriplet.scala | 4 +- .../scala/org/apache/spark/graph/Graph.scala | 18 +++---- .../spark/graph/GraphKryoRegistrator.scala | 2 +- .../org/apache/spark/graph/GraphLab.scala | 24 +++++----- .../org/apache/spark/graph/GraphOps.scala | 30 +++++++----- .../spark/graph/PartitionStrategy.scala | 14 +++--- .../scala/org/apache/spark/graph/Pregel.scala | 8 ++-- .../org/apache/spark/graph/VertexRDD.scala | 42 ++++++++--------- .../algorithms/ConnectedComponents.scala | 4 +- .../spark/graph/algorithms/PageRank.scala | 4 +- .../StronglyConnectedComponents.scala | 6 +-- .../apache/spark/graph/algorithms/Svdpp.scala | 12 ++--- .../graph/algorithms/TriangleCount.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 16 +++---- .../graph/impl/EdgePartitionBuilder.scala | 10 ++-- .../graph/impl/EdgeTripletIterator.scala | 2 +- .../apache/spark/graph/impl/GraphImpl.scala | 32 ++++++------- .../spark/graph/impl/MessageToPartition.scala | 12 ++--- .../graph/impl/ReplicatedVertexView.scala | 29 ++++++------ .../spark/graph/impl/RoutingTable.scala | 16 +++---- .../apache/spark/graph/impl/Serializers.scala | 10 ++-- .../spark/graph/impl/VertexPartition.scala | 47 ++++++++++--------- .../org/apache/spark/graph/package.scala | 6 +-- .../spark/graph/util/GraphGenerators.scala | 12 ++--- .../apache/spark/graph/GraphOpsSuite.scala | 18 +++---- .../org/apache/spark/graph/GraphSuite.scala | 31 ++++++------ .../org/apache/spark/graph/PregelSuite.scala | 10 ++-- .../apache/spark/graph/SerializerSuite.scala | 18 +++---- .../spark/graph/impl/EdgePartitionSuite.scala | 2 +- 31 files changed, 234 insertions(+), 221 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 5ac77839eb866..19c28bea685ba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -11,11 +11,11 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** * The vertex id of the source vertex */ - var srcId: Vid = 0, + var srcId: VertexID = 0, /** * The vertex id of the target vertex. */ - var dstId: Vid = 0, + var dstId: VertexID = 0, /** * The attribute associated with the edge. */ @@ -27,7 +27,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] * @param vid the id one of the two vertices on the edge. * @return the id of the other vertex on the edge. */ - def otherVertexId(vid: Vid): Vid = + def otherVertexId(vid: VertexID): VertexID = if (srcId == vid) dstId else { assert(dstId == vid); srcId } /** @@ -38,13 +38,13 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] * @return the relative direction of the edge to the corresponding * vertex. */ - def relativeDirection(vid: Vid): EdgeDirection = + def relativeDirection(vid: VertexID): EdgeDirection = if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } } object Edge { def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { override def compare(a: Edge[ED], b: Edge[ED]): Int = - Ordering[(Vid, Vid)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) + Ordering[(VertexID, VertexID)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 230202d6b0a6f..fd933593523a8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag]( def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) - (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { @@ -66,7 +66,7 @@ class EdgeRDD[@specialized ED: ClassTag]( }) } - def collectVids(): RDD[Vid] = { + def collectVertexIDs(): RDD[VertexID] = { partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 5a384a5f84dd9..a5103ed3cbffc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -47,7 +47,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { * @param vid the id one of the two vertices on the edge. * @return the attribute for the other vertex on the edge. */ - def otherVertexAttr(vid: Vid): VD = + def otherVertexAttr(vid: VertexID): VD = if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } /** @@ -56,7 +56,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { * @param vid the id of one of the two vertices on the edge * @return the attr for the vertex with that id. */ - def vertexAttr(vid: Vid): VD = + def vertexAttr(vid: VertexID): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 420d01b426180..dd0799142ef74 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -125,7 +125,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * }}} * */ - def mapVertices[VD2: ClassTag](map: (Vid, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassTag](map: (VertexID, VD) => VD2): Graph[VD2, ED] /** * Construct a new graph where the value of each edge is @@ -253,7 +253,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * satisfy the predicates. */ def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), - vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] + vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] /** * Subgraph of this graph with only vertices and edges from the other graph. @@ -302,7 +302,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * vertex * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") - * val inDeg: RDD[(Vid, Int)] = + * val inDeg: RDD[(VertexID, Int)] = * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) * }}} * @@ -314,7 +314,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * */ def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) : VertexRDD[A] @@ -341,15 +341,15 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") - * val outDeg: RDD[(Vid, Int)] = rawGraph.outDegrees() + * val outDeg: RDD[(VertexID, Int)] = rawGraph.outDegrees() * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) * } * }}} * */ - def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(Vid, U)]) - (mapFunc: (Vid, VD, Option[U]) => VD2) + def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(VertexID, U)]) + (mapFunc: (VertexID, VD, Option[U]) => VD2) : Graph[VD2, ED] // Save a copy of the GraphOps object so there is always one unique GraphOps object @@ -377,7 +377,7 @@ object Graph { * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. */ def fromEdgeTuples[VD: ClassTag]( - rawEdges: RDD[(Vid, Vid)], + rawEdges: RDD[(VertexID, VertexID)], defaultValue: VD, uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) @@ -419,7 +419,7 @@ object Graph { * partitioning the edges. */ def apply[VD: ClassTag, ED: ClassTag]( - vertices: RDD[(Vid, VD)], + vertices: RDD[(VertexID, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { GraphImpl(vertices, edges, defaultVertexAttr) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index b8c1b5b0f032d..296f3848f1c61 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -14,7 +14,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) - kryo.register(classOf[(Vid, Object)]) + kryo.register(classOf[(VertexID, Object)]) kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index c1ce5cd9ccada..22f48540193ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -42,11 +42,12 @@ object GraphLab extends Logging { (graph: Graph[VD, ED], numIter: Int, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) - (gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, + (gatherFunc: (VertexID, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, - applyFunc: (Vid, VD, Option[A]) => VD, - scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, - startVertices: (Vid, VD) => Boolean = (vid: Vid, data: VD) => true): Graph[VD, ED] = { + applyFunc: (VertexID, VD, Option[A]) => VD, + scatterFunc: (VertexID, EdgeTriplet[VD, ED]) => Boolean, + startVertices: (VertexID, VD) => Boolean = (vid: VertexID, data: VD) => true) + : Graph[VD, ED] = { // Add an active attribute to all vertices to track convergence. @@ -56,7 +57,7 @@ object GraphLab extends Logging { // The gather function wrapper strips the active attribute and // only invokes the gather function on active vertices - def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { + def gather(vid: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { if (e.vertexAttr(vid)._1) { val edgeTriplet = new EdgeTriplet[VD,ED] edgeTriplet.set(e) @@ -70,7 +71,7 @@ object GraphLab extends Logging { // The apply function wrapper strips the vertex of the active attribute // and only invokes the apply function on active vertices - def apply(vid: Vid, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { + def apply(vid: VertexID, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { val (active, vData) = data if (active) (true, applyFunc(vid, vData, accum)) else (false, vData) @@ -78,8 +79,8 @@ object GraphLab extends Logging { // The scatter function wrapper strips the vertex of the active attribute // and only invokes the scatter function on active vertices - def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { - val vid = e.otherVertexId(rawVid) + def scatter(rawVertexID: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { + val vid = e.otherVertexId(rawVertexID) if (e.vertexAttr(vid)._1) { val edgeTriplet = new EdgeTriplet[VD,ED] edgeTriplet.set(e) @@ -92,7 +93,8 @@ object GraphLab extends Logging { } // Used to set the active status of vertices for the next round - def applyActive(vid: Vid, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { + def applyActive( + vid: VertexID, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { val (prevActive, vData) = data (newActiveOpt.getOrElse(false), vData) } @@ -103,7 +105,7 @@ object GraphLab extends Logging { while (i < numIter && numActive > 0) { // Gather - val gathered: RDD[(Vid, A)] = + val gathered: RDD[(VertexID, A)] = activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply @@ -113,7 +115,7 @@ object GraphLab extends Logging { // Scatter is basically a gather in the opposite direction so we reverse the edge direction // activeGraph: Graph[(Boolean, VD), ED] - val scattered: RDD[(Vid, Boolean)] = + val scattered: RDD[(VertexID, Boolean)] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 11c6120beb414..e41287c1ed4f1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -112,7 +112,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * */ def aggregateNeighbors[A: ClassTag]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) : VertexRDD[A] = { @@ -151,25 +151,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * @return the vertex set of neighboring ids for each vertex. */ def collectNeighborIds(edgeDirection: EdgeDirection) : - VertexRDD[Array[Vid]] = { + VertexRDD[Array[VertexID]] = { val nbrs = if (edgeDirection == EdgeDirection.Both) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), reduceFunc = _ ++ _ ) } else if (edgeDirection == EdgeDirection.Out) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.srcId, Array(et.dstId))), reduceFunc = _ ++ _) } else if (edgeDirection == EdgeDirection.In) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.dstId, Array(et.srcId))), reduceFunc = _ ++ _) } else { throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.") } - graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[Vid]) } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[VertexID]) + } } // end of collectNeighborIds @@ -187,14 +189,16 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * vertex. */ def collectNeighbors(edgeDirection: EdgeDirection) : - VertexRDD[ Array[(Vid, VD)] ] = { - val nbrs = graph.aggregateNeighbors[Array[(Vid,VD)]]( + VertexRDD[ Array[(VertexID, VD)] ] = { + val nbrs = graph.aggregateNeighbors[Array[(VertexID,VD)]]( (vid, edge) => Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), (a, b) => a ++ b, edgeDirection) - graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[(Vid, VD)]) } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[(VertexID, VD)]) + } } // end of collectNeighbor @@ -228,9 +232,9 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * }}} * */ - def joinVertices[U: ClassTag](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + def joinVertices[U: ClassTag](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD) : Graph[VD, ED] = { - val uf = (id: Vid, data: VD, o: Option[U]) => { + val uf = (id: VertexID, data: VD, o: Option[U]) => { o match { case Some(u) => mapFunc(id, data, u) case None => data @@ -259,7 +263,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * val degrees: VertexSetRDD[Int] = graph.outDegrees * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} * }, - * vpred = (vid: Vid, deg:Int) => deg > 0 + * vpred = (vid: VertexID, deg:Int) => deg > 0 * ) * }}} * @@ -267,7 +271,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { def filter[VD2: ClassTag, ED2: ClassTag]( preprocess: Graph[VD, ED] => Graph[VD2, ED2], epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, - vpred: (Vid, VD2) => Boolean = (v:Vid, d:VD2) => true): Graph[VD, ED] = { + vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = { graph.mask(preprocess(graph).subgraph(epred, vpred)) } } // end of GraphOps diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index 293a9d588afda..c01b4b9439589 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph sealed trait PartitionStrategy extends Serializable { - def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid + def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid } @@ -51,9 +51,9 @@ sealed trait PartitionStrategy extends Serializable { * */ case object EdgePartition2D extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt - val mixingPrime: Vid = 1125899906842597L + val mixingPrime: VertexID = 1125899906842597L val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts @@ -62,8 +62,8 @@ case object EdgePartition2D extends PartitionStrategy { case object EdgePartition1D extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + val mixingPrime: VertexID = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } } @@ -74,7 +74,7 @@ case object EdgePartition1D extends PartitionStrategy { * random vertex cut. */ case object RandomVertexCut extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { math.abs((src, dst).hashCode()) % numParts } } @@ -86,7 +86,7 @@ case object RandomVertexCut extends PartitionStrategy { * will end up on the same partition. */ case object CanonicalRandomVertexCut extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { val lower = math.min(src, dst) val higher = math.max(src, dst) math.abs((lower, higher).hashCode()) % numParts diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 4664091b5714b..3b84e2e5e4195 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -25,9 +25,9 @@ import scala.reflect.ClassTag * // Set the vertex attributes to the initial pagerank values * .mapVertices( (id, attr) => 1.0 ) * - * def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = * resetProb + (1.0 - resetProb) * msgSum - * def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = + * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Option[Double] = * Some(edge.srcAttr * edge.attr) * def messageCombiner(a: Double, b: Double): Double = a + b * val initialMessage = 0.0 @@ -88,8 +88,8 @@ object Pregel { */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( - vprog: (Vid, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], + vprog: (VertexID, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index c5fb4aeca73a0..25b0aed85aae9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -29,7 +29,7 @@ import org.apache.spark.graph.impl.VertexPartition /** - * A `VertexRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is + * A `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is * only one entry for each vertex and by pre-indexing the entries for fast, * efficient joins. * @@ -40,12 +40,12 @@ import org.apache.spark.graph.impl.VertexPartition * @example Construct a `VertexRDD` from a plain RDD * {{{ * // Construct an intial vertex set - * val someData: RDD[(Vid, SomeType)] = loadData(someFile) + * val someData: RDD[(VertexID, SomeType)] = loadData(someFile) * val vset = VertexRDD(someData) * // If there were redundant values in someData we would use a reduceFunc * val vset2 = VertexRDD(someData, reduceFunc) * // Finally we can use the VertexRDD to index another dataset - * val otherData: RDD[(Vid, OtherType)] = loadData(otherFile) + * val otherData: RDD[(VertexID, OtherType)] = loadData(otherFile) * val vset3 = VertexRDD(otherData, vset.index) * // Now we can construct very fast joins between the two sets * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) @@ -54,7 +54,7 @@ import org.apache.spark.graph.impl.VertexPartition */ class VertexRDD[@specialized VD: ClassTag]( val partitionsRDD: RDD[VertexPartition[VD]]) - extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + extends RDD[(VertexID, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) @@ -104,9 +104,9 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Provide the `RDD[(Vid, VD)]` equivalent output. + * Provide the `RDD[(VertexID, VD)]` equivalent output. */ - override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = { + override def compute(part: Partition, context: TaskContext): Iterator[(VertexID, VD)] = { firstParent[VertexPartition[VD]].iterator(part, context).next.iterator } @@ -125,14 +125,14 @@ class VertexRDD[@specialized VD: ClassTag]( * given predicate. * * @param pred the user defined predicate, which takes a tuple to conform to - * the RDD[(Vid, VD)] interface + * the RDD[(VertexID, VD)] interface * * @note The vertex set preserves the original index structure * which means that the returned RDD can be easily joined with * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexRDD[VD] = + override def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) /** @@ -160,7 +160,7 @@ class VertexRDD[@specialized VD: ClassTag]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassTag](f: (Vid, VD) => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** @@ -197,7 +197,7 @@ class VertexRDD[@specialized VD: ClassTag]( * */ def leftZipJoin[VD2: ClassTag, VD3: ClassTag] - (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { + (other: VertexRDD[VD2])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true ) { (thisIter, otherIter) => @@ -228,8 +228,8 @@ class VertexRDD[@specialized VD: ClassTag]( * VertexRDD with the attribute emitted by f. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: RDD[(Vid, VD2)]) - (f: (Vid, VD, Option[VD2]) => VD3) + (other: RDD[(VertexID, VD2)]) + (f: (VertexID, VD, Option[VD2]) => VD3) : VertexRDD[VD3] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. @@ -254,7 +254,7 @@ class VertexRDD[@specialized VD: ClassTag]( * must have the same index. */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) - (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true ) { (thisIter, otherIter) => @@ -269,8 +269,8 @@ class VertexRDD[@specialized VD: ClassTag]( * Replace vertices with corresponding vertices in `other`, and drop vertices without a * corresponding vertex in `other`. */ - def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(Vid, U)]) - (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient innerZipJoin other match { @@ -293,7 +293,7 @@ class VertexRDD[@specialized VD: ClassTag]( * co-indexed with this one. */ def aggregateUsingIndex[VD2: ClassTag]( - messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = + messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => @@ -319,8 +319,8 @@ object VertexRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassTag](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { - val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { + def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)]): VertexRDD[VD] = { + val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } @@ -339,9 +339,9 @@ object VertexRDD { * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassTag](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = + def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { - val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { + val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } @@ -351,7 +351,7 @@ object VertexRDD { new VertexRDD(vertexPartitions) } - def apply[VD: ClassTag](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) + def apply[VD: ClassTag](vids: RDD[VertexID], rdd: RDD[(VertexID, VD)], defaultVal: VD) : VertexRDD[VD] = { VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala index 7cd947d2ba7df..2a6b8c0999ab0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala @@ -16,10 +16,10 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(edge: EdgeTriplet[Vid, ED]) = { + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { if (edge.srcAttr < edge.dstAttr) { Iterator((edge.dstId, edge.srcAttr)) } else if (edge.srcAttr > edge.dstAttr) { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala index f77dffd7b4156..26b8dc5ab60c6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala @@ -65,7 +65,7 @@ object PageRank extends Logging { // Define the three functions needed to implement PageRank in the GraphX // version of Pregel - def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = resetProb + (1.0 - resetProb) * msgSum def sendMessage(edge: EdgeTriplet[Double, Double]) = Iterator((edge.dstId, edge.srcAttr * edge.attr)) @@ -129,7 +129,7 @@ object PageRank extends Logging { // Define the three functions needed to implement PageRank in the GraphX // version of Pregel - def vertexProgram(id: Vid, attr: (Double, Double), msgSum: Double): (Double, Double) = { + def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { val (oldPR, lastDelta) = attr val newPR = oldPR + (1.0 - resetProb) * msgSum (newPR, newPR - oldPR) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala index c324c984d7692..8031aa10ce3ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala @@ -15,7 +15,7 @@ object StronglyConnectedComponents { * * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[Vid, ED] = { + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } @@ -52,7 +52,7 @@ object StronglyConnectedComponents { // collect min of all my neighbor's scc values, update if it's smaller than mine // then notify any neighbors with scc values larger than mine - sccWorkGraph = GraphLab[(Vid, Boolean), ED, Vid](sccWorkGraph, Integer.MAX_VALUE)( + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( (vid, e) => e.otherVertexAttr(vid)._1, (vid1, vid2) => math.min(vid1, vid2), (vid, scc, optScc) => @@ -62,7 +62,7 @@ object StronglyConnectedComponents { // start at root of SCCs. Traverse values in reverse, notify all my neighbors // do not propagate if colors do not match! - sccWorkGraph = GraphLab[(Vid, Boolean), ED, Boolean]( + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( sccWorkGraph, Integer.MAX_VALUE, EdgeDirection.Out, diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 18395bdc5f230..85fa23d309469 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -51,12 +51,12 @@ object Svdpp { // calculate initial bias and norm var t0 = g.mapReduceTriplets(et => Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) - g = g.outerJoinVertices(t0) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => + g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) } def mapTrainF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) - : Iterator[(Vid, (RealVector, RealVector, Double))] = { + : Iterator[(VertexID, (RealVector, RealVector, Double))] = { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) @@ -73,19 +73,19 @@ object Svdpp { for (i <- 0 until conf.maxIters) { // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) - g = g.outerJoinVertices(t1) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => + g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd } // phase 2, update p for user nodes and q, y for item nodes val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) - g = g.outerJoinVertices(t2) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => + g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) } } // calculate error on training set - def mapTestF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(Vid, Double)] = { + def mapTestF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) @@ -95,7 +95,7 @@ object Svdpp { Iterator((et.dstId, err)) } val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) - g = g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd } (g, u) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala index a6384320bab6b..81774d52e4291 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -46,7 +46,7 @@ object TriangleCount { (vid, _, optSet) => optSet.getOrElse(null) } // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(Vid, Int)] = { + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = { assert(et.srcAttr != null) assert(et.dstAttr != null) val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 7ae4d7df43a2f..b4311fa9f894e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -16,10 +16,10 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * @tparam ED the edge attribute type. */ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( - val srcIds: Array[Vid], - val dstIds: Array[Vid], + val srcIds: Array[VertexID], + val dstIds: Array[VertexID], val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[Vid, Int]) extends Serializable { + val index: PrimitiveKeyOpenHashMap[VertexID, Int]) extends Serializable { /** * Reverse all the edges in this partition. @@ -101,8 +101,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { val builder = new EdgePartitionBuilder[ED] var firstIter: Boolean = true - var currSrcId: Vid = nullValue[Vid] - var currDstId: Vid = nullValue[Vid] + var currSrcId: VertexID = nullValue[VertexID] + var currDstId: VertexID = nullValue[VertexID] var currAttr: ED = nullValue[ED] var i = 0 while (i < size) { @@ -136,7 +136,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) */ def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgePartition[ED2]) - (f: (Vid, Vid, ED, ED2) => ED3): EdgePartition[ED3] = { + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgePartition[ED3] = { val builder = new EdgePartitionBuilder[ED3] var i = 0 var j = 0 @@ -193,14 +193,14 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * iterator is generated using an index scan, so it is efficient at skipping edges that don't * match srcIdPred. */ - def indexIterator(srcIdPred: Vid => Boolean): Iterator[Edge[ED]] = + def indexIterator(srcIdPred: VertexID => Boolean): Iterator[Edge[ED]] = index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) /** * Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The * cluster must start at position `index`. */ - private def clusterIterator(srcId: Vid, index: Int) = new Iterator[Edge[ED]] { + private def clusterIterator(srcId: VertexID, index: Int) = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] private[this] var pos = index diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index ae3f3a6d03145..56624ef60adc0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -13,22 +13,22 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ - def add(src: Vid, dst: Vid, d: ED) { + def add(src: VertexID, dst: VertexID, d: ED) { edges += Edge(src, dst, d) } def toEdgePartition: EdgePartition[ED] = { val edgeArray = edges.trim().array Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) - val srcIds = new Array[Vid](edgeArray.size) - val dstIds = new Array[Vid](edgeArray.size) + val srcIds = new Array[VertexID](edgeArray.size) + val dstIds = new Array[VertexID](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[Vid, Int] + val index = new PrimitiveKeyOpenHashMap[VertexID, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { index.update(srcIds(0), 0) - var currSrcId: Vid = srcIds(0) + var currSrcId: VertexID = srcIds(0) var i = 0 while (i < edgeArray.size) { srcIds(i) = edgeArray(i).srcId diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala index 4d5eb240a91b7..e95d79e3d63d6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -25,7 +25,7 @@ class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( // allocating too many temporary Java objects. private val triplet = new EdgeTriplet[VD, ED] - private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) + private val vmap = new PrimitiveKeyOpenHashMap[VertexID, VD](vidToIndex, vertexArray) override def hasNext: Boolean = pos < edgePartition.size diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 2ce5404e94744..6eb401b3b57ac 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -89,7 +89,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def statistics: Map[String, Any] = { // Get the total number of vertices after replication, used to compute the replication ratio. - def numReplicatedVertices(vid2pids: RDD[Array[Array[Vid]]]): Double = { + def numReplicatedVertices(vid2pids: RDD[Array[Array[VertexID]]]): Double = { vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble } @@ -157,7 +157,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - override def mapVertices[VD2: ClassTag](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)) @@ -208,7 +208,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, - vpred: (Vid, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + vpred: (VertexID, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { // Filter the vertices, reusing the partitioner and the index from this graph val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) @@ -250,7 +250,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( ////////////////////////////////////////////////////////////////////////////////////////////////// override def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { @@ -280,14 +280,14 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val edgeIter = activeDirectionOpt match { case Some(EdgeDirection.Both) => if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID)) .filter(e => vPart.isActive(e.dstId)) } else { edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) } case Some(EdgeDirection.Out) => if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID)) } else { edgePartition.iterator.filter(e => vPart.isActive(e.srcId)) } @@ -318,7 +318,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } // end of mapReduceTriplets override def outerJoinVertices[U: ClassTag, VD2: ClassTag] - (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { + (updates: RDD[(VertexID, U)])(updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // updateF preserves type, so we can use incremental replication val newVerts = vertices.leftJoin(updates)(updateF) @@ -360,7 +360,7 @@ object GraphImpl { } def apply[VD: ClassTag, ED: ClassTag]( - vertices: RDD[(Vid, VD)], + vertices: RDD[(VertexID, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { @@ -369,7 +369,7 @@ object GraphImpl { // Get the set of all vids val partitioner = Partitioner.defaultPartitioner(vertices) val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVidsFromEdges(edgeRDD, partitioner) + val vidsFromEdges = collectVertexIDsFromEdges(edgeRDD, partitioner) val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } @@ -381,7 +381,7 @@ object GraphImpl { /** * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges - * data structure (RDD[(Vid, Vid, ED)]). + * data structure (RDD[(VertexID, VertexID, ED)]). * * The edge RDD contains multiple partitions, and each partition contains only one RDD key-value * pair: the key is the partition id, and the value is an EdgePartition object containing all the @@ -404,19 +404,19 @@ object GraphImpl { defaultVertexAttr: VD): GraphImpl[VD, ED] = { edges.cache() // Get the set of all vids - val vids = collectVidsFromEdges(edges, new HashPartitioner(edges.partitions.size)) + val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) new GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ - private def collectVidsFromEdges( + private def collectVertexIDsFromEdges( edges: EdgeRDD[_], - partitioner: Partitioner): RDD[(Vid, Int)] = { + partitioner: Partitioner): RDD[(VertexID, Int)] = { // TODO: Consider doing map side distinct before shuffle. - new ShuffledRDD[Vid, Int, (Vid, Int)]( - edges.collectVids.map(vid => (vid, 0)), partitioner) - .setSerializer(classOf[VidMsgSerializer].getName) + new ShuffledRDD[VertexID, Int, (VertexID, Int)]( + edges.collectVertexIDs.map(vid => (vid, 0)), partitioner) + .setSerializer(classOf[VertexIDMsgSerializer].getName) } } // end of object GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index bf033945dee24..2d03f75a28a25 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -3,15 +3,15 @@ package org.apache.spark.graph.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{Pid, Vid} +import org.apache.spark.graph.{Pid, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: Pid, - var vid: Vid, + var vid: VertexID, var data: T) - extends Product2[Pid, (Vid, T)] with Serializable { + extends Product2[Pid, (VertexID, T)] with Serializable { override def _1 = partition @@ -41,7 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[Pid, (Vid, T), VertexBroadcastMsg[T]](self, partitioner) + val rdd = new ShuffledRDD[Pid, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { @@ -77,8 +77,8 @@ object MsgRDDFunctions { new VertexBroadcastMsgRDDFunctions(rdd) } - def partitionForAggregation[T: ClassTag](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { - val rdd = new ShuffledRDD[Vid, T, (Vid, T)](msgs, partitioner) + def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexID, T)], partitioner: Partitioner) = { + val rdd = new ShuffledRDD[VertexID, T, (VertexID, T)](msgs, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 970acfed27425..9d2d242ffa8d9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -31,9 +31,9 @@ class ReplicatedVertexView[VD: ClassTag]( * vids from both the source and destination of edges. It must always include both source and * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. */ - private val localVidMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { + private val localVertexIDMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { case Some(prevView) => - prevView.localVidMap + prevView.localVertexIDMap case None => edges.partitionsRDD.mapPartitions(_.map { case (pid, epart) => @@ -43,7 +43,7 @@ class ReplicatedVertexView[VD: ClassTag]( vidToIndex.add(e.dstId) } (pid, vidToIndex) - }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVidMap") + }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIDMap") } private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) @@ -104,8 +104,8 @@ class ReplicatedVertexView[VD: ClassTag]( case None => // Within each edge partition, place the shipped vertex attributes into the correct - // locations specified in localVidMap - localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => + // locations specified in localVertexIDMap + localVertexIDMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map @@ -128,15 +128,15 @@ class ReplicatedVertexView[VD: ClassTag]( object ReplicatedVertexView { protected def buildBuffer[VD: ClassTag]( - pid2vidIter: Iterator[Array[Array[Vid]]], + pid2vidIter: Iterator[Array[Array[VertexID]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { - val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + val pid2vid: Array[Array[VertexID]] = pid2vidIter.next() val vertexPart: VertexPartition[VD] = vertexPartIter.next() Iterator.tabulate(pid2vid.size) { pid => val vidsCandidate = pid2vid(pid) val size = vidsCandidate.length - val vids = new PrimitiveVector[Vid](pid2vid(pid).size) + val vids = new PrimitiveVector[VertexID](pid2vid(pid).size) val attrs = new PrimitiveVector[VD](pid2vid(pid).size) var i = 0 while (i < size) { @@ -152,16 +152,16 @@ object ReplicatedVertexView { } protected def buildActiveBuffer( - pid2vidIter: Iterator[Array[Array[Vid]]], + pid2vidIter: Iterator[Array[Array[VertexID]]], activePartIter: Iterator[VertexPartition[_]]) - : Iterator[(Int, Array[Vid])] = { - val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + : Iterator[(Int, Array[VertexID])] = { + val pid2vid: Array[Array[VertexID]] = pid2vidIter.next() val activePart: VertexPartition[_] = activePartIter.next() Iterator.tabulate(pid2vid.size) { pid => val vidsCandidate = pid2vid(pid) val size = vidsCandidate.length - val actives = new PrimitiveVector[Vid](vidsCandidate.size) + val actives = new PrimitiveVector[VertexID](vidsCandidate.size) var i = 0 while (i < size) { val vid = vidsCandidate(i) @@ -175,7 +175,8 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) +class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexID], val attrs: Array[VD]) extends Serializable { - def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } + def iterator: Iterator[(VertexID, VD)] = + (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala index b6cd048b33bb1..9e6f78197e7f9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -14,12 +14,12 @@ import org.apache.spark.util.collection.PrimitiveVector */ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { - val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) - val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) - val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) - val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) + val bothAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(true, true) + val srcAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(true, false) + val dstAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(false, true) + val noAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(false, false) - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = (includeSrcAttr, includeDstAttr) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly @@ -28,9 +28,9 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { } private def createPid2Vid( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => + val vid2pid: RDD[(VertexID, Pid)] = edges.partitionsRDD.mapPartitions { iter => val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet @@ -53,7 +53,7 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { val numPartitions = vertices.partitions.size vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[VertexID]) for ((vid, pid) <- iter) { pid2vid(pid) += vid } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index dcf619fa85e8a..a3b0ea7689fc7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -7,12 +7,12 @@ import org.apache.spark.SparkConf import org.apache.spark.graph._ import org.apache.spark.serializer._ -class VidMsgSerializer(conf: SparkConf) extends Serializer { +class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, _)] + val msg = t.asInstanceOf[(VertexID, _)] writeVarLong(msg._1, optimizePositive = false) this } @@ -101,7 +101,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Int)] + val msg = t.asInstanceOf[(VertexID, Int)] writeVarLong(msg._1, optimizePositive = false) writeUnsignedVarInt(msg._2) this @@ -124,7 +124,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Long)] + val msg = t.asInstanceOf[(VertexID, Long)] writeVarLong(msg._1, optimizePositive = false) writeVarLong(msg._2, optimizePositive = true) this @@ -147,7 +147,7 @@ class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Double)] + val msg = t.asInstanceOf[(VertexID, Double)] writeVarLong(msg._1, optimizePositive = false) writeDouble(msg._2) this diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 7048a40f42364..91244daa54a57 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -10,18 +10,18 @@ import org.apache.spark.graph._ private[graph] object VertexPartition { - def apply[VD: ClassTag](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[Vid, VD] + def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)]): VertexPartition[VD] = { + val map = new PrimitiveKeyOpenHashMap[VertexID, VD] iter.foreach { case (k, v) => map(k) = v } new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) } - def apply[VD: ClassTag](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) + def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)], mergeFunc: (VD, VD) => VD) : VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[Vid, VD] + val map = new PrimitiveKeyOpenHashMap[VertexID, VD] iter.foreach { case (k, v) => map.setMerge(k, v, mergeFunc) } @@ -44,15 +44,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( def size: Int = mask.cardinality() /** Return the vertex attribute for the given vertex ID. */ - def apply(vid: Vid): VD = values(index.getPos(vid)) + def apply(vid: VertexID): VD = values(index.getPos(vid)) - def isDefined(vid: Vid): Boolean = { + def isDefined(vid: VertexID): Boolean = { val pos = index.getPos(vid) pos >= 0 && mask.get(pos) } /** Look up vid in activeSet, throwing an exception if it is None. */ - def isActive(vid: Vid): Boolean = { + def isActive(vid: VertexID): Boolean = { activeSet.get.contains(vid) } @@ -72,7 +72,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * each of the entries in the original VertexRDD. The resulting * VertexPartition retains the same index. */ - def map[VD2: ClassTag](f: (Vid, VD) => VD2): VertexPartition[VD2] = { + def map[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation val newValues = new Array[VD2](capacity) var i = mask.nextSetBit(0) @@ -92,7 +92,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * RDD can be easily joined with the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - def filter(pred: (Vid, VD) => Boolean): VertexPartition[VD] = { + def filter(pred: (VertexID, VD) => Boolean): VertexPartition[VD] = { // Allocate the array to store the results into val newMask = new BitSet(capacity) // Iterate over the active bits in the old mask and evaluate the predicate @@ -130,7 +130,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexPartition[VD2]) - (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") leftJoin(createUsingIndex(other.iterator))(f) @@ -149,14 +149,14 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( /** Left outer join another iterator of messages. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: Iterator[(Vid, VD2)]) - (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + (other: Iterator[(VertexID, VD2)]) + (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { leftJoin(createUsingIndex(other))(f) } /** Inner join another VertexPartition. */ def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) - (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") innerJoin(createUsingIndex(other.iterator))(f) @@ -176,15 +176,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Inner join an iterator of messages. */ def innerJoin[U: ClassTag, VD2: ClassTag] - (iter: Iterator[Product2[Vid, U]]) - (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + (iter: Iterator[Product2[VertexID, U]]) + (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = { innerJoin(createUsingIndex(iter))(f) } /** * Similar effect as aggregateUsingIndex((a, b) => a) */ - def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[Vid, VD2]]) + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexID, VD2]]) : VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) @@ -202,7 +202,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in * the partition, hidden by the bitmask. */ - def innerJoinKeepLeft(iter: Iterator[Product2[Vid, VD]]): VertexPartition[VD] = { + def innerJoinKeepLeft(iter: Iterator[Product2[VertexID, VD]]): VertexPartition[VD] = { val newMask = new BitSet(capacity) val newValues = new Array[VD](capacity) System.arraycopy(values, 0, newValues, 0, newValues.length) @@ -217,8 +217,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( } def aggregateUsingIndex[VD2: ClassTag]( - iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = - { + iter: Iterator[Product2[VertexID, VD2]], + reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) iter.foreach { product => @@ -237,7 +237,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( new VertexPartition[VD2](index, newValues, newMask) } - def replaceActives(iter: Iterator[Vid]): VertexPartition[VD] = { + def replaceActives(iter: Iterator[VertexID]): VertexPartition[VD] = { val newActiveSet = new VertexSet iter.foreach(newActiveSet.add(_)) new VertexPartition(index, values, mask, Some(newActiveSet)) @@ -247,7 +247,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): VertexPartition[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexID, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- this.iterator) { hashMap.setMerge(k, v, arbitraryMerge) @@ -255,7 +255,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) } - def iterator: Iterator[(Vid, VD)] = mask.iterator.map(ind => (index.getValue(ind), values(ind))) + def iterator: Iterator[(VertexID, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) - def vidIterator: Iterator[Vid] = mask.iterator.map(ind => index.getValue(ind)) + def vidIterator: Iterator[VertexID] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 655ae53bf8bc5..823d47c359b09 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -5,15 +5,15 @@ import org.apache.spark.util.collection.OpenHashSet package object graph { - type Vid = Long + type VertexID = Long // TODO: Consider using Char. type Pid = Int - type VertexSet = OpenHashSet[Vid] + type VertexSet = OpenHashSet[VertexID] // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap - type VertexIdToIndexMap = OpenHashSet[Vid] + type VertexIdToIndexMap = OpenHashSet[VertexID] /** * Return the default null-like value for a data type T. diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d61f358bb08bd..51f45cb8922b3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -70,7 +70,7 @@ object GraphGenerators { val sigma = 1.3 //val vertsAndEdges = (0 until numVertices).flatMap { src => { - val vertices: RDD[(Vid, Int)] = sc.parallelize(0 until numVertices).map{ + val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{ src => (src, sampleLogNormal(mu, sigma, numVertices)) } @@ -92,11 +92,11 @@ object GraphGenerators { } - def generateRandomEdges(src: Int, numEdges: Int, maxVid: Int): Array[Edge[Int]] = { + def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = { val rand = new Random() var dsts: Set[Int] = Set() while (dsts.size < numEdges) { - val nextDst = rand.nextInt(maxVid) + val nextDst = rand.nextInt(maxVertexID) if (nextDst != src) { dsts += nextDst } @@ -251,9 +251,9 @@ object GraphGenerators { */ def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = { // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Vid = r * cols + c + def sub2ind(r: Int, c: Int): VertexID = r * cols + c - val vertices: RDD[(Vid, (Int,Int))] = + val vertices: RDD[(VertexID, (Int,Int))] = sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) val edges: RDD[Edge[Double]] = vertices.flatMap{ case (vid, (r,c)) => @@ -273,7 +273,7 @@ object GraphGenerators { * being the center vertex. */ def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { - val edges: RDD[(Vid, Vid)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) + val edges: RDD[(VertexID, VertexID)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) Graph.fromEdgeTuples(edges, 1) } // end of starGraph diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala index 9e9213631ff63..132e6be24a04a 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala @@ -11,7 +11,8 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { test("aggregateNeighbors") { withSpark { sc => val n = 3 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 1) + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID))), 1) val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), @@ -26,21 +27,22 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { assert(outdegrees.collect().toSet === Set((0, n))) val noVertexValues = star.aggregateNeighbors[Int]( - (vid: Vid, edge: EdgeTriplet[Int, Int]) => None, + (vid: VertexID, edge: EdgeTriplet[Int, Int]) => None, (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), EdgeDirection.In) - assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)]) + assert(noVertexValues.collect().toSet === Set.empty[(VertexID, Int)]) } } test("joinVertices") { withSpark { sc => - val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val vertices = + sc.parallelize(Seq[(VertexID, String)]((1, "one"), (2, "two"), (3, "three")), 2) val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) val g: Graph[String, String] = Graph(vertices, edges) - val tbl = sc.parallelize(Seq[(Vid, Int)]((1, 10), (2, 20))) - val g1 = g.joinVertices(tbl) { (vid: Vid, attr: String, u: Int) => attr + u } + val tbl = sc.parallelize(Seq[(VertexID, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (vid: VertexID, attr: String, u: Int) => attr + u } val v = g1.vertices.collect().toSet assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) @@ -67,7 +69,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { test ("filter") { withSpark { sc => val n = 5 - val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) val graph: Graph[Int, Int] = Graph(vertices, edges) val filteredGraph = graph.filter( @@ -75,7 +77,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, - vpred = (vid: Vid, deg:Int) => deg > 0 + vpred = (vid: VertexID, deg:Int) => deg > 0 ) val v = filteredGraph.vertices.collect().toSet diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index e6c19dbc40358..41f3a8311d5a8 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -9,7 +9,7 @@ import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = { - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") } test("Graph.fromEdgeTuples") { @@ -39,7 +39,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } - val vertices: RDD[(Vid, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) + val vertices: RDD[(VertexID, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) val graph = Graph(vertices, edges, false) assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically @@ -56,7 +56,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val star = starGraph(sc, n) assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === - (1 to n).map(x => (0: Vid, x: Vid, "v", "v")).toSet) + (1 to n).map(x => (0: VertexID, x: VertexID, "v", "v")).toSet) } } @@ -92,7 +92,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val p = 100 val verts = 1 to n val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => - verts.filter(y => y % x == 0).map(y => (x: Vid, y: Vid))), p), 0) + verts.filter(y => y % x == 0).map(y => (x: VertexID, y: VertexID))), p), 0) assert(graph.edges.partitions.length === p) val partitionedGraph = graph.partitionBy(EdgePartition2D) assert(graph.edges.partitions.length === p) @@ -118,10 +118,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = starGraph(sc, n) // mapVertices preserving type val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") - assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: Vid, "v2")).toSet) + assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, "v2")).toSet) // mapVertices changing type val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) - assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: Vid, 1)).toSet) + assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, 1)).toSet) } } @@ -150,7 +150,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: Vid, 1)).toSet) + assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexID, 1)).toSet) } } @@ -173,7 +173,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mask") { withSpark { sc => val n = 5 - val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) val graph: Graph[Int, Int] = Graph(vertices, edges) @@ -199,7 +199,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val star = starGraph(sc, n) val doubleStar = Graph.fromEdgeTuples( - sc.parallelize((1 to n).flatMap(x => List((0: Vid, x: Vid), (0: Vid, x: Vid))), 1), "v") + sc.parallelize((1 to n).flatMap(x => + List((0: VertexID, x: VertexID), (0: VertexID, x: VertexID))), 1), "v") val star2 = doubleStar.groupEdges { (a, b) => a} assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) @@ -218,7 +219,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) // activeSetOpt - val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) + val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexID, y: VertexID) val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) val vids = complete.mapVertices((vid, attr) => vid).cache() val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } @@ -229,11 +230,11 @@ class GraphSuite extends FunSuite with LocalSparkContext { } Iterator((et.srcId, 1)) }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet - assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) + assert(numEvenNeighbors === (1 to n).map(x => (x: VertexID, n / 2)).toSet) // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ring = Graph.fromEdgeTuples(sc.parallelize((0 until n).map(x => (x: Vid, (x+1) % n: Vid)), 3), 0) - .mapVertices((vid, attr) => vid).cache() + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) + val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { @@ -243,7 +244,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } Iterator((et.dstId, 1)) }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet - assert(numOddNeighbors === (2 to n by 2).map(x => (x: Vid, 1)).toSet) + assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexID, 1)).toSet) } } @@ -258,7 +259,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), (a: Int, b: Int) => a + b).collect.toSet - assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) + assert(neighborDegreeSums === Set((0: VertexID, n)) ++ (1 to n).map(x => (x: VertexID, 0))) // outerJoinVertices preserving type val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } val newReverseStar = diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala index 44182e85eeafe..de7e3872cad6f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala @@ -10,7 +10,8 @@ class PregelSuite extends FunSuite with LocalSparkContext { test("1 iteration") { withSpark { sc => val n = 5 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") val result = Pregel(star, 0)( (vid, attr, msg) => attr, et => Iterator.empty, @@ -23,11 +24,12 @@ class PregelSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val chain = Graph.fromEdgeTuples( - sc.parallelize((1 until n).map(x => (x: Vid, x + 1: Vid)), 3), + sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), 0).cache() - assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: Vid, 0)).toSet) + assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } - assert(chainWithSeed.vertices.collect.toSet === Set((1: Vid, 1)) ++ (2 to n).map(x => (x: Vid, 0)).toSet) + assert(chainWithSeed.vertices.collect.toSet === + Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) val result = Pregel(chainWithSeed, 0)( (vid, attr, msg) => math.max(msg, attr), et => Iterator((et.dstId, et.srcAttr)), diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 4014cbe440d8d..2864ffd1ca31e 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -82,7 +82,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("IntAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 5) + val outMsg = (4: VertexID, 5) val bout = new ByteArrayOutputStream val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -90,8 +90,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Int) = inStrm.readObject() - val inMsg2: (Vid, Int) = inStrm.readObject() + val inMsg1: (VertexID, Int) = inStrm.readObject() + val inMsg2: (VertexID, Int) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) @@ -102,7 +102,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("LongAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 1L << 32) + val outMsg = (4: VertexID, 1L << 32) val bout = new ByteArrayOutputStream val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -110,8 +110,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Long) = inStrm.readObject() - val inMsg2: (Vid, Long) = inStrm.readObject() + val inMsg1: (VertexID, Long) = inStrm.readObject() + val inMsg2: (VertexID, Long) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) @@ -122,7 +122,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("DoubleAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 5.0) + val outMsg = (4: VertexID, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -130,8 +130,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Double) = inStrm.readObject() - val inMsg2: (Vid, Double) = inStrm.readObject() + val inMsg1: (VertexID, Double) = inStrm.readObject() + val inMsg2: (VertexID, Double) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index f951fd7a82741..fd0beee2f6408 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -62,7 +62,7 @@ class EdgePartitionSuite extends FunSuite { test("innerJoin") { def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { val builder = new EdgePartitionBuilder[A] - for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } + for ((src, dst, attr) <- xs) { builder.add(src: VertexID, dst: VertexID, attr) } builder.toEdgePartition } val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) From 43e1bdc80c2b19533596df74fd7b97a2d7b84bb6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:59:48 -0800 Subject: [PATCH 1283/1571] Pid -> PartitionID --- .../scala/org/apache/spark/graph/EdgeRDD.scala | 8 ++++---- .../scala/org/apache/spark/graph/Graph.scala | 5 +++-- .../apache/spark/graph/PartitionStrategy.scala | 16 ++++++++-------- .../org/apache/spark/graph/impl/GraphImpl.scala | 8 ++++---- .../spark/graph/impl/MessageToPartition.scala | 14 +++++++------- .../spark/graph/impl/ReplicatedVertexView.scala | 14 +++++++------- .../apache/spark/graph/impl/RoutingTable.scala | 4 ++-- .../scala/org/apache/spark/graph/package.scala | 2 +- 8 files changed, 36 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index fd933593523a8..78821bf568158 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -9,7 +9,7 @@ import org.apache.spark.storage.StorageLevel class EdgeRDD[@specialized ED: ClassTag]( - val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -17,7 +17,7 @@ class EdgeRDD[@specialized ED: ClassTag]( override protected def getPartitions: Array[Partition] = partitionsRDD.partitions /** - * If partitionsRDD already has a partitioner, use it. Otherwise assume that the Pids in + * If partitionsRDD already has a partitioner, use it. Otherwise assume that the PartitionIDs in * partitionsRDD correspond to the actual partitions and create a new partitioner that allows * co-partitioning with partitionsRDD. */ @@ -25,7 +25,7 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - firstParent[(Pid, EdgePartition[ED])].iterator(part, context).next._2.iterator + firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context).next._2.iterator } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -44,7 +44,7 @@ class EdgeRDD[@specialized ED: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassTag](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index dd0799142ef74..86282e607ebb7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -169,7 +169,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassTag](map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] + def mapEdges[ED2: ClassTag]( + map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] /** * Construct a new graph where the value of each edge is @@ -220,7 +221,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * */ def mapTriplets[ED2: ClassTag]( - map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): + map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] /** diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index c01b4b9439589..bc05fb812c008 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph sealed trait PartitionStrategy extends Serializable { - def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid + def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID } @@ -51,18 +51,18 @@ sealed trait PartitionStrategy extends Serializable { * */ case object EdgePartition2D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { - val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: VertexID = 1125899906842597L - val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts } } case object EdgePartition1D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { val mixingPrime: VertexID = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } @@ -74,7 +74,7 @@ case object EdgePartition1D extends PartitionStrategy { * random vertex cut. */ case object RandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { math.abs((src, dst).hashCode()) % numParts } } @@ -86,7 +86,7 @@ case object RandomVertexCut extends PartitionStrategy { * will end up on the same partition. */ case object CanonicalRandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { val lower = math.min(src, dst) val higher = math.max(src, dst) math.abs((lower, higher).hashCode()) % numParts diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 6eb401b3b57ac..8f42e7d5929dd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -69,7 +69,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val numPartitions = edges.partitions.size val edTag = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => - val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) @@ -173,13 +173,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } override def mapEdges[ED2: ClassTag]( - f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + f: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) } override def mapTriplets[ED2: ClassTag]( - f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdTag = classTag[VD] @@ -354,7 +354,7 @@ object GraphImpl { } def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( - edgePartitions: RDD[(Pid, EdgePartition[ED])], + edgePartitions: RDD[(PartitionID, EdgePartition[ED])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 2d03f75a28a25..b2fa7284827a4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -3,15 +3,15 @@ package org.apache.spark.graph.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{Pid, VertexID} +import org.apache.spark.graph.{PartitionID, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( - @transient var partition: Pid, + @transient var partition: PartitionID, var vid: VertexID, var data: T) - extends Product2[Pid, (VertexID, T)] with Serializable { + extends Product2[PartitionID, (VertexID, T)] with Serializable { override def _1 = partition @@ -27,9 +27,9 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( - @transient var partition: Pid, + @transient var partition: PartitionID, var data: T) - extends Product2[Pid, T] with Serializable { + extends Product2[PartitionID, T] with Serializable { override def _1 = partition @@ -41,7 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[Pid, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) + val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { @@ -62,7 +62,7 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { - new ShuffledRDD[Pid, T, MessageToPartition[T]](self, partitioner) + new ShuffledRDD[PartitionID, T, MessageToPartition[T]](self, partitioner) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 9d2d242ffa8d9..7d29861db1e3a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -46,12 +46,12 @@ class ReplicatedVertexView[VD: ClassTag]( }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIDMap") } - private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) - private lazy val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(true, false) - private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) - private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) + private lazy val bothAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(true, true) + private lazy val srcAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(true, false) + private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) + private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly @@ -63,7 +63,7 @@ class ReplicatedVertexView[VD: ClassTag]( def get( includeSrc: Boolean, includeDst: Boolean, - actives: VertexRDD[_]): RDD[(Pid, VertexPartition[VD])] = { + actives: VertexRDD[_]): RDD[(PartitionID, VertexPartition[VD])] = { // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is @@ -81,7 +81,7 @@ class ReplicatedVertexView[VD: ClassTag]( } private def create(includeSrc: Boolean, includeDst: Boolean) - : RDD[(Pid, VertexPartition[VD])] = { + : RDD[(PartitionID, VertexPartition[VD])] = { val vdTag = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala index 9e6f78197e7f9..96d9e9d7f8c8e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -30,8 +30,8 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(VertexID, Pid)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val vid2pid: RDD[(VertexID, PartitionID)] = edges.partitionsRDD.mapPartitions { iter => + val (pid: PartitionID, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet if (includeSrcAttr) { // Add src vertices to the set. diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 823d47c359b09..b98a11b918a21 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -8,7 +8,7 @@ package object graph { type VertexID = Long // TODO: Consider using Char. - type Pid = Int + type PartitionID = Int type VertexSet = OpenHashSet[VertexID] From 100718bcd3f6ade1a93256458ec1528bb9142b5e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 14:05:09 -0800 Subject: [PATCH 1284/1571] Svdpp -> SVDPlusPlus --- .../algorithms/{Svdpp.scala => SVDPlusPlus.scala} | 12 ++++++------ .../{SvdppSuite.scala => SVDPlusPlusSuite.scala} | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) rename graph/src/main/scala/org/apache/spark/graph/algorithms/{Svdpp.scala => SVDPlusPlus.scala} (89%) rename graph/src/test/scala/org/apache/spark/graph/algorithms/{SvdppSuite.scala => SVDPlusPlusSuite.scala} (72%) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala similarity index 89% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala rename to graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala index 85fa23d309469..083aa305388e8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala @@ -5,7 +5,7 @@ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ -class SvdppConf( // Svdpp parameters +class SVDPlusPlusConf( // SVDPlusPlus parameters var rank: Int, var maxIters: Int, var minVal: Double, @@ -15,7 +15,7 @@ class SvdppConf( // Svdpp parameters var gamma6: Double, var gamma7: Double) extends Serializable -object Svdpp { +object SVDPlusPlus { /** * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model", * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. @@ -23,12 +23,12 @@ object Svdpp { * * @param edges edges for constructing the graph * - * @param conf Svdpp parameters + * @param conf SVDPlusPlus parameters * * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]], conf: SvdppConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { + def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { // generate default vertex attribute def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { @@ -55,7 +55,7 @@ object Svdpp { (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) } - def mapTrainF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) : Iterator[(VertexID, (RealVector, RealVector, Double))] = { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) @@ -85,7 +85,7 @@ object Svdpp { } // calculate error on training set - def mapTestF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { + def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala similarity index 72% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala rename to graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala index 411dd3d336c2a..a0a6eb33e36fc 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala @@ -9,21 +9,21 @@ import org.apache.spark.graph.util.GraphGenerators import org.apache.spark.rdd._ -class SvdppSuite extends FunSuite with LocalSparkContext { +class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { test("Test SVD++ with mean square error on training set") { withSpark { sc => - val SvdppErr = 8.0 + val svdppErr = 8.0 val edges = sc.textFile("mllib/data/als/test.data").map { line => val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } - val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = Svdpp.run(edges, conf) + val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + var (graph, u) = SVDPlusPlus.run(edges, conf) val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size - assert(err <= SvdppErr) + assert(err <= svdppErr) } } From 731f56f309914e3fc7c22c8ef1c8cb9dd40d42c1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 14:31:33 -0800 Subject: [PATCH 1285/1571] graph -> graphx --- {graph => graphx}/pom.xml | 0 .../scala/org/apache/spark/graphx}/Analytics.scala | 8 ++++---- .../main/scala/org/apache/spark/graphx}/Edge.scala | 2 +- .../org/apache/spark/graphx}/EdgeDirection.scala | 4 ++-- .../scala/org/apache/spark/graphx}/EdgeRDD.scala | 4 ++-- .../scala/org/apache/spark/graphx}/EdgeTriplet.scala | 4 ++-- .../main/scala/org/apache/spark/graphx}/Graph.scala | 4 ++-- .../apache/spark/graphx}/GraphKryoRegistrator.scala | 4 ++-- .../scala/org/apache/spark/graphx}/GraphLab.scala | 2 +- .../scala/org/apache/spark/graphx}/GraphLoader.scala | 6 +++--- .../scala/org/apache/spark/graphx}/GraphOps.scala | 2 +- .../org/apache/spark/graphx}/PartitionStrategy.scala | 2 +- .../main/scala/org/apache/spark/graphx}/Pregel.scala | 2 +- .../scala/org/apache/spark/graphx}/VertexRDD.scala | 6 +++--- .../graphx}/algorithms/ConnectedComponents.scala | 4 ++-- .../apache/spark/graphx}/algorithms/PageRank.scala | 4 ++-- .../spark/graphx}/algorithms/SVDPlusPlus.scala | 4 ++-- .../algorithms/StronglyConnectedComponents.scala | 4 ++-- .../spark/graphx}/algorithms/TriangleCount.scala | 4 ++-- .../apache/spark/graphx}/impl/EdgePartition.scala | 4 ++-- .../spark/graphx}/impl/EdgePartitionBuilder.scala | 4 ++-- .../spark/graphx}/impl/EdgeTripletIterator.scala | 4 ++-- .../org/apache/spark/graphx}/impl/GraphImpl.scala | 10 +++++----- .../spark/graphx}/impl/MessageToPartition.scala | 4 ++-- .../spark/graphx}/impl/ReplicatedVertexView.scala | 4 ++-- .../org/apache/spark/graphx}/impl/RoutingTable.scala | 4 ++-- .../org/apache/spark/graphx}/impl/Serializers.scala | 4 ++-- .../apache/spark/graphx}/impl/VertexPartition.scala | 8 ++++---- .../scala/org/apache/spark/graphx}/package.scala | 2 +- .../org/apache/spark/graphx}/perf/BagelTest.scala | 4 ++-- .../org/apache/spark/graphx}/perf/SparkTest.scala | 4 ++-- .../apache/spark/graphx}/util/BytecodeUtils.scala | 2 +- .../apache/spark/graphx}/util/GraphGenerators.scala | 12 ++++++------ .../org/apache/spark/graphx}/util/HashUtils.scala | 2 +- .../src/test/resources/log4j.properties | 0 .../org/apache/spark/graphx}/GraphOpsSuite.scala | 6 +++--- .../scala/org/apache/spark/graphx}/GraphSuite.scala | 4 ++-- .../org/apache/spark/graphx}/LocalSparkContext.scala | 4 ++-- .../scala/org/apache/spark/graphx}/PregelSuite.scala | 2 +- .../org/apache/spark/graphx}/SerializerSuite.scala | 6 +++--- .../org/apache/spark/graphx}/VertexRDDSuite.scala | 6 +++--- .../algorithms/ConnectedComponentsSuite.scala | 6 +++--- .../spark/graphx}/algorithms/PageRankSuite.scala | 6 +++--- .../spark/graphx}/algorithms/SVDPlusPlusSuite.scala | 6 +++--- .../StronglyConnectedComponentsSuite.scala | 6 +++--- .../graphx}/algorithms/TriangleCountSuite.scala | 6 +++--- .../spark/graphx}/impl/EdgePartitionSuite.scala | 4 ++-- .../spark/graphx}/impl/VertexPartitionSuite.scala | 4 ++-- .../spark/graphx}/util/BytecodeUtilsSuite.scala | 2 +- project/SparkBuild.scala | 12 ++++++------ 50 files changed, 111 insertions(+), 111 deletions(-) rename {graph => graphx}/pom.xml (100%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Analytics.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Edge.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeDirection.scala (88%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeRDD.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeTriplet.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Graph.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphKryoRegistrator.scala (92%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphLab.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphLoader.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphOps.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/PartitionStrategy.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Pregel.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/VertexRDD.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/ConnectedComponents.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/PageRank.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/SVDPlusPlus.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/StronglyConnectedComponents.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/TriangleCount.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgePartition.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgePartitionBuilder.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgeTripletIterator.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/GraphImpl.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/MessageToPartition.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/ReplicatedVertexView.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/RoutingTable.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/Serializers.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/VertexPartition.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/package.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/perf/BagelTest.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/perf/SparkTest.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/BytecodeUtils.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/GraphGenerators.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/HashUtils.scala (91%) rename {graph => graphx}/src/test/resources/log4j.properties (100%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/GraphOpsSuite.scala (96%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/GraphSuite.scala (99%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/LocalSparkContext.scala (92%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/PregelSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/SerializerSuite.scala (98%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/VertexRDDSuite.scala (95%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/ConnectedComponentsSuite.scala (95%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/PageRankSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/SVDPlusPlusSuite.scala (87%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/StronglyConnectedComponentsSuite.scala (92%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/TriangleCountSuite.scala (94%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/impl/EdgePartitionSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/impl/VertexPartitionSuite.scala (98%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/util/BytecodeUtilsSuite.scala (98%) diff --git a/graph/pom.xml b/graphx/pom.xml similarity index 100% rename from graph/pom.xml rename to graphx/pom.xml diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Analytics.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 14b9be73f1651..0cafc3fdf94ce 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark._ -import org.apache.spark.graph.algorithms._ +import org.apache.spark.graphx.algorithms._ /** @@ -49,7 +49,7 @@ object Analytics extends Logging { val serializer = "org.apache.spark.serializer.KryoSerializer" System.setProperty("spark.serializer", serializer) //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") taskType match { case "pagerank" => { @@ -400,7 +400,7 @@ object Analytics extends Logging { // System.setProperty("spark.serializer", "spark.KryoSerializer") // //System.setProperty("spark.shuffle.compress", "false") - // System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + // System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") // taskType match { // case "pagerank" => { diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/Edge.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 19c28bea685ba..29b46674f149c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx /** diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala similarity index 88% rename from graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index a1468a152baa9..785f941650139 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx /** @@ -7,7 +7,7 @@ package org.apache.spark.graph */ sealed abstract class EdgeDirection { /** - * Reverse the direction of an edge. An in becomes out, + * Reverse the direction of an edge. An in becomes out, * out becomes in and both remains both. */ def reverse: EdgeDirection = this match { diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 78821bf568158..e4ef460e6f88a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index a5103ed3cbffc..b0565b7e0e07c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph +package org.apache.spark.graphx -import org.apache.spark.graph.impl.VertexPartition +import org.apache.spark.graphx.impl.VertexPartition /** * An edge triplet represents two vertices and edge along with their diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Graph.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 86282e607ebb7..2b7c0a2583f21 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.graph.impl._ +import org.apache.spark.graphx.impl._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala similarity index 92% rename from graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index 296f3848f1c61..f8aab951f0637 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import com.esotericsoftware.kryo.Kryo -import org.apache.spark.graph.impl._ +import org.apache.spark.graphx.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet import org.apache.spark.util.BoundedPriorityQueue diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/GraphLab.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 22f48540193ea..437288405f307 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 7daac4fcc56c3..473cfb18cf481 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import java.util.{Arrays => JArrays} import scala.reflect.ClassTag -import org.apache.spark.graph.impl.EdgePartitionBuilder +import org.apache.spark.graphx.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} +import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl} import org.apache.spark.util.collection.PrimitiveVector diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/GraphOps.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index e41287c1ed4f1..cacfcb1c90f49 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala rename to graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index bc05fb812c008..5e80a535f1c81 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx sealed trait PartitionStrategy extends Serializable { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Pregel.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 3b84e2e5e4195..8ddb78813557f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala rename to graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 25b0aed85aae9..cfee9b089f9e4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag @@ -24,8 +24,8 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graph.impl.MsgRDDFunctions -import org.apache.spark.graph.impl.VertexPartition +import org.apache.spark.graphx.impl.MsgRDDFunctions +import org.apache.spark.graphx.impl.VertexPartition /** diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala index 2a6b8c0999ab0..a0dd36da60f32 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object ConnectedComponents { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 26b8dc5ab60c6..0292b7316d392 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.apache.spark.Logging -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object PageRank extends Logging { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala index 083aa305388e8..8fdfa3d9077a4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.apache.spark.rdd._ -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import scala.util.Random import org.apache.commons.math.linear._ diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index 8031aa10ce3ea..f64fc3ef0f52a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object StronglyConnectedComponents { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala index 81774d52e4291..b5a93c1bd19bb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object TriangleCount { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index b4311fa9f894e..4176563d22555 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 56624ef60adc0..d4f08497a280a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Sorting -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index e95d79e3d63d6..79fd962ffdc15 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 8f42e7d5929dd..be9f1881500f5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -1,14 +1,14 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.util.collection.PrimitiveVector import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.impl.GraphImpl._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ -import org.apache.spark.graph.util.BytecodeUtils +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.GraphImpl._ +import org.apache.spark.graphx.impl.MsgRDDFunctions._ +import org.apache.spark.graphx.util.BytecodeUtils import org.apache.spark.rdd.{ShuffledRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ClosureCleaner diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index b2fa7284827a4..ad5daf8f6ae32 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{PartitionID, VertexID} +import org.apache.spark.graphx.{PartitionID, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 7d29861db1e3a..63180bc3af473 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} @@ -6,7 +6,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} -import org.apache.spark.graph._ +import org.apache.spark.graphx._ /** * A view of the vertices after they are shipped to the join sites specified in diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala index 96d9e9d7f8c8e..3bd8b24133244 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.PrimitiveVector diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index a3b0ea7689fc7..1c3c87f08dcb9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -1,10 +1,10 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer import org.apache.spark.SparkConf -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.serializer._ class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 91244daa54a57..7c83497ca9564 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -1,14 +1,14 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} import org.apache.spark.Logging -import org.apache.spark.graph._ +import org.apache.spark.graphx._ -private[graph] object VertexPartition { +private[graphx] object VertexPartition { def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)]): VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[VertexID, VD] @@ -30,7 +30,7 @@ private[graph] object VertexPartition { } -private[graph] +private[graphx] class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( val index: VertexIdToIndexMap, val values: Array[VD], diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/package.scala rename to graphx/src/main/scala/org/apache/spark/graphx/package.scala index b98a11b918a21..96f0d91c9bcf9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -3,7 +3,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -package object graph { +package object graphx { type VertexID = Long diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala rename to graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala index eaff27a33eda5..81332e0800b94 100644 --- a/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala @@ -1,6 +1,6 @@ ///// This file creates circular dependencies between examples bagle and graph -// package org.apache.spark.graph.perf +// package org.apache.spark.graphx.perf // import org.apache.spark._ // import org.apache.spark.SparkContext._ @@ -8,7 +8,7 @@ // import org.apache.spark.examples.bagel // //import org.apache.spark.bagel.examples._ -// import org.apache.spark.graph._ +// import org.apache.spark.graphx._ // object BagelTest { diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala rename to graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala index 01bd96855082c..24262640ab0da 100644 --- a/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala @@ -1,13 +1,13 @@ ///// This file creates circular dependencies between examples bagle and graph -// package org.apache.spark.graph.perf +// package org.apache.spark.graphx.perf // import org.apache.spark._ // import org.apache.spark.SparkContext._ // import org.apache.spark.bagel.Bagel // import org.apache.spark.bagel.examples._ -// import org.apache.spark.graph._ +// import org.apache.spark.graphx._ // object SparkTest { diff --git a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index bc00ce215153d..ec8d534333de5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 51f45cb8922b3..57117241ad818 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import scala.annotation.tailrec import scala.math._ @@ -10,10 +10,10 @@ import org.apache.spark.serializer._ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.Graph -import org.apache.spark.graph.Edge -import org.apache.spark.graph.impl.GraphImpl +import org.apache.spark.graphx._ +import org.apache.spark.graphx.Graph +import org.apache.spark.graphx.Edge +import org.apache.spark.graphx.impl.GraphImpl /** * @todo cleanup and modularize code @@ -31,7 +31,7 @@ object GraphGenerators { val serializer = "org.apache.spark.serializer.KryoSerializer" System.setProperty("spark.serializer", serializer) //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") val host = "local[4]" val sc = new SparkContext(host, "Lognormal graph generator") diff --git a/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala similarity index 91% rename from graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala index cb18ef3d26bae..7a79d33350b1c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util object HashUtils { diff --git a/graph/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties similarity index 100% rename from graph/src/test/resources/log4j.properties rename to graphx/src/test/resources/log4j.properties diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala similarity index 96% rename from graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 132e6be24a04a..cc281fce997bf 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd._ import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala similarity index 99% rename from graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 41f3a8311d5a8..094fa722a0030 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.FunSuite import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ +import org.apache.spark.graphx.Graph._ import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { diff --git a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala similarity index 92% rename from graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala rename to graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index 5c20d559aa973..6aec2ea8a994f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach @@ -12,7 +12,7 @@ import org.apache.spark.SparkContext */ trait LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ def withSpark[T](f: SparkContext => T) = { diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index de7e3872cad6f..429622357f0bd 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 2864ffd1ca31e..3ba412c1f84f4 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} @@ -7,8 +7,8 @@ import scala.util.Random import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.graph.impl._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ +import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.serializer.SerializationStream diff --git a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala similarity index 95% rename from graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index e876b8e4e8bfe..573b708e89dd9 100644 --- a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd._ import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala similarity index 95% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 81a1b7337f2a4..5e2ecfcde9b0c 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index 81d82a5a6b719..e365b1e2305b4 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -1,13 +1,13 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx.util.GraphGenerators object GridPageRank { diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala similarity index 87% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala index a0a6eb33e36fc..06604198d73a0 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala similarity index 92% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala index 4afb158a68ed4..696b80944ed02 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala similarity index 94% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala index 274ab11f0cbd1..0e59912754c05 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index fd0beee2f6408..eb82436f0964c 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.graph._ +import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index 72579a48c2885..d37d64e8c849e 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.scalatest.FunSuite class VertexPartitionSuite extends FunSuite { diff --git a/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala index d85e877ddfaf1..11db339750920 100644 --- a/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import org.scalatest.FunSuite diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c2cd6fb45abda..c2b1c0c35cc5b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -48,20 +48,20 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(core, graph, bagel, mllib) + .dependsOn(core, graphx, bagel, mllib) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) + lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graph, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) + .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -111,10 +111,10 @@ object SparkBuild extends Build { lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graph, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graph) ++ maybeYarnRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) @@ -308,7 +308,7 @@ object SparkBuild extends Build { name := "spark-tools" ) ++ assemblySettings ++ extraAssemblySettings - def graphSettings = sharedSettings ++ Seq( + def graphxSettings = sharedSettings ++ Seq( name := "spark-graphx" ) From 2af98198ad6e12bc8438488d306667020dbbc657 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Jan 2014 15:57:44 -0800 Subject: [PATCH 1286/1571] Send logs to stderr by default (instead of stdout). --- conf/log4j.properties.template | 1 + .../main/resources/org/apache/spark/log4j-defaults.properties | 1 + 2 files changed, 2 insertions(+) diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index d72dbadc3904f..17d1978dde4d7 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -1,6 +1,7 @@ # Set everything to be logged to the console log4j.rootCategory=INFO, console log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index d72dbadc3904f..17d1978dde4d7 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -1,6 +1,7 @@ # Set everything to be logged to the console log4j.rootCategory=INFO, console log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n From dd7fd0c4433398b2e0a86e008a4d64dafc2f11be Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 10 Jan 2014 08:53:59 +0800 Subject: [PATCH 1287/1571] Upgrade Kafka dependecy to 0.8.0 release version --- external/kafka/pom.xml | 4 ++-- project/SparkBuild.scala | 18 +++++++++--------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index f782e0e126d45..23b2fead657e6 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -45,9 +45,9 @@ test - com.sksamuel.kafka + org.apache.kafka kafka_${scala.binary.version} - 0.8.0-beta1 + 0.8.0 com.sun.jmx diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bd5816ea280db..c1a5c4df6f308 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -90,7 +90,7 @@ object SparkBuild extends Build { lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]() lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]() - lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) + lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) .dependsOn(streaming % "compile->compile;test->test") lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings) @@ -98,23 +98,23 @@ object SparkBuild extends Build { lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings) .dependsOn(streaming % "compile->compile;test->test") - + lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings) .dependsOn(streaming % "compile->compile;test->test") - + lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings) .dependsOn(streaming % "compile->compile;test->test") lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) - + lazy val examples = Project("examples", file("examples"), settings = examplesSettings) .dependsOn(core, mllib, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef - lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) + lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", @@ -320,7 +320,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( - "commons-io" % "commons-io" % "2.4" + "commons-io" % "commons-io" % "2.4" ) ) @@ -387,19 +387,19 @@ object SparkBuild extends Build { "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) ) ) - + def kafkaSettings() = sharedSettings ++ Seq( name := "spark-streaming-kafka", libraryDependencies ++= Seq( "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), - "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" + "org.apache.kafka" %% "kafka" % "0.8.0" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") exclude("net.sf.jopt-simple", "jopt-simple") excludeAll(excludeNetty) ) ) - + def flumeSettings() = sharedSettings ++ Seq( name := "spark-streaming-flume", libraryDependencies ++= Seq( From 142921c6c0d0d9134d5fa0e307575d8f6749fb9a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 9 Jan 2014 00:05:53 -0800 Subject: [PATCH 1288/1571] Add some missing Java API methods --- .../scala/org/apache/spark/SparkContext.scala | 12 ++++- .../apache/spark/api/java/JavaDoubleRDD.scala | 12 +++-- .../apache/spark/api/java/JavaPairRDD.scala | 6 +++ .../org/apache/spark/api/java/JavaRDD.scala | 6 +++ .../apache/spark/api/java/JavaRDDLike.scala | 6 +++ .../spark/api/java/JavaSparkContext.scala | 54 ++++++++++++++++++- 6 files changed, 90 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f91392b3510e2..66c226e491362 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -244,6 +244,10 @@ class SparkContext( localProperties.set(new Properties()) } + /** + * Set a local property that affects jobs submitted from this thread, such as the + * Spark fair scheduler pool. + */ def setLocalProperty(key: String, value: String) { if (localProperties.get() == null) { localProperties.set(new Properties()) @@ -255,6 +259,10 @@ class SparkContext( } } + /** + * Get a local property set in this thread, or null if it is missing. See + * [[org.apache.spark.SparkContext.setLocalProperty]]. + */ def getLocalProperty(key: String): String = Option(localProperties.get).map(_.getProperty(key)).getOrElse(null) @@ -265,7 +273,7 @@ class SparkContext( } /** - * Assigns a group id to all the jobs started by this thread until the group id is set to a + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a * different value or cleared. * * Often, a unit of execution in an application consists of multiple Spark actions or jobs. @@ -288,7 +296,7 @@ class SparkContext( setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) } - /** Clear the job group id and its description. */ + /** Clear the current thread's job group ID and its description. */ def clearJobGroup() { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index da30cf619a1d0..b0dedc6f4eb13 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -207,13 +207,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav * e.g. for the array * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1,0,0 - * + * And on the input of 1 and 50 we would have a histogram of 1,0,0 + * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. - * buckets array must be at least two elements + * buckets array must be at least two elements * All NaN entries are treated the same. If you have a NaN bucket it must be * the maximum value of the last position and all NaN entries will be counted * in that bucket. @@ -225,6 +225,12 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = { srdd.histogram(buckets.map(_.toDouble), evenBuckets) } + + /** Assign a name to this RDD */ + def setName(name: String): JavaDoubleRDD = { + srdd.setName(name) + this + } } object JavaDoubleRDD { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 55c87450ac65a..0fb7e195b34c4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -647,6 +647,12 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = { rdd.countApproxDistinctByKey(relativeSD, numPartitions) } + + /** Assign a name to this RDD */ + def setName(name: String): JavaPairRDD[K, V] = { + rdd.setName(name) + this + } } object JavaPairRDD { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 037cd1c774691..7d48ce01cf2cc 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -127,6 +127,12 @@ JavaRDDLike[T, JavaRDD[T]] { wrapRDD(rdd.subtract(other, p)) override def toString = rdd.toString + + /** Assign a name to this RDD */ + def setName(name: String): JavaRDD[T] = { + rdd.setName(name) + this + } } object JavaRDD { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 924d8af0602f4..ebbbbd88061a1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -244,6 +244,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an array that contains all of the elements in this RDD. + */ + def toArray(): JList[T] = collect() + /** * Return an array that contains all of the elements in a specific partition of this RDD. */ @@ -455,4 +460,5 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD) + def name(): String = rdd.name } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index e93b10fd7eecb..7a6f044965027 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -425,6 +425,51 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def clearCallSite() { sc.clearCallSite() } + + /** + * Set a local property that affects jobs submitted from this thread, such as the + * Spark fair scheduler pool. + */ + def setLocalProperty(key: String, value: String): Unit = sc.setLocalProperty(key, value) + + /** + * Get a local property set in this thread, or null if it is missing. See + * [[org.apache.spark.api.java.JavaSparkContext.setLocalProperty]]. + */ + def getLocalProperty(key: String): String = sc.getLocalProperty(key) + + /** + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a + * different value or cleared. + * + * Often, a unit of execution in an application consists of multiple Spark actions or jobs. + * Application programmers can use this method to group all those jobs together and give a + * group description. Once set, the Spark web UI will associate such jobs with this group. + * + * The application can also use [[org.apache.spark.api.java.JavaSparkContext.cancelJobGroup]] + * to cancel all running jobs in this group. For example, + * {{{ + * // In the main thread: + * sc.setJobGroup("some_job_to_cancel", "some job description"); + * rdd.map(...).count(); + * + * // In a separate thread: + * sc.cancelJobGroup("some_job_to_cancel"); + * }}} + */ + def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description) + + /** Clear the current thread's job group ID and its description. */ + def clearJobGroup(): Unit = sc.clearJobGroup() + + /** + * Cancel active jobs for the specified group. See + * [[org.apache.spark.api.java.JavaSparkContext.setJobGroup]] for more information. + */ + def cancelJobGroup(groupId: String): Unit = sc.cancelJobGroup(groupId) + + /** Cancel all jobs that have been scheduled or are running. */ + def cancelAllJobs(): Unit = sc.cancelAllJobs() } object JavaSparkContext { @@ -436,5 +481,12 @@ object JavaSparkContext { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to SparkContext. */ - def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray + def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray + + /** + * Find the JAR that contains the class of a particular object, to make it easy for users + * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in + * your driver program. + */ + def jarOfObject(obj: AnyRef): Array[String] = SparkContext.jarOfObject(obj).toArray } From c43eb006442bbab14f5cd0898b6ad0f39f273506 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 9 Jan 2014 18:14:20 -0800 Subject: [PATCH 1289/1571] Fix bug added when we changed AppDescription.maxCores to an Option The Scala compiler warned about this -- we were comparing an Option against an integer now. --- .../org/apache/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index dbb0cb90f5186..9485bfd89eb57 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -67,11 +67,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {

F9zWf*Q@v;4=nMcpS-UX~2>%7{$nq9d$jReQETx(e<=T1YZob$&@3@S8LIzVLlv} z#$FMgoSqtQ6KON*FPs>%b24l_)hc?1O|gqES9+_bm#aK_dQY#d)#|C+eWLJ4EfF` z)n{T90BZ7)RZZ?*Z5Lgx^vl&v8p&(&!Vh*~vuje_50+~0yX14IPM3}N3Sk8G zy3Q^cM!blwG@G&;F}Eob`@#DEAXaX_CiDBf{tiNd z&=GPkt+v@NA!;uTUtg+ADs(Sv)n8QEW)E7f@Wk-3_4}hT`atPw%qs4Cb*bu>I%z?p z*4X3W2}o0$FE8&h^0Dc$m2qT zqBfDnxS;M+7N@>zQy6h!KVlAV68>dvvVm2^g*tB4C-_LY6L z6IgdJId-U9c?VKYqmK`+~D?;?Z zWR>JD$NTxxxp3Md#FwYrb^DK!{)3PA`~RNL_8 zryj3@Ai9o;PgvXx%9$|zofen+UiEf@99%%Ene%MOJyg2&Ku#XtYH~_T7{wiQzQ{2-(q6Lej`{0fel;r<0*XL&-euN| zvPXci*Hok-fz6&6DD$U9ly;T~jQ`{@z#g_KfaP!ymeI3L5xPuxBHA6_6;vrY$3o?m zvNun)s~@6Vw1UH%_O>|tMcCdEvGvZ}dbIl}4k4LuJ!=qezZ}aZBzb8@y{)yYHcCPV zJ@|m~ZK6F=utQUI++%{_dWA_&I9>?X#h6m~jTvkQTCQ*Viv2dxDYP2U*F!+!8Zf-q zXtO^w9{$ZfQALA#*Ql5%O!Fzi(v#PxuOva-B#7gkM#TyLNn8xBa8?O>`8sMKJ@Z3# z;mOMOe&T-;W^2y!bgj};euY3+v>5JU8z<*PV5g8_$)1LRiu=XI`4M;Wf;CABq{#gtv*YM_l~Dw`h^h{=*`?XG;JDgFy*>NazZysE#4JN!WV*pY9`#l_ycg*KE+XI)Yyd&U39jB6btpRx@4?wxL z+Ch~={m;HhUxvZDUvX`hX*gsD)ZCxR*-z`7{RW#N9PU=u+BhV`4z|8`{50h6Z`Q-c z^lBmR>u;X51KuSzDh3NJa>d_WxZp!qr+6ZAc?nI$!QtL{UA8GZ$vO&?Jy?k`49V*F z6@hufzY&jnLShmma(CQ#F$WUm#mP0T+{JzvKbJ&A;id}5%Xm7S3@eVDJz%zf9ojNg z7K-zLRrxOo+q@CWbVHZB*!20Q~ePB6v%~z%CWkBKS$U}H~ zyCKsyH)~pF;v4W8A}pmzA$8O5igvHbimFMyD!a7s^8b?rP9%OBh_-egVEQ z3Vd+!GpWu0urm*F97#}D)>=7hE}5a8@vvwiD$pou#+!W{7_2fkD~!@ zhoKzb&QiU$Eike#R~wPx-(7td(g~qv!1EZN?_)dh%BYaC{P2b`V7IkmUvsmMt&RvU zuGOphm6^>K@KYPw$JUDmjrZnD9|k(^Db{qW3me-<*NFrrm;JJs%+dmIKGq!6%s#j> z3iNKXvcqtot!vZ2g?(ss1b7=#yy9iBaUYdN?P{&1O?*#}p*`WL~eLokcuaIc0owRUM|2V36a9nwm#e+z~0St)$8s}3|P zQ`FsWwd~2uak-JjoWxh1WvDH>+h)u49sln1h2CHd!dH_qzYWr1dg2m7gZh5B&8y=) zdSMjT1jA+C<$Ltdq55E=m^SbQTFxG*<{6rH{;B2}*x@j2&D-Id&+L{UH_66ouFcK@ z-2*)i--BHDbGy-044rqlwkB@t_7_r>#6{X3Eh5xz8ODQD@-p;$eQmK9TyRbqt$8-g z)#10pxev@J=nn9X?e$qf<0Lv#ik2h%{FdmT%^NcnX>_snnX?aJG#Y~0NbcItmfD8O zZP5CA|KF-W(Q)|rQlHZn8%&b}%CUue6>d&T39s?-G0HbT>R@f|9mHw!+26vN&+&0} zOK-D_D_7U6V)%+QAZbfFZz%8V!>Xs3ynKQdwFkB)Jb}lKa@+MZ*5B*+_4Im0XM?y0 z@q^oU#||AT&EM(z?Vq_@#U29lE@BeT#a8^R$Hhh}^O#jPIQvwXcR3QqpdOs7ePo|~ zczOK|IpgTEqzLj-479mhOgvy0SB^boNZChIVOR0F2yGZgFHY)UmD^Z)O{?;(H;RrY zL^Wz&IHk|it+dva4_*BjDpKyk4oC%Zh-A)BZ1 zOt<38$&G*Q_*FlccM-+zO!F~&WN?coq$Dg z5U>IQc)zzMS7CKbV*V2wp&Ye7c(-UKIgxb@5Bb1;yn5uCK4NMQI+Ngnxf!m0ll=Dc zm7~`$Z#K=OM+2?R-?otbc%2dKGT*A}J3_$MRv!Aku>Atmqu4P3ucPeE3D2qTzJ~83e3qY= zs}w2M?AMImg_7{MR7U>HOM_|*6F@Jk!2Y4CMYF)thDajK#^-|1@_Up`!dppnb-9~I zOma}jKtEjM9$`?ugg83)rtu%bDd_MGkGI>ZXK9wj^Q3!3vT&B2dI?FPA@PyQ>$VCs z4E@s)u>`Yq8jb7J(w88&W%(_UF(L3a+9FiNQhfZUuj}dZk{KMf_M$>oiU3 zOL2k8aX87UXVW`RpF}@~cKwh*xH1h>-L=RplF6K^$(P)p=Z|*)@ic0Qd$vO}TSUb( zG6iDK{*PXlrm}Nnz$dnlisjY6>MlM^uXS1@Pqws$RIKq=dV1?`dPZf8m6qMw*g`5c zBwTCUFyWEEknT?$XBNVwwsM^s<3el>1)TVktwONs0CSZ4evF?RY=;)m*Qnb{yQ*22pYKZCde? zm5CT>WygJiHi%8-I#%24;YIV!^ui*qnW8;y5EXZ3k8iSNwD(=hv z#>;W_SP+XMrt$X zV#7!kTdw8w+glxLpiiJ{Hn9yx_$Bn$lKf+{W3C)FcHPuv4}hRKR~{MvjLL>3;d$(6 zJm?&nkQk3Et9_&Ye>yjfu9@DC6@k6?a{NBGF$T9A;h>!`MGpu=R^Of<8YHf2p6DJ8;H z0});nRdCcV>C#h_s{=~w!b%$XjTRRaizPlh_^am}PdfK<3I?ksELW59)=tRcB?02* zl^hci9TgT7g6FETFDp~!tal%@aJT?!5=`mLsKJx;sPalFyz<%RIixlI23Qn-_KG>klXas{+iKr#XT({@D|f_zJeqNId|Rt@25~H0~rh! zdxT$s{}px^nh=^8qw2r2*)5fz!F_rCn$&@1QefI@E>@_C%u zKL{ke3W{hM8{_LGFVe;Mk(1v8Lzg}Ao;_`C7N#4uU|iwtJv7XKho4k$=sg)QaUM+v z{fy5UpXKM6l7+Ig8C#XL`q$t3$t(P3HjQcuYihWSLSDFU8+KVY7i#Rm$DL;M`Yesw z4U7#98Q3W{Y=E3`Y*B%eXYlvb5px3tZ^&ZMJ)qdk5|fA?b8wXMqQy^!rBoza)73D% z%7$;uLK(YXeay&mliR)l=6hhUH$(WPZOX#HjrAe_RvlYt8~*eyKCVmdEm;uX!O6C2 zsoaacLnt!TfiAp7yaUG1JJ91}UwPJa#|oA2%g(>?aZy*q$p;AkRRo*h;i8=)r@Qg~J4;Whm;+K*X zyuSmIXV15PMo$u&SkD+T0Xg!@d8&%16HJR z9i4VBHh{SrOzv3s(xir~U#|iFrM20*#>*b%0fO$eXv3;=-hNRv?--ieH-f?Qj4WrtQiTA|B)9MALoYhHz8)<)v@vGolu4 z@H&ByH>?S#MPAljT-hgn&{S0~&p&Aw`iS0OTLv9lF&#^u6`8tD3chhFi{cbD5V`uh zH}(0NHhXP?INonQIFUe&wYGpVhK958jVskVYG~!B2Cv|iLt0}gVIucW*$FC zQ*>9YWUTsw%ZN+1v}{u_YN-7$6!HH0laPV5$Lb+u)Mw?`c}J9C8cUdUnjM6q$ zU6)>(Qf)EFf#`#Ie0u(uNH;Z9W-Gb#lrOe^8$GDx1t^~xyme0mB5{iyIM~ny>$3@h zvU-Tu53YhcphFCk zy>QU&?9*!GZRscoza8S!Oi{4ev{))lg6n1n7li$AWQAD*6^;MbU8VLEhmdf5JpKp1 z5``e?jYi6=i2s#S+quSo(9=rQb+0)^80koC)CuDb<@F+OpZ31&2mwKe86K=6M!pq= zXge7iF1k|w=9>La!ow|bQPouX*1fUj7|QejB6*sS`Cb$!%}Ektjn$!I?@M{(L}!X) z!RAx+F1d>s$G4_5*fZ8TkSHp0ocD{NC4svOSLJ6z^L`*AvuzNAQ6~}^yl~=Wpj&bb zF=5L4YNno!dsG-^FOD|A+f!0L5iX>T7*5FDdP&-jV0p@zQK*P4*G9hBpk94#@9|ZQ z{s-x)onnY&yn2!giz?|hOT{Eum5*5#K$|JZ!3vx zb@GKiRN06RrHyImk7}cPg18}?JBxI!n<|vuwPMpkItY0KGk4tU&$@^>T|=mz7=Ah8 z?VJ=?VGH!-eOs>@B3xz(3>Cu+3Cc{OZH;>kgMg+(gnW z2CeuE#wQI$N&ub)#bb){2J~I2NcV#<*k*vRaZULB?Jh!mqHy0=wzVSgd{8%v{0&qs z=~d$KqVwU4$I;xmKjv>La&;L~I^tJ@mGXW^CeqwQvzx{ML$WeD zRhau|)=SjPNX*jlS)F7*5lRDO-1C^ExT!R=Zu1BK17Z|n2XFX0yNMXHo#meL{Aofr zx~=OB?CQY^xbQO(qVN=Di&Av`?HwIX%|@2<8CFmD5=MGIQJfI-<$9Mn)?g0>mZoT$ z9JIW@2(l!i33AZG>O}%}!2q=Pq8c;6xPc4DIn^4<2# zBryr1424fvTDaza5M%Ji`kE@(-tw99{v0II#_TFgYO^F!sy1~|Z0$N&ywot7u%PXM z9t_uasEA`aYGb!K%@CYOrF~_$sN*gWcN8Cw-ff48;zaFLlT$5A$4AjsqlK*Mc++r^ z#ZkA?TZnla_(e>JDcMwMc)W6f zagAYuSNM298!=e~n;gj4ValUV(%T5*?wh61lJ_pJridVuhRTa{rW|j(b~$vZiGn(_ zg8qtoye&g2>_uUV)w<^k40?hAnG2R_cndo{ zM?^4f6t(C%Z(k(c_DAO_YB6<6EEicu|F<|X&G{Ht%@mC)Fvev&y_HQhjv~>y%lj9*TY(jUNp{_|^X*yYXwA zeZl%ui`Rq9fAELASDTe0%GE8LB3LKS9M<*<%H(OquDDUjw$w`;EoZ74*174Ir_KU< zB6R1&%%II8TiaqWuiQ2y!Qd68?u_#v_;|}ub5-!EeXk!yboT`HhPhkECan7ijH&o| zyLZ?oGLQ(GryCU$KjTxn>>!^$4;%8#AoM2@WLIXOaPa>%f7xLG6s7gO>Vn(AD|U!7 zq?2|E=IVd*uKWf%pTH2&nK4s#iA-Vg%ecu!?%KP*TmR1xXcvZHndTqv7d6*yUlcrIBO}S(6y_dx zA+x*(#N>sM_^2WLpeUwONGR4f!{t_<-LA+thY&oldaQ{Bf2WHOU0oGJu9`b`g}s9f z=t4^#=Zj>B?5ujC9@bS6SE5sjW@g=u;m>T^WxnYB4d(dD(S^#0*msPi{y ze8qFtIgykjQ^>s||Ifa$l*;^pkI!qZxgd)29!g59cDn_9`yiR(Sps~4&plSXBvQnT zn&xx|#ztoEtx_hOi*NthY8=#~1CPGUF;-vEgImHxme~Fw30b=hEW-p<f>^b^*Kd19q;MXQ%rH$$;qMYYM! z?~gGl%Dbmf1MCv{($e$(}?xB-u&GfsKPb4w@B*w^2N_1>dVf64F zbbr+@7@lu>JmV%VP9cvX;uF^IX(*wddwYU*P+oHCkQB-Dz+Fo1KDm#Gb)u7kY zcQfgX$LFvUpVv84SAgvAn9*2}?C$7}G+698&p3r_|l(4QrsA<{N}Ze=0&~=r1Siy4}1-9h9aI zO2b>$==LJPI0Ytzc{%OlkQGl;K-&TuPxM}O5NTNnlAJQIYUc>?~K#db96sKH9|417_tD3-v=pmN0aXgXWh% z-;|D=6oq0uE+j>YAk#)doFp^E$CwTQ_;>M^BbSW}cKQNkOo!R3uxeV^pCN@+bc&{m zTK{6+#3J6%j^<9eR|gDaIOA4F%X)}hxte8`PuT$yJxAP*DR(DYq%j||L=(;QyX7C4 z!i=K(oRboIGy{%X1U1H|l1j#jLQO`NEB?REGrNz5;s$VPGsRsK7(-)KgLxdLa`5F} zPOozv+?Fs1cYwEvB3HJZN>O9!E9$Qy2j~w6;Hh%`Ay!CvY5h+W`0Wd5MLja#NONVF zh+xgmQX5oF*fFUpv(K#1lTARU&D*ZbmMceybhgM5hKFmUdqXKS?ejY1y$?mMfRvMu zXFnb(615ow-$ACZ=hK-NcTo#?kB={)bQ&YVna^tTSsYA?R@o!zM9p?ZVQD%L$OG$> zu}T3JYx5limjB+=CKQG83rZaoi2~8hrhG;-@oO$qb7IKC56e1Cq$=0jwth!`L-(}+ zeREY-&uu1%GEAKhGoRco=eTVBM4s6NPgP}h_4)og@|I(FWAD>wZKjoX=btX>B}>LE zgF#iP`~O(**iWFJ2b~WyU(XQfVwbBOW-2Oe{yew(Q{W_Y#X1jUg=dLq({?1e4Sk;U z-%96IGQr_P)ys25hB(a0j_S(uDhh>HuXT%|&6oYrF8LYyP76d~n2_mgJg_I(NI*4q z`yZD})4*|tC-Gt4vc)3D9FI&}wdLiXn2}kWLYoh4fD3qqU-I`NN?Sd`jD+%({lE8Q+nztk8g`~_^l*88bV z@mNtyCng+C34*Hcp($`+p^CB0#`+W(`r$TeiZn?oGJ8LVlLhpjpt`DU- zp%UP`2o;HLZ?nj@I`ya+1>M=YrcYJap*rlq`{LbOL|*p9M6z(oOkVtA)!(+I);0v# zDojQ9pG2hA+a&Y%vpp)pkXuenBCj<*9NjOF;td1mpwy zrI;tAMqt|QYo4)}mwLj9l1rxn;0ZNrFg0KPEJBPki`l^?J}HI>$Yb-Zy&^*v`QyynVinG~u69Tns?u$T4<9>Emo-A|7Z!>jSq{Peo+a=wRA;|%&cF*vn{YR5ac=b z_J<-?zqS?dJ$Y~Jq!R&moVBT9&|@2Lxe*E z7SWi3&vdM*F0wHV6CJ0s^MP1fuKO+h4XhVnRcFrBswQU%8`9uRm~m7tV4Nz_Dx*1= zkMQwqu1{@|84!i;z|alTiqf4_q+W#?8@{Igbu>)Lef@eJ5heDo;*mqe#ANRYkA12^ zJ30){Gl5U)iC8v5#SNV}VXunZLsyUJN84~NSs8MRn}}r7$uus2F;&du**m{Ki%KaE zE{_$r8i-8lXXS4AdAqMm4~Ir&5wUofecezbXq!m_U6yJ&d=-Rq9me&0xs;6+zgSBb zrCq|uCr>Un7NsapKqwmGIIkutL%{S!392yX9MV+eYPDD3de$8`h;W|+#}BNxYA)h* z+hkO@((ku!z4>)CCNlft*i2ET?uRX$viF60#D(CZ9qnpF=k5FdrmfU;wwAZ))DEI< zVo_ZcIbQpa54-L*v-oIYeK@?w)7#1Lk zF&P)LAk5sfgtI@O_%yx3cf$+{wp&GWwr!)rRz-$veu0)00dc%18UL9m ziUOdGDgA)T2di%eh7Q5-o+Q1mi1MV?X-V4g@~L-BV!ZVDDH-G5^f?u2GPWS{Se~#S z`&`7rnX*A%O08i=8*0WEz>UJQ4!f}Uqr Qo^joCIeZs^2<7nq0M|&~&j0`b diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd deleted file mode 100644 index 681fe00f92108..0000000000000 --- a/sbt/sbt.cmd +++ /dev/null @@ -1,25 +0,0 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -set EXTRA_ARGS= -if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java - -set SPARK_HOME=%~dp0.. - -java -Xmx1200M -XX:MaxPermSize=200m -XX:ReservedCodeCacheSize=256m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" diff --git a/spark-class b/spark-class index 1858ea62476d9..254ddee04ae7b 100755 --- a/spark-class +++ b/spark-class @@ -104,7 +104,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 + echo "You need to build Spark with 'sbt assembly' before running this program." >&2 exit 1 fi if [ "$num_jars" -gt "1" ]; then From 436f3d28560bed9f428efce6f7c4caf44111c60e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 16:08:35 +0530 Subject: [PATCH 1119/1571] ignoring tests for now, contrary to what I assumed these tests make sense given what they are testing. --- .../test/resources/uncommons-maths-1.2.2.jar | Bin 49019 -> 0 bytes .../org/apache/spark/FileServerSuite.scala | 6 +++--- 2 files changed, 3 insertions(+), 3 deletions(-) delete mode 100644 core/src/test/resources/uncommons-maths-1.2.2.jar diff --git a/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar deleted file mode 100644 index e126001c1c270aa1b149970d07c53dbe0d13514e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 49019 zcmb@ub98Q9vM(I(*tTsuJGO1xHg@dn*tTu$*tTuk$(Q$ZpYz@Bd%7>iea5Q)YCfxK z&Kj$Jr70%~3<3!N01f~^9T~z4@IU+@06+kwg_Zbe#AQV2K1Trn+;PW@=eI!^k1yf{4(Mq!iq|?(js@#<6}~iG_*4?k~EZ)<5Tqt^z%&H2M*(ci1yTC zQj=oJ0O5cnH20Bitr-ytNRkSQPMMVarxW+kyHQe3@zCSWQBl%PFfd+GQBmW_%-B$n z4hksTcX1EQ?yX)(Om|M1j))NvmB@Zze&}t0VI&y%xO8a$B`3gt$oW5L^sNv8fUScm z-G6HGKXbtTox|D2(AL`8*2eMQ@cyOsU;p<1^IGdWnf<#JSpQ;$fw_&ogZsaE72&^F z_$}mLYn^Nz%nkoQn7SIQbzY_T45bwrJ?CmQxHR&UWRIrLm5~U zFpH3uE}ez+UuQjuqa|(EucrXLf_(jvfdKlkP2Y!xZp(y7m+#{!@&R$`_IA$xs_iv7 z!kPei8q#re-hOr9zIoSWdiS84?FFkxrL$Or%EUh&eBBa^c@CxTi=OEw& z(?2tRpA`~wDHAyk)i$!Q?=~^s}H6vSFtGxc_-3b-%~n!Yp)9 zd&Ibk3Gz}ZlO8qlSDfhq7-MuMMQfG0Cvg74MS?uK0(@3&UDHmYmSWHSmmNmc@;)>d z*Y5@%Cd)QK6y?q9!yhi{DFB3=VJ5ONdil3J-^sdH$0LdnP|5Tn0yu)FD%g+386N{`jsByTN`qI33G8O4XsG z`m*}2i_WUJJi$b;G_{zq6qRbj|LAR$n^|f#sZF>W zW;rqnT3J?`Lgz1xBCJHV0S+ERL&%r2%N)3gCm|B`aBjdl;~0*LY<4zh60^yQXVUJQ zobLWL!?QH%=Be(k*g^sk@PRO_+4OrbPfkWg2GS}qM`|U>S-LO(tfXQyrBM$BM?BA< z#KRi~X1TauGUm(s&yb)H7^=}>wDh4`N4)x_IQvOCmGp{FQ4it3Mc8dlR~OB=;$ zp=Y$upcBrkRevUb3j*>EsC>8{U@>yT<|#OlTEBPiwG)WxH%)D;gQlCVXnSnlA7*lR zPclTbUATGFDv`5S6|WiXt3wsh!Zk1B^*bU!@3C9<^c~#i=x#anaCWR%Hql?=SbW(1LPnkJ;lZh70 z;kF4FW7ILDLg2E&|MOB4Yh$swOYSyq2$V%t-j+SKfG>VKx27Q14VVVjn>5yOBb( zW6d;XNn-u-gYL*cVawLZo%8(;e8UDy@N@9N-cm%LGnO0172$VB9hwuGWLSsF7~R^p zb8b0AjAg+%>tSkKvDb#J_a%F6rGnPnzJMAbZKn^--bK>><{ey`B90w`-ly-H;AP3V zU{y->ETe*aQxDRtV*?i67;x?H73uVp_2C;S{kVhdvW0QJ{X6<0f<&S zeXy^3WU9eY^|u!|(4Ir1qrUBg=VC11k5wU9OpPyCb7t$c`Fo+#tBx(kf{3pn8Fq+Ml??eU=-v$Mw}0iGRivTNId3wRtGh00rQcB|DJs-Zd61%vWiY8k1zNKg z3Vm?Ikb`~}qmP#;JwnY3G)s#fqXyZ|Ttky8l+e(mqJXwvImVT#Kjx{gAa`M~okyugsO^_fH$+#;y_OZVmgo)f-AQuc#xm49MWV<z;EqL7k(YDv$_$m~_4kW)HtTa-EDGhW8!o1`K37QUQb@K$2p1cE zY17#Tyqf2c4;%8+-bIm*Gr5(Rv0;*NA1zV}-=&an3alwz67Va6c)R`X|B{sMKdVQ$ z&tU@s4gwEXgZ>k#2O(v*HCWnUolepxqax90Ja=QV7=9RT+8XT-5>cvJsOa6Q@^D{Q zYjJv5c5)njuD78*8)2MpfXq{`fP+XdhM>ld_NN?SSxtrBsm|P-gNj8qIkBvrDy5Bo z;;{l{4H4rT^Zki0jTmwbwiX~n?#JNv>UNji_{ELs9>iEldPtb+!nO&LP6m>aua!XB}0SC2+jMMoG*&6!iwBwb?MWhXf&1&b=amJj=?dWZ^9-WId3uQli*&^h^ zA#?GZX4_V6X+gAUQhz^~YNW(MLVeu=x`T^b>gBFBq)vwCbkv4iOZbLPl@;zO|K+lU zSH2_Lpz==HpzY5Wl{8)<3LBm2KOx@lfU#`|xPv}b1l<0g!2x(?dq-(qK}lX-+B91~ zElGV0?7ZuodzTK*I>SYOw0jvMo&IsXb}s$J*6;|mx0IVzb#-+~`Vwe&`^6>= z?^&?N59XVYi->+`8~4gALQ*2$wUjyrC^ z$Pr24VTkkWQV=_cc280fcXI|_LX~asMgww(lZyrZMU1!79FV%$#{+z(I16YzBJ`1Z- z`_krX8N@VQgolRzva+kNp8d;<)VBBv4*_?OV3gk#Q_eGY^xzV(ba0bo@FMWz#kIDN zOI&)^&ll2F8F7Wrbs-5nzVV-7vj(Hpb)wdHuEfdrq*{^IFlmYyx7G49VVO%gr*q1JisLFAp?0Ke#m}yN3L8TW@0-0H zFmgX1C6lqn8=(&S=gmYpn8-MDajy7ET*|GJtME4~2hCa><**RudbCe+JYaVCt<}yA zGmq_JbldOFXq*mza4@q?H}>3+dtyKTz9p)AX3*C%*z*?1$*#Mjt1sj=l&-EpsU{4V zitS6D5FwT$Wgsg5ZIn-X?8v9?n>M9kTvk7?)a26U9@0TO1qdqLUU{h5xTCdM9JPO2 z#+o+v)>x~>n-g_(%YRX;rE_PAlmK{PKbrFhVl2ibhOrozu-1TVER>$NUL~#AWX_au zv0A(SbR7mIu`=JzlQ3kwIrR?nkM0GkYP~+-+qKXl0RT|`uiT51qLYKUjp^UxA5}{` zr2`C~u89@NMHqDfR`|MKiRmLYVu8DaOQ?jy7WiVnBjk37J>x2DCI>g_*DWXpO72bQ zbSM|{3YE(Y%2kk1peE<-S7pAOS9LF9K993MYv;_1GtFtdH?K0RSYtV*OuElJ-8au% zZ_mEEJTd-2YC#xs$bjA)z{`Z}^a(bK=}85L^kL&d?8#?9>`^_JLF>lXSO2()q94xfTMK4Zc=Fcf9_c;lmbMY)mT7G&y?e$;eG)H3b=b^~0Vv z^2kW$$Bh>}ANY_NeW~aZ~=71)Cz|w5!@Q%g(tf+ zT3wQWx9nt-OM>WY<*rRV-&A~XR`mLaNR-4iT~DZ8qI+M*&ktaIYUuijyB6xXPf zNxRZ=KGXDpJ?(TynQX=J6~Qy)v{go)r&Q7Ls=D|w*9*oAD}AHP5@n7cyA|5x3hEPz zS#$=;C#rem3xb&s2+|-qK8P#Q0D+1LlHv46x#VK0g4dYP7nILa%b-<)v6CBag}FuA zu5F0M_O6zYa}F*`oV`c0Trrwoz1w;tyUscLGcRk5PoY_u-W9`$*yBec58Ct8i4}Pu z_{kG34urr#br<*MCTS%?4eHdiJN?skw+HOcEQ!6;JX_Yuq$h$hJyD^(9ZfDv#5c@5 z+zGWfmcx{@6>YQfjt$l8y0b2n1*=tb>=kw@XRc28sQaGN&5vn5;zZr;Ed0Oh$tMbt?Xf&J#5ymzarETilonMu%q=)vLsyhmK zoguxmq@&^wWn;{8y1CiOYts4b+gmfnOdEaQfnWTkfmg}*IO`Y5dNfr?>NSPV$=SUR zW@ZFtYU?zkVbtNleT z-^*}ZjWf6RqE?v|&)VtcYGKzj9Tsq#DpwpMcY&+yDF+EFGBUn&uoxY(p@fVQC0!_i zQ1oxiibW08|W=lrvXPROQpydG{}uYL<)iM^`$d*p!9&BgS+gh1Pv&Ii5Wj6 zWp4INR3urIj{3hwdJBcq<(9kAVCk{X95(%2DR~6H1S{{l(Fh_YjS{5~6+~G7#Rb|c z1Om6sA4IYg&ti2@(2*6HdWbWl(=hR#WKjF_Y90!!kOWM6kV&$;JId9Kdmsmb$PF#o z-;K_n(FeB&LJPHIKW1_qHHd?WzSVpEfxl{c!L+vf$dPQ_uxWrNUZE_FZY{$a$B}Wi zFpo)a%qvmN=62PL?dkBi#JK(!ZMRDLXM#dh{Fh=?dKG~TTdkYsQ}NBhXvC8BZKeEi zNm8w6iyAv+lB^($^Jv*V)J&$SX)#eTN5(KHzj1k1x0b($zr?wp)m97F7OOo~tx-`mglp2z z4sM6x|76*#?cWbE`4V2ok5wJQ{xE33aKAtw`uWXeL6fms`+VP4-xx;RAQOJ678WQ zG{m6k&vcltJ6L?K-H#9Of&Bv=63QMr7nnoj-A9ARud}8x6Ji&_u)RqUsG-_mj@PQN zoA~FNSz<)DT z)0Jg<%jD>@BJUN+#y_~CQtZ!_u0WZUF1(A&DyU6z*@_<97)0nnN_tSD>{wyJ~H! z#hVCeOc~mXjvq_C8GhEAh1xULkB}Fh1K0;mc>dE^Kf6%S`@)=-# z0VI0Ff-xmRVPT5)baMZpPV1>oi&*S}8t_~cyMUp-E;XrTs1%GI$2dPy7C64vsw0jD! zN66CA^7wwYCEYEmY@IahG}Wf`5KAd10X{W67;&bgX$9S5FtIvGc+%oR_4^llnBLp4 z{P0-_EeX#RX3;NAi=btJvs7kkDtTEw45)ex_tvC(w0;=XP!U|^<Pir)rq3r|4f){!>Cr z>tm;qi%>?~Esxq2BP1)q%^8igHALtWU!Zf=8c6bnl_aBwL9l+$yIN&dJ$|rP$3ThMO;RGx2J4D$IFca4I-% z(AlKu{04Qsq$P#0>te6=!*_~^RIe4fBWj-vkNX3okTLQj6TIxAe^Jt^eY(s=wMJf* zxuJcxag$()Of(%OvdX}45FzMHqL#+bbnr5Lul+9JA*t=zNp4jyxUr)+cm z_~eF(Q};J1+6M7g{oA%jG-w-zN zn`r($l(o}0wA42>rZKlMvHcsG<$ADP;spZ(69jW{0aI}SV-*3b+J2wS+gEBH&rlKp z>+$d3KkvU++8FO>@anASuM-jhdw9=t`k3w5e_vmJ?=bjS?~jW#_$V0f=+BezHadue z2EvDVdHobuFC8I~f`*QS{s{z)j*;?<7zr5azsf{^OY~TrZT|@b0sz1d&`d$B!uMs6vLRM$%%~xXJozVIdyVzDfiL~bN;Z5DABbS_qN(C6^TM1+%j5|afPD5t|wMi4rz)` z2{GcD_Q3e`lvyX5WhLGk?YjLeuvW`d8pPz+vlg{c; zob!OUx||3XkixlH1BypfqUS?fq= ztlBGb71>DKXlRUK6(q*0xC7C^Z^}U?s8X0i(B#T$n4JgujX`(cKnaHI!1=-ApyT#I zLw6%|IjZXL@bvY^uX{eNdrrPwa%^#^c7I+?p#oYBMd7d7t;WEy;iCxQ%<`)ZKi9JO z*^G}|$KuxwgX$B}^a8mWPf@LJX_+xjY?_bY|ctPVm?b)Yh!;g>|UB&O7 zEAngg76&{-^HO1DTmCsR{`AMwg*L$HBF59T>5qR7hn9W0Spf5*36&OmyLrHG+D`ZJ z8jG>DG^1A#S*^xmOV4Q?@F*Cc@rs?#jR!SO+D>E~mbAZbPUT@5x< zhp$O`rgbLVORE0+q52r=v3Cv=B*|E;6$Lz!&mF5c9C!Zq-MDDhJcYR%Jyc0=uqh)1 zW2|w(=w=cWm2vr0NQdK*u`CsCrGVMtA?TFtLocm`?dRsbe8+jngL{#x@o1kMCQwI3q{~?vBaP1 zJZy0YRX1-4FqKN^)rZpU#u@dJLT`?5_39to5a<_GBfEof6>%=yg(=vT!tKV?#E}^n z8sKHis1*&$xfraD9j=lM?ZiTaXJ|#LkjX%81RCyb!ELHjCT#YEBkT*Q*eXid7Mgxn?S%~gKyISm=MyUgw=M+Vj) zaS5RJGVu8z_e#2@j+aK7f>&=BY`1guyG9M^4p{nh;P%EC`w#`P-NeJoY!?J~hJ-)B z&lJ95gU{`*gzn!r)FQ+4sDdTB6_Na*if~t0pc+Bk;)B`I&$!*Igs<9B1HV(#Yc3rf zOqp0tD%kD0QGZkuU_cMPU2dsXuqEC-Tn#F3Q|d>>iC`PWiuID2WZZ3D@?*1J0)WXS zi0}`f;7Mk3aHu`QblJ!fevGrolWeBi`AW$oIrUj^+bfM6_*g8gqp~&L(ZClmB0e@} zQoj`o$`{-i<8i8N+&}+u;x)>8q;NjiIkVa}mR2E~Vk+f+`C)pmw27XH05X)sgS(A# zm~wp+nV~s|wsi^67n=beAz4C&OT(%}A^mWA-LW-^ycrMH#YY)XV5v@@D5vuVi4Y;7 z6dHk1W92@%au*R&HyHVRPPJ^?5Lqv%GP(G&_OtKMEHNZE4dY&90oc>jy;w1f&FSz@ zJ{cm)y_(1RbIGHq?bDI`6UgFSu`$-l1~8pQfxCTY6kovE^qB1 z^#?=0=5>h)?}q){NB+n3#(=6u9k!wnrF~eg*gg!6 zd%MfyumPJK|94 z&6r|w#m)zHgrk?OTOvLLlGP2~@@Dn|*PN-OdGq zbD&iF!f;e)+$a?!UAudpq4L2i2u~$wJBJYtg9C@X0Sq~vkk1p6FJfgDgudN zd)XT191mn?4;brSK?md*=TuKDKA|jbLCfx_#hs^>LD}+*axw3?NpqvOVxxCu1TWG= z3+v(HzSdz`r8lw2YI{YiEA`&|;2-QfqO7fb-Sf@ZQ|LhgxaNjV|ipV0gEg1 zt9S$UI$PkDj&tEQa^GjpXzUHO=EWKTVkz4PbuS!Xv)}I&wv%#DBH=j01NWEYYGrxz zrlM!eqerm_SEnMaX>T}9w9b-}sew}M;YOCY1{|hwr|}Oaz;{5vj)+9JnD-B+rfoW_ zKHmf7&q<-L0QVO%8ON-?Zacs)7Yb+p;=OPM&E@SR&gp?NYF#HuhMuYeqbK?YV>&w| zP14jIGf?}6LBw5s6OH^RNsob1%YLJd9`-Hsa{92v-|+O=DNVi zN}6Z~MvtzEtG{^W0&iE>aVp<8PZ-4LFtbo>GNxTU?=HMx;ZoBVLn26+{3W*9X%;(i zAQ|!)YaKLOsY$~2yTMg&suGdme&ob#DR|-IBatQ1A)BoqA;vGEO|vq}d97Td{Xp=N zeDTc25X)Bb8`gKuhur2}g-yFY7nL*8yO_J3DwnMUAI%GIdn{PfTIS{Ll>Rbxv-$JK z890C*hFX9957fGG%1iZWCtBb{(KX38tVj6^*2CJC{RQju)e*Sqc?c%lx-m%pR83L6 z*8Spw>{z!vz+lWw&AW9f4%aVbLxVPd%3yHFt)Ovv&$NP4e$jsjYWNlYs|R&C1KH2m=f2l3k1VUVF0@~ z$u5YtcZ+D#TSBVJdsscY58(sBnlI#q>krRM${W-0zfe8(poL-(QUOfwk z!}aV8!&$Q&s>P9&uv^-kw^2Opuiaa05x1+^ljDMradYa(s$g5mbT&0fFZk&;pP;b$ z`<9%fk}yJvQcQ`^MbJN@pAQ~IV3$k)Nd6{3<=rgyXUSo^>tOwq1bN)kCfCt7w0E8S zm`uH~-TWBf^n1G&0MWcIf$T)RLjd_5sRtMwzTvzaRBs?AToc-$mKkBkt{VegM=8Z{ z6lf%VjEm$_0dB>Puz%Acr&Tf_%%H1qBLrgooZ$BC#KRjfX7F4QKDDrWhu}n5x(kKi zv=>2{rEo(6+9)zOj9FwzWyZ_oF5|0(H+*bjq37|L5$hC;GCxtESai8hBglU|eoiec zEg76R!POQ&nTm8Mz-fN%v^^f^pJHtE(2&ZiI0U6xq7+TJe7KLzqexkev14MY#BbRV zezzI##F3?$a}+$%V2(XzdIV#sl079i%bY5Cni}7hh&q);zfWCc?($_cDFL%TVMLo= znEk8bl6n=`D@~RuAtO;{6XV0$j36VKz^k_O9JRwO^)Xx@ z=Ca%gUKzDiM056}Bt!49E)H8|rsWjY96V0DU1Fo( zQ-c%hZceaEy3~kR{*x)S3ByydqnTV%U~nQTF;6YkjA_Ig<%o^NbzrPg=>~b0#*(c# zQr`ixfa9Ek??`*uvt=dKLEIRd3TC>R{{Y+fNEicsg@l&ole|irixy2d69p^V&TDf^ zQJFyjCF~C)6QgI&t&RwlurQ)|;=ui;3K3IlMVZ`{&5z$$H1fQ9v5Ym z&`m{3H`c(@^#}Q&y-?4u5UBeFCwXdKA=8;@D;}zvGzwGsIqS+24J*P-lpZ!T2l5DM zVuj~Veq>RG%H39SpU@rhtf3fWCsiYh;KRr~W}IwoX6gVr=!x+}_WAd$&1&;co@YB_ zLj-ZDLn)C2wuTOx$2zDj^}coQY#{PbnH1zmQ>QgnD}-fb&;VDH*g{=#ftif(7LGkN zp^L{+BT`d^tvYqY!p!1M%uou3TVr<>Sx9`3U}5y^-!u&xhDD2{G{#>>cq*A0OZ1j2 zcE;l!+lWsJVplN6gtKzkBgV<0D^C4T5t(H*BaD-f*}z#J9&;`9`m514q^;FtIy?G# z7UP7dCMK5V8>31XoOf9BXsDxzZ7ZEG9_Z0qCvPoJ#C%3BHvu67(eP&Xo7h|bO!9eV0l8f-&?M5JAjcpI+te7 zo$^g+zp2n3K1kZ&p4)Kw@_rCs^20s3!?|ke651olgLe0$0(FK}0#@Ya0#Fk+OV2(l zA^5T4{lv;VrvjWl1cq<>;PoT0LgI;@0{R0A_or%GOmu5XzTF2V%HZ-iMMov!49oWM zCJe%o`lq>Y${xhw5wh(ONRweS2j@i}q#d^YCbIq_uv@pQVU^Gyg_MOKJn)9xL{9pz z7Vo83B_fdVvo)UVTKcr->7FHJtzxYlcVEnG!Ehdjyi%;-+5taRT@NVdr=0`+`6!sY zYKQFvgLc5SuGw)PX}6L_fZEK}vPzDPmsHGNb>{EMffizCFQFE}j8 zehV5?pFgg*wE;oFN#cugGI!Qh_X56>jQQz>&hKVD_3 z!~Q0+C5D7|6*zE9l+zW08gt-M8ScVE8iag#i3_Z>zk+oVj|2ZL$Qu!Iom&SG?1c)Y zP2dMn6@6>54xoS=k#3u7EfZW5e8K2>aj??fBU7)D&}lODf|{B;f_OBM<*3Sbz{N+J zDeF1uEB(fj@f0Ri8}3RDf0DXd&x?r-Js_aHG~GysdWr{42GXGBb|viuVrRk8zy8rk z?#fk=x)o>BO|-nEPT5`j?%Wi~C-x_Q{;F3cwV57cAz;r+3UGyyPeA?mg+w2bRuBj_ zU-?@~>+GM$7j=|20X?So37bc-1IA8pp=Rw}J~Fi#j-~m=;LK`>Sn_JBsXcY^sO#R) zbF2>sXr9$>|2AXez1Km$0?r1znw-moQGi6-)|h+;2R5m(Qp;5?R`k5u9y@AesOq20 z`KxNn8+cG-4EtGnS;fhMSY!I64>8`3foKd8tBdA?u#X%ISDkk^qU^zP??!T&jXFCD=J+e;4*x)mHlLdg!UZV#aWDN|D$X)o3hdtt{83Hv7(J~*60 zB+WqQY6$5M;Gmvy4XM>6C)ruFzFpu3@gwQxghYz>dz1OLSsU$FbnGZbb0!$L#KPuvQV3wcp4fj{S6Zo9}LXp3ZJMbG> zIDgL}7+4w0{Kx#mf2HsLj)da1Ws&%igSSq8tR95 zqthlE14mp`*tp7n{rc+frxM2pk>20LGv=;eTU-P}q?e6{pk?Yq2 z`WO~Vk-!g>!iHE!IJDtIHhPN{)JgGAk&#S^&5>vsf-hYe)C|@#7TsDfd$dFXIfo-To^H^;(S!-DG2qgq6oci` zX>%8>v9xg!Jo<5Q8b5(ky%YY><31&A!`WsD^=IZ%kgwAchub$73GF!A*j_RS+df^b z&uPgYXrT3__5hA*C4Lk|xy~nc5v?r7bMFB5WMMGcyv0-?%Rfo2k!OfLTYWcauXxi) zWIrxeAU4;{gzNX!@%qIn6h<;Vz9GCI;^ml|-*<_D0M*#6ms=<-*C`f}3Hn15>~#5V z=@+Ivg0BdNZOmNu+x3!c^XK{qS9*-WE}vwHAU7^|Q;Oh7V6b?o>KOi9x^y_utRid} zW}M5)Q#^Qlb}#)4Q9(zsd+1>7I)U2d4}r4~a6J6DF+BV>(O>i;Bcuof>163edyoy@ zn5tqJFsyr&Re)V+#8_wH+O|V+*1xg#H>?c33K^vGlh#D{!M&kHe;*fvhr?0~UW4|H zZAk36O-&^T1Y3bAexlpOn}PB0%aA3%7|ZWB;-Y*kS&|5cU)?s5F%K0lAEoir8Uf!^`5lj$kL zF~r@|*B#W(FN)YejS+sp0f-8sg0yCk-ln~-xJ3ziMIwF6uDs89j`OJG&FI8sx7mMT_NxQ)7B{v)y{ge)NyLYy&GMTrg*4E z*S@ppKyvNEWh8dtplqSuXxb+64RWz>Evl(z;l?()w+ex8r0mSIiQV~LtQBGHO@kNw7qY)fhgBh300fR^r4AfpJo zikr~B4?YH5xaTh=%8u#eG~6$05E9Vg~=kN>O>UhW!q}H>#$`@En)?bUD%5 zdNp+isuxrm@~1EG588Wfbp>;V9hST15`-#Ngx0MTy>)`{I(@j|S z$JC`yg`>i?k|uMDSiVBn)L2m-5uBFYK!eg4>Cc+ql=&hBb!7!5<^%VYMR~&Jbo}&Z z$&ujTU}nMWAcE{F?J6+zb!*FhSM#5B%WY#ZqX(&7d~mBXb|=C7rA9x`A4pTHkKUK@*_c%x9EQu7?0w%To5pIn#N zzMijiKOwiFTFylAs6k6Lv~Kdj%=s8=yv>A?M%uE6mHDc6IPkKD*xVldvrXu3&(pcz zP~&Zx+{55}MSI4$eUihpUb672x0OMcK9=DkMl7Xl=wG_`L^?I8QmjLx-a-Y zk4!E@Kqm**V<5F)9I>Q)#`j6%v&o-J{ar!UlL_w}f#75{Pu$ooMJkPE8JTtpOsY+K zO;}qiiu>-;tksH=(2?aPGx1V&BgdiKP{~M-hQ$8#58t==`QG%d8@D##%Qdqz;Du?q zOwRgS+OQIC6S;e6)K@n6r64XN%}G<4r@WwC{np2Pq!)*kl766$-Iu2-g5qYjPI;{m zsw<%(Gb^-T4Yvp*p|PgFqHeoTCK62Bok3>+r0A)yBB^UwLQT<`TVyd;ey%{ko68r8 zJKtfVS!f1P;<6@=_pqX%%imouFh8^BnK;^Q!^#J2kVfSm#VeCDNMvrTx-s znmki7VU&E)fZAxDHco0ql;{?Xjlr}C^Qcv+zY6Oiz#>UkmJa=uPjD_7iGj2(mg&`Q)kj|jWJ!+)^fJKn#1Y{{vL=P>=$bvni zNORFD;Rj@0c<`hHYtuOBORYsN>9KQ99a+uHG>kJTb&NjN#fLrW$0YykQa3a{ReQ`2 z$}BxbKYq||xf{uMiJofZcd$K5pNL}Rci5{^H$1$OXF{K9gQO7-sgjibz@JaFR?0Ud zyxpO9ke@lbg75NrU%-fh1EjnLx3%HbJ9YlDO6!9-C}YnF;t9FYCE$C364qJ6`BJ!s zx9supQZm2%)papwN0%&a%7!7xA~;^H7b4U3C0uvSwd)@uc_>|nO8l8JbnDP$P?VIJ zDJa{Z!(?fSSOO4fe8PZ)>NKE?8#;?wRL}GiG9zNuXA^U)MuW2JsvM>=&fVq4*3wxH zRs?LlmICIVT+I-F0OxaA zFKy^P-LH;38QKIrY?3XTLrMP{ZOjsv$QOB4!ynrb#4<(Y>9Nkxa`2kRz;hfN%DgFH z%8D~2T8hKY{%wdflAlJ%DYSle{WG_gtp#-EV9O27^D6hyAm0+|a*C^sH{hs&Yz%-lNeFJNUA9l!`Dh3z%^vqhO`wg|Pw*+d6qkSo$=jO(p+2TTqWwsO z)?*Q1Wjj6seIp{+I7h*H7C<3uXOc-I-?A48-)+sen2&8zR6aw{_Z5YICkw{%G*d$Al@#947Vz3y= zu(@Cd%kV{W=d7~Vve!OJfxC$jbYjjHpMLo=c0?5p>ZKj;5({IBZlkU5i0Wg!(bvYe zI(>sQ4V!{{!Xg4*@q<{%1#89;Fkn~L!MP}x7+yss9WsNi0mxfIXf}zUObSkFf!Hnr zV2&P7cK4Y($6P%x@auc+C5>0mrbXbUi@)Y{V0%ich_0@t#*CWLjK80$OY-(y{Pxnc zbKg(A6-T8i_Z6~?gljNIv#u_6f+JmHjvboA{#FBl#I4OI6JwE&_R|Y@3j}4G?g#sU zi!Y^4>FNf@H{=EItJ3pWtn(-)tI!`XQB2hBY}xu5TsmoNGg|h(0L|t5buP4y@^CR`frM!@gc{S78unrh zlIWT4$^&-L3bq|$KVjv6bndh7jSc+KA~nbEvRNGddYwem?c!D1ve!$zxz?@X=5opb z;Ww#v199F(ZxhO)hHArUbY2dr-Y#c^ItCt;cS(HzQ{4Mh^RJ8MHGE zbOlG=W>Pg!r@q!g7md@pFq!Pf#Bp9+SA`AGFJj z3$=t*e+zp5P5%m#{{4;seqlvHTW1?5V+REp(f@3ai;9$n{7xAJZJ9M{fJTtn?FeY0 zdw?LIpiwR(^|RiGxB!Wt?P$3Hz>$b(wK@j7$G`<&y|nQG2%*1SdxHexFII}rSCt^^ zw`~q$Z8+qQA2jGD-jYJZ(~u=Lftrdg>~lUa5K=de2#tkRSx%_T*Ns%t$0=$@;Eiw- z`qA^!Q`MIiG>#(n#%L(TG1+$|QJRBC_(tuFzi>XwEYV=QAGbbEyK*?8wcx$VvAg|~ zv1t@*dBJ?A8X&*#lVJH5r}f_||GgEG#_oT^rilvDwsSw>y)DUH3QnB1bPk;{1zYE#hc__biqvLD0|hI$ zuX5~Hw{RWqOkWh$;=!7ii78%?lL~-E+-koL{e(UiacNv)-l6yVl+8o!7Vy@@%<0^W z%bl#T$0XOz$C5UXZSX!=)IEYL@JF1?Ar`C*B%}t*G$Fra(@9y@vy(kR5qxQ8n0gKA zC8ASNqC(gVd{1w)7+2Q<(;iA9U5*f_5w0F5E{AHMR6yhD<7O^Ro5zvOgeAN-e3-@{RgPjjJnV(f_;tv-y%>bs%1^T%e><54D zk_i&oER2@otg#CYTS6=wibK>m7)_MSr9P;tU8+k0RQEr{dp6rZ*FxVz#|+TllVSe{ z_Vu4Z&);swV3lqs*^tm62N%Yw(Zke8Quui(v4%`osHaE^irh3ZODpXw)8_O2DA^N0bSfY5F2?m7aTTUm%BwfjOfnIMQNZGobqF$eD@~J4O`Vd4^aeU0X-Bypd!3+cfF#2 z1RRWkm^CR}5rMk5hJ8es-oJw0Frt#^r|7_A1Ei{O7orXb47rdUnCZY61IixdWT~6> zfIFyMPNCJB6PJSsuk9g<(z#U8Q_3DBsw-)`3-0kHSC$MZS4^>@Xxz}++gqGDsDuQ+ zbBN&)Pv0XfW~;xt$k6{s*gHm9)-3D7-DTUhZC96Vo2zWQx{NM#xvI;yZQHhOTVL&W zzxSN8_xy&cAvU@ z+B5@9#~C%(O_mcIg#$Q@emDHM{u5KC@W7PWu9C7>0Smf zEd(DnX@Rz%cpXn8)9`!g^kb9Y$e|j{EO6F(uOU;?MB`hdAD(0#=L1B0uE}EJiU6kC zj)4ZJZU~Sf zw4qCh|MWUiEbGA5N#A!0@k5a$7$K2u<)QoR+xEeg^fg#x_JObG>E;# z&&d=)$#V2;1U4UdWe6hSdGWyWMR69YPFd+Ok_g(kE^1y z6pWd#IHoXhs*Esm4w9V+=Ezc?VRHaXcyzTGdO1|5Sdf=Kr7u(`k~k-a22>@eXbzb8 zsjlaUX9EFrsB>krfMwCBS)mAe#DzYXG)dW)VC}=QA~<*7dFeo~(4(l+a|h0@mR-A& zZ5U3YmJFOkZ`qLtuQ-P-6%;FlBlQWnBS{||HGzJ`GEwGzG@m^|^#IN3^LR@UrO|rY zCC3->^hgJMB<4^{(o2VA`_7Q~ox9|{%5FFk5M>SYZt%z{daNKOTX-?D-bP-0 z$y<5=rLxiA0EjeSU+T?%(B?yK@fgtmeiZpwZw*9U8?rC0Jx<=scvuda*R3RZ{oSGlS~qRNwVC)lhJwynA9JY3g$ zgZ?D8-4I0MmncN82`zcD!SD0k*f#WoG3tb|CC?b7ju#&C%pYf&?ze-umLVoq9S&qc zGXHmKM~6?=B;D6%mHjWK^8X8F{yX0JDuQ8aZEj%oZ!mMd>ffa>Y9v+wuL3J5*(kt* zsdV|@`?LK0;?aswit*L`Gz9iO`b-Kqd={e=V7t-C3lc_LQTrQ7>sQ}>LqvR zq)M`kwH}ZU&BhRt5Lns=fL@1TAEc!`1Of^tQ``2G;i%=C{IekkX$yIEov2Vl7H3eE z#ujGQN>P$i`A%)gF-iJ2%H2B!t;u9DZFaP}iBJt1$Z>nV$^v(x{j@PE@)Fh{5wOD* zLcVsIejaO-h*7x6Xr_Jx@e-KB7Mvop8EiUI8K=@JW18_hc1oQUwh4-Rvi(fw^`xin zD@%s=)!mouu{O6rADbhrg8(TDue#?*CRy5{W&dP0>u5#sJh0S05hm73b>n(0;x6Th zgW?$ls&@0)W*v^6x?4GroOyd)Lue=uXy+$7O+y!}sB4Ku%4E>dhCZTYiAeZZ&-yJ_ z%}1q-0PSIWAJ|*%Z7}L^wSKencr*7w$6sfGq@xS96+^v>QiBM|Q95oyVx8KtOA@>9 zK!<~2#L{$FE>7BF8Y?LY@ubDxP%hLhKtT8lI zc`0RFtWxTZH0Ge3Fj7+NC|0Fxfl$(8%mZ<+ams6Z-(xuf)BGn)J%tAgtG?3eih8>? zFsQgVa6nTR3G;KqC!LxchIfjhumb(p>2C>!o_KH_74H!?+~KSyQ>zPm;KD1q+@4Ej zvD0J6aZ}@Dyf3&Ck~PcC^~e=ZQ)NrY6mtVk4^J_lUv|m&*%Q3NUQl8@J-}Y|LgUxy zrdSkDA5n4n2DK60QXFK}q03l~<*Nh4+~mP-V=wKX&3VHFWnELkAO}Vb6T#xlkl;iw zW8tN4O$F$=@X+1$T7uK>K-azpb>t%U?PE-#bJBqaxV47C8NjE%E~bIoNi|?hHyS5R z<7IQ7__Sab%(B;&*12L+%SW_JL0}3=L14coX7LgDV3=!qibD^@>_KSS4*`GVO3@uBt+Q({UC6&jZrwC%4#&F4R`6O)QE6MUv?0MozA?PoUd+(-m8C zPBgijTI8VCI8blVXN~nM>mJM58rFgkLU<^oShL zDRXT>wV0+mgzMrH9>a#1bX$RGv&nK6nNAjC#Q6DM$g`37_}P)hXY|nYbCFS@776Q0 zmG7}OLb8l`(*}6DtHh6eqb#5NNjmdZQ}2|$MAo$UFf@QpnLMuP1PBaTHA3>^&f3NF zDU7W_LXO#_*`jwK6sR=##NS|4=twe4J!lJ$$Y@^g2^%Kv6w#Dhtxbayr(czbO~ykW zwd%IHH+`9=%utAR=|!=Z`9yVaPsWB9>n{jpT4|K~)CV5Oe+YXYz-?yXLCnGO*|Z>9 zm^VlN`xkm%;uK$xubU?5KWa4o@3#RV88smm22QrGi}v4l0hP^vlpy+GTQ@#-k(!-i zf-)lpD$p|jkp%>rXC8+IV=W_Doi!!n00})zXT=7Ci+d*^8qD8~ZtC?QU~2V?37C(A z?@7#WvwgVE9*jD5+0*CC?D8JA>Ro)i?>w4*i|S#G>8`^ukvib~;PWPvZ!PF2B8VpYIvyd#DLc)5wkCbKzR7#`m zF`>#;fM(0S%A06_cOJH^TW!h#nM!9&Mu(gG%TtUJa(}aijjP-05)yVgY6e8w6g(M# z_kG+Z>Fm5d$&#V=k$Ok=h-s=zWhcJnPmDz59|$QZC0va@u9%yuAQVxW2*W^#_KPs}D#ikX z&hScZUE1`@MUnAhIna950KZgPa}EMaiJITW1Q!i!KUm_#$ZeA1J|gpkX*HY$O|=sh z$pEy<^7VG#z+oYv+2w2PzF<&HEi_N$5LO(I_5cz%Em+if7SA;kYlSt$vfw<0(njb6 z@Y#7|kJKql`(E?;-kDCynZeodQeUMYa2FcZ7F>XCg8x{xg|$$fIT9%&iQGr!MFmx` zQcyWFO_1N|JW6(MLd#`!1>r(HDuZ4JDb@C#FV2tU`!s#lbcG!xMsjN8y=h*=z?6A; z7WDTv@?xwwkR*1YvV94MZCqWsSGR;VP+-q1Hqjz-cV-ceTJYEFi-LA+NC~(T$YJPx zV&4QD6(+)B3Cw-0nbcj5a^|CKu(P_%AJ{z6&cbX{x4$%#4$KsR3a6rHIRwWo#BSL> zekz)94M3z&4oV{?m`Aw#1mJa3GXYX43z^Ps#9Wyq&N*Zmc0rueZ-f%wjSD9(xtARu zz;axO8o+w-aNL@MaUYcK=xn>Rg*KP;yfW<{@uxmf794VXfw?nRjZQJ1fUj9$R*d9+ ztM^uvm%v^7fb&ChAu=}45}YoFfEub9>-^#vwG~2>(m_3Bqn@Dbb6-ZtoK5GFqrg6@ z_3hu{3U0cbOik^do2#cIc;y;)lD?r(h8j0c6WBB=IMN!dl4%ywGiZcgxefe&hegOgb zqvwR94XVsHkJ}7FUTe&-_Y66{MNSPyv1fZnM58RTPK>q}e0={`covl@SarBPsPS<{BA!NO!tVQ9utl>v9VL3X(q-Y|Wy>7B)m`uC z+tp2(c1(hd#xhLG!um#sgOclD{(0((hv!Nw;n7cN4Mt52>V2qu*yinPFHZ1eKeN-9 zj0$K?&z1$76Dz&e7E{s#C^Yxbr7x87$jWo>BLvXK7SjFi~d_26}=I!YZ zYQo6$RR^_0%>gN)UfTEEppR=qMm>u9r~c|hXZH*A=P6k{y6xMUeblp@T!!D9-;1@m zlrgAiL*f_Nt@JCYMZs^pLME~Wea@f6^*WZ~PO%SCj?akL+X&TgZg!oRlvcR*L(4jYq#ml&uyCk);P*e+3oeoJiFpQ?*x_dQbXxpJIv zV{y$$SBiTX9B8>W+ZOdTR?r*_4umZ8a3|9{hk%aEu|FYUqVJq|F-qU6PnsRI)c6>Q z9o0KHjV4y}Oe)i!zssx)SI@2TwSxR*(h z*mOjL4-W1_Jw=bWd`6WR)I6~6jej&O$or^&F~}wDSFckIT_W=dt8gD`9|7Jy`2C#uWIr;SX77pbXx20r<{jIA(9=D zQvBI_tA@kT=xt!lW6aN|(42odlf_1@QhMK;-$-fVjNEu^d+c$eHBihnE=uD+bu?QkTb2ve1l= znZ@42YE*(&6`eYOLFpMbd zY@+|XHrDl(KMPaaqqj2YqioUp%HC<0B~+}M09X~D7 zf=m=Lh0{(3Oyf&#nO1d^$)!AxH`AFsE179)7tPTG8 z8-K5bR_@e3DlL_73T=j;E7RqDsC!#AIPR6)D*?5**WyvIr8yP;9zA1`qqW)I#l*2z z^?YfKqt&1Lggap^rP=54AgS?St(7!oB0qL8sUKGAGfp2XV|2%5 z?BBEQQ;yR^oRx+{(ClVlJ#f+)Ps^qqBbL4(wvf3XjhZ`65leXsO*X$_VI19K@0)H3 zV@>NRYjT zdc<8Kz>>iuMtacIO0*`Er+Ou+XF(^Ali3nz-a#BtdO$g552uDI$?%C?Jsza?8G2LR zvZRml=)9(wn!(;mTbpD6ZYb;5b`_pP8!H_7?j%x%zn23K_NTSxmGz zje;ulE9Mf$T&ztiW>m)JS4OpSiO1mL$U|?X`F^BTTBf9vWs=h|xN}ykVyEb6>g$@- zBsC4~p&6zxFt^qZoiVycsaKw+VBbj|^m{PgBI)#=vAU-7|;#?5|$UH;K$@IvBr87tyQr1MZ ziwl`UWG=`>r8gu|>QXr^ZeXf)^m^JP=?&0iHe@1F&d4M+4WmvuZY{={DyNsna6M2Q z3x|_JEoFQ}+w#Kg(;JxdN){O_7xTxk?}F(%ULwi(T|371R1al5YIgUz2CJneUD^a) zxop_n$DSqAcia+&Gt(;e~_3B6Le=Trs znB4#$rb=JfcW9t<+02*X*fKr`+?JEMa%>i?;xwjsjk|NRAFkGqU&F;+a(+gp-oo+D z+ZW=UH;wMuPPm@zN43iE?RzD$acqpcI(LMm-nO@lRi~b(UoyKdJ^R>9z?6!O z@EY24od~(d8T`&WaSg$Km`j%F!dH+UHRxQrYedFx|M8_7GFR`ly|n|+Fs1`ves7fA z5oXlq*1;Qz>Ed&ptn*u8#^Dov+$I-)zppO`zx566p=lE*|LFKL=R)Z1Ee-o2y1L*5 z{fWt7k~-{!sZ-1DXv^?5YFs8)=SaANPbBi`TZ`_&ljy*0D$cK4Ev|znHughT^}&ft zi-~LA^`j>~_Ct7e+X;LV-pqOTDEwo%!DvpdL%BQQLAiVJ!AbS?iOkxv^w06}Xa2Rc zPKM`Sw@F{WhUuKX<1MZqqLub-{bABonb|$>rlCMAAi!AuK(O6@!4v{Q zRat}hcMXW)Qs0A$nLqjIjKfU`;_yvHV$Jr~d?J$q3yQ5QfhpjG056 zuzt5V{&6Nw zT0#u4g6w0tfE{AxIKkTJ2u)QGx-BB~S%w~nFww`lBaZ(v(-poY`hz~)_LnHfJ!b5m zqLeR@_<;xu{a+&E|1;_@QS3iQk=zocpilqH#J@#fqyAq0uZjN{^_S>>F8{aa@3p>^ z_{S4N|MA5CDf)Z)zg7Kf`M>n~$MRpQ{@3zf6aN;O>4N-IufL!8^^V56UlWu6Th;#~ z`uiRKYvPv@|Kp$d?@=}zuXzs}SH78W@7-LP%XlB5I`^*sdRIc6m4_qz$}Yh8!hE>@ z1Kwk6<795*Z0qc(VqtOwFuzR}FUrYwx*>e|b9rNNjC?5j5*KetAx`K$K zBHF#cw5oM(>OE8ol5T?aN9qlQ0*#gR&Falv{ z^zC-Vh3&CipBYGtszhvR#^r6ny7$*qlL$MB;VEm@!i1Yr2ioP^ciRn$nvFBzTeU@2 zCcC^Ga3V2uUGlJ=ijPJ)5)JnTc{uT&N0jef^Tdf_Mx5?+g~b&6ZG{l2GQ{T>U~g)R z!G00(NHKY4FaN3I$a8=DQP8)P5kyXlq|Flw=1*RnGxlEmvwcHiNp3}rjY8T%bvLK9 ze6(y;Sf1;5@kxj+5%1&8X2{CaG98uM*$$UBI$*51$&V9UPMfDc7ImFX42Ouju}c#DkcDd6TDsa-Ed$Anz(}CumPgPrt-8S>_ee&X%6W{4m``Cb&NUMEUqAwS z#L=urn+gM52Cy^;Cy0`y^28tmGs4))IVk}E#{a(ORM*vz)et_{z0PaO@FR1;gc}!DekMQp&4F1; z5eTCAk^PLUoBu)_E!n>sb4X>^TaPx-6c3?(DqF@t9ALSwR%uEl0aQH)Sfas<|ZR7?>XAsvt^qu?N(`2)$w z3w1LJ*fO8t4d+*TN=CkO=V^fsutI*+E$%S1B| zEr-3`P4(FYZ;v&dE5~N2)Rd|)Nvb-#W@k7#m0GFA5qs%AO0!S8yT@y-P3x>?+l`o+ z5XE8t$r#(ml$|(7F=(~i%Q3*bAPv@UGfgC9u}Y%Me;TGEFYtmSnyFDa8X1ZGoAbe|?3v0(st zViI~(wQ?VJgmERU)!t!UtfGa&DtYY&B2v68m0vztQe755XT@nuM4Z7r`)y zV-F4ct!tB0f^0`>>nR(i#|9Exty)DQ>`e7yCn5QONI1`zc<% z&EC!{TGEqe7+#czj;mfTu2<$95la_&O76hy_?{a#b?x*&(`hyiNFVu2-Xz!e=Lcr_ z_Lo>Mn&wWSuahntlURAn>AKrjo$u1FM4}54vyCj5M41A3@0K#nR5YfmoJ>hm9)=?< zL%)GbHPj1J7W8k)t{Qo{??wP$H#?1b`8SB6y0SFDaC3@yB)K%}qN3;PmfHWaiR*P@ zZQtAQ(jB@`hW??u>XSQzdj?qCC^GhK1K_@DqI40=2O4XxR;>@&CF&OG-0|g0p$a(d z=%f*R@On&xA&{k9;O}tvqYASKYqlf~5edw2=0un^J#sD~Z<0!s3ZfX}5%tdY6~hv9 zi3eKrYHTWiaGYJrFBL-Ykd1hKhF0>*S$1~DlFm{#v(0?mp>$h%pCuW2M&JZ zi@&+8j~x2_EXO&yF4x#T2&!2!Ur#8!L&C-|*2`pwI;6EpRu`0qU=`nf6udmxrRFu% zFzx4=XGb*j?;*6OpDrZfMMendkQLw^7eYI5S>aN9sAN~ZU4wIh-!y!p3G*O-fIc1t zvYaFS9(RGT33Ou7AF^_BNh1o~JZ5!5#0beGM0f^X58A?t`t8M{2T?9Sd~xnY*cNQm z*B<8^ycIW2#umIq(T9%I3-IL-{RI72d$}bn4D$R%y(0eNrm+9-o>~~-qWD+Kr2ij_ znV_sChpmS20oQS6$cB!i4Mb8rOBR`p!A|0=C}`Hi$;j%`41&6!t+N1L8`J)4!7}zO zt^{5MwK(TQX+~0=O2zF|$oGe@?CND1E(5HLid04lv*|%9r(;KL=jZb%^|!@pv|e{( z*0mE+ew)FmAi4PDDLb)2V;F@sl~YRCOp7LTA@~}uf`p7PLO%__cyX0GEGO=xGtbm- zH7!?G8nY*9Aprt5_Rfd>8z=jUItsg%tEDwY=7!YaWfgZZ0xomz27XcmN!UurK-2Xa z6b_{-Tn(!^w!B6Q8|C9LbnVfyij<7<1#8o#&6-mncBl%qVHTXx@_FsTjN+6JRo>kc z+2wM;!QO;}ez~|t2R-g_O2=t+jeo!-c4_`2}lcj#&b;vjm(R%Zk7}usyI$V zI{i3*{a%A@D;%TY1|bWvibtOVb`-r`b%FwGx}=GSChX)xrn8A?=}hsH7EfvrW+28(6Mhyf9N=_?((RUy$-64pf-k{f1)oaX$4yb-lcZ>=#M1*6jq#Q;j6 zA9NGN)F-_R21eiSa19oDVF`l!#4zf+-YC$Du_OnHV-Uxf3>tDsb(6wjRks?emxJV< zkF~B?i=8#dTAy{zK?zeVmwFoBC_>YkRpK?7=pwMhYD0b0U9c3siYt5v5Zh)HmzR}B zGcFSmk6~-M3vm}6**}S-r}uf6WS|~QfaX~~8uK|$Cco3q)Nmlv=oTt#p2Wn@&D-;# z3mIHrrEcaMpbv{;G3Mtt28}fF_F)uMjOr4bbZfEkbL(RUQg6jA?23mo3AS>g_Z$u4 zz9GmI+>CkJN$;Et+S45-evU(dg4-Qct@4)e{8RDHfKA- zZT+1siBfTh7{=x)WC4@4P73jsx(6znI`{qun;4 zp(9xN^{W$8^Xqk&G8*P4SFd6COP;|mxoo&|*0mw#R!rpY{9zlKmV;40C5ZjCDK>)CG%xX*#3hxF$b6# zI2cRV*g5}Sgie)JhgD@19dZ{cF@=jqU4L_Pdt7oGU>0jbX>*&V`dzk0Btgn>TJDh{ z#owfV+HUd37{>0Jc}OsslwRh*gMGisF*@&YIMaFS^KfasIy~rFW|r^re?Gsn{#;#B zWDFcelO{y%KBSs?BJzKwx~Es+>8kU`L`58v1ju%`g$sD)6Xb4X`-@R={Z^TdN(*P( z6U6X-6&0aI%lnfwYShf5P*tQakwsct_2$LZSyM$rzyxGH8>Msaw5l~vyNS^Bo5^}s z3Odk`T@AEnRyJlKs>~c4+6T~Umvj&NEAgUCX{uahoi*Rb7{)WzL`9$Hv}r%j7ClWe zXgLmbeYK2CXPu5KVK2hm>coc8$o6%k%>cXNg7X69QPpV{?I^n3wDAuAJ}-ZDe=qj% zkBd9ikDOM(!3F0aErR{|O z=OjaSl*;bf?RBg=ftK-ZB;FqtjGnVYi@{=_K4bNXF-4#k6^2@U{t88}w!fBpRkWKq zT%lpD5~D#aUAd}Y8@mjeL`%`J18^Sov2~iL!Gzh+#DGsb^dW5BZkU%_j1_Md7Bq|1 z(a#LJY3>$v30fyQP;=D+hKmi9Ubt-06toHyaJfri*L>lzl_VK#i6YkmjxEL7V!Z6H z9gYPP&k}zv-AfkCnv8+gbniRZ7{!+y|2BZuI>^?+j2kVsE-9bl6;N;RI%6}>9Ll#h zM0~-`w=;-y|8mThXU+gkv8q^`Tt)woQ8~24n8JoLDqg-j=fx~EY>)013bZ&mYB$k1 zZ-GtE)QzgRRk5LNBwdaQa}u7OewXdqfSgP19d3Q*hPet|LAaq9gI;lr>6hJX{5&r+ z;=iv9yO9~t#`hhL!#9b7Udd7rUQnBJvCiwqD0#ep4QB`ZH5Ejdg9@eJDa5Xk*ARO- zT3~6D=(2zGktGz;+ek9P{`%ADK_=rB_Ma9ZPY9`HX-vMZAUCtOZ;}rPG4*b4Uih@l zF}y{#Q-3&ql351(|50;KmHL5xg?agvgE9z`y=n9=LE?5UM}Vpyws5iyR_&Mgz0WT0 z2}u}p5W=1`3SrSJ*9$wBR62#k#fML9>C<0O_J`^o+;oFWs0$yiCGqRgn%E*=i(~d( z!bOJOR6^)wwky0he^B43@2BipHvcsma6B;mG$&?3jyQngZIJrFhUgc|ZGk=i@A{0W z>r3z<`OzCQ2jL|03misXgEy#WUceoc2?hSOVvP2#us38o#vzr4DdsaJLYqW||1w^U zo-4&E4*Am>Y{|we{J+L5n@}jVL|^Cv%9o|#{7+WK(Z~Vd1o(eo3yNB{NCGGyU}&m# zv?%7q`6`ySRVx+qPr^cmELcc}$O;+UUGX-(6MqaEk0)pHly?L8yZ%uiq1F7ip`jP& zqhp8T>+|U={5Np7L@_vcho+#AKUQWp6wxwP`_+-@JjJbK7Gx)6R>5_#Lcxl!5t)wU zFl#Zl8KZuetQbt;quKiUv|yu-`KLj;NzsB#0C)-5KQIZJrOwQa8vq8~XW(#}IvvW5 zAYHj4_%?PVtkKI`g(PaV)39j>*>%R=Uk#PPvJ|f!htCdYZaf^g+*BneZ7ze}q(IjS zE#w_H8&H%e@Luy%=hwfA9~>05LBiUUO9x@2%&-bcff-S_^4~&uFMf|TM{&pVK%nNk z6aYnenC9TK96XTc?&S?QYlS%Krw0DMe9Aug#a@x&`EDz8ergOY!qZ@fofrg!MzhxK zA4ZP$P-WXweT4%4^)E(mKl{=|Wpi*71iW7&&pgiE75}P*@6tbko|EbSL5KGZrUyKL=tg>foSwf0D8#Yw8Gs3V)Z zGZklCU5X%Uj-`6^)BEPXZam|guFbKpjmP~(Z1@kP84))-TN}U^wDo`Ikt%B`&Hh07 zn1RkVKa(lS$=NC=)CSt9hfs!k`I(b7_Jkox)9ut72y&go@d2HJc*V5)boP0z!rKb@ z7-ckgJa%!+>Dh6)>L~j#SV_P6Eka*Y0L`1yIO{fFZ{Vjt3m7U1L*DN}d4>o?&_z4m zNF1A?reXNCo$6Jli=ig!-HgCM)0f0RKg`8a?C+(`k-?)25y8jwm6$YUG@2RL!dCFK zCesFs)KcQ(tIj4beqzPQw0ThzWyz-n+sS1%%Io^m)gwkA=#T0;f`l+&9p@O1WLgf` zC>GfDDd*7=MP}vVJaF@kKa$d&lEm#@2250wuh&Rm(^kq;Cj5_=-+cF^pGF`41jZD! zW&B_*U{lpGVccX(54fkq8}Mvx`SqNWpv6+IdYg(`Z*lumIp2oSvR98G{U?Z%nfIW= zvAP`#ibeL++V{h=op6=g(a%kLzy@sFA6Dzz94FS&9fV0Gdnf>^JBqSXj`&xN>jR0K zC$if|0_BJuz7UMvQFF{)Dl++)>mU6jB~)}f;el%kepChW+uc$~5on&i5OigUAn%)_ zBAfmLt7Mh8F5?}sQ^hK&E@evG8#Eay++1MFRvVuT3R|<{#t)A8aX1d#bl(1uM7d@j z5;$nY0lFTHH3a&7@>}HuuOz&Jj84C0bO^Zx_(~X_TaGp@S9a@6?{URFKG6(cy3lq~ z_riB!%0VT+!DxoW!P=GlcRwb6p06CLvCIGpc8UY<6 zro^~;Q6te!2f6%mcqH+iuP|?C)#y9F0zsK!(7Gc z;<;kDYE9@wkRW^HJY2!hD|S_H*r#Qht7$hoHMb4H zZ?<+Yo~N9xehE*s2E)`!3xxViL#niKTtqqr!?<=S=jwG*(^XStU1-v))LOG~X7FxV z3UMU|<-d1WtMJL-?kuGhS{BTHMOV!Vt{Q*bV5@v_^7pfFg^h9QVa&+^r1NnyyXe(xa6)ZuZ}N6Aa8 zDtaaSj%E=v!<&lkz6%oyyc-q@ye}3KhJ%1N3LIeJ4eEk9PZPm$?BkN7an~kRhQEpF zcK#vyy)zWVQCw6|3xnObT;52nrCCbfuOn4)ogz{&JqY9dXT%5a^y01_B6l!Rk2V6F zShW524>G8f5}J2^V^pm?z@Hf=VXm)6Ucy|W{=y`AB9QqN=2IQ9WZ#ir(1oKA3Mt=U zse0*2qO#iaor?fEM2wlKM21YxQczR=OnyK2!jD$Kf&^-x(B`ANI6&JeGR8{ zB1So1(f`$Z1ee^y(|?6W=~%hvDg1=v4Gqx3 zUXJ&i_0@Gxm&;|H_xFoQwQn|A6-HRhH%c@1aJ86AyysH5oNH6|$c-!!(ZBZ_f}2?m zoR6L)2Z7L>y=c>=urAD=g15DE6*bnal^dyB_XkrGz<^7cjMaLwh|_*parz{YQ(FEZ zoZt3}CahL6!RXip&3H;AG)HS3szWb%vIpX6uv)=6nH~v#Dg{I8D6bu0p?UZ{8V6BSswUB@ zaI;VU{qq#YV?Ss#6Dk^rb?P>FEWHK<*O~YNLKi+jRj4`(hZU^6Ut}_t=~pafJE9A5 zQl~C zrSFcmmZNCWR6xEpQuVUy)7nDi{z}OCQ6+}S(t$5nn;)hqf+xX#r72{bd<`}Jux;7Y zI&=tH(?YdyLUQg7h@7ixRPs%HZnh$#+2AToEJqsm+$z`s&K%Mlth67iVmjxr>k1L6 zDT2{U+9Sh-&F|JJ$DGJJ)C%1`tRv~TE|A3h#{w7FXg+)lhu{snGr+%tOF|zEO!6s^ zB^Nz{`YApSq@_ghJ1USQMb@$Q3w$0D2e6snH#F3#AYD5cl6oK&xmt0L@V!GQFQ$qc z1(=M+f@nskKm*Gcgca;OJ;ObgL5jW4;s_Siqy18E_ZxEen{ZG~*=B3Tm-plV$glQ? zrz&*s%r(9OzC4MX{?{v#c*L)xzOid!VDMpc_%G@yp^zp|KqfG}Vu~+J3zrz2eH00A zr;u}14%UwX#sNF4znI?t zbV>;-*3w#wm_AXLtYiZqyMbOiW9Ky!7*#?uu?-|Fzd6RBw@{Fg#5EZnU^%KQm0thE zCMpKk)T?M(Clr{!SF0}1Rol$Rmj&hN)oHX^tX!HaJ(}WA`>4G#%lu(^NHYpZr_I$M z%Dm;f z1Z*@P%B>J!PmUzcHAVLxjBLu79d@w!t}Ml^OOPfk+51pizhYEKhrNR1>bQdw8++UK za5gd${>3&)p7xnyVkz|gpv-Dl^uCXVONKMXZ6c6IhYpF37$73vp1#=bwv+xL3}OLq zI^KrLUY_mNqHmVxax26Si)|!~4|^T^v%Rv^R4(=BIg3#mc@#c8e8huue-z@Ks9c3# zKHUkm> z*?QgVa-X~3kIUvKWgg`!#bbFQOEReDGy_e-XGVN12D3xA9^D=|FcX^ZVgupH5=NR? zCqG7{HPV6pbee}ogw#p|^qz(f~@>@>&skrvv>6x;x1Z&=w7 zDKj81O9$`60CpCP`TBU{mZgg(hxTD-S->)-_i2gQr#5>@G~Z8y`rLZ(dRZLosZ&pX zo6jFm>)iCY;!dCi6(SFSQuhbJ@pPNK$hk@>cF(9t?Sz_NqY(9PaxR$kR2h~Gv~)_S zEVC5Qh7rk}buigVy2*y>#U>TZM&-*SbZsqKc*3VNXubKJy<7@jvfAk4F#kV;{UQRCv~_g+f* zEXnzt98Yh`qaeN?(MEq^x%#S4Ew_K$Sl<|>R~JE+Nh2q3!alxw+#|xX-veysx@C@T zauoqwOULtz^WL5xu4#%!=_Qy)`t*|&Y9rxh8+?vx!3a0}`>sD?V;-E!ore^Foq7a8 z)ASf|*Q)JzgJI)nd2noKkt1UhH;Fe0!&^0IXnZkLidqZQ*sbry0VP4YqBQAu>-#N_Zc zV0ov!C^ku?!vlMC&O2`_ASO(%)Un7?L4_e-_Id5~DVJb-0L}}u)!a8|vIO=K(YGGx zH7u3N?q~8zb~|u6>PLBBH&k1|rx5CrP+K98Es;%W7i^y=^%dHVH|mm=;NltZu96mGpvy`DW*auFm2=G^J<-T=)LjkcP2(YfoaD4k+KXCL z&dNErTGL&$v7FUNqbGps+GMP!q3)i3v4a*vzhXGL*Tzmc{0UyW)2dL`zhKs^w7CQO zZRf0qaIU(2bqSMxp!kU-!7cGU**Iw;7v4P0FYJ)R7agr*kRP&AABw-5#b>nf;n5As zw`=x^Iive9xqp)(#8)5hvo9Xusi3IaF$Z??*XyPa!-kanMAu1XU=y-s`UYBzV5|lStCAn%vvqVk|a!Y*3WOIl)G4z zp`=#1Too0#os!-!K0zy%l3LcH{5c{}*XR{-=2pZa_ec#%%1)q{7U>%vYgEI-*mqXx zv@QUeqh&^J*Q&L8kLwxrDuv9HPf!=OliF_g<BA}P^YRqc(7*bEH8-$zSPsoYG58p}e5kI0*t)}RMknw+cyjjRFc|QHbH(j+h)~shUK#ZnR zppoAa!GCQe%e7w9;mgg8$$zTHs;)Pkk%JuewRwG=Fp&8LcbY*jve`3Xd4fuFYJxu~ z;JkqR3HGm!Q~vZPg3VX7O85okGyd<6Q{L9x(eZ!u8~CT&Ojp+V3R_WlEwQ1Y`#DH{ zH%JtxlXZu}@Bi`_4GhMUJ_UZRt8>nKc($JF{rq_8`>iR66&ZbNkKt#kMH67FA&loDd1$LbB83w8 z@_xq~x&5iDFXU;ckHzHWiyMobtYXQ}$QTl?`C9Df4y*zU4Rb8+1_S&{Zq0fzEFYhv zv|U`mIlk97Bf6?)@gq?k$C3-EIJLqyME{|Rslm?GBWQi*TfXX z(YuBzX^I5tN6k}+HGus+GlR70EP~#Pb5x!(Q)KY?J)&YZ>Y!I9cE0lnIzzdvSvci8Dx5GNDArgZ+-uaJm$YI_Jlv-R5i#ze)UxN!IIbY|3xVh#!kbUp&b8YNaP({Qqz=l`6j_48wY8Z1*P-nhzybG?`ikxf3DaK+AToL;@5%N+hYi~SFLAJGR&hfT(Z zyG*sDM}?<-`7s6IXgcd1e8)ai{k13Ywatb zs%*M&QKY3o8Wf~kQo0eO8>A0ihf-2Hq`N~}x>LHlkp>Cr4r$~bzu)iU^Zbhcz3aYf zy^8~jd7hd5&Y5@i%%1(!z|nUv8DQD5Nz@M|1T#5~9<;=bZ`v;!Z3QaqsH9HgY()*& z?;!8TuqU6+nU`j~_EFzny|zz2O0-EXbG^RY0zyFRP})DB@3LnEt;$Ho`U`fvgwL{) z>If7FP{mmT*sJp6%R6&_&?ZEHr_&DO)s)?*iau+8=G!p->;kzhq)2lugiWuGa+ww` zY~^Y;-)pRVC$dyA&q$f7a>ao9qw<@wV2gsusBMQ%x^<{l=DJfKQlX=O^Ed~%{xT*9 zIH&O~U$l%wuV~cPXLX}ONj+a_M@(lmb5Ci%ZsG(MA)}bQaOn=x(v!L4yZk}j1 zXrsm}^r~*Bb38{ha6sdbA2q%3NiJ~oiAzEc_qmJ~a|1-N_d@!jslDvOtD@c_%Y*f| z%>|yqbv*2SHP%m+V=VPmG8}ufQv7QYC0tLg@mA_fbcJ`AwcXM z6?+CAjjIf!Qwo_E^(CsIEdpfWlgc}LKE66J#XeBcPs){mUyAJdxLXYMO2A-ZZh5MW z%W2-`p=K=-)l2T;M$Ri<6Syf`E4Kv_VJ0I6QnBlxK|Iozr@2UQi3su`$8)wNz?Zbc_`n)gzt5w-HyIzrJcGes%hACY%A~ zzG+nMmN8ToPPB;R5QV*;G352NXSqCcUSUb|#q;wSD#!iUhyDspp-ZnTJe=zCMQ&Ly27ob_vK=n zt#E`fskNUeDMrdD>1a!KaDoD|6dnSE+V9!stgnW6uBUOZvii^N7fZ2JKiP-fb1Jpk z&6amIvC&0(Bec&gG`|Qi-lUUkYakd$PiW<2I5qy#w9ZVEqaHlNupvF|!7T}J7~faF zk!Pxabosn=2u;*ArI;+b`^4hIV1b0zE4UD1&;|h2u;C#DeWX1R#@?G<2*;`s?oBFg zq7R?opTutzWL7=-N>=C+Y!94?q91m|l6<+-Kl5((<>|M;b=r@kUaiNoAr#!wsJqPJ z?1Hm~en5QDHdXYs7FpD=n`K7h$%6;MP^gA*-N4r$fF#pzR+!55D~DRJlpEdfYusNP z8PKXAxU*+S&1;bCiO|b=)FE%UUFo^KnDwq3>kxZ6^X_`Gsp{#O!i_-_xfL=r&Dx7J zg@|>sLUqB!oBBEjtb@!q`gl&epD57NT=ch_aZg&{W$h-i-wV0&ar05QjEB#ATrBoO z>kxSj(X!Y^%LEz89d(}~u7?o$6Sm2?4M49`h~5lAd}DqPJl;gUuK>}2fvwMwR1<8Z z$8-dP7AZ*0QuGy1D1nWsQIN;(s&dc;#?KU)xhz27o)+r$#FvS!u_0i0$>%Up3;%_9 zqLlFkey$)x3TI@FeVQGsujHnZANjbhT#!RLUp?&v@*Q|qB|W#L$fQ59l78diFgiHwmZKbUZe-Sv9qLvvGI zM>=H=C8mHdhex0f6-{qmf^{?c@|-=?zY!^(vDK9PVOi|DwYdJk<-LcYkdmc9WSz`bX zbp~6ou^v5kA?cv7-lSGB7gT+EaDv~8`Dpjbx+aa`8G>dX%XVZ$fF$%x@v?Mv9OVCvePV+`~_3^D80y(KnhEwd_Zn$$_Ke&q~r#QWuI=+R)GsFy(13 zq5<+sz4Mj`KFx>9rz6y*Edyr=5@DkHtv*181V@Iakxlstx@uzTIl(3qP` z<>j0zs0&wUl{5Uaybmpunj1VUUY#+Lh4(NUv^>g+%wK3C8A~PbXYRoi5wGUtIyPZ= zFNHuA`;g*`-k3vx)jLD!fR@l6!yEY5EM9&fFt;u#RNU~h3gOWh2jftuu zZ`tW^pY(X0-A@y<%C^q0N};Y``SQ#?_(4PNvM+mPN`ugSjB?&A=!{+N{J}u%a{5p4 z*3mopfQ)tFt-3%%)9KgN1qcm|>tEY-dwYB5pzh5@(iXfGfN5Y)XCxE zVW)Vd&^fqovvEmpdxh2*F4%%@F#*BZIe{wp!C!0Z4ACVorNkDF^4;#!KFz*M8a$mZ z@XQ2zi*|Asdn?wjI*Az4mqX6bA(M)^drBk%;??B0W0aI*Th!2UEC^hqy@ymd*d_LF zn2;6nH_i|YRK!E#j3PW3gwMS(>Z63a) z8e?oX9NpVQA%(8nP@lMe*3vo1#ly38sT*P=0F#%s!$&(N@PV)xU_^L90S-XtI^rE( zmYv$BVSL16C_5?Wl(p2nqpD;=#~PirkF%7{(v3>Pb@M!iSmXT@-jlP1oeE--D{jdq zBePcrMs`D9kUOMsa^lx?IqaFFti*=rM*C_)j}Rx&;m!?MqKj!M(yf`c><@G1NRjms zLxBF`B%MKtO{gyMW1j8zFkud$Y|(a?s$_AJmuj2_Yl002?zg~Ld3=_dIs4$S>P(*B zokO^de=ytI-Fh%s5%K}sPFObt$XA1|&fQ-@%tzRcQ|Rb^@0z_=qq*!1zQ$fP?n$Lb zHCz0yXvc2aHW6%z$Y3S&oX8bji=FJZMe}3kij1d1LMG^#^=OjSWtmUv;G(u>RS9e6 z1nqYgMLWA2+`Fb*FWsBOp-hd4J_{(8O6^Vaef7M5B?NIA1!NNLC9-vKV{kQO5Y<{q zZg0@mO7*j`XPiXX4PyGJ{jkMW<^$kzU5$Irr1|kOo6wrih5`N>qes!fy~JkWI=82p zfd|Vr9^vmrkwzo!QHz%4P_mBt6tNmnV6ayhNR!j^3B9_sg#B8*tnXovw}oQ1Ql^GI z2I|ayNzzoaPo898RP;JOd*u1n^N?Ch6&OA|DwLl8q_& zRkq~#C+{2Y3%zUy^ve&d!^Q3$2ZgV>rm%v2wvc9pZ3$qih9YMLY@Xq2iuOVynJ-vs zIj91K%bf`d-kK;}!}?NORlsMa66cP~%!Jux1+-Q8-WX?x#<%9oWY2p<=U>i32lK$n zHtL;Ufg2P7i zd79QG`TCLSD(zF9zUo`AwfEbbq>gm*MDCIluXk@e8oiQu9CHJk5@pAU^E_gTKtEBx zrf{##CFwk}q$or98E_)E@LKxq zehJdYiVvOw!WF3gGZu|DU!r+Mw}vZ&0jDT)0#4SD1gA1ecp{-PIDn30sUc@MTLN>4 z4+5l|4arJuB{P`L+%j(uTz_8cxDJXe@&m1Lya(NXNLDGQZ>g{Im+s0x`*Vr`j7T1Q zD8e@jJ-JZ_7?afp@Qr1bxo#+aH3%;@9;<| zXcNdF%Qkc3Mo|*Rz7!9xl7dZ8-)#x&l{!2!o;dV`Q(Z5>d5f8d;KC3LV2VwCOOSZJ z1-G@K!*MdM-lLQoiv|mq6-L~bLe_fK`CtQVyYcdCSn421ZNU(%2tf9saBt@mfm|BhHe;I6$s+GN z5x=5N*XCDKweHMuba`lM_@y!ojCbeZRv&}OE z`C~U;l4?qYyLV|qdB!sJyWm0}ScHn%&_}RvhXY!cUE+o3g}jZ!GfcKg-3f#0X@FK) z@9Eq2y=90IY~!tL*r@1QxKK8C3I=8j^C=%ujgZ_oM@{Val-V}zZhN$gVwQBl5kW=V zSH;lTY=*o=#upc13 zy>%MQ=Zmkv&{`QruFTLoP77}dCwZc$)S(DH3((R5x(ewZFy#^rEKb;5MWto6C9FSN zgH?XXIB}+K=-{1lnVDiz_9l6Ek@ESc@PUq{1DZSs6737>O^8-M8g->0v__5gj$@9LueS>y`}?UlOEh~aZ%9VaU!gEH$d zWyL6^4w%H2lpwQsyxlwWkVV3DHLW3<8V!rb4OyRy7QL>dIAAonam{%R_?cJpGlcp= zu?Jc_8Gf}bJr`?Hq+1}b@wIdZmAK~661Ti z7sh8C3juH;ncX%ekpucILr1Ui9pm6_F^r?^QKi@XuxwaH!`RSt-(2Ir^Hnfm(Y7QB zd&*Vb#rx=8Me`(^K3B-s2po&KXvkzzArtd_?d(9?M=(nw(^yxQ))pMO6s?S@9Yg1P zc-Zk;qvKRkvKjAL^B@eWF*JI7BC(9@9Ir_y1L{Agsg!J@DiQ(`q;?cBH#no0Qv5Y^ zv6W~I=SBjFcPxrrD}@%>@&zR|5I9w%3Un-xm_Bfv#r>$t;1zXIjdU4HLyutF1)6+--&F>;Y+cs&ADD)O1*e&kL}~! z0^6|MG;AqY71m4G0sOhStDEJsPN48j);}i%Q<0XFH!?P`{b#%lQq-^+RYKtwH=|~U zkJYA#9Eu<05Px@`Mamc8nQ9>k!Rh5(;ivnV{r)TXQVGE<>2s26$hRp>k^;RCZ7+s= zzoqcJaXgtmUw2Zep&>CNc6?)#HuDG!|T)uv|X;W7Y${yA`iy4 zcB*!sXp7P4>MViLYCm9sCzI;EsAm_k(yD{eCG@BCr}V=KSVn~p*HiBUprezKNlM^M zD)+2d3`QTy*Gz4HQ7P1G>cU&cIj9#|w_mK5H_erL6w8dVES^082plw2!y ze<7IT?;gBCa~-ecV$wv%WtGesXRtp0O#d@AnSSE=@ij$;)xL@9kzV@`th)~iXREMC7Sm=JLg~)IkW>cRuMCr**DC#(O_7Xuq{o$NmhlzE$3UFdHtP=8gVZ)YSfkgq-J2Se*>ri+A7yF33guvh4c>gJ zPxtrSXgP6vsk7jyNPA&av+9&j-Tko=a(9qdcE$3H7n)(uQEHz!;?w@SMEUG`j42yT zuPLlllfZKa=`p+rUUyjD@na!}#IHIH*AJcF(P@efi{4;=79u!%hsBb5t$-I&bs&T% zZL*E|#5lvkPj)qB$=G}zF+GBC2+-&e_M$Ky0GY(Qu?$&mMQjQ6s8BaF`4fDH>O)jg zSs&DJicTQ{j$uCHa|B`nd(ku+^uTo?U$y2Pp{1l{n5HJE3)Cgyq$|1Fp3iaW^G)cW zJYXlLZDefPg{h_s$Zat5Ps4dr)>uv?PC7vG1>=M&zF+Yi$BXS2Fi+;nU^sopLyKWZ zysl5QBZo|pJ78h9`6EJS3yy&a+*M4?zn| z>ZTvg0z3mzm1EOP34CGUYS31DHW3r%T{hw-sU^{oUre-Rl+#iu6Ia$vPsFUNYHDbV zQeUpb2lREXF7*Ta#)f-Nm5B=soh=y-mn)2Bc+uYGiK@7ZZs~{t@*oqIp_5yL5XHtQ z1w)w6wvGI}xJc^n84N8vbKai^ zvCq{!0)G;0_%Xx!|08_x^%uei*J*8akyq;f4}uMu=9Z;8phYYrq~C86fb`aELD4YA z02_KC(6m=x-}c9QACwFiPZ^WfbjQTp!TkB8MPi$kMpcDgcL~i}dbL<0NAxoLw-?3u zA3B&f94o@u!peJ0qC(^-y!pf%wV?^!$m1(*W8F6x#)8}fb)b{&y&B!_`M?m`do21L zaw_Llq*2HWuFu-+C11~d1|`VdyEwf$UtvWh-I#ge=szxa%#1_3|IAE+HD*fBv*>CW z{h+ff=&|Crr^%{jgOvAc}biQ<8VQG(vUwwP4!kdEf^HH-hFh`8OuqHMVtu1{+HXGK2(uYdTJ<43y2JHxht?`^sWFK>s6Nho>8@gd8SHZ99 zK5w+(=)G`6qJAG|7>hYqV!unoD!HWeR}w@R#u!Z#5791mx5Fd zGghBD7Y}wUaV$d1T~d&KpxxL;GSRv)M&opGwS)#Y*-7*Shm%E zf$_OeiU+)k{k^-^sA7zxleMg~sWG8_bHD@>ct|E! zs{KqHX52smk6P+AWnkv$M^EN)=#rRcMXz3D2a$FQ5Fak(gqP51iNL9$lKRUtR6A!{ z^Smv{=QJQwbfB}r$Brgc*;&F!*?IOcBB$A~(3o4RFODAC`%-fQH|6l0(KBR20< zOu!d^7mbN^6asx4m2^~pqsWwMk#w_gvyMr8JthelOC%P@sD|8O%6PdAQ{1_%qP%0o zpxq%;+_w?^#sWD=*Y91>F;9#nJ~Q_bkB%AnAmtr@6afxd1lINA1tft zS$)M-|H1|h(dA>MNRWyN@dvC@zp^c5iNY$~FWdJ^hcOA_i(DO#-PuN}#0$Hxy*nm= zIB7Y$!@gSa=EYBu#-E0P6sH2w1kQVZl*=TB&=oq^%Eu|yPU?d=gFQM}r zG;{HbFnDWLtB)Vh6@IY7tQgY7&467@0gMX?#&%Nd4%#jHZo(05v6VI4;Ms z5n=$?7>00*MHx5)U&E#2ZeiDojtmMdFfF;5dk~~P*5*Z!&hr{(=kBd~UKI;-01YLy z-8{){;t$12iDLTs>GMbPPrd84^8>~tO`1`rpVC(D@fesBTN>74vp#j}9YfxM9Fvlk zBex#*G~)`Ec&IxAIcXgj495|oSb4l&Y$u@?;74%!?i#U8A<*=FxXKm4e zk`_<0o3bp^H|!9md(U+(+ZQOq^3+-Fn=RW|s~sQDpiQOG(LHfSbYa`97PO;3a3vyB z+j{m|_EFRnUkA-lt=y>4GEb7#`+-1LJ}F0-43ZN@?h{CpYs?+ZdQtAi`Ke5ep$h&> zT<;tO8#NBDFB;CCj`nb}^d1-)r{p{@WM~@Hk4l#souM6a>#hPtv!Dp-eOFvn>@@ct zHy3?!D?fXD6!KtrlfMBnbdU9c>OOzNp5lfV1i#5{-cefKlKUB}8dZa^)AEe?F1rM3 z^TGTV-viC(5nd!6W>`CAK}iKL**GTzX4iBTBThz1r^DC00{uLEkk{C+&vR17w5G<> z$NZNYCsgo}mSGHSZ0F!h0wyz0Ajy z;o`Yt5=nCJpczC+`x_Mnsvj%Mm^7@4a6f{_{o=u1N7WED*oxpbl{>gyOkRpfd|9Jq z%7(cSd12aC;aDuW6kN#XjBGWVPJ;Xna!h0MYef)@ucHTdb6#>2FXGPjNGB1AG4==& z-!yDSklg#o4F0p$5&n|PC$Jl>bfzdMUPJe^*5A?Fc46VB(%TDY?W+bm^iMxDkTx6U ztM5?=>M>~a!ZMAo56`YNN&C{*n)6Wxa%9z0-Q>kmH}q3h+|bbdxdn>r#-C>z%@|eZ z*dQl84Rrr*E$3gaYGHFrTZbR-YcPjBDU|UgBtOVgB#TiG95xa{kY%rJBNpW|CuYUX zBk?BRV2dGDt(ifrvf+1)j?!L4-WT<=ksWI3(TII}w7QJAec8E%=XUP|chaIoaHcaj zM8o|elQq8ZLGIkVP*ZIs6X{TJx$H7=LhhBOl>8fxFiAa*uo}8c;Apz{oBLQ{{cAcp zXbJk`)eI5kmZ>CS(_vET>^;%Pwj^R5{vsiMYT=az6hndLVdR@iF*?4ka@R05;rC$x zc=>v26j(;+ec14cX)XvXwhsormAz++p3*&vtwR z^2FbP?%xGJ{mb6}J-d_atw#iDL-{Q3wSe@bVs?IucHxPsSdQNE)33T_ ziYDc_pUzyL&@`gkg+mfa6--P+c!goy9InfH^%q>{&8ivQ-ePJ zQ&f^)N=!spUV%!QjuUMKD2NlAtfa_rRyWuMBroP$<-| zn)u@(fyqk1%Gr)&}#2L39|kc zzXRvW-y2Jf1r0l|(I6nW{%*rx=9wn=bu<3V^=pF_@m)jt{<2klWd943jJ~z`>+f#u z|In(J`Oik`1{ImuLT{}{1wBGVT51LiOmJg>Yfs*ylL`J`p#NKm5}X2De(;vU zNBDoI_;m*dmy){0ek5@xHs~GmD`6?{x`T_Q+~P;d-iiNT(kbA`;1UM6$SO*ABmY*= z0Gt9m^XDyv7vOG+KjZ-gCjw9OcuVw6|4yRcr+oyc0MCtfOF?J)2a4ZkN&_bWPnmQ} z;taf-9;tE5L!a;mp8ty50LKOoKe)xNO#CDEA0~w0d;PcAnMrqI z|JTkxc&))V;BJvk((Xk5^9%@lbL Date: Thu, 2 Jan 2014 17:31:33 +0530 Subject: [PATCH 1120/1571] Removed a repeated test and changed tests to not use uncommons jar --- .../org/apache/spark/FileServerSuite.scala | 94 ++++++++++--------- 1 file changed, 50 insertions(+), 44 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 063b5fbab4a69..a15c3751c21be 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -17,15 +17,46 @@ package org.apache.spark +import java.io._ +import java.util.jar.{JarEntry, JarOutputStream} + +import SparkContext._ import com.google.common.io.Files import org.scalatest.FunSuite -import java.io.{File, PrintWriter, FileReader, BufferedReader} -import SparkContext._ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ - @transient var testJarFile: File = _ + @transient var testJarFile: String = _ + + + override def beforeAll() { + super.beforeAll() + val buffer = new Array[Byte](10240) + val tmpdir = new File(Files.createTempDir(), "test") + tmpdir.mkdir() + val tmpJarEntry = new File(tmpdir, "FileServerSuite2.txt") + val pw = new PrintWriter(tmpJarEntry) + pw.println("test String in the file named FileServerSuite2.txt") + pw.close() + // The ugliest code possible, was translated from java. + val tmpFile2 = new File(tmpdir, "test.jar") + val stream = new FileOutputStream(tmpFile2) + val jar = new JarOutputStream(stream, new java.util.jar.Manifest()) + val jarAdd = new JarEntry(tmpJarEntry.getName) + jarAdd.setTime(tmpJarEntry.lastModified) + jar.putNextEntry(jarAdd) + val in = new FileInputStream(tmpJarEntry) + var nRead = 0 + while (nRead <= 0) { + nRead = in.read(buffer, 0, buffer.length) + jar.write(buffer, 0, nRead) + } + in.close() + jar.close() + stream.close() + testJarFile = tmpFile2.getAbsolutePath + } override def beforeEach() { super.beforeEach() @@ -75,20 +106,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - ignore ("Dynamically adding JARS locally") { + test ("Dynamically adding JARS locally") { sc = new SparkContext("local[4]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) + sc.addJar(testJarFile) + val testData = Array((1, 1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } } test("Distributing files on a standalone cluster") { @@ -105,35 +131,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - ignore ("Dynamically adding JARS on a standalone cluster") { + test ("Dynamically adding JARS on a standalone cluster") { sc = new SparkContext("local-cluster[1,1,512]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) + sc.addJar(testJarFile) + val testData = Array((1,1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } } - ignore ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile.replace("file", "local")) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) - } } From b810a85cdddb247e1a104f4daad905b97222ad85 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 18:37:40 +0530 Subject: [PATCH 1121/1571] spark-shell -> bin/spark-shell --- README.md | 2 +- docs/index.md | 2 +- docs/mllib-guide.md | 2 +- docs/quick-start.md | 4 ++-- docs/running-on-yarn.md | 2 +- docs/scala-programming-guide.md | 10 +++++----- docs/spark-debugger.md | 2 +- docs/spark-standalone.md | 4 ++-- make-distribution.sh | 2 +- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/README.md b/README.md index 1550a8b5512d9..170e96485102e 100644 --- a/README.md +++ b/README.md @@ -19,7 +19,7 @@ which is packaged with it. To build Spark and its example programs, run: Once you've built Spark, the easiest way to start using it is the shell: - ./spark-shell + ./bin/spark-shell Or, for the Python API, the Python shell (`./pyspark`). diff --git a/docs/index.md b/docs/index.md index d3ac696d1e818..a5c179c19c250 100644 --- a/docs/index.md +++ b/docs/index.md @@ -34,7 +34,7 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using `local` for testing. -Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or +Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or Python interpreter (`./pyspark`). These are a great way to learn the framework. # Launching on a Cluster diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index c1ff9c417c353..95537ef18520f 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -87,7 +87,7 @@ svmAlg.optimizer.setNumIterations(200) val modelL1 = svmAlg.run(parsedData) {% endhighlight %} -Both of the code snippets above can be executed in `spark-shell` to generate a +Both of the code snippets above can be executed in `bin/spark-shell` to generate a classifier for the provided dataset. Available algorithms for binary classification: diff --git a/docs/quick-start.md b/docs/quick-start.md index bdf127d40467b..912f332236123 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -20,7 +20,7 @@ $ sbt/sbt assembly ## Basics Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running `./spark-shell` in the Spark directory. +Start the shell by running `./bin/spark-shell` in the Spark directory. Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: @@ -99,7 +99,7 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). +It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). # A Standalone App in Scala Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 00cad998810e8..be32034b3e5fb 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -112,7 +112,7 @@ For example: SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ - MASTER=yarn-client ./spark-shell + MASTER=yarn-client ./bin/spark-shell # Building Spark for Hadoop/YARN 2.2.x diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 1db255ca53457..16ae15046a14d 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -13,7 +13,7 @@ At a high level, every Spark application consists of a *driver program* that run A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums. -This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `spark-shell` script. We highly recommend doing that to follow along! +This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `bin/spark-shell` script. We highly recommend doing that to follow along! # Linking with Spark @@ -54,16 +54,16 @@ object for more advanced configuration. The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use {% highlight bash %} -$ MASTER=local[4] ./spark-shell +$ MASTER=local[4] ./bin/spark-shell {% endhighlight %} Or, to also add `code.jar` to its classpath, use: {% highlight bash %} -$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell +$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell {% endhighlight %} ### Master URLs @@ -95,7 +95,7 @@ If you want to run your application on a cluster, you will need to specify the t * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). * `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. -If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. +If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. # Resilient Distributed Datasets (RDDs) diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index d6315d97f4d59..11c51d5cde7c9 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP ### Loading the event log into the debugger -1. Run a Spark shell with `MASTER=host ./spark-shell`. +1. Run a Spark shell with `MASTER=host ./bin/spark-shell`. 2. Use `EventLogReader` to load the event log as follows: {% highlight scala %} spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log")) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c2ce23081a4c4..ec5ae5b4f7c10 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -143,9 +143,9 @@ constructor](scala-programming-guide.html#initializing-spark). To run an interactive Spark shell against the cluster, run the following command: - MASTER=spark://IP:PORT ./spark-shell + MASTER=spark://IP:PORT ./bin/spark-shell -Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will +Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. diff --git a/make-distribution.sh b/make-distribution.sh index 82638a23a089e..8765c7e620506 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -34,7 +34,7 @@ # 2) cd to deploy dir; ./bin/start-master.sh # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./bin/start-slave.sh 1 <> -# 5) MASTER="spark://my-master-ip:7077" ./spark-shell +# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell # # Figure out where the Spark framework is installed From 94b7a7fe37a4b1459bfdbece2a4162451d6a8ac2 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 18:41:21 +0530 Subject: [PATCH 1122/1571] run-example -> bin/run-example --- README.md | 4 ++-- bin/run-example | 2 +- bin/run-example2.cmd | 2 +- docs/bagel-programming-guide.md | 4 ++-- docs/index.md | 6 +++--- docs/java-programming-guide.md | 4 ++-- docs/running-on-yarn.md | 2 +- docs/scala-programming-guide.md | 4 ++-- docs/streaming-programming-guide.md | 4 ++-- .../apache/spark/streaming/examples/JavaKafkaWordCount.java | 2 +- .../apache/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../org/apache/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../org/apache/spark/streaming/examples/MQTTWordCount.scala | 4 ++-- .../apache/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../apache/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../streaming/examples/clickstream/PageViewStream.scala | 4 ++-- 19 files changed, 31 insertions(+), 31 deletions(-) diff --git a/README.md b/README.md index 170e96485102e..7154165ab17fb 100644 --- a/README.md +++ b/README.md @@ -24,9 +24,9 @@ Once you've built Spark, the easiest way to start using it is the shell: Or, for the Python API, the Python shell (`./pyspark`). Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./run-example `. For example: +To run one of them, use `./bin/run-example `. For example: - ./run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example org.apache.spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. diff --git a/bin/run-example b/bin/run-example index f2699c38a9a0a..a3976beb12141 100755 --- a/bin/run-example +++ b/bin/run-example @@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then fi if [ -z "$1" ]; then - echo "Usage: run-example []" >&2 + echo "Usage: bin/run-example []" >&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index 6861334cb07e5..d9c10545e7490 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [^] + echo Usage: bin/run-example ^ [^] goto exit :arg_given diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index de001e6c52f85..c4f1f6d6ade8c 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -157,8 +157,8 @@ trait Message[K] { # Where to Go from Here -Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.: +Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.: - ./run-example org.apache.spark.examples.bagel.WikipediaPageRank + ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank Each example program prints usage help when run without any arguments. diff --git a/docs/index.md b/docs/index.md index a5c179c19c250..f0990fff73251 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE # Running the Examples and Shell Spark comes with several sample programs in the `examples` directory. -To run one of the samples, use `./run-example ` in the top-level Spark directory -(the `run-example` script sets up the appropriate paths and launches that program). -For example, try `./run-example org.apache.spark.examples.SparkPi local`. +To run one of the samples, use `./bin/run-example ` in the top-level Spark directory +(the `bin/run-example` script sets up the appropriate paths and launches that program). +For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. Each example prints usage help when run with no parameters. Note that all of the sample programs take a `` parameter specifying the cluster URL diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 53085cc6719b0..07732fa1229f3 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -190,9 +190,9 @@ We hope to generate documentation with Java-style syntax in the future. Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the -`run-example` script included in Spark; for example: +`bin/run-example` script included in Spark; for example: - ./run-example org.apache.spark.examples.JavaWordCount + ./bin/run-example org.apache.spark.examples.JavaWordCount Each example program prints usage help when run without any arguments. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index be32034b3e5fb..e2b21f9cde05a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -107,7 +107,7 @@ For example: SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ - ./run-example org.apache.spark.examples.SparkPi yarn-client + ./bin/run-example org.apache.spark.examples.SparkPi yarn-client SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 16ae15046a14d..c1ef46a1cded7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -366,9 +366,9 @@ res2: Int = 10 # Where to Go from Here You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website. -In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example: +In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: - ./run-example org.apache.spark.examples.SparkPi + ./bin/run-example org.apache.spark.examples.SparkPi Each example program prints usage help when run without any arguments. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6f97db6a3e733..1c9ece6270781 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -245,7 +245,7 @@ $ nc -lk 9999 Then, in a different terminal, you can start NetworkWordCount by using {% highlight bash %} -$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 {% endhighlight %} This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen. @@ -283,7 +283,7 @@ Time: 1357008430000 ms -You can find more examples in `/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. +You can find more examples in `/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. # DStream Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec71c..75b588e4b85a9 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -41,7 +41,7 @@ * is the number of threads the kafka consumer should use * * Example: - * `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, + * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, * zoo03 my-consumer-group topic1,topic2 1` */ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 2402409e6ef87..3641517934b74 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -134,9 +134,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index bc8564b3ba080..61be1ce4b1726 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 570ba4c81a1d2..8dc8a3531a014 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ff332a0282129..ea138f55e872b 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -79,9 +79,9 @@ object MQTTPublisher { * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` */ object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index e2487dca5f2cc..ce8df8c5023ee 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index cb30c4edb30ce..e55d71edfcdc9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index e83ce78aa5f1b..2948aa7cc4944 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -62,9 +62,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index de70c50473558..1a40fdb9a3153 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -39,8 +39,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 8282cc9269c13..968b578487a5e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ object PageViewStream { def main(args: Array[String]) { From a3f90a2ecf14a01aa27fc95c133b1ff375583adb Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 18:50:12 +0530 Subject: [PATCH 1123/1571] pyspark -> bin/pyspark --- README.md | 2 +- docs/index.md | 2 +- docs/python-programming-guide.md | 28 ++++++++++++++-------------- docs/quick-start.md | 4 ++-- python/pyspark/shell.py | 2 +- 5 files changed, 19 insertions(+), 19 deletions(-) diff --git a/README.md b/README.md index 7154165ab17fb..f7ad26de5b8d9 100644 --- a/README.md +++ b/README.md @@ -21,7 +21,7 @@ Once you've built Spark, the easiest way to start using it is the shell: ./bin/spark-shell -Or, for the Python API, the Python shell (`./pyspark`). +Or, for the Python API, the Python shell (`./bin/pyspark`). Spark also comes with several sample programs in the `examples` directory. To run one of them, use `./bin/run-example `. For example: diff --git a/docs/index.md b/docs/index.md index f0990fff73251..86d574daaab4a 100644 --- a/docs/index.md +++ b/docs/index.md @@ -35,7 +35,7 @@ or `local` to run locally with one thread, or `local[N]` to run locally with N t `local` for testing. Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or -Python interpreter (`./pyspark`). These are a great way to learn the framework. +Python interpreter (`./bin/pyspark`). These are a great way to learn the framework. # Launching on a Cluster diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 96f93e24fe41d..dc187b3efec9b 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -47,7 +47,7 @@ PySpark will automatically ship these functions to workers, along with any objec Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. -In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell. +In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell. # Installing and Configuring PySpark @@ -60,17 +60,17 @@ By default, PySpark requires `python` to be available on the system `PATH` and u All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark applications should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. -The script automatically adds the `pyspark` package to the `PYTHONPATH`. +Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. +The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. # Interactive Use -The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: +The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: {% highlight bash %} $ sbt/sbt assembly -$ ./pyspark +$ ./bin/pyspark {% endhighlight %} The Python shell can be used explore data interactively and is a simple way to learn the API: @@ -82,35 +82,35 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `pyspark` shell creates SparkContext that runs applications locally on a single core. +By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core. To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. -For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): +For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} -$ MASTER=spark://IP:PORT ./pyspark +$ MASTER=spark://IP:PORT ./bin/pyspark {% endhighlight %} Or, to use four cores on the local machine: {% highlight bash %} -$ MASTER=local[4] ./pyspark +$ MASTER=local[4] ./bin/pyspark {% endhighlight %} ## IPython It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. -To do this, set the `IPYTHON` variable to `1` when running `pyspark`: +To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`: {% highlight bash %} -$ IPYTHON=1 ./pyspark +$ IPYTHON=1 ./bin/pyspark {% endhighlight %} Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support: {% highlight bash %} -$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark +$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark {% endhighlight %} IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable. @@ -118,7 +118,7 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env # Standalone Programs -PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`. +PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application. Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor: @@ -153,6 +153,6 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples). You can run them by passing the files to `pyspark`; e.g.: - ./pyspark python/examples/wordcount.py + ./bin/pyspark python/examples/wordcount.py Each program prints usage help when run without arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 912f332236123..2fa2bd718b244 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -277,11 +277,11 @@ We can pass Python functions to Spark, which are automatically serialized along For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleApp` is simple enough that we do not need to specify any code dependencies. -We can run this application using the `pyspark` script: +We can run this application using the `bin/pyspark` script: {% highlight python %} $ cd $SPARK_HOME -$ ./pyspark SimpleApp.py +$ ./bin/pyspark SimpleApp.py ... Lines with a: 46, Lines with b: 23 {% endhighlight python %} diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index ef07eb437baac..1602227a273e7 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -47,7 +47,7 @@ if add_files != None: print "Adding files: [%s]" % ", ".join(add_files) -# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, +# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, # which allows us to execute the user's PYTHONSTARTUP file: _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP') if _pythonstartup and os.path.isfile(_pythonstartup): From 66d501276b5a066bd9abaa4e284cfad557665948 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 2 Jan 2014 16:17:57 +0000 Subject: [PATCH 1124/1571] Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance --- .../spark/network/netty/FileClient.java | 5 +- .../netty/FileClientChannelInitializer.java | 2 +- .../spark/network/netty/FileServer.java | 8 +-- .../netty/FileServerChannelInitializer.java | 4 +- .../network/netty/FileServerHandler.java | 6 +-- .../spark/network/netty/PathResolver.java | 52 +++++++++---------- .../org/apache/spark/examples/JavaHdfsLR.java | 32 +++++++----- .../org/apache/spark/examples/JavaKMeans.java | 26 +++++++--- .../apache/spark/examples/JavaLogQuery.java | 23 ++++---- .../apache/spark/examples/JavaPageRank.java | 14 +++-- .../apache/spark/examples/JavaSparkPi.java | 11 ++-- .../org/apache/spark/examples/JavaTC.java | 19 ++++--- .../apache/spark/examples/JavaWordCount.java | 13 ++++- .../apache/spark/mllib/examples/JavaALS.java | 21 +++++--- .../spark/mllib/examples/JavaKMeans.java | 19 ++++--- .../apache/spark/mllib/examples/JavaLR.java | 22 +++++--- .../examples/JavaFlumeEventCount.java | 5 +- .../examples/JavaKafkaWordCount.java | 16 ++++-- .../examples/JavaNetworkWordCount.java | 15 ++++-- .../streaming/examples/JavaQueueStream.java | 11 ++-- 20 files changed, 203 insertions(+), 121 deletions(-) diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 46d61503bc562..d2d778b7567bc 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -31,7 +31,8 @@ class FileClient { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); + private final FileClientHandler handler; private Channel channel = null; private Bootstrap bootstrap = null; @@ -39,7 +40,7 @@ class FileClient { private final int connectTimeout; private final int sendTimeout = 60; // 1 min - public FileClient(FileClientHandler handler, int connectTimeout) { + FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; this.connectTimeout = connectTimeout; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index fb61be1c12d79..264cf97d0209f 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -25,7 +25,7 @@ class FileClientChannelInitializer extends ChannelInitializer { private final FileClientHandler fhandler; - public FileClientChannelInitializer(FileClientHandler handler) { + FileClientChannelInitializer(FileClientHandler handler) { fhandler = handler; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index aea75344594f5..c93425e2787dc 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -33,15 +33,14 @@ */ class FileServer { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName()); private EventLoopGroup bossGroup = null; private EventLoopGroup workerGroup = null; private ChannelFuture channelFuture = null; private int port = 0; - private Thread blockingThread = null; - public FileServer(PathResolver pResolver, int port) { + FileServer(PathResolver pResolver, int port) { InetSocketAddress addr = new InetSocketAddress(port); // Configure the server. @@ -70,7 +69,8 @@ public FileServer(PathResolver pResolver, int port) { * Start the file server asynchronously in a new thread. */ public void start() { - blockingThread = new Thread() { + Thread blockingThread = new Thread() { + @Override public void run() { try { channelFuture.channel().closeFuture().sync(); diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index 3f15ff898fc1e..46efec8f8d963 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -25,9 +25,9 @@ class FileServerChannelInitializer extends ChannelInitializer { - PathResolver pResolver; + private final PathResolver pResolver; - public FileServerChannelInitializer(PathResolver pResolver) { + FileServerChannelInitializer(PathResolver pResolver) { this.pResolver = pResolver; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index e2d9391b4cc93..3ac045f9444f2 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -31,11 +31,11 @@ class FileServerHandler extends SimpleChannelInboundHandler { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName()); private final PathResolver pResolver; - public FileServerHandler(PathResolver pResolver){ + FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; } @@ -61,7 +61,7 @@ public void channelRead0(ChannelHandlerContext ctx, String blockIdString) { ctx.flush(); return; } - int len = new Long(length).intValue(); + int len = (int) length; ctx.write((new FileHeader(len, blockId)).buffer()); try { ctx.write(new DefaultFileRegion(new FileInputStream(file) diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java index 9f7ced44cf93d..7ad8d03efbadc 100755 --- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java +++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java @@ -1,26 +1,26 @@ -/* - * 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.network.netty; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; - -public interface PathResolver { - /** Get the file segment in which the given block resides. */ - public FileSegment getBlockLocation(BlockId blockId); -} +/* + * 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.network.netty; + +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.FileSegment; + +public interface PathResolver { + /** Get the file segment in which the given block resides. */ + FileSegment getBlockLocation(BlockId blockId); +} diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c5df..9f0e3412a62e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -24,19 +24,22 @@ import java.io.Serializable; import java.util.Arrays; -import java.util.StringTokenizer; import java.util.Random; +import java.util.regex.Pattern; /** * Logistic regression based classification. */ -public class JavaHdfsLR { +public final class JavaHdfsLR { - static int D = 10; // Number of dimensions - static Random rand = new Random(42); + private static final int D = 10; // Number of dimensions + private static final Random rand = new Random(42); + + private JavaHdfsLR() { + } static class DataPoint implements Serializable { - public DataPoint(double[] x, double y) { + DataPoint(double[] x, double y) { this.x = x; this.y = y; } @@ -46,20 +49,22 @@ public DataPoint(double[] x, double y) { } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public DataPoint call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - double y = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double y = Double.parseDouble(tok[0]); double[] x = new double[D]; - int i = 0; - while (i < D) { - x[i] = Double.parseDouble(tok.nextToken()); - i += 1; + for (int i = 0; i < D; i++) { + x[i] = Double.parseDouble(tok[i+1]); } return new DataPoint(x, y); } } static class VectorSum extends Function2 { + @Override public double[] call(double[] a, double[] b) { double[] result = new double[D]; for (int j = 0; j < D; j++) { @@ -70,12 +75,13 @@ public double[] call(double[] a, double[] b) { } static class ComputeGradient extends Function { - double[] weights; + private final double[] weights; - public ComputeGradient(double[] weights) { + ComputeGradient(double[] weights) { this.weights = weights; } + @Override public double[] call(DataPoint p) { double[] gradient = new double[D]; for (int i = 0; i < D; i++) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eaefd6..1671d0cdc8dfb 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -27,19 +27,27 @@ import java.util.List; import java.util.Map; +import java.util.regex.Pattern; /** * K-means clustering using Java API. */ -public class JavaKMeans { +public final class JavaKMeans { + + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKMeans() { + } /** Parses numbers split by whitespace to a vector */ static Vector parseVector(String line) { - String[] splits = line.split(" "); + String[] splits = SPACE.split(line); double[] data = new double[splits.length]; int i = 0; - for (String s : splits) - data[i] = Double.parseDouble(splits[i++]); + for (String s : splits) { + data[i] = Double.parseDouble(s); + i++; + } return new Vector(data); } @@ -82,7 +90,7 @@ public static void main(String[] args) throws Exception { JavaRDD data = sc.textFile(path).map( new Function() { @Override - public Vector call(String line) throws Exception { + public Vector call(String line) { return parseVector(line); } } @@ -96,7 +104,7 @@ public Vector call(String line) throws Exception { JavaPairRDD closest = data.map( new PairFunction() { @Override - public Tuple2 call(Vector vector) throws Exception { + public Tuple2 call(Vector vector) { return new Tuple2( closestPoint(vector, centroids), vector); } @@ -107,7 +115,8 @@ public Tuple2 call(Vector vector) throws Exception { JavaPairRDD> pointsGroup = closest.groupByKey(); Map newCentroids = pointsGroup.mapValues( new Function, Vector>() { - public Vector call(List ps) throws Exception { + @Override + public Vector call(List ps) { return average(ps); } }).collectAsMap(); @@ -122,8 +131,9 @@ public Vector call(List ps) throws Exception { } while (tempDist > convergeDist); System.out.println("Final centers:"); - for (Vector c : centroids) + for (Vector c : centroids) { System.out.println(c); + } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 407cd7ccfaee0..1ce53fe403cc2 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -35,9 +35,9 @@ /** * Executes a roll up-style query against Apache logs. */ -public class JavaLogQuery { +public final class JavaLogQuery { - public static List exampleApacheLogs = Lists.newArrayList( + public static final List exampleApacheLogs = Lists.newArrayList( "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + @@ -51,14 +51,17 @@ public class JavaLogQuery { "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + "0 73.23.2.15 images.com 1358492557 - Whatup"); - public static Pattern apacheLogRegex = Pattern.compile( + public static final Pattern apacheLogRegex = Pattern.compile( "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + private JavaLogQuery() { + } + /** Tracks the total query count and number of aggregate bytes for a particular group. */ public static class Stats implements Serializable { - private int count; - private int numBytes; + private final int count; + private final int numBytes; public Stats(int count, int numBytes) { this.count = count; @@ -92,12 +95,12 @@ public static Stats extractStats(String line) { if (m.find()) { int bytes = Integer.parseInt(m.group(7)); return new Stats(1, bytes); - } - else + } else { return new Stats(1, 0); + } } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { if (args.length == 0) { System.err.println("Usage: JavaLogQuery [logFile]"); System.exit(1); @@ -110,14 +113,14 @@ public static void main(String[] args) throws Exception { JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { @Override - public Tuple2, Stats> call(String s) throws Exception { + public Tuple2, Stats> call(String s) { return new Tuple2, Stats>(extractKey(s), extractStats(s)); } }); JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { @Override - public Stats call(Stats stats, Stats stats2) throws Exception { + public Stats call(Stats stats, Stats stats2) { return stats.merge(stats2); } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 89aed8f279654..447ba93bd6bdd 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.ArrayList; +import java.util.regex.Pattern; /** * Computes the PageRank of URLs from an input file. Input file should @@ -38,7 +39,12 @@ * ... * where URL and their neighbors are separated by space(s). */ -public class JavaPageRank { +public final class JavaPageRank { + private static final Pattern SPACES = Pattern.compile("\\s+"); + + private JavaPageRank() { + } + private static class Sum extends Function2 { @Override public Double call(Double a, Double b) { @@ -66,7 +72,7 @@ public static void main(String[] args) throws Exception { JavaPairRDD> links = lines.map(new PairFunction() { @Override public Tuple2 call(String s) { - String[] parts = s.split("\\s+"); + String[] parts = SPACES.split(s); return new Tuple2(parts[0], parts[1]); } }).distinct().groupByKey().cache(); @@ -74,7 +80,7 @@ public Tuple2 call(String s) { // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) throws Exception { + public Double call(List rs) { return 1.0; } }); @@ -97,7 +103,7 @@ public Iterable> call(Tuple2, Double> s) { // Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(new Sum()).mapValues(new Function() { @Override - public Double call(Double sum) throws Exception { + public Double call(Double sum) { return 0.15 + sum * 0.85; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5af5..d2a2a1db7c77f 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -26,8 +26,10 @@ import java.util.List; /** Computes an approximation to pi */ -public class JavaSparkPi { +public final class JavaSparkPi { + private JavaSparkPi() { + } public static void main(String[] args) throws Exception { if (args.length == 0) { @@ -41,21 +43,22 @@ public static void main(String[] args) throws Exception { int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; List l = new ArrayList(n); - for (int i = 0; i < n; i++) + for (int i = 0; i < n; i++) { l.add(i); + } JavaRDD dataSet = jsc.parallelize(l, slices); int count = dataSet.map(new Function() { @Override - public Integer call(Integer integer) throws Exception { + public Integer call(Integer integer) { double x = Math.random() * 2 - 1; double y = Math.random() * 2 - 1; return (x * x + y * y < 1) ? 1 : 0; } }).reduce(new Function2() { @Override - public Integer call(Integer integer, Integer integer2) throws Exception { + public Integer call(Integer integer, Integer integer2) { return integer + integer2; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b776d1..e61b9c4f0ea33 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -31,11 +31,14 @@ /** * Transitive closure on a graph, implemented in Java. */ -public class JavaTC { +public final class JavaTC { - static int numEdges = 200; - static int numVertices = 100; - static Random rand = new Random(42); + private static final int numEdges = 200; + private static final int numVertices = 100; + private static final Random rand = new Random(42); + + private JavaTC() { + } static List> generateGraph() { Set> edges = new HashSet>(numEdges); @@ -43,15 +46,18 @@ static List> generateGraph() { int from = rand.nextInt(numVertices); int to = rand.nextInt(numVertices); Tuple2 e = new Tuple2(from, to); - if (from != to) edges.add(e); + if (from != to) { + edges.add(e); + } } return new ArrayList>(edges); } static class ProjectFn extends PairFunction>, Integer, Integer> { - static ProjectFn INSTANCE = new ProjectFn(); + static final ProjectFn INSTANCE = new ProjectFn(); + @Override public Tuple2 call(Tuple2> triple) { return new Tuple2(triple._2()._2(), triple._2()._1()); } @@ -76,6 +82,7 @@ public static void main(String[] args) { // Because join() joins on keys, the edges are stored in reversed order. JavaPairRDD edges = tc.map( new PairFunction, Integer, Integer>() { + @Override public Tuple2 call(Tuple2 e) { return new Tuple2(e._2(), e._1()); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index bd6383e13df7a..ed4e9b49d0a0a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -27,8 +27,14 @@ import java.util.Arrays; import java.util.List; +import java.util.regex.Pattern; + +public final class JavaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaWordCount() { + } -public class JavaWordCount { public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaWordCount "); @@ -40,18 +46,21 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { + @Override public Iterable call(String s) { - return Arrays.asList(s.split(" ")); + return Arrays.asList(SPACE.split(s)); } }); JavaPairRDD ones = words.map(new PairFunction() { + @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); JavaPairRDD counts = ones.reduceByKey(new Function2() { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 45a0d237da314..b33e6481475a5 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -26,28 +26,35 @@ import org.apache.spark.mllib.recommendation.Rating; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; import scala.Tuple2; /** * Example using MLLib ALS from Java. */ -public class JavaALS { +public final class JavaALS { + + private JavaALS() { + } static class ParseRating extends Function { + private static final Pattern COMMA = Pattern.compile(","); + + @Override public Rating call(String line) { - StringTokenizer tok = new StringTokenizer(line, ","); - int x = Integer.parseInt(tok.nextToken()); - int y = Integer.parseInt(tok.nextToken()); - double rating = Double.parseDouble(tok.nextToken()); + String[] tok = COMMA.split(line); + int x = Integer.parseInt(tok[0]); + int y = Integer.parseInt(tok[1]); + double rating = Double.parseDouble(tok[2]); return new Rating(x, y, rating); } } static class FeaturesToString extends Function, String> { + @Override public String call(Tuple2 element) { - return element._1().toString() + "," + Arrays.toString(element._2()); + return element._1() + "," + Arrays.toString(element._2()); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9fee..a9db04d5254fa 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -25,20 +25,25 @@ import org.apache.spark.mllib.clustering.KMeansModel; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Example using MLLib KMeans from Java. */ -public class JavaKMeans { +public final class JavaKMeans { + + private JavaKMeans() { + } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public double[] call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - int numTokens = tok.countTokens(); - double[] point = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - point[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double[] point = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + point[i] = Double.parseDouble(tok[i]); } return point; } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6ba6..56341315bfc34 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -27,22 +27,28 @@ import org.apache.spark.mllib.regression.LabeledPoint; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Logistic regression based classification using ML Lib. */ -public class JavaLR { +public final class JavaLR { + + private JavaLR() { + } static class ParsePoint extends Function { + private static final Pattern COMMA = Pattern.compile(","); + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public LabeledPoint call(String line) { - String[] parts = line.split(","); + String[] parts = COMMA.split(line); double y = Double.parseDouble(parts[0]); - StringTokenizer tok = new StringTokenizer(parts[1], " "); - int numTokens = tok.countTokens(); - double[] x = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - x[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(parts[1]); + double[] x = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + x[i] = Double.parseDouble(tok[i]); } return new LabeledPoint(y, x); } diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f39c..bd0bbb56ffeda 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -36,7 +36,10 @@ * creates a server and listens for flume events. * is the port the Flume receiver will listen on. */ -public class JavaFlumeEventCount { +public final class JavaFlumeEventCount { + private JavaFlumeEventCount() { + } + public static void main(String[] args) { if (args.length != 3) { System.err.println("Usage: JavaFlumeEventCount "); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec71c..17eb87190813b 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -19,6 +19,7 @@ import java.util.Map; import java.util.HashMap; +import java.util.regex.Pattern; import com.google.common.collect.Lists; import org.apache.spark.api.java.function.FlatMapFunction; @@ -45,7 +46,12 @@ * zoo03 my-consumer-group topic1,topic2 1` */ -public class JavaKafkaWordCount { +public final class JavaKafkaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKafkaWordCount() { + } + public static void main(String[] args) { if (args.length < 5) { System.err.println("Usage: KafkaWordCount "); @@ -67,7 +73,7 @@ public static void main(String[] args) { JavaDStream lines = messages.map(new Function, String>() { @Override - public String call(Tuple2 tuple2) throws Exception { + public String call(Tuple2 tuple2) { return tuple2._2(); } }); @@ -75,19 +81,19 @@ public String call(Tuple2 tuple2) throws Exception { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199be57..fb090cc2621dc 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -27,6 +27,8 @@ import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import java.util.regex.Pattern; + /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: NetworkWordCount @@ -38,7 +40,12 @@ * and then run the example * `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` */ -public class JavaNetworkWordCount { +public final class JavaNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaNetworkWordCount() { + } + public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: NetworkWordCount \n" + @@ -56,18 +63,18 @@ public static void main(String[] args) { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1bba..6be967237c828 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -31,8 +31,11 @@ import java.util.List; import java.util.Queue; -public class JavaQueueStream { - public static void main(String[] args) throws InterruptedException { +public final class JavaQueueStream { + private JavaQueueStream() { + } + + public static void main(String[] args) throws Exception { if (args.length < 1) { System.err.println("Usage: JavaQueueStream "); System.exit(1); @@ -62,14 +65,14 @@ public static void main(String[] args) throws InterruptedException { JavaPairDStream mappedStream = inputStream.map( new PairFunction() { @Override - public Tuple2 call(Integer i) throws Exception { + public Tuple2 call(Integer i) { return new Tuple2(i % 10, 1); } }); JavaPairDStream reducedStream = mappedStream.reduceByKey( new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); From 59e8009b8d5e51b6f776720de8c9ecb09e1072dc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 18:54:08 +0530 Subject: [PATCH 1125/1571] a few left over document change --- docs/spark-standalone.md | 2 +- ec2/spark_ec2.py | 2 +- make-distribution.sh | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ec5ae5b4f7c10..baa0a062f704c 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -20,7 +20,7 @@ then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all You can start a standalone master server by executing: - ./bin/start-master.sh + ./sbin/start-master.sh Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it, or pass as the "master" argument to `SparkContext`. You can also find this URL on diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index a2b0e7e7f4748..d82a1e1490cc0 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -436,7 +436,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) - ssh(master, opts, "/root/spark/bin/start-all.sh") + ssh(master, opts, "/root/spark/sbin/start-all.sh") def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") diff --git a/make-distribution.sh b/make-distribution.sh index 8765c7e620506..0463d14762e79 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -31,9 +31,9 @@ # # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host -# 2) cd to deploy dir; ./bin/start-master.sh +# 2) cd to deploy dir; ./sbin/start-master.sh # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. -# 4) ./bin/start-slave.sh 1 <> +# 4) ./sbin/start-slave.sh 1 <> # 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell # From 5a3c00c9581f81522a32c0b5d21ba81498c2d9c3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 30 Dec 2013 12:23:18 -0800 Subject: [PATCH 1126/1571] Removed redundant TaskSetManager.error() function. This function was leftover from a while ago, and now just passes all calls through to the abort() function, so this commit deletes it. --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d94b706854f3a..c4ac8337c568e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -328,7 +328,7 @@ private[spark] class TaskSchedulerImpl( // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { - manager.error(message) + manager.abort(message) } catch { case e: Exception => logError("Exception in error callback", e) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 67ad99a4d72ee..6dd1469d8f801 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -548,11 +548,6 @@ private[spark] class TaskSetManager( } } - def error(message: String) { - // Save the error message - abort("Error: " + message) - } - def abort(message: String) { // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message) From a1b438d94de10506dc7dcac54eb331ee2c0479aa Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 2 Jan 2014 12:34:46 -0800 Subject: [PATCH 1127/1571] Remove erroneous FAILED state for killed tasks. Currently, when tasks are killed, the Executor first sends a status update for the task with a "KILLED" state, and then sends a second status update with a "FAILED" state saying that the task failed due to an exception. The second FAILED state is misleading/unncessary, and occurs due to a NonLocalReturnControl Exception that gets thrown due to the way we kill tasks. This commit eliminates that problem. --- .../org/apache/spark/executor/Executor.scala | 24 ++++++++++--------- .../spark/scheduler/TaskSchedulerImpl.scala | 3 ++- 2 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 3c92c205ea978..e51d274d33874 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -141,11 +141,6 @@ private[spark] class Executor( val tr = runningTasks.get(taskId) if (tr != null) { tr.kill() - // We remove the task also in the finally block in TaskRunner.run. - // The reason we need to remove it here is because killTask might be called before the task - // is even launched, and never reaching that finally block. ConcurrentHashMap's remove is - // idempotent. - runningTasks.remove(taskId) } } @@ -167,6 +162,8 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { + object TaskKilledException extends Exception + @volatile private var killed = false @volatile private var task: Task[Any] = _ @@ -200,9 +197,11 @@ private[spark] class Executor( // If this task has been killed before we deserialized it, let's quit now. Otherwise, // continue executing the task. if (killed) { - logInfo("Executor killed task " + taskId) - execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - return + // Throw an exception rather than returning, because returning within a try{} block + // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl + // exception will be caught by the catch block, leading to an incorrect ExceptionFailure + // for the task. + throw TaskKilledException } attemptedTask = Some(task) @@ -216,9 +215,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - logInfo("Executor killed task " + taskId) - execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - return + throw TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -260,6 +257,11 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } + case TaskKilledException => { + logInfo("Executor killed task " + taskId) + execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) + } + case t: Throwable => { val serviceTime = (System.currentTimeMillis() - taskStart).toInt val metrics = attemptedTask.flatMap(t => t.metrics) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d94b706854f3a..1b67332ff8f9f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -285,7 +285,8 @@ private[spark] class TaskSchedulerImpl( } } case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") + logInfo("Ignoring update with state %s from TID %s because its task set is gone" + .format(state, tid)) } } catch { case e: Exception => logError("Exception in statusUpdate", e) From 8831923219faf1599957056dd5f406a22d1f1128 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 2 Jan 2014 13:52:35 -0800 Subject: [PATCH 1128/1571] TempBlockId takes UUID and is explicitly non-serializable --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 7 ++++--- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index bcc3101485703..301d784b350a3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.util.UUID + /** * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different @@ -68,8 +70,8 @@ private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends B def name = "input-" + streamId + "-" + uniqueId } -/** Block associated with temporary data managed as blocks. */ -private[spark] case class TempBlockId(id: String) extends BlockId { +/** Id associated with temporary data managed as blocks. Not serializable. */ +private[spark] case class TempBlockId(id: UUID) extends BlockId { def name = "temp_" + id } @@ -85,7 +87,6 @@ private[spark] object BlockId { val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r - val TEMP = "temp_(.*)".r val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 32da458a6f570..e25bc90c4f495 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -92,9 +92,9 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD /** Produces a unique block id and File suitable for intermediate results. */ def createTempBlock(): (TempBlockId, File) = { - var blockId = new TempBlockId(UUID.randomUUID().toString) + var blockId = new TempBlockId(UUID.randomUUID()) while (getFile(blockId).exists()) { - blockId = new TempBlockId(UUID.randomUUID().toString) + blockId = new TempBlockId(UUID.randomUUID()) } (blockId, getFile(blockId)) } From c6de982be69cd50e66375cfea3d6c3267a01583b Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 2 Jan 2014 16:50:35 -0600 Subject: [PATCH 1129/1571] Fix yarn build after sparkConf changes --- .../spark/deploy/yarn/ApplicationMaster.scala | 52 ++++++------------ .../org/apache/spark/deploy/yarn/Client.scala | 10 ++-- .../spark/deploy/yarn/ClientArguments.scala | 3 +- .../spark/deploy/yarn/WorkerLauncher.scala | 8 ++- .../deploy/yarn/YarnAllocationHandler.scala | 27 +++++---- .../spark/deploy/yarn/ApplicationMaster.scala | 55 ++++++------------- .../org/apache/spark/deploy/yarn/Client.scala | 10 ++-- .../spark/deploy/yarn/ClientArguments.scala | 3 +- .../spark/deploy/yarn/WorkerLauncher.scala | 11 ++-- .../deploy/yarn/YarnAllocationHandler.scala | 25 ++++++--- 10 files changed, 95 insertions(+), 109 deletions(-) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 91e35e2d34e2e..7c32e0ab9bcb5 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils @@ -60,14 +60,19 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var isLastAMRetry: Boolean = true private var amClient: AMRMClient[ContainerRequest] = _ + private val sparkConf = new SparkConf() // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3).toString()).toInt + private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", + math.max(args.numWorkers * 2, 3)) def run() { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. - conf.set("spark.local.dir", getLocalDirs()) + System.setProperty("spark.local.dir", getLocalDirs()) + + // set the web ui port to be ephemeral for yarn so we don't conflict with + // other spark processes running on the same box + System.setProperty("spark.ui.port", "0") // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -89,8 +94,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. - waitForSparkMaster() - waitForSparkContextInitialized() // Do this after Spark master is up and SparkContext is created so that we can register UI Url. @@ -134,30 +137,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) } - private def waitForSparkMaster() { - logInfo("Waiting for Spark driver to be reachable.") - var driverUp = false - var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt - while (!driverUp && tries < numTries) { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") - try { - val socket = new Socket(driverHost, driverPort.toInt) - socket.close() - logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) - driverUp = true - } catch { - case e: Exception => { - logWarning("Failed to connect to driver at %s:%s, retrying ...". - format(driverHost, driverPort)) - Thread.sleep(100) - tries = tries + 1 - } - } - } - } - private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName( @@ -199,7 +178,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -215,7 +194,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e amClient, appAttemptId, args, - sparkContext.preferredNodeLocationData) + sparkContext.preferredNodeLocationData, + sparkContext.getConf) } else { logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". format(numTries * waitTime, maxNumTries)) @@ -223,7 +203,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, amClient, appAttemptId, - args) + args, + sparkContext.getConf) } } } finally { @@ -265,7 +246,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -343,7 +325,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1bba6a5ae4ac8..a75066888c22f 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -57,6 +57,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() + private val sparkConf = new SparkConf + // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) @@ -244,7 +246,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -437,7 +439,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def monitorApplication(appId: ApplicationId): Boolean = { - val interval = new SparkConf().getOrElse("spark.yarn.report.interval", "1000").toLong + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) while (true) { Thread.sleep(interval) @@ -501,7 +503,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 1a9bb97b3ecec..7aac2328dad60 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.{ArrayBuffer, HashMap} +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} import org.apache.spark.util.IntParam import org.apache.spark.util.MemoryParam @@ -35,7 +36,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = new SparkConf().get("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index f7d73f0d83f47..32c774c90e6de 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo @@ -48,7 +48,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var amClient: AMRMClient[ContainerRequest] = _ - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + conf = new SparkConf)._1 var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. @@ -157,7 +158,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte amClient, appAttemptId, args, - preferredNodeLocationData) + preferredNodeLocationData, + new SparkConf) logInfo("Allocating " + args.numWorkers + " workers.") // Wait until all containers have finished diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index abc3447746f9e..85ab08ef34db4 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -64,7 +64,8 @@ private[yarn] class YarnAllocationHandler( val workerMemory: Int, val workerCores: Int, val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int]) + val preferredRackToCount: Map[String, Int], + val sparkConf: SparkConf) extends Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set @@ -254,8 +255,8 @@ private[yarn] class YarnAllocationHandler( } else { val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), - conf.get("spark.driver.port"), + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) @@ -565,7 +566,8 @@ object YarnAllocationHandler { conf: Configuration, amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments + args: ApplicationMasterArguments, + sparkConf: SparkConf ): YarnAllocationHandler = { new YarnAllocationHandler( conf, @@ -575,7 +577,8 @@ object YarnAllocationHandler { args.workerMemory, args.workerCores, Map[String, Int](), - Map[String, Int]()) + Map[String, Int](), + sparkConf) } def newAllocator( @@ -584,7 +587,8 @@ object YarnAllocationHandler { appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, map: collection.Map[String, - collection.Set[SplitInfo]] + collection.Set[SplitInfo]], + sparkConf: SparkConf ): YarnAllocationHandler = { val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map) new YarnAllocationHandler( @@ -595,7 +599,8 @@ object YarnAllocationHandler { args.workerMemory, args.workerCores, hostToSplitCount, - rackToSplitCount) + rackToSplitCount, + sparkConf) } def newAllocator( @@ -605,7 +610,8 @@ object YarnAllocationHandler { maxWorkers: Int, workerMemory: Int, workerCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]] + map: collection.Map[String, collection.Set[SplitInfo]], + sparkConf: SparkConf ): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) new YarnAllocationHandler( @@ -616,7 +622,8 @@ object YarnAllocationHandler { workerMemory, workerCores, hostToCount, - rackToCount) + rackToCount, + sparkConf) } // A simple method to copy the split info map. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index dc9228180f60a..7cf120d3eb8a4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -36,10 +36,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils - class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) @@ -57,14 +56,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - // default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3).toString()).toInt + + private val sparkConf = new SparkConf() + // Default to numWorkers * 2, with minimum of 3 + private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", + math.max(args.numWorkers * 2, 3)) def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. - conf.set("spark.local.dir", getLocalDirs()) + System.setProperty("spark.local.dir", getLocalDirs()) + + // set the web ui port to be ephemeral for yarn so we don't conflict with + // other spark processes running on the same box + System.setProperty("spark.ui.port", "0") // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -99,8 +104,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. - waitForSparkMaster() - waitForSparkContextInitialized() // Do this after spark master is up and SparkContext is created so that we can register UI Url @@ -161,30 +164,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e resourceManager.registerApplicationMaster(appMasterRequest) } - private def waitForSparkMaster() { - logInfo("Waiting for spark driver to be reachable.") - var driverUp = false - var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt - while(!driverUp && tries < numTries) { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") - try { - val socket = new Socket(driverHost, driverPort.toInt) - socket.close() - logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) - driverUp = true - } catch { - case e: Exception => { - logWarning("Failed to connect to driver at %s:%s, retrying ...". - format(driverHost, driverPort)) - Thread.sleep(100) - tries = tries + 1 - } - } - } - } - private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName( @@ -226,7 +205,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -242,7 +221,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e resourceManager, appAttemptId, args, - sparkContext.preferredNodeLocationData) + sparkContext.preferredNodeLocationData, + sparkContext.getConf) } else { logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". format(count * waitTime, numTries)) @@ -250,7 +230,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, resourceManager, appAttemptId, - args) + args, + sparkContext.getConf) } } } finally { @@ -294,7 +275,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -377,7 +358,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 595a7ee8c3d83..2bd047c97adba 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -54,6 +54,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() + private val sparkConf = new SparkConf // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) @@ -230,7 +231,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -422,7 +423,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def monitorApplication(appId: ApplicationId): Boolean = { - val interval = new SparkConf().getOrElse("spark.yarn.report.interval", "1000").toLong + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) while (true) { Thread.sleep(interval) @@ -485,8 +486,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") - .toBoolean + val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false").toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index e9e46a193b145..9075ca71e7efc 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.yarn +import org.apache.spark.SparkConf import org.apache.spark.util.MemoryParam import org.apache.spark.util.IntParam import collection.mutable.{ArrayBuffer, HashMap} @@ -33,7 +34,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = new SparkConf().get("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index c1e79cbe665a5..28259de68fffd 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo @@ -47,7 +47,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var yarnAllocator: YarnAllocationHandler = null private var driverClosed:Boolean = false - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + conf = new SparkConf)._1 var actor: ActorRef = null // This actor just working as a monitor to watch on Driver Actor. @@ -175,9 +176,11 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private def allocateWorkers() { // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map() + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = + scala.collection.immutable.Map() - yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData) + yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, + args, preferredNodeLocationData, new SparkConf) logInfo("Allocating " + args.numWorkers + " workers.") // Wait until all containers have finished diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5966a0f757757..c8af653b3ffab 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -62,7 +62,8 @@ private[yarn] class YarnAllocationHandler( val workerMemory: Int, val workerCores: Int, val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int]) + val preferredRackToCount: Map[String, Int], + val sparkConf: SparkConf) extends Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set @@ -239,7 +240,7 @@ private[yarn] class YarnAllocationHandler( // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) @@ -552,7 +553,8 @@ object YarnAllocationHandler { conf: Configuration, resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments): YarnAllocationHandler = { + args: ApplicationMasterArguments, + sparkConf: SparkConf): YarnAllocationHandler = { new YarnAllocationHandler( conf, @@ -562,7 +564,8 @@ object YarnAllocationHandler { args.workerMemory, args.workerCores, Map[String, Int](), - Map[String, Int]()) + Map[String, Int](), + sparkConf) } def newAllocator( @@ -571,7 +574,8 @@ object YarnAllocationHandler { appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, map: collection.Map[String, - collection.Set[SplitInfo]]): YarnAllocationHandler = { + collection.Set[SplitInfo]], + sparkConf: SparkConf): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) new YarnAllocationHandler( @@ -582,7 +586,8 @@ object YarnAllocationHandler { args.workerMemory, args.workerCores, hostToCount, - rackToCount) + rackToCount, + sparkConf) } def newAllocator( @@ -592,7 +597,8 @@ object YarnAllocationHandler { maxWorkers: Int, workerMemory: Int, workerCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + map: collection.Map[String, collection.Set[SplitInfo]], + sparkConf: SparkConf): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) @@ -604,7 +610,8 @@ object YarnAllocationHandler { workerMemory, workerCores, hostToCount, - rackToCount) + rackToCount, + sparkConf) } // A simple method to copy the split info map. From fced7885cb6cd09761578f960540d739bcbb465a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 2 Jan 2014 17:11:16 -0600 Subject: [PATCH 1130/1571] fix yarn-client --- .../org/apache/spark/deploy/yarn/WorkerLauncher.scala | 9 +++++---- .../org/apache/spark/deploy/yarn/WorkerLauncher.scala | 9 +++++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 32c774c90e6de..99b824e1295a7 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -47,9 +47,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var driverClosed:Boolean = false private var amClient: AMRMClient[ContainerRequest] = _ + private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = new SparkConf)._1 + conf = sparkConf)._1 var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. @@ -137,8 +138,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - conf.set("spark.driver.host", driverHost) - conf.set("spark.driver.port", driverPort.toString) + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -159,7 +160,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte appAttemptId, args, preferredNodeLocationData, - new SparkConf) + sparkConf) logInfo("Allocating " + args.numWorkers + " workers.") // Wait until all containers have finished diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 28259de68fffd..a8de89c670811 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -46,9 +46,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var yarnAllocator: YarnAllocationHandler = null private var driverClosed:Boolean = false + private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = new SparkConf)._1 + conf = sparkConf)._1 var actor: ActorRef = null // This actor just working as a monitor to watch on Driver Actor. @@ -163,8 +164,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - conf.set("spark.driver.host", driverHost) - conf.set("spark.driver.port", driverPort.toString) + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -180,7 +181,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte scala.collection.immutable.Map() yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, - args, preferredNodeLocationData, new SparkConf) + args, preferredNodeLocationData, sparkConf) logInfo("Allocating " + args.numWorkers + " workers.") // Wait until all containers have finished From c189c8362caeaa7a0f46af1c8e0d8d37fd171d7b Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 2 Jan 2014 15:22:20 -0800 Subject: [PATCH 1131/1571] Added Scala and Python examples for mllib --- docs/mllib-guide.md | 313 ++++++++++++++++++++++++++++++++++++-------- 1 file changed, 261 insertions(+), 52 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index c1ff9c417c353..e9d3785427b56 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -39,56 +39,9 @@ underlying gradient descent primitive (described parameter (*regParam*) along with various parameters associated with gradient descent (*stepSize*, *numIterations*, *miniBatchFraction*). -The following code snippet illustrates how to load a sample dataset, execute a -training algorithm on this training data using a static method in the algorithm -object, and make predictions with the resulting model to compute the training -error. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.regression.LabeledPoint - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_svm_data.txt") -val parsedData = data.map { line => - val parts = line.split(' ') - LabeledPoint(parts(0).toDouble, parts.tail.map(x => x.toDouble).toArray) -} - -// Run training algorithm -val numIterations = 20 -val model = SVMWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("trainError = " + trainErr) -{% endhighlight %} - -The `SVMWithSGD.train()` method by default performs L2 regularization with the -regularization parameter set to 1.0. If we want to configure this algorithm, we -can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in -this way as well. For example, the following code produces an L1 regularized -variant of SVMs with regularization parameter set to 0.1, and runs the training -algorithm for 200 iterations. -{% highlight scala %} -import org.apache.spark.mllib.optimization.L1Updater -val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) -{% endhighlight %} -Both of the code snippets above can be executed in `spark-shell` to generate a -classifier for the provided dataset. Available algorithms for binary classification: @@ -121,14 +74,14 @@ of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression models are trained for each cluster). MLlib supports -[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, arguably -the most commonly used clustering approach that clusters the data points into -*k* clusters. The MLlib implementation includes a parallelized +[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of +the most commonly used clustering algorithms that clusters the data points into +predfined number of clusters. The MLlib implementation includes a parallelized variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). The implementation in MLlib has the following parameters: -* *k* is the number of clusters. +* *k* is the number of desired clusters. * *maxIterations* is the maximum number of iterations to run. * *initializationMode* specifies either random initialization or initialization via k-means\|\|. @@ -169,7 +122,7 @@ the entries in the user-item matrix as *explicit* preferences given by the user It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with such data is taken from -[Collaborative Filtering for Implicit Feedback Datasets](http://research.yahoo.com/pub/2433). +[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf). Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as a combination of binary preferences and *confidence values*. The ratings are then related to the level of confidence in observed user preferences, rather than explicit ratings given to items. @@ -210,3 +163,259 @@ at each iteration. Available algorithms for gradient descent: * [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) + +# Using MLLib in Scala + +Following code snippets can be executed in `spark-shell`. + +## Binary Classification + +The following code snippet illustrates how to load a sample dataset, execute a +training algorithm on this training data using a static method in the algorithm +object, and make predictions with the resulting model to compute the training +error. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.classification.SVMWithSGD +import org.apache.spark.mllib.regression.LabeledPoint + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_svm_data.txt") +val parsedData = data.map { line => + val parts = line.split(' ') + LabeledPoint(parts(0).toDouble, parts.tail.map(x => x.toDouble).toArray) +} + +// Run training algorithm to build the model +val numIterations = 20 +val model = SVMWithSGD.train(parsedData, numIterations) + +// Evaluate model on training examples and compute training error +val labelAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +println("Training Error = " + trainErr) +{% endhighlight %} + + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight scala %} +import org.apache.spark.mllib.optimization.L1Updater + +val svmAlg = new SVMWithSGD() +svmAlg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) +val modelL1 = svmAlg.run(parsedData) +{% endhighlight %} + +## Linear Regression +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The +example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We +compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit) + +{% highlight scala %} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.regression.LabeledPoint + +// Load and parse the data +val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, parts(1).split(' ').map(x => x.toDouble).toArray) +} + +// Building the model +val numIterations = 20 +val model = LinearRegressionWithSGD.train(parsedData, numIterations) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} + + +Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training +[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error). + +## Clustering +In the following example after loading and parsing data, we use the KMeans object to cluster the data +into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within +Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the +optimal *k* is usually one where there is an "elbow" in the WSSSE graph. + +{% highlight scala %} +import org.apache.spark.mllib.clustering.KMeans + +// Load and parse the data +val data = sc.textFile("kmeans_data.txt") +val parsedData = data.map( _.split(' ').map(_.toDouble)) + +// Cluster the data into two classes using KMeans +val numIterations = 20 +val numClusters = 2 +val clusters = KMeans.train(parsedData, numClusters, numIterations) + +// Evaluate clustering by computing Within Set Sum of Squared Errors +val WSSSE = clusters.computeCost(parsedData) +println("Within Set Sum of Squared Errors = " + WSSSE) +{% endhighlight %} + + +## Collaborative Filtering +In the following example we load rating data. Each row consists of a user, a product and a rating. +We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation +model by measuring the Mean Squared Error of rating prediction. + +{% highlight scala %} +import org.apache.spark.mllib.recommendation.ALS +import org.apache.spark.mllib.recommendation.Rating + +// Load and parse the data +val data = sc.textFile("mllib/data/als/test.data") +val ratings = data.map(_.split(',') match { + case Array(user, item, rate) => Rating(user.toInt, item.toInt, rate.toDouble) +}) + +// Build the recommendation model using ALS +val numIterations = 20 +val model = ALS.train(ratings, 1, 20, 0.01) + +// Evaluate the model on rating data +val ratesAndPreds = ratings.map{ case Rating(user, item, rate) => (rate, model.predict(user, item))} +val MSE = ratesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/ratesAndPreds.count +{% endhighlight %} + +If the rating matrix is derived from other source of information (i.e., it is inferred from +other signals), you can use the trainImplicit method to get better results. + +{% highlight scala %} +val model = ALS.trainImplicit(ratings, 1, 20, 0.01) +{% endhighlight %} + +# Using MLLib in Python +Following examples can be tested in the PySpark shell. + +## Binary Classification +The following example shows how to load a sample dataset, build Logistic Regression model, +and make predictions with the resulting model to compute the training error. + +{% highlight python %} +from pyspark.mllib.classification import LogisticRegressionWithSGD +from numpy import array + +# Load and parse the data +data = sc.textFile("mllib/data/sample_svm_data.txt") +parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) +model = LogisticRegressionWithSGD.train(sc, parsedData) + +# Build the model +labelsAndPreds = parsedData.map(lambda point: (int(point.item(0)), + model.predict(point.take(range(1, point.size))))) + +# Evaluating the model on training data +trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) +print("Training Error = " + str(trainErr)) +{% endhighlight %} + +## Linear Regression +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The +example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We +compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit) + +{% highlight python %} +from pyspark.mllib.regression import LinearRegressionWithSGD +from numpy import array + +# Load and parse the data +data = sc.textFile("mllib/data/ridge-data/lpsa.data") +parsedData = data.map(lambda line: array([float(x) for x in line.replace(',', ' ').split(' ')])) + +# Build the model +model = LinearRegressionWithSGD.train(sc, parsedData) + +# Evaluate the model on training data +valuesAndPreds = parsedData.map(lambda point: (point.item(0), + model.predict(point.take(range(1, point.size))))) +MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) +{% endhighlight %} + + +## Clustering +In the following example after loading and parsing data, we use the KMeans object to cluster the data +into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within +Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the +optimal *k* is usually one where there is an "elbow" in the WSSSE graph. + +{% highlight python %} +from pyspark.mllib.clustering import KMeans +from numpy import array +from math import sqrt + +# Load and parse the data +data = sc.textFile("kmeans_data.txt") +parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) + +# Build the model (cluster the data) +clusters = KMeans.train(sc, parsedData, 2, maxIterations=10, + runs=30, initialization_mode="random") + +# Evaluate clustering by computing Within Set Sum of Squared Errors +def error(point): + center = clusters.centers[clusters.predict(point)] + return sqrt(sum([x**2 for x in (point - center)])) + +WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) +print("Within Set Sum of Squared Error = " + str(WSSSE)) +{% endhighlight %} + +Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared +Errors. + +## Collaborative Filtering +In the following example we load rating data. Each row consists of a user, a product and a rating. +We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation +model by measuring the Mean Squared Error of rating prediction. + +{% highlight python %} +from pyspark.mllib.recommendation import ALS +from numpy import array + +# Load and parse the data +data = sc.textFile("mllib/data/als/test.data") +ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) + +# Build the recommendation model using Alternating Least Squares +model = ALS.train(sc, ratings, 1, 20) + +# Evaluate the model on training data +ratesAndPreds = ratings.map(lambda p: (p[2], model.predict(int(p[0]), int(p[1])))) +MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) + +{% endhighlight %} + +If the rating matrix is derived from other source of information (i.e., it is inferred from other +signals), you can use the trainImplicit method to get better results. + +{% highlight python %} +# Build the recommendation model using Alternating Least Squares based on implicit ratings +model = ALS.trainImplicit(sc, ratings, 1, 20) +{% endhighlight %} \ No newline at end of file From 81989e26647ede54e19ef8058846e1bd42c0bfb5 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 2 Jan 2014 16:22:13 -0800 Subject: [PATCH 1132/1571] Commented the last part of collaborative filtering examples that lead to errors --- docs/mllib-guide.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index e9d3785427b56..0bebc41137c0c 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -297,8 +297,9 @@ val numIterations = 20 val model = ALS.train(ratings, 1, 20, 0.01) // Evaluate the model on rating data -val ratesAndPreds = ratings.map{ case Rating(user, item, rate) => (rate, model.predict(user, item))} -val MSE = ratesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/ratesAndPreds.count +//val ratesAndPreds = ratings.map{ case Rating(user, item, rate) => (rate, model.predict(user, item))} +//val MSE = ratesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/ratesAndPreds.count +//println("Mean Squared Error = " + MSE) {% endhighlight %} If the rating matrix is derived from other source of information (i.e., it is inferred from @@ -406,9 +407,9 @@ ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) model = ALS.train(sc, ratings, 1, 20) # Evaluate the model on training data -ratesAndPreds = ratings.map(lambda p: (p[2], model.predict(int(p[0]), int(p[1])))) -MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count() -print("Mean Squared Error = " + str(MSE)) +#ratesAndPreds = ratings.map(lambda p: (p[2], model.predict(int(p[0]), int(p[1])))) +#MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count() +#print("Mean Squared Error = " + str(MSE)) {% endhighlight %} From a1b8dd53e3474dae2c49b30bc9719c7f6b98c7cc Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 2 Jan 2014 19:07:22 -0800 Subject: [PATCH 1133/1571] Added StreamingContext.getOrCreate to for automatic recovery, and added RecoverableNetworkWordCount example to use it. --- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../RecoverableNetworkWordCount.scala | 58 +++++++++++++++++++ .../apache/spark/streaming/Checkpoint.scala | 12 +++- .../spark/streaming/StreamingContext.scala | 18 +++++- 4 files changed, 85 insertions(+), 5 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 921b887a8978a..0615f7b565e40 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -47,7 +47,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{WORKER_URL}}", "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala new file mode 100644 index 0000000000000..0e5f39f7722ab --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala @@ -0,0 +1,58 @@ +package org.apache.spark.streaming.examples + +import org.apache.spark.streaming.{Time, Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.util.IntParam +import java.io.File +import org.apache.spark.rdd.RDD +import com.google.common.io.Files +import java.nio.charset.Charset + +object RecoverableNetworkWordCount { + + def createContext(master: String, ip: String, port: Int, outputPath: String) = { + + val outputFile = new File(outputPath) + if (outputFile.exists()) outputFile.delete() + + // Create the context with a 1 second batch size + println("Creating new context") + val ssc = new StreamingContext(master, "RecoverableNetworkWordCount", Seconds(1), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + // Create a NetworkInputDStream on target ip:port and count the + // words in input stream of \n delimited test (eg. generated by 'nc') + val lines = ssc.socketTextStream(ip, port) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.foreach((rdd: RDD[(String, Int)], time: Time) => { + val counts = "Counts at time " + time + " " + rdd.collect().mkString("[", ", ", "]") + println(counts) + println("Appending to " + outputFile.getAbsolutePath) + Files.append(counts + "\n", outputFile, Charset.defaultCharset()) + }) + ssc + } + + def main(args: Array[String]) { + if (args.length != 5) { + System.err.println("You arguments were " + args.mkString("[", ", ", "]")) + System.err.println( + """ + |Usage: RecoverableNetworkWordCount + | + |In local mode, should be 'local[n]' with n > 1 + |Both and should be full paths + """.stripMargin + ) + System.exit(1) + } + val Array(master, ip, IntParam(port), checkpointDirectory, outputPath) = args + val ssc = StreamingContext.getOrCreate(checkpointDirectory, + () => { + createContext(master, ip, port, outputPath) + }) + ssc.start() + + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 7b343d2376add..139e2c08b5d88 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -24,7 +24,7 @@ import java.util.concurrent.RejectedExecutionException import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.MetadataCleaner @@ -141,9 +141,15 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private[streaming] object CheckpointReader extends Logging { + def doesCheckpointExist(path: String): Boolean = { + val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk")) + val fs = new Path(path).getFileSystem(new Configuration()) + (attempts.count(p => fs.exists(p)) > 1) + } + def read(path: String): Checkpoint = { val fs = new Path(path).getFileSystem(new Configuration()) - val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) + val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk")) val compressionCodec = CompressionCodec.createCodec() @@ -175,7 +181,7 @@ object CheckpointReader extends Logging { } }) - throw new Exception("Could not read checkpoint from path '" + path + "'") + throw new SparkException("Could not read checkpoint from path '" + path + "'") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 41da028a3cf9f..01b213ab42c9b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -570,12 +570,28 @@ class StreamingContext private ( } -object StreamingContext { +object StreamingContext extends Logging { implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { new PairDStreamFunctions[K, V](stream) } + def getOrCreate( + checkpointPath: String, + creatingFunc: () => StreamingContext, + createOnCheckpointError: Boolean = false + ): StreamingContext = { + if (CheckpointReader.doesCheckpointExist(checkpointPath)) { + logInfo("Creating streaming context from checkpoint file") + new StreamingContext(checkpointPath) + } else { + logInfo("Creating new streaming context") + val ssc = creatingFunc() + ssc.checkpoint(checkpointPath) + ssc + } + } + protected[streaming] def createNewSparkContext( master: String, appName: String, From 3dc379ce5aa51cc9c41f590d79c350b5dea33fc3 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 4 Dec 2013 13:20:27 +0800 Subject: [PATCH 1134/1571] Reorganize yarn related codes into sub projects to remove duplicate files. --- project/SparkBuild.scala | 23 +- {new-yarn => yarn/2.0}/pom.xml | 0 .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/WorkerLauncher.scala | 0 .../spark/deploy/yarn/WorkerRunnable.scala | 0 .../deploy/yarn/YarnAllocationHandler.scala | 0 yarn/2.2/pom.xml | 161 +++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/WorkerLauncher.scala | 0 .../spark/deploy/yarn/WorkerRunnable.scala | 0 .../deploy/yarn/YarnAllocationHandler.scala | 0 .../yarn/ApplicationMasterArguments.scala | 0 .../spark/deploy/yarn/ClientArguments.scala | 0 .../yarn/ClientDistributedCacheManager.scala | 0 .../deploy/yarn/YarnSparkHadoopUtil.scala | 0 .../ClientDistributedCacheManagerSuite.scala | 0 yarn/scheduler/pom.xml | 161 +++++++++++++ .../cluster/YarnClientClusterScheduler.scala | 0 .../cluster/YarnClientSchedulerBackend.scala | 0 .../cluster/YarnClusterScheduler.scala | 0 .../yarn/ApplicationMasterArguments.scala | 94 -------- .../spark/deploy/yarn/ClientArguments.scala | 147 ----------- .../yarn/ClientDistributedCacheManager.scala | 228 ------------------ .../deploy/yarn/YarnSparkHadoopUtil.scala | 43 ---- .../cluster/YarnClientClusterScheduler.scala | 48 ---- .../cluster/YarnClientSchedulerBackend.scala | 110 --------- .../cluster/YarnClusterScheduler.scala | 59 ----- .../ClientDistributedCacheManagerSuite.scala | 220 ----------------- 30 files changed, 337 insertions(+), 957 deletions(-) rename {new-yarn => yarn/2.0}/pom.xml (100%) rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%) rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%) rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) create mode 100644 yarn/2.2/pom.xml rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%) rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%) rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala (100%) rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala (100%) rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala (100%) rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala (100%) rename {new-yarn => yarn/common}/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala (100%) create mode 100644 yarn/scheduler/pom.xml rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala (100%) rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (100%) rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (100%) delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala delete mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2eef2dfa5e72a..b34d42da16f73 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -85,12 +85,11 @@ object SparkBuild extends Build { } // Conditionally include the yarn sub-project - lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core) - - //lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core) - - lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val yarnCommon = Project("yarn-common", file("yarn/common"), settings = yarnCommonSettings) dependsOn(core) + lazy val yarnAPI = Project("yarn-api", file(if (isNewHadoop) "yarn/2.2" else "yarn/2.0"), settings = yarnAPISettings) dependsOn(yarnCommon) + lazy val yarnScheduler = Project("yarn", file("yarn/scheduler"), settings = yarnSchedulerSettings) dependsOn(yarnAPI) + lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarnCommon, yarnAPI, yarnScheduler) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarnCommon, yarnAPI, yarnScheduler) else Seq[ProjectReference]() // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef @@ -320,10 +319,18 @@ object SparkBuild extends Build { ) ) - def yarnSettings = sharedSettings ++ Seq( - name := "spark-yarn" + def yarnAPISettings = sharedSettings ++ Seq( + name := "spark-yarn-api" ) ++ extraYarnSettings + def yarnCommonSettings = sharedSettings ++ Seq( + name := "spark-yarn-common" + ) + + def yarnSchedulerSettings = sharedSettings ++ Seq( + name := "spark-yarn" + ) + // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq() diff --git a/new-yarn/pom.xml b/yarn/2.0/pom.xml similarity index 100% rename from new-yarn/pom.xml rename to yarn/2.0/pom.xml diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala similarity index 100% rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala similarity index 100% rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/yarn/2.2/pom.xml b/yarn/2.2/pom.xml new file mode 100644 index 0000000000000..8a065c6d7d1d7 --- /dev/null +++ b/yarn/2.2/pom.xml @@ -0,0 +1,161 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-yarn_2.9.3 + jar + Spark Project YARN Support + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-core_2.9.3 + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.scalatest + scalatest_2.9.3 + test + + + org.mockito + mockito-all + test + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/.. + 1 + ${spark.classpath} + + + + + + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala similarity index 100% rename from new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala rename to yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala diff --git a/yarn/scheduler/pom.xml b/yarn/scheduler/pom.xml new file mode 100644 index 0000000000000..4847d5269477a --- /dev/null +++ b/yarn/scheduler/pom.xml @@ -0,0 +1,161 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-yarn_2.9.3 + jar + Spark Project YARN Scheduler + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-core_2.9.3 + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.scalatest + scalatest_2.9.3 + test + + + org.mockito + mockito-all + test + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/.. + 1 + ${spark.classpath} + + + + + + diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala rename to yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala rename to yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 100% rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala deleted file mode 100644 index f76a5ddd39e90..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ /dev/null @@ -1,94 +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.yarn - -import org.apache.spark.util.IntParam -import collection.mutable.ArrayBuffer - -class ApplicationMasterArguments(val args: Array[String]) { - var userJar: String = null - var userClass: String = null - var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 - var workerCores = 1 - var numWorkers = 2 - - parseArgs(args.toList) - - private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer = new ArrayBuffer[String]() - - 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 ("--args") :: value :: tail => - userArgsBuffer += value - args = tail - - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value - args = tail - - case ("--worker-memory") :: IntParam(value) :: tail => - workerMemory = value - args = tail - - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value - args = tail - - case Nil => - if (userJar == null || userClass == null) { - printUsageAndExit(1) - } - - case _ => - printUsageAndExit(1, args) - } - } - - userArgs = userArgsBuffer.readOnly - } - - def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { - if (unknownParam != null) { - System.err.println("Unknown/unsupported param " + unknownParam) - } - System.err.println( - "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" + - "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") - System.exit(exitCode) - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala deleted file mode 100644 index 9075ca71e7efc..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ /dev/null @@ -1,147 +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.yarn - -import org.apache.spark.SparkConf -import org.apache.spark.util.MemoryParam -import org.apache.spark.util.IntParam -import collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} - -// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! -class ClientArguments(val args: Array[String]) { - var addJars: String = null - var files: String = null - var archives: String = null - var userJar: String = null - var userClass: String = null - var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 - var workerCores = 1 - var numWorkers = 2 - var amQueue = new SparkConf().get("QUEUE", "default") - var amMemory: Int = 512 - var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" - var appName: String = "Spark" - // TODO - var inputFormatInfo: List[InputFormatInfo] = null - - parseArgs(args.toList) - - private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() - - 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 ("--args") :: value :: tail => - userArgsBuffer += value - args = tail - - case ("--master-class") :: value :: tail => - amClass = value - args = tail - - case ("--master-memory") :: MemoryParam(value) :: tail => - amMemory = value - args = tail - - case ("--worker-memory") :: MemoryParam(value) :: tail => - workerMemory = value - args = tail - - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value - args = tail - - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value - args = tail - - case ("--queue") :: value :: tail => - amQueue = value - args = tail - - case ("--name") :: value :: tail => - appName = value - args = tail - - case ("--addJars") :: value :: tail => - addJars = value - args = tail - - case ("--files") :: value :: tail => - files = value - args = tail - - case ("--archives") :: value :: tail => - archives = value - args = tail - - case Nil => - if (userJar == null || userClass == null) { - printUsageAndExit(1) - } - - case _ => - printUsageAndExit(1, args) - } - } - - userArgs = userArgsBuffer.readOnly - inputFormatInfo = inputFormatMap.values.toList - } - - - def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { - if (unknownParam != null) { - System.err.println("Unknown/unsupported param " + unknownParam) - } - System.err.println( - "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + - "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + - " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + - " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + - " --name NAME The name of your application (Default: Spark)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + - " --files files Comma separated list of files to be distributed with the job.\n" + - " --archives archives Comma separated list of archives to be distributed with the job." - ) - System.exit(exitCode) - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala deleted file mode 100644 index 5f159b073f537..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ /dev/null @@ -1,228 +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.yarn - -import java.net.URI - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType -import org.apache.hadoop.yarn.util.{Records, ConverterUtils} - -import org.apache.spark.Logging - -import scala.collection.mutable.HashMap -import scala.collection.mutable.LinkedHashMap -import scala.collection.mutable.Map - - -/** Client side methods to setup the Hadoop distributed cache */ -class ClientDistributedCacheManager() extends Logging { - private val distCacheFiles: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() - private val distCacheArchives: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() - - - /** - * Add a resource to the list of distributed cache resources. This list can - * be sent to the ApplicationMaster and possibly the workers so that it can - * be downloaded into the Hadoop distributed cache for use by this application. - * Adds the LocalResource to the localResources HashMap passed in and saves - * the stats of the resources to they can be sent to the workers and verified. - * - * @param fs FileSystem - * @param conf Configuration - * @param destPath path to the resource - * @param localResources localResource hashMap to insert the resource into - * @param resourceType LocalResourceType - * @param link link presented in the distributed cache to the destination - * @param statCache cache to store the file/directory stats - * @param appMasterOnly Whether to only add the resource to the app master - */ - def addResource( - fs: FileSystem, - conf: Configuration, - destPath: Path, - localResources: HashMap[String, LocalResource], - resourceType: LocalResourceType, - link: String, - statCache: Map[URI, FileStatus], - appMasterOnly: Boolean = false) = { - val destStatus = fs.getFileStatus(destPath) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] - amJarRsrc.setType(resourceType) - val visibility = getVisibility(conf, destPath.toUri(), statCache) - amJarRsrc.setVisibility(visibility) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath)) - amJarRsrc.setTimestamp(destStatus.getModificationTime()) - amJarRsrc.setSize(destStatus.getLen()) - if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") - localResources(link) = amJarRsrc - - if (appMasterOnly == false) { - val uri = destPath.toUri() - val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) - if (resourceType == LocalResourceType.FILE) { - distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } else { - distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } - } - } - - /** - * Adds the necessary cache file env variables to the env passed in - * @param env - */ - def setDistFilesEnv(env: Map[String, String]) = { - val (keys, tupleValues) = distCacheFiles.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - - if (keys.size > 0) { - env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } - } - } - - /** - * Adds the necessary cache archive env variables to the env passed in - * @param env - */ - def setDistArchivesEnv(env: Map[String, String]) = { - val (keys, tupleValues) = distCacheArchives.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - - if (keys.size > 0) { - env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc,n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } - } - } - - /** - * Returns the local resource visibility depending on the cache file permissions - * @param conf - * @param uri - * @param statCache - * @return LocalResourceVisibility - */ - def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { - if (isPublic(conf, uri, statCache)) { - return LocalResourceVisibility.PUBLIC - } - return LocalResourceVisibility.PRIVATE - } - - /** - * Returns a boolean to denote whether a cache file is visible to all(public) - * or not - * @param conf - * @param uri - * @param statCache - * @return true if the path in the uri is visible to all, false otherwise - */ - def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { - val fs = FileSystem.get(uri, conf) - val current = new Path(uri.getPath()) - //the leaf level file should be readable by others - if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { - return false - } - return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache) - } - - /** - * Returns true if all ancestors of the specified path have the 'execute' - * permission set for all users (i.e. that other users can traverse - * the directory heirarchy to the given path) - * @param fs - * @param path - * @param statCache - * @return true if all ancestors have the 'execute' permission set for all users - */ - def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, - statCache: Map[URI, FileStatus]): Boolean = { - var current = path - while (current != null) { - //the subdirs in the path should have execute permissions for others - if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { - return false - } - current = current.getParent() - } - return true - } - - /** - * Checks for a given path whether the Other permissions on it - * imply the permission in the passed FsAction - * @param fs - * @param path - * @param action - * @param statCache - * @return true if the path in the uri is visible to all, false otherwise - */ - def checkPermissionOfOther(fs: FileSystem, path: Path, - action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { - val status = getFileStatus(fs, path.toUri(), statCache) - val perms = status.getPermission() - val otherAction = perms.getOtherAction() - if (otherAction.implies(action)) { - return true - } - return false - } - - /** - * Checks to see if the given uri exists in the cache, if it does it - * returns the existing FileStatus, otherwise it stats the uri, stores - * it in the cache, and returns the FileStatus. - * @param fs - * @param uri - * @param statCache - * @return FileStatus - */ - def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { - val stat = statCache.get(uri) match { - case Some(existstat) => existstat - case None => - val newStat = fs.getFileStatus(new Path(uri)) - statCache.put(uri, newStat) - newStat - } - return stat - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala deleted file mode 100644 index 2ba2366ead171..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ /dev/null @@ -1,43 +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.yarn - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.conf.Configuration - -/** - * Contains util methods to interact with Hadoop from spark. - */ -class YarnSparkHadoopUtil extends SparkHadoopUtil { - - // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. - override def isYarnMode(): Boolean = { true } - - // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems - // Always create a new config, dont reuse yarnConf. - override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) - - // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster - override def addCredentials(conf: JobConf) { - val jobCreds = conf.getCredentials() - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) - } -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala deleted file mode 100644 index 522e0a9ad7eeb..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ /dev/null @@ -1,48 +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.scheduler.cluster - -import org.apache.spark._ -import org.apache.hadoop.conf.Configuration -import org.apache.spark.deploy.yarn.YarnAllocationHandler -import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.Utils - -/** - * - * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. - */ -private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { - - def this(sc: SparkContext) = this(sc, new Configuration()) - - // By default, rack is unknown - override def getRackForHost(hostPort: String): Option[String] = { - val host = Utils.parseHostPort(hostPort)._1 - val retval = YarnAllocationHandler.lookupRack(conf, host) - if (retval != null) Some(retval) else None - } - - override def postStartHook() { - - // The yarn application is running, but the worker might not yet ready - // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt - Thread.sleep(2000L) - logInfo("YarnClientClusterScheduler.postStartHook done") - } -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala deleted file mode 100644 index 4b69f5078b0ab..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ /dev/null @@ -1,110 +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.scheduler.cluster - -import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} -import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments} -import org.apache.spark.scheduler.TaskSchedulerImpl - -private[spark] class YarnClientSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) - with Logging { - - var client: Client = null - var appId: ApplicationId = null - - override def start() { - super.start() - - val defalutWorkerCores = "2" - val defalutWorkerMemory = "512m" - val defaultWorkerNumber = "1" - - val userJar = System.getenv("SPARK_YARN_APP_JAR") - var workerCores = System.getenv("SPARK_WORKER_CORES") - var workerMemory = System.getenv("SPARK_WORKER_MEMORY") - var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") - - if (userJar == null) - throw new SparkException("env SPARK_YARN_APP_JAR is not set") - - if (workerCores == null) - workerCores = defalutWorkerCores - if (workerMemory == null) - workerMemory = defalutWorkerMemory - if (workerNumber == null) - workerNumber = defaultWorkerNumber - - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") - val hostport = driverHost + ":" + driverPort - - val argsArray = Array[String]( - "--class", "notused", - "--jar", userJar, - "--args", hostport, - "--worker-memory", workerMemory, - "--worker-cores", workerCores, - "--num-workers", workerNumber, - "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" - ) - - val args = new ClientArguments(argsArray) - client = new Client(args) - appId = client.runApp() - waitForApp() - } - - def waitForApp() { - - // TODO : need a better way to find out whether the workers are ready or not - // maybe by resource usage report? - while(true) { - val report = client.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" - ) - - // Ready to go, or already gone. - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.RUNNING) { - return - } else if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - throw new SparkException("Yarn application already ended," + - "might be killed or not able to launch application master.") - } - - Thread.sleep(1000) - } - } - - override def stop() { - super.stop() - client.stop() - logInfo("Stoped") - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala deleted file mode 100644 index 2d9fbcb400e5b..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ /dev/null @@ -1,59 +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.scheduler.cluster - -import org.apache.hadoop.conf.Configuration - -import org.apache.spark._ -import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} -import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.Utils - -/** - * - * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of - * ApplicationMaster, etc. is done - */ -private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) - extends TaskSchedulerImpl(sc) { - - logInfo("Created YarnClusterScheduler") - - def this(sc: SparkContext) = this(sc, new Configuration()) - - // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate - // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?) - // Subsequent creations are ignored - since nodes are already allocated by then. - - - // By default, rack is unknown - override def getRackForHost(hostPort: String): Option[String] = { - val host = Utils.parseHostPort(hostPort)._1 - val retval = YarnAllocationHandler.lookupRack(conf, host) - if (retval != null) Some(retval) else None - } - - override def postStartHook() { - val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) - if (sparkContextInitialized){ - // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt - Thread.sleep(3000L) - } - logInfo("YarnClusterScheduler.postStartHook done") - } -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala deleted file mode 100644 index 2941356bc55f9..0000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ /dev/null @@ -1,220 +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.yarn - -import java.net.URI - -import org.scalatest.FunSuite -import org.scalatest.mock.MockitoSugar -import org.mockito.Mockito.when - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType -import org.apache.hadoop.yarn.util.{Records, ConverterUtils} - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map - - -class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar { - - class MockClientDistributedCacheManager extends ClientDistributedCacheManager { - override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { - return LocalResourceVisibility.PRIVATE - } - } - - test("test getFileStatus empty") { - val distMgr = new ClientDistributedCacheManager() - val fs = mock[FileSystem] - val uri = new URI("/tmp/testing") - when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val stat = distMgr.getFileStatus(fs, uri, statCache) - assert(stat.getPath() === null) - } - - test("test getFileStatus cached") { - val distMgr = new ClientDistributedCacheManager() - val fs = mock[FileSystem] - val uri = new URI("/tmp/testing") - val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus) - val stat = distMgr.getFileStatus(fs, uri, statCache) - assert(stat.getPath().toString() === "/tmp/testing") - } - - test("test addResource") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", - statCache, false) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 0) - assert(resource.getSize() === 0) - assert(resource.getType() === LocalResourceType.FILE) - - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0") - assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0") - assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - - //add another one and verify both there and order correct - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing2")) - val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") - when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus) - distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", - statCache, false) - val resource2 = localResources("link2") - assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2) - assert(resource2.getTimestamp() === 10) - assert(resource2.getSize() === 20) - assert(resource2.getType() === LocalResourceType.FILE) - - val env2 = new HashMap[String, String]() - distMgr.setDistFilesEnv(env2) - val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val files = env2("SPARK_YARN_CACHE_FILES").split(',') - val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',') - assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(timestamps(0) === "0") - assert(sizes(0) === "0") - assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name()) - - assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2") - assert(timestamps(1) === "10") - assert(sizes(1) === "20") - assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name()) - } - - test("test addResource link null") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) - - intercept[Exception] { - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, - statCache, false) - } - assert(localResources.get("link") === None) - assert(localResources.size === 0) - } - - test("test addResource appmaster only") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", - statCache, true) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 10) - assert(resource.getSize() === 20) - assert(resource.getType() === LocalResourceType.ARCHIVE) - - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - } - - test("test addResource archive") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", - statCache, false) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 10) - assert(resource.getSize() === 20) - assert(resource.getType() === LocalResourceType.ARCHIVE) - - val env = new HashMap[String, String]() - - distMgr.setDistArchivesEnv(env) - assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10") - assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20") - assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) - } - - -} From d1528c7c8c8bf37c471f399658881efea35b9741 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 4 Dec 2013 16:17:49 +0800 Subject: [PATCH 1135/1571] Fix pom for yarn code reorgnaize commit --- assembly/pom.xml | 10 +++ pom.xml | 55 +++---------- yarn/2.0/pom.xml | 124 +---------------------------- yarn/2.2/pom.xml | 124 +---------------------------- yarn/assembly/pom.xml | 45 +++++++++++ yarn/common/pom.xml | 122 +++++++++++++++++++++++++++++ yarn/pom.xml | 173 ++++++++++------------------------------- yarn/scheduler/pom.xml | 146 +++++++--------------------------- 8 files changed, 264 insertions(+), 535 deletions(-) create mode 100644 yarn/assembly/pom.xml create mode 100644 yarn/common/pom.xml diff --git a/assembly/pom.xml b/assembly/pom.xml index fc2adc1fbb5d2..7a719ad0d880c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -133,6 +133,16 @@ + + new-yarn + + + org.apache.spark + spark-yarn_${scala.binary.version} + ${project.version} + + + bigtop-dist + + 4.0.0 + + org.apache.spark + spark-yarn-aggregator_${scala.binary.version} + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-yarn_2.10 + jar + Spark Project YARN Assembly + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-yarn-scheduler_${scala.binary.version} + ${project.version} + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + diff --git a/yarn/common/pom.xml b/yarn/common/pom.xml new file mode 100644 index 0000000000000..d194ff486ee96 --- /dev/null +++ b/yarn/common/pom.xml @@ -0,0 +1,122 @@ + + + + 4.0.0 + + org.apache.spark + spark-yarn-aggregator_${scala.binary.version} + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-yarn-common_2.10 + jar + Spark Project YARN Common + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-all + test + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/../.. + 1 + ${spark.classpath} + + + + + + diff --git a/yarn/pom.xml b/yarn/pom.xml index bc64a190fd284..2c2a31011c071 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,137 +25,50 @@ org.apache.spark - spark-yarn_2.10 - jar - Spark Project YARN Support + spark-yarn-aggregator_${scala.binary.version} + pom + Spark Project YARN Aggregator http://spark.incubator.apache.org/ + + + common + scheduler + assembly + - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.hadoop - hadoop-client - ${yarn.version} - - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.mockito - mockito-all - test - - + + + maven-root + Maven root repository + http://repo1.maven.org/maven2 + + true + + + false + + + + + + + + + + + + hadoop2-yarn + + 2.0 + + + + + new-yarn + + 2.2 + + + - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/.. - 1 - ${spark.classpath} - - - - - diff --git a/yarn/scheduler/pom.xml b/yarn/scheduler/pom.xml index 4847d5269477a..8c3656571d354 100644 --- a/yarn/scheduler/pom.xml +++ b/yarn/scheduler/pom.xml @@ -19,13 +19,13 @@ 4.0.0 org.apache.spark - spark-parent + spark-yarn-aggregator_${scala.binary.version} 0.9.0-incubating-SNAPSHOT ../pom.xml org.apache.spark - spark-yarn_2.9.3 + spark-yarn-scheduler_2.10 jar Spark Project YARN Scheduler http://spark.incubator.apache.org/ @@ -33,129 +33,37 @@ org.apache.spark - spark-core_2.9.3 + spark-yarn-common_${scala.binary.version} ${project.version} - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.hadoop - hadoop-client - ${yarn.version} - - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - - - org.scalatest - scalatest_2.9.3 - test - - - org.mockito - mockito-all - test - + + + hadoop2-yarn + + + org.apache.spark + spark-yarn-2.0-api_${scala.binary.version} + ${project.version} + + + + + + new-yarn + + + org.apache.spark + spark-yarn-2.2-api_${scala.binary.version} + ${project.version} + + + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/.. - 1 - ${spark.classpath} - - - - From aec96dd108940b14004bb0895d20f4beceda9ff4 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Fri, 6 Dec 2013 10:30:16 +0800 Subject: [PATCH 1136/1571] Change profile name new-yarn to hadoop2.2-yarn --- assembly/pom.xml | 2 +- pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/scheduler/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7a719ad0d880c..bd3420deb99a9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -134,7 +134,7 @@ - new-yarn + hadoop2.2-yarn org.apache.spark diff --git a/pom.xml b/pom.xml index d1cf2c23b553e..a2c1f6d53d2fb 100644 --- a/pom.xml +++ b/pom.xml @@ -738,7 +738,7 @@ - new-yarn + hadoop2.2-yarn 2 2.2.0 diff --git a/yarn/pom.xml b/yarn/pom.xml index 2c2a31011c071..d3c417f8f55ab 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -64,7 +64,7 @@ - new-yarn + hadoop2.2-yarn 2.2 diff --git a/yarn/scheduler/pom.xml b/yarn/scheduler/pom.xml index 8c3656571d354..8723b7bfd511e 100644 --- a/yarn/scheduler/pom.xml +++ b/yarn/scheduler/pom.xml @@ -51,7 +51,7 @@ - new-yarn + hadoop2.2-yarn org.apache.spark From 96e25e567cddb5327af0b56e810d3f8124677288 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Fri, 6 Dec 2013 16:04:54 +0800 Subject: [PATCH 1137/1571] Fix yarn/assemble pom file --- yarn/assembly/pom.xml | 46 ++++++++++++++++++++++++++ yarn/assembly/src/assemble/uberjar.xml | 29 ++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 yarn/assembly/src/assemble/uberjar.xml diff --git a/yarn/assembly/pom.xml b/yarn/assembly/pom.xml index 7f3539cd4292d..659b8c5ca7a66 100644 --- a/yarn/assembly/pom.xml +++ b/yarn/assembly/pom.xml @@ -41,5 +41,51 @@ target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-assembly-plugin + + false + + src/assemble/uberjar.xml + + + + + uberjar + package + + single + + + + + + + + + hadoop2-yarn + + + org.apache.spark + spark-yarn-2.0-api_2.10 + ${project.version} + + + + + + hadoop2.2-yarn + + + org.apache.spark + spark-yarn-2.2-api_2.10 + ${project.version} + + + + + diff --git a/yarn/assembly/src/assemble/uberjar.xml b/yarn/assembly/src/assemble/uberjar.xml new file mode 100644 index 0000000000000..0fbcdce17b41b --- /dev/null +++ b/yarn/assembly/src/assemble/uberjar.xml @@ -0,0 +1,29 @@ + + + + uberjar + + jar + + false + + + true + runtime + true + false + + + From 8818661721a2705a0f3d9aff6187f2eee12aa2eb Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Tue, 17 Dec 2013 10:14:26 +0800 Subject: [PATCH 1138/1571] Fix pom file for scala binary version --- yarn/2.0/pom.xml | 2 +- yarn/2.2/pom.xml | 2 +- yarn/assembly/pom.xml | 6 +++--- yarn/common/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/scheduler/pom.xml | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/yarn/2.0/pom.xml b/yarn/2.0/pom.xml index e53686f40c37f..7106a803454f2 100644 --- a/yarn/2.0/pom.xml +++ b/yarn/2.0/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/2.2/pom.xml b/yarn/2.2/pom.xml index aa1146e765dd9..8acb749728010 100644 --- a/yarn/2.2/pom.xml +++ b/yarn/2.2/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/assembly/pom.xml b/yarn/assembly/pom.xml index 659b8c5ca7a66..7bac1ed503958 100644 --- a/yarn/assembly/pom.xml +++ b/yarn/assembly/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml @@ -70,7 +70,7 @@ org.apache.spark - spark-yarn-2.0-api_2.10 + spark-yarn-2.0-api_${scala.binary.version} ${project.version} @@ -81,7 +81,7 @@ org.apache.spark - spark-yarn-2.2-api_2.10 + spark-yarn-2.2-api_${scala.binary.version} ${project.version} diff --git a/yarn/common/pom.xml b/yarn/common/pom.xml index d194ff486ee96..2ffe97228897b 100644 --- a/yarn/common/pom.xml +++ b/yarn/common/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d3c417f8f55ab..689fe86bbba08 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,7 +25,7 @@ org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 pom Spark Project YARN Aggregator http://spark.incubator.apache.org/ diff --git a/yarn/scheduler/pom.xml b/yarn/scheduler/pom.xml index 8723b7bfd511e..cbc5b30bdfa84 100644 --- a/yarn/scheduler/pom.xml +++ b/yarn/scheduler/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_${scala.binary.version} + spark-yarn-aggregator_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml From e867e31145b100997232720399155d158241b125 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 16 Dec 2013 14:39:37 +0800 Subject: [PATCH 1139/1571] some code clean up for Yarn 2.2 --- .../scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 4 ++-- .../org/apache/spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 99b824e1295a7..191ef798d706a 100644 --- a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -56,7 +56,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte // This actor just working as a monitor to watch on Driver Actor. class MonitorActor(driverUrl: String) extends Actor { - var driver: ActorSelection = null + var driver: ActorSelection = _ override def preStart() { logInfo("Listen to driver: " + driverUrl) @@ -66,7 +66,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte override def receive = { case x: DisassociatedEvent => - logInfo("Driver terminated or disconnected! Shutting down.") + logInfo("Driver terminated or disconnected! Shutting down. $x") driverClosed = true } } diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 85ab08ef34db4..8a9a73f5b4cd6 100644 --- a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.util.{RackResolver, Records} -object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { +object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value } From 67cd752e746bf2c2202949e15dbb85faf8e3c482 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Fri, 6 Dec 2013 10:26:59 +0800 Subject: [PATCH 1140/1571] Add README for yarn modules --- yarn/README.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 yarn/README.md diff --git a/yarn/README.md b/yarn/README.md new file mode 100644 index 0000000000000..3639f2c364ef7 --- /dev/null +++ b/yarn/README.md @@ -0,0 +1,16 @@ +# YARN DIRECTORY LAYOUT + +Hadoop Yarn related code are organized + +Hadoop Yarn related codes are organized in separate modules with layered dependency. + + * common : Common codes that will be called into by other modules. + + * 2.0 / 2.2 : Codes that involve specific version of Hadoop YARN API. Depends on common module. + + 2.0 actually represents 0.23 and 2.0 + 2.2 actually represents 2.2 and later, until the API is break again. + + * Scheduler : Implementation of various YARN Scheduler and backend. Depends on 2.0 / 2.2 modules. + * Assembly : For maven build to assembly all other modules in Yarn into one single jar. + From be343d2a567b8ba3824c5e186e046f96ff77e334 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 9 Dec 2013 09:47:52 +0800 Subject: [PATCH 1141/1571] Fix yarn/README.md and update docs/running-on-yarn.md --- docs/running-on-yarn.md | 2 +- yarn/README.md | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index aa75ca43241fb..5f995ce51b64c 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -116,7 +116,7 @@ For example: # Building Spark for Hadoop/YARN 2.2.x -Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. +Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. diff --git a/yarn/README.md b/yarn/README.md index 3639f2c364ef7..e2c3aea9a8ba3 100644 --- a/yarn/README.md +++ b/yarn/README.md @@ -1,7 +1,5 @@ # YARN DIRECTORY LAYOUT -Hadoop Yarn related code are organized - Hadoop Yarn related codes are organized in separate modules with layered dependency. * common : Common codes that will be called into by other modules. From 7815a3ace97848b1d4bab4ef6823fc6d55cfb72e Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Fri, 6 Dec 2013 10:36:59 +0800 Subject: [PATCH 1142/1571] Update maven build documentation --- docs/building-with-maven.md | 10 +++------- docs/running-on-yarn.md | 2 +- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index c709001632261..699b10b188612 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -37,7 +37,7 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Cloudera CDH 4.2.0 with MapReduce v1 $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" profile and set the "yarn.version" property: +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" or "hadoop2.2-yarn" profile and set the "hadoop.version", "yarn.version" property: # Apache Hadoop 2.0.5-alpha $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package @@ -45,12 +45,8 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with # Cloudera CDH 4.2.0 with MapReduce v2 $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package -Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows: - - # Apache Hadoop 2.2.X and newer - $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn - -The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. + # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer + $ mvn -Phadoop2.2-yarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package ## Spark Tests in Maven ## diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 5f995ce51b64c..f47098554e145 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -116,7 +116,7 @@ For example: # Building Spark for Hadoop/YARN 2.2.x -Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. +Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5). See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. From dd6d347f4f9bee6f7509fffe07a1a2d74803f783 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Tue, 17 Dec 2013 13:19:14 +0800 Subject: [PATCH 1143/1571] A few clean up for yarn 2.0 code --- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 13 ++++++------- .../apache/spark/deploy/yarn/WorkerRunnable.scala | 2 +- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index a8de89c670811..1ef099aceb0d1 100644 --- a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -39,28 +39,27 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte def this(args: ApplicationMasterArguments) = this(args, new Configuration()) private val rpc: YarnRPC = YarnRPC.create(conf) - private var resourceManager: AMRMProtocol = null - private var appAttemptId: ApplicationAttemptId = null - private var reporterThread: Thread = null + private var resourceManager: AMRMProtocol = _ + private var appAttemptId: ApplicationAttemptId = _ + private var reporterThread: Thread = _ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - private var yarnAllocator: YarnAllocationHandler = null + private var yarnAllocator: YarnAllocationHandler = _ private var driverClosed:Boolean = false private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 - var actor: ActorRef = null + var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. class MonitorActor(driverUrl: String) extends Actor { - var driver: ActorSelection = null + var driver: ActorSelection = _ override def preStart() { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) - driver ! "hello" context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 6a90cc51cfbaf..4f34bd913eb8f 100644 --- a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -51,7 +51,7 @@ class WorkerRunnable( extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) - var cm: ContainerManager = null + var cm: ContainerManager = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { From ad60710010c4f1cefaab443f8c5e01f495ccaa60 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Tue, 17 Dec 2013 16:08:12 +0800 Subject: [PATCH 1144/1571] Need to send dummy hello message to actually estabilish akka connection. --- .../scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 2 ++ .../scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 1ef099aceb0d1..e64530702c7d5 100644 --- a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -60,6 +60,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte override def preStart() { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) + // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events. + driver ! "Hello" context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 191ef798d706a..4d9cca0766eac 100644 --- a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -61,6 +61,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte override def preStart() { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) + // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events. + driver ! "Hello" context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } From c5422e02b868dd69d1078c246ba15e4a02a7b8b9 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 23 Dec 2013 10:33:33 +0800 Subject: [PATCH 1145/1571] merge yarn/scheduler yarn/common code into one directory --- .../spark/scheduler/cluster/YarnClientClusterScheduler.scala | 0 .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 0 .../org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename yarn/{scheduler => common}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala (100%) rename yarn/{scheduler => common}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (100%) rename yarn/{scheduler => common}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (100%) diff --git a/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala similarity index 100% rename from yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala diff --git a/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala similarity index 100% rename from yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala diff --git a/yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 100% rename from yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala From d1a6f7aabc739e275ed1e39a8db522ef9f3161da Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 23 Dec 2013 13:32:20 +0800 Subject: [PATCH 1146/1571] Use unmanaged source dir to include common yarn code --- project/SparkBuild.scala | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b34d42da16f73..d58f6aaec0638 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -85,11 +85,11 @@ object SparkBuild extends Build { } // Conditionally include the yarn sub-project - lazy val yarnCommon = Project("yarn-common", file("yarn/common"), settings = yarnCommonSettings) dependsOn(core) - lazy val yarnAPI = Project("yarn-api", file(if (isNewHadoop) "yarn/2.2" else "yarn/2.0"), settings = yarnAPISettings) dependsOn(yarnCommon) - lazy val yarnScheduler = Project("yarn", file("yarn/scheduler"), settings = yarnSchedulerSettings) dependsOn(yarnAPI) - lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarnCommon, yarnAPI, yarnScheduler) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarnCommon, yarnAPI, yarnScheduler) else Seq[ProjectReference]() + lazy val yarn20 = Project("yarn2-alpha", file("yarn/2.0"), settings = yarn20Settings) dependsOn(core) + lazy val yarn22 = Project("yarn2-stable", file("yarn/2.2"), settings = yarn22Settings) dependsOn(core) + + lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn22 else yarn20) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn22 else yarn20) else Seq[ProjectReference]() // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef @@ -319,16 +319,20 @@ object SparkBuild extends Build { ) ) - def yarnAPISettings = sharedSettings ++ Seq( - name := "spark-yarn-api" + def yarnCommonSettings = sharedSettings ++ Seq( + unmanagedSourceDirectories in Compile <++= baseDirectory { base => + Seq( + base / "../common/src/main/scala" + ) + } ) ++ extraYarnSettings - def yarnCommonSettings = sharedSettings ++ Seq( - name := "spark-yarn-common" + def yarn20Settings = yarnCommonSettings ++ Seq( + name := "spark-yarn-2.0" ) - def yarnSchedulerSettings = sharedSettings ++ Seq( - name := "spark-yarn" + def yarn22Settings = yarnCommonSettings ++ Seq( + name := "spark-yarn-2.2" ) // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain From 7c96faee747e90c1bb5210de01e425fef25cd4f3 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 23 Dec 2013 16:17:40 +0800 Subject: [PATCH 1147/1571] Fix pom for build yarn/2.x with yarn/common into one jar --- assembly/pom.xml | 4 +-- yarn/2.0/pom.xml | 91 +++++++++++++++++++++++++++++++++++++++++------- yarn/2.2/pom.xml | 91 +++++++++++++++++++++++++++++++++++++++++------- yarn/pom.xml | 52 +++++++++++++++++++++------ 4 files changed, 202 insertions(+), 36 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index bd3420deb99a9..bd948344156ea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -128,7 +128,7 @@ org.apache.spark - spark-yarn_${scala.binary.version} + spark-yarn-2.0_${scala.binary.version} ${project.version} @@ -138,7 +138,7 @@ org.apache.spark - spark-yarn_${scala.binary.version} + spark-yarn-2.2_${scala.binary.version} ${project.version} diff --git a/yarn/2.0/pom.xml b/yarn/2.0/pom.xml index 7106a803454f2..ec6738fa38279 100644 --- a/yarn/2.0/pom.xml +++ b/yarn/2.0/pom.xml @@ -25,21 +25,88 @@ org.apache.spark - spark-yarn-2.0-api_2.10 + spark-yarn-2.0_2.10 jar - Spark Project YARN 2.0 API + Spark Project YARN 2.0 http://spark.incubator.apache.org/ - - - org.apache.spark - spark-yarn-common_${scala.binary.version} - ${project.version} - - - - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + ../common/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + ../common/src/test/scala + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/../.. + 1 + ${spark.classpath} + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes diff --git a/yarn/2.2/pom.xml b/yarn/2.2/pom.xml index 8acb749728010..345462a2c09e0 100644 --- a/yarn/2.2/pom.xml +++ b/yarn/2.2/pom.xml @@ -25,21 +25,88 @@ org.apache.spark - spark-yarn-2.2-api_2.10 + spark-yarn-2.2_2.10 jar - Spark Project YARN 2.2 API + Spark Project YARN 2.2 http://spark.incubator.apache.org/ - - - org.apache.spark - spark-yarn-common_${scala.binary.version} - ${project.version} - - - - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + ../common/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + ../common/src/test/scala + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/../.. + 1 + ${spark.classpath} + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes diff --git a/yarn/pom.xml b/yarn/pom.xml index 689fe86bbba08..3300a465ef7dc 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -30,12 +30,6 @@ Spark Project YARN Aggregator http://spark.incubator.apache.org/ - - common - scheduler - assembly - - maven-root @@ -50,10 +44,48 @@ - - - - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-all + test + + From 79b6b4ddc2859596fb7ddbf76b5bd9e8faef17b2 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 23 Dec 2013 16:25:06 +0800 Subject: [PATCH 1148/1571] Clean up unused files for yarn --- yarn/assembly/pom.xml | 91 ------------------ yarn/assembly/src/assemble/uberjar.xml | 29 ------ yarn/common/pom.xml | 122 ------------------------- yarn/scheduler/pom.xml | 69 -------------- 4 files changed, 311 deletions(-) delete mode 100644 yarn/assembly/pom.xml delete mode 100644 yarn/assembly/src/assemble/uberjar.xml delete mode 100644 yarn/common/pom.xml delete mode 100644 yarn/scheduler/pom.xml diff --git a/yarn/assembly/pom.xml b/yarn/assembly/pom.xml deleted file mode 100644 index 7bac1ed503958..0000000000000 --- a/yarn/assembly/pom.xml +++ /dev/null @@ -1,91 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-yarn-aggregator_2.10 - 0.9.0-incubating-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-yarn_2.10 - jar - Spark Project YARN Assembly - http://spark.incubator.apache.org/ - - - - org.apache.spark - spark-yarn-scheduler_${scala.binary.version} - ${project.version} - - - - - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes - - - org.apache.maven.plugins - maven-assembly-plugin - - false - - src/assemble/uberjar.xml - - - - - uberjar - package - - single - - - - - - - - - - hadoop2-yarn - - - org.apache.spark - spark-yarn-2.0-api_${scala.binary.version} - ${project.version} - - - - - - hadoop2.2-yarn - - - org.apache.spark - spark-yarn-2.2-api_${scala.binary.version} - ${project.version} - - - - - - diff --git a/yarn/assembly/src/assemble/uberjar.xml b/yarn/assembly/src/assemble/uberjar.xml deleted file mode 100644 index 0fbcdce17b41b..0000000000000 --- a/yarn/assembly/src/assemble/uberjar.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - uberjar - - jar - - false - - - true - runtime - true - false - - - diff --git a/yarn/common/pom.xml b/yarn/common/pom.xml deleted file mode 100644 index 2ffe97228897b..0000000000000 --- a/yarn/common/pom.xml +++ /dev/null @@ -1,122 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-yarn-aggregator_2.10 - 0.9.0-incubating-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-yarn-common_2.10 - jar - Spark Project YARN Common - http://spark.incubator.apache.org/ - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.hadoop - hadoop-client - ${yarn.version} - - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.mockito - mockito-all - test - - - - - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/../.. - 1 - ${spark.classpath} - - - - - - diff --git a/yarn/scheduler/pom.xml b/yarn/scheduler/pom.xml deleted file mode 100644 index cbc5b30bdfa84..0000000000000 --- a/yarn/scheduler/pom.xml +++ /dev/null @@ -1,69 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-yarn-aggregator_2.10 - 0.9.0-incubating-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-yarn-scheduler_2.10 - jar - Spark Project YARN Scheduler - http://spark.incubator.apache.org/ - - - - org.apache.spark - spark-yarn-common_${scala.binary.version} - ${project.version} - - - - - - hadoop2-yarn - - - org.apache.spark - spark-yarn-2.0-api_${scala.binary.version} - ${project.version} - - - - - - hadoop2.2-yarn - - - org.apache.spark - spark-yarn-2.2-api_${scala.binary.version} - ${project.version} - - - - - - - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes - - From ddc5054b35e0d23e76fb895f2455b75ceebdc206 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 23 Dec 2013 16:28:52 +0800 Subject: [PATCH 1149/1571] Fix yarn/README.md --- yarn/README.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/yarn/README.md b/yarn/README.md index e2c3aea9a8ba3..0d63bd9352d75 100644 --- a/yarn/README.md +++ b/yarn/README.md @@ -1,14 +1,12 @@ # YARN DIRECTORY LAYOUT -Hadoop Yarn related codes are organized in separate modules with layered dependency. +Hadoop Yarn related codes are organized in separate directories for easy management. - * common : Common codes that will be called into by other modules. + * common : Common codes that do not depending on specific version of Hadoop. - * 2.0 / 2.2 : Codes that involve specific version of Hadoop YARN API. Depends on common module. + * 2.0 / 2.2 : Codes that involve specific version of Hadoop YARN API. 2.0 actually represents 0.23 and 2.0 2.2 actually represents 2.2 and later, until the API is break again. - * Scheduler : Implementation of various YARN Scheduler and backend. Depends on 2.0 / 2.2 modules. - * Assembly : For maven build to assembly all other modules in Yarn into one single jar. - +2.0 / 2.2 will build together with common dir into a single jar From a47ebf7228acb22f23293cc7197a14b88c1ea65a Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 2 Jan 2014 10:12:06 +0800 Subject: [PATCH 1150/1571] Add yarn/common/src/test dir in building script --- project/SparkBuild.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d58f6aaec0638..bc9c02d96ff18 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -324,7 +324,14 @@ object SparkBuild extends Build { Seq( base / "../common/src/main/scala" ) + }, + + unmanagedSourceDirectories in Test <++= baseDirectory { base => + Seq( + base / "../common/src/test/scala" + ) } + ) ++ extraYarnSettings def yarn20Settings = yarnCommonSettings ++ Seq( From ebdfa6bb9766209bc5a3c4241fa47141c5e9c5cb Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 2 Jan 2014 11:23:33 +0800 Subject: [PATCH 1151/1571] Using name yarn-alpha/yarn instead of yarn-2.0/yarn-2.2 --- assembly/pom.xml | 8 ++++---- docs/building-with-maven.md | 8 ++++---- pom.xml | 4 ++-- project/SparkBuild.scala | 16 ++++++++-------- yarn/README.md | 8 ++++---- yarn/{2.2 => alpha}/pom.xml | 4 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/WorkerLauncher.scala | 0 .../spark/deploy/yarn/WorkerRunnable.scala | 0 .../deploy/yarn/YarnAllocationHandler.scala | 0 yarn/pom.xml | 8 ++++---- yarn/{2.0 => stable}/pom.xml | 4 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/WorkerLauncher.scala | 0 .../spark/deploy/yarn/WorkerRunnable.scala | 0 .../deploy/yarn/YarnAllocationHandler.scala | 0 18 files changed, 30 insertions(+), 30 deletions(-) rename yarn/{2.2 => alpha}/pom.xml (97%) rename yarn/{2.0 => alpha}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename yarn/{2.0 => alpha}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename yarn/{2.0 => alpha}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%) rename yarn/{2.0 => alpha}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%) rename yarn/{2.0 => alpha}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) rename yarn/{2.0 => stable}/pom.xml (97%) rename yarn/{2.2 => stable}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename yarn/{2.2 => stable}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename yarn/{2.2 => stable}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%) rename yarn/{2.2 => stable}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%) rename yarn/{2.2 => stable}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) diff --git a/assembly/pom.xml b/assembly/pom.xml index bd948344156ea..079509bec6fb7 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -124,21 +124,21 @@ - hadoop2-yarn + yarn-alpha org.apache.spark - spark-yarn-2.0_${scala.binary.version} + spark-yarn-alpha_${scala.binary.version} ${project.version} - hadoop2.2-yarn + yarn org.apache.spark - spark-yarn-2.2_${scala.binary.version} + spark-yarn_${scala.binary.version} ${project.version} diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 699b10b188612..b9ff0af76f647 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -37,16 +37,16 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Cloudera CDH 4.2.0 with MapReduce v1 $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" or "hadoop2.2-yarn" profile and set the "hadoop.version", "yarn.version" property: +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property: # Apache Hadoop 2.0.5-alpha - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer - $ mvn -Phadoop2.2-yarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package ## Spark Tests in Maven ## diff --git a/pom.xml b/pom.xml index a2c1f6d53d2fb..aa2f076aacc24 100644 --- a/pom.xml +++ b/pom.xml @@ -722,7 +722,7 @@ - hadoop2-yarn + yarn-alpha 2 @@ -738,7 +738,7 @@ - hadoop2.2-yarn + yarn 2 2.2.0 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bc9c02d96ff18..b0749cc9c466f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -85,11 +85,11 @@ object SparkBuild extends Build { } // Conditionally include the yarn sub-project - lazy val yarn20 = Project("yarn2-alpha", file("yarn/2.0"), settings = yarn20Settings) dependsOn(core) - lazy val yarn22 = Project("yarn2-stable", file("yarn/2.2"), settings = yarn22Settings) dependsOn(core) + lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) + lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) - lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn22 else yarn20) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn22 else yarn20) else Seq[ProjectReference]() + lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]() // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef @@ -334,12 +334,12 @@ object SparkBuild extends Build { ) ++ extraYarnSettings - def yarn20Settings = yarnCommonSettings ++ Seq( - name := "spark-yarn-2.0" + def yarnAlphaSettings = yarnCommonSettings ++ Seq( + name := "spark-yarn-alpha" ) - def yarn22Settings = yarnCommonSettings ++ Seq( - name := "spark-yarn-2.2" + def yarnSettings = yarnCommonSettings ++ Seq( + name := "spark-yarn" ) // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain diff --git a/yarn/README.md b/yarn/README.md index 0d63bd9352d75..9a7a1dd838dea 100644 --- a/yarn/README.md +++ b/yarn/README.md @@ -4,9 +4,9 @@ Hadoop Yarn related codes are organized in separate directories for easy managem * common : Common codes that do not depending on specific version of Hadoop. - * 2.0 / 2.2 : Codes that involve specific version of Hadoop YARN API. + * alpha / stable : Codes that involve specific version of Hadoop YARN API. - 2.0 actually represents 0.23 and 2.0 - 2.2 actually represents 2.2 and later, until the API is break again. + alpha represents 0.23 and 2.0.x + stable represents 2.2 and later, until the API is break again. -2.0 / 2.2 will build together with common dir into a single jar +alpha / stable will build together with common dir into a single jar diff --git a/yarn/2.2/pom.xml b/yarn/alpha/pom.xml similarity index 97% rename from yarn/2.2/pom.xml rename to yarn/alpha/pom.xml index 345462a2c09e0..94812f8c45c52 100644 --- a/yarn/2.2/pom.xml +++ b/yarn/alpha/pom.xml @@ -25,9 +25,9 @@ org.apache.spark - spark-yarn-2.2_2.10 + spark-yarn-alpha_2.10 jar - Spark Project YARN 2.2 + Spark Project YARN Alpha API http://spark.incubator.apache.org/ diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala similarity index 100% rename from yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala similarity index 100% rename from yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala diff --git a/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/yarn/pom.xml b/yarn/pom.xml index 3300a465ef7dc..7c145ef362600 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -89,16 +89,16 @@ - hadoop2-yarn + yarn-alpha - 2.0 + alpha - hadoop2.2-yarn + yarn - 2.2 + stable diff --git a/yarn/2.0/pom.xml b/yarn/stable/pom.xml similarity index 97% rename from yarn/2.0/pom.xml rename to yarn/stable/pom.xml index ec6738fa38279..45a14313f69b4 100644 --- a/yarn/2.0/pom.xml +++ b/yarn/stable/pom.xml @@ -25,9 +25,9 @@ org.apache.spark - spark-yarn-2.0_2.10 + spark-yarn_2.10 jar - Spark Project YARN 2.0 + Spark Project YARN Stable API http://spark.incubator.apache.org/ diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala similarity index 100% rename from yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala similarity index 100% rename from yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala diff --git a/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala From c59029402d0348a5721cc128c20b79683a75f4ca Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 2 Jan 2014 14:04:56 +0800 Subject: [PATCH 1152/1571] move duplicate pom config into parent pom --- yarn/alpha/pom.xml | 82 +------------------------------------ yarn/pom.xml | 99 +++++++++++++++++++++++++++++++++++++-------- yarn/stable/pom.xml | 82 +------------------------------------ 3 files changed, 84 insertions(+), 179 deletions(-) diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 94812f8c45c52..8291e9e7a36ce 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_2.10 + yarn-parent_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml @@ -28,85 +28,5 @@ spark-yarn-alpha_2.10 jar Spark Project YARN Alpha API - http://spark.incubator.apache.org/ - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-scala-sources - generate-sources - - add-source - - - - src/main/scala - ../common/src/main/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - ../common/src/test/scala - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/../.. - 1 - ${spark.classpath} - - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - diff --git a/yarn/pom.xml b/yarn/pom.xml index 7c145ef362600..aea8b0cddefa2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,25 +25,10 @@ org.apache.spark - spark-yarn-aggregator_2.10 + yarn-parent_2.10 pom - Spark Project YARN Aggregator - http://spark.incubator.apache.org/ + Spark Project YARN Parent POM - - - maven-root - Maven root repository - http://repo1.maven.org/maven2 - - true - - - false - - - - org.apache.spark @@ -103,4 +88,84 @@ + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + ../common/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + ../common/src/test/scala + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/../.. + 1 + ${spark.classpath} + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 45a14313f69b4..62fe3e274250f 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-yarn-aggregator_2.10 + yarn-parent_2.10 0.9.0-incubating-SNAPSHOT ../pom.xml @@ -28,85 +28,5 @@ spark-yarn_2.10 jar Spark Project YARN Stable API - http://spark.incubator.apache.org/ - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-scala-sources - generate-sources - - add-source - - - - src/main/scala - ../common/src/main/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - ../common/src/test/scala - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/../.. - 1 - ${spark.classpath} - - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - From 18b3633e54a8c902083f15e64983abca1eb0de7e Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 2 Jan 2014 14:20:20 +0800 Subject: [PATCH 1153/1571] minor fix for loginfo --- .../scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 4d9cca0766eac..9b898b582916b 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -68,7 +68,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte override def receive = { case x: DisassociatedEvent => - logInfo("Driver terminated or disconnected! Shutting down. $x") + logInfo(s"Driver terminated or disconnected! Shutting down. $x") driverClosed = true } } From f442afc22ef2e8a10ea22d5a7b392d41a1c7fdf8 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 2 Jan 2014 14:24:48 +0800 Subject: [PATCH 1154/1571] fix docs for yarn --- docs/running-on-yarn.md | 3 --- yarn/README.md | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f47098554e145..2c1b2cc294931 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -116,8 +116,6 @@ For example: # Building Spark for Hadoop/YARN 2.2.x -Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5). - See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. # Important Notes @@ -126,4 +124,3 @@ See [Building Spark with Maven](building-with-maven.html) for instructions on ho - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. -- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally. diff --git a/yarn/README.md b/yarn/README.md index 9a7a1dd838dea..65ee85447e04a 100644 --- a/yarn/README.md +++ b/yarn/README.md @@ -1,12 +1,12 @@ # YARN DIRECTORY LAYOUT -Hadoop Yarn related codes are organized in separate directories for easy management. +Hadoop Yarn related codes are organized in separate directories to minimize duplicated code. * common : Common codes that do not depending on specific version of Hadoop. * alpha / stable : Codes that involve specific version of Hadoop YARN API. alpha represents 0.23 and 2.0.x - stable represents 2.2 and later, until the API is break again. + stable represents 2.2 and later, until the API changes again. alpha / stable will build together with common dir into a single jar From 010e72c079274cab7c86cbde3bc7fa5c447e2072 Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Fri, 3 Jan 2014 15:01:38 +0800 Subject: [PATCH 1155/1571] Modify spark on yarn to create SparkConf process --- .gitignore | 2 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 8 +++---- .../org/apache/spark/deploy/yarn/Client.scala | 22 +++++++++---------- .../spark/deploy/yarn/ClientArguments.scala | 4 ++-- .../spark/deploy/yarn/WorkerLauncher.scala | 7 +++--- .../spark/deploy/yarn/WorkerRunnable.scala | 5 +++-- .../deploy/yarn/YarnAllocationHandler.scala | 1 + .../cluster/YarnClientSchedulerBackend.scala | 4 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 9 ++++---- .../org/apache/spark/deploy/yarn/Client.scala | 18 ++++++++------- .../spark/deploy/yarn/ClientArguments.scala | 4 ++-- .../spark/deploy/yarn/WorkerLauncher.scala | 7 +++--- .../spark/deploy/yarn/WorkerRunnable.scala | 5 +++-- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 ++-- 15 files changed, 56 insertions(+), 46 deletions(-) diff --git a/.gitignore b/.gitignore index b3c4363af038e..399362f7d3e46 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,8 @@ *~ *.swp +*.ipr *.iml +*.iws .idea/ .settings .cache diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 7c32e0ab9bcb5..609e4e47e3048 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,11 +42,12 @@ import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + def this(args: ApplicationMasterArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + + def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) - private var rpc: YarnRPC = YarnRPC.create(conf) private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var appAttemptId: ApplicationAttemptId = _ private var userThread: Thread = _ @@ -60,7 +61,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var isLastAMRetry: Boolean = true private var amClient: AMRMClient[ContainerRequest] = _ - private val sparkConf = new SparkConf() // Default to numWorkers * 2, with minimum of 3 private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3)) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a75066888c22f..952171cd0ae15 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -50,23 +50,23 @@ import org.apache.spark.deploy.SparkHadoopUtil * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, * which will launch a Spark master process and negotiate resources throughout its duration. */ -class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { +class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) extends YarnClientImpl with Logging { + + def this(args: ClientArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + + def this(args: ClientArguments) = this(args, new SparkConf()) var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() - private val sparkConf = new SparkConf - // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) // App files are world-wide readable and owner writable -> rw-r--r-- val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) - def this(args: ClientArguments) = this(new Configuration(), args) - def runApp(): ApplicationId = { validateArgs() // Initialize and start the client service. @@ -326,7 +326,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val env = new HashMap[String, String]() - Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env) + Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir @@ -482,10 +482,10 @@ object Client { // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - // see Client#setupLaunchEnv(). System.setProperty("SPARK_YARN_MODE", "true") + val sparkConf = new SparkConf() + val args = new ClientArguments(argStrings, sparkConf) - val args = new ClientArguments(argStrings) - - (new Client(args)).run() + new Client(args, sparkConf).run() } // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps @@ -495,7 +495,7 @@ object Client { } } - def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) { + def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { @@ -503,7 +503,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 7aac2328dad60..1419f215c78e5 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.MemoryParam // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -class ClientArguments(val args: Array[String]) { +class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -36,7 +36,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = new SparkConf().get("QUEUE", "default") + var amQueue = sparkConf.get("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 99b824e1295a7..09ac8d77ca0b7 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -35,9 +35,11 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + + def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) private var appAttemptId: ApplicationAttemptId = _ private var reporterThread: Thread = _ @@ -47,7 +49,6 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var driverClosed:Boolean = false private var amClient: AMRMClient[ContainerRequest] = _ - private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 9f5523c4b97a8..b7699050bbaeb 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -39,12 +39,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} class WorkerRunnable( container: Container, conf: Configuration, + sparkConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, @@ -197,7 +198,7 @@ class WorkerRunnable( def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) + Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 85ab08ef34db4..9fbc783d60126 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -280,6 +280,7 @@ private[yarn] class YarnAllocationHandler( val workerRunnable = new WorkerRunnable( container, conf, + sparkConf, driverUrl, workerId, workerHostname, diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 4b69f5078b0ab..324ef4616fe26 100644 --- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -67,8 +67,8 @@ private[spark] class YarnClientSchedulerBackend( "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" ) - val args = new ClientArguments(argsArray) - client = new Client(args) + val args = new ClientArguments(argsArray, conf) + client = new Client(args, conf) appId = client.runApp() waitForApp() } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 7cf120d3eb8a4..69170c74277b4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,11 +39,13 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + def this(args: ApplicationMasterArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) - private var rpc: YarnRPC = YarnRPC.create(conf) + def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) + + private val rpc: YarnRPC = YarnRPC.create(conf) private var resourceManager: AMRMProtocol = _ private var appAttemptId: ApplicationAttemptId = _ private var userThread: Thread = _ @@ -57,7 +59,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - private val sparkConf = new SparkConf() // Default to numWorkers * 2, with minimum of 3 private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3)) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2bd047c97adba..525ea7276208b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -45,16 +45,17 @@ import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil -class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { +class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) extends YarnClientImpl with Logging { - def this(args: ClientArguments) = this(new Configuration(), args) + def this(args: ClientArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + + def this(args: ClientArguments) = this(args, new SparkConf()) var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() - private val sparkConf = new SparkConf // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) @@ -307,7 +308,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val env = new HashMap[String, String]() - Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env) + Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir @@ -466,9 +467,10 @@ object Client { // Note that anything with SPARK prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val args = new ClientArguments(argStrings) + val sparkConf = new SparkConf + val args = new ClientArguments(argStrings,sparkConf) - new Client(args).run + new Client(args,sparkConf).run } // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps @@ -478,7 +480,7 @@ object Client { } } - def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) { + def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { @@ -486,7 +488,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false").toBoolean + val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 9075ca71e7efc..09303ae5c25f8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -24,7 +24,7 @@ import collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! -class ClientArguments(val args: Array[String]) { +class ClientArguments(val args: Array[String],val sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -34,7 +34,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = new SparkConf().get("QUEUE", "default") + var amQueue = sparkConf.get("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index a8de89c670811..1a792ddf6669c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -34,9 +34,11 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + + def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) private val rpc: YarnRPC = YarnRPC.create(conf) private var resourceManager: AMRMProtocol = null @@ -46,7 +48,6 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte private var yarnAllocator: YarnAllocationHandler = null private var driverClosed:Boolean = false - private val sparkConf = new SparkConf val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 6a90cc51cfbaf..5e5d0421bacb4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -37,12 +37,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} class WorkerRunnable( container: Container, conf: Configuration, + sparkConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, @@ -200,7 +201,7 @@ class WorkerRunnable( def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) + Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c8af653b3ffab..e91257be8ed00 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -261,7 +261,7 @@ private[yarn] class YarnAllocationHandler( } new Thread( - new WorkerRunnable(container, conf, driverUrl, workerId, + new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId, workerHostname, workerMemory, workerCores) ).start() } diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 4b69f5078b0ab..324ef4616fe26 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -67,8 +67,8 @@ private[spark] class YarnClientSchedulerBackend( "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" ) - val args = new ClientArguments(argsArray) - client = new Client(args) + val args = new ClientArguments(argsArray, conf) + client = new Client(args, conf) appId = client.runApp() waitForApp() } From b27b75f1c595139bdcebbadb43e89b0a7eadf2b5 Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Fri, 3 Jan 2014 15:34:24 +0800 Subject: [PATCH 1156/1571] Modify spark on yarn to create SparkConf process --- .../spark/deploy/yarn/ApplicationMaster.scala | 20 ++++++++++------- .../org/apache/spark/deploy/yarn/Client.scala | 10 +++++---- .../spark/deploy/yarn/WorkerLauncher.scala | 20 +++++++++-------- .../spark/deploy/yarn/ApplicationMaster.scala | 21 +++++++++++------- .../org/apache/spark/deploy/yarn/Client.scala | 18 ++++++++------- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../spark/deploy/yarn/WorkerLauncher.scala | 22 ++++++++++--------- 7 files changed, 65 insertions(+), 48 deletions(-) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 609e4e47e3048..69ae14ce8385c 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,9 +42,11 @@ import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, + sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) @@ -115,7 +117,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) + .getOrElse("")) if (localDirs.isEmpty()) { throw new Exception("Yarn Local dirs can't be empty") @@ -137,11 +139,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) } - private def startUserClass(): Thread = { + private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName( args.userClass, - false /* initialize */, + false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { @@ -257,7 +259,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s } private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { @@ -316,7 +318,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s logInfo("finishApplicationMaster with " + status) // Set tracking URL to empty since we don't have a history server. - amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) } /** @@ -351,6 +353,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() } } + } object ApplicationMaster { @@ -401,6 +404,7 @@ object ApplicationMaster { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) + override def run() { logInfo("Invoking sc stop from shutdown hook") sc.stop() @@ -409,7 +413,7 @@ object ApplicationMaster { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } } - } ) + }) } // Wait for initialization to complete and atleast 'some' nodes can get allocated. diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 952171cd0ae15..440ad5cde54eb 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -50,9 +50,11 @@ import org.apache.spark.deploy.SparkHadoopUtil * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, * which will launch a Spark master process and negotiate resources throughout its duration. */ -class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) extends YarnClientImpl with Logging { +class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) + extends YarnClientImpl with Logging { - def this(args: ClientArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ClientArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ClientArguments) = this(args, new SparkConf()) @@ -143,7 +145,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) e clusterMetrics.getNumNodeManagers) val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, + logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, queueApplicationCount = %s, queueChildQueueCount = %s""".format( queueInfo.getQueueName, queueInfo.getCurrentCapacity, @@ -347,7 +349,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) e val prefix = " --args " val args = clientArgs.userArgs val retval = new StringBuilder() - for (arg <- args){ + for (arg <- args) { retval.append(prefix).append(" '").append(arg).append("' ") } retval.toString diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 09ac8d77ca0b7..e4c6ab212c56b 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -35,9 +35,11 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) + extends Logging { - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) @@ -50,7 +52,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar private var amClient: AMRMClient[ContainerRequest] = _ - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 var actor: ActorRef = _ @@ -93,7 +95,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // must be <= timeoutInterval/ 2. // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. - val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L)) reporterThread = launchReporterThread(interval) // Wait for the reporter thread to Finish. @@ -139,8 +141,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar Thread.sleep(100) } } - sparkConf.set("spark.driver.host", driverHost) - sparkConf.set("spark.driver.port", driverPort.toString) + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -169,7 +171,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // TODO: Handle container failure yarnAllocator.addResourceRequests(args.numWorkers) - while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + while (yarnAllocator.getNumWorkersRunning < args.numWorkers) { yarnAllocator.allocateResources() Thread.sleep(100) } @@ -180,7 +182,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // TODO: We might want to extend this to allocate more containers in case they die ! private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { @@ -212,7 +214,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar def finishApplicationMaster(status: FinalApplicationStatus) { logInfo("finish ApplicationMaster with " + status) - amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 69170c74277b4..2bb11e54c549a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,9 +39,11 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.util.Utils -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, + sparkConf: SparkConf) extends Logging { - def this(args: ApplicationMasterArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) @@ -126,7 +128,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) + .getOrElse("")) if (localDirs.isEmpty()) { throw new Exception("Yarn Local dirs can't be empty") @@ -165,11 +167,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s resourceManager.registerApplicationMaster(appMasterRequest) } - private def startUserClass(): Thread = { + private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName( args.userClass, - false /* initialize */, + false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { @@ -231,7 +233,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s yarnConf, resourceManager, appAttemptId, - args, + args, sparkContext.getConf) } } @@ -286,7 +288,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s } private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { @@ -385,6 +387,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, s if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() } } + } object ApplicationMaster { @@ -394,6 +397,7 @@ object ApplicationMaster { // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. private val ALLOCATOR_LOOP_WAIT_COUNT = 30 + def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { @@ -432,6 +436,7 @@ object ApplicationMaster { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) + override def run() { logInfo("Invoking sc stop from shutdown hook") sc.stop() @@ -440,7 +445,7 @@ object ApplicationMaster { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } } - } ) + }) } // Wait for initialization to complete and atleast 'some' nodes can get allocated. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 525ea7276208b..6abb4d501779f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -45,9 +45,11 @@ import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil -class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) extends YarnClientImpl with Logging { +class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) + extends YarnClientImpl with Logging { - def this(args: ClientArguments,sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ClientArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ClientArguments) = this(args, new SparkConf()) @@ -123,7 +125,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) e clusterMetrics.getNumNodeManagers) val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, + logInfo( """Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, queueApplicationCount = %s, queueChildQueueCount = %s""".format( queueInfo.getQueueName, queueInfo.getCurrentCapacity, @@ -143,7 +145,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) e } val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD if (amMem > maxMem) { - logError("AM size is to large to run on this cluster " + amMem) + logError("AM size is to large to run on this cluster " + amMem) System.exit(1) } @@ -328,7 +330,7 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf) e val prefix = " --args " val args = clientArgs.userArgs val retval = new StringBuilder() - for (arg <- args){ + for (arg <- args) { retval.append(prefix).append(" '").append(arg).append("' ") } retval.toString @@ -467,10 +469,10 @@ object Client { // Note that anything with SPARK prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf - val args = new ClientArguments(argStrings,sparkConf) + val sparkConf = new SparkConf + val args = new ClientArguments(argStrings, sparkConf) - new Client(args,sparkConf).run + new Client(args, sparkConf).run } // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 09303ae5c25f8..8254d628fb918 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -24,7 +24,7 @@ import collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! -class ClientArguments(val args: Array[String],val sparkConf: SparkConf) { +class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 1a792ddf6669c..300e78612e001 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -34,9 +34,11 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) + extends Logging { - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) + def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = + this(args, new Configuration(), sparkConf) def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) @@ -47,9 +49,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = null - private var driverClosed:Boolean = false + private var driverClosed: Boolean = false - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf)._1 var actor: ActorRef = null @@ -83,7 +85,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar if (minimumMemory > 0) { val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD - val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) if (numCore > 0) { // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 @@ -104,7 +106,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // must be <= timeoutInterval/ 2. // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. - val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L)) reporterThread = launchReporterThread(interval) // Wait for the reporter thread to Finish. @@ -165,8 +167,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar Thread.sleep(100) } } - sparkConf.set("spark.driver.host", driverHost) - sparkConf.set("spark.driver.port", driverPort.toString) + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -188,7 +190,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + while (yarnAllocator.getNumWorkersRunning < args.numWorkers) { yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) Thread.sleep(100) } @@ -199,7 +201,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // TODO: We might want to extend this to allocate more containers in case they die ! private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { From 94f2fffa23436ed66a24c705f88dabe59bf54037 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 14:37:42 +0530 Subject: [PATCH 1157/1571] fixed review comments --- README.md | 19 ++++++++++-------- .../scala/org/apache/spark/SparkContext.scala | 7 ++++++- .../scala/org/apache/spark/DriverSuite.scala | 9 ++++++--- .../deploy/worker/ExecutorRunnerTest.scala | 8 +++++++- docs/running-on-yarn.md | 4 ++-- project/SparkBuild.scala | 14 ++++++++----- python/lib/py4j-0.8.1.zip | Bin 0 -> 809541 bytes python/lib/py4j0.7.egg | Bin 191756 -> 0 bytes .../org/apache/spark/repl/SparkILoop.scala | 4 +++- 9 files changed, 44 insertions(+), 21 deletions(-) create mode 100644 python/lib/py4j-0.8.1.zip delete mode 100644 python/lib/py4j0.7.egg diff --git a/README.md b/README.md index 22e7ab824577a..8f686743f042b 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained from [here](http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html). To build Spark and its example programs, run: +which can be obtained from [here](http://www.scala-sbt.org). To build Spark and its example programs, run: sbt assembly @@ -38,19 +38,22 @@ locally with one thread, or "local[N]" to run locally with N threads. ## Running tests -### With sbt. (you need sbt installed) -Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows on *nix based systems using sbt. +### With sbt (Much faster to run compared to maven) +Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows using sbt. -`SPARK_HOME=$(pwd) SPARK_TESTING=1 sbt test` - -TODO: figure out instructions for windows. +`sbt test` ### With maven. +1. Export these necessary environment variables as follows. + + `export SCALA_HOME=` + + `export MAVEN_OPTS="-Xmx1512m -XX:MaxPermSize=512m"` -1. Build assembly by +2. Build assembly by `mvn package -DskipTests` -2. Run tests +3. Run tests `mvn test` ## A Note About Hadoop Versions diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4d6a97e255d34..c6f6883b01152 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -169,10 +169,15 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING"); + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); value <- Option(System.getenv(key))) { executorEnvs(key) = value } + // A workaround for SPARK_TESTING and SPARK_HOME + for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } // Since memory can be set with a system property too, use that executorEnvs("SPARK_MEM") = executorMemory + "m" executorEnvs ++= conf.getExecutorEnv diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 6d1695eae74be..89c5631ad8f07 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -30,13 +30,16 @@ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - assert(System.getenv("SPARK_HOME") != null) + val sparkHome = Option(System.getenv("SPARK_HOME")) + .orElse(Option(System.getProperty("spark.home"))).get // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(60 seconds) { - Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(System.getenv("SPARK_HOME"))) + Utils.executeAndGetOutput( + Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 4cb4ddc9cd66a..894a72284b3fe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -18,13 +18,19 @@ package org.apache.spark.deploy.worker import java.io.File + +import scala.util.Try + import org.scalatest.FunSuite + import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} + class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env("SPARK_HOME") + val sparkHome = Try(sys.env("SPARK_HOME")).toOption + .orElse(Option(System.getProperty("spark.home"))).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 13d5fd3685bfb..aded6430b3767 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -12,7 +12,7 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly The assembled JAR will be something like this: `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. @@ -72,7 +72,7 @@ The command to launch the YARN Client is as follows: For example: # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly # Configure logging $ cp conf/log4j.properties.template conf/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b335b5a20af80..8290e7cf438c7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -20,6 +20,7 @@ import sbt.Classpaths.publishTask import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ +import scala.util.Properties // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -68,10 +69,12 @@ object SparkBuild extends Build { // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") - + val sparkHome = System.getProperty("user.dir") + System.setProperty("spark.home", sparkHome) + System.setProperty("spark.testing", "1") // Allows build configuration to be set through environment variables - lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) - lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { + lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) + lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { case None => { val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) @@ -79,7 +82,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } - lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match { + lazy val isYarnEnabled = Properties.envOrNone("SPARK_YARN") match { case None => DEFAULT_YARN case Some(v) => v.toBoolean } @@ -112,8 +115,9 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, + javaOptions += "-Dspark.home=" + sparkHome, + javaOptions += "-Dspark.testing=1", javaOptions += "-Xmx3g", - // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), diff --git a/python/lib/py4j-0.8.1.zip b/python/lib/py4j-0.8.1.zip new file mode 100644 index 0000000000000000000000000000000000000000..3231e31164e51daad4254e8f09a49336205563b8 GIT binary patch literal 809541 zcmZs?V~j3N@GdyEZJ)6{^Nwv>XKdTHZQHhO+cwVF+TZSexc9$zJ6%bq(_Kj=^*mLn z>Q;~j1w#V@0)hgX;Ug0QC+QD>1_c6Y0|x@a_;1(VgVmCbk)D&Di9tqESX53~l-}9h zIYoKKc8e3K_mK*Mm(BlD)@y4Am2QC@SaUg1MTZbQN)AfHP+$T2;RPT}QNBgjdOlLh zm3?bBS`8;(x4_Rs{^s5uz|hv_=lkLK`R)Wf#aAls=6FP-@*R}r$uGtu_g`w`V3 zf7erpJ$SD>NeuUP%|j7R|4f;1_wjirQ}SwE8viwbuJOnMAVvY!QI;J-Jn;NWL@q2t`g3uIj8Sl?-798C=!XnFTYTNTo@m-)SgMUfV`Bzh zol@yIFLn;lgn8sxo;+^mCWSGs&klzMDH>$XP0L9@!%2!k*K(^#sq7Fu14M)SI9b;? zc2-WPF%g%%f=OW?-@&fifbAZIYPpjHlUO$syZF8-p{mJKoCOPq~i-0ft(}tzy^*O|)a(?49GxT`Jyq-pwKSd7B*QFiMKjwe2+;ph z)g+oJ%+}Rz4h{kY#P=T#|39h}q{ZnZ<;3JuRJLNX7?FI=Xig!8I&GHKc-SncUD~pu zMBF}k(xKfkx5f!t4{reo##z-$6>q@6d}$f?0q{(zc2y8@zqq6b=rLF9k-Zn0>9=nZmX8s0j>UJvNjpsSCX}M z+I_8~4%vtX@qk{B=uyF>mT?&f6T>|#%E?~$US8+F8v=JQ(DUW_PZz8a8I z5RxpwBAaSD&OT(ko_1kqdz>Yrsm+|o(`CBxyGx^;K`smv!!)|cd$uiaVI>RQ;n@39 zXSa|`jB%S^WIRrN7gv6Om+Jt1+VH0zs35WV;<_vp_+87jU{B)N(6 zx>e7X%6b8=Q8!g2^bYQJOg@)0XK)61KJG@%n-kEiy*}K)wHO2LP~)W#7KEHru_NojoH7oMcd_J zs$DYY9&kNNYU~{a;Vayxk)`q`b4?2489c@SKk=7Ye;8L1@3=mtw3(>Lmb@lmq!)?d zz6~xm@W8&@A0!TdAJNb7-H6!JR=$1N=ZVUniF^2^7Tg%g6VleC&Gw+PyLxK&Ny`bY zHW*@YRf(%8X^3c{Bk$>XU2$KIRlAh2`C}8aqgXi+cSvhAV-2|2DL9crQ(SrCm|==}C8VWXAu< zPx^Sh=91*elo&JC%&hvshkZHox%*YHdwWqisz0dR5C;IPdgD(xXV1OY$b*gv{}**9 z|3}@-P@V=g|AFuHKf(T=AT+i!VxTv4v9LC_ur;;&6Ek2tM1&-A`xBYqSVOV}7Wjx; zv-}7Wr8yBeF0eFbr$Z>t^0*70H%Io4Sv!S6#D(#7r&nSuGX(xulDg13T7=C(R2uHD zV{*&wa;#<@-knsoeCY8rmg-O63QcdE6wYqG4vygkm$yICXRE=P@>impeY67e-!SF7 z{_%wFd)AGS4f7Qqt||()MCP^(9W0!hXJDtL1uRs|v!GIw=-j93Y$d&=ozBlY$p2|$ z!zLz)E;JBO;(y`p|HnoHLpv8|dUIzR>l=-4$E|kc-!0Ebx+SrfWs;g>&lnp?d~G!x zvoBTY*&SK(qX3EUU|AG6AStyhuY9inK7rejP9TZMk9yp*AIq{_84-}E5yRKEKHd7q zuB54-^Rg;n5$cG8%hR{(W<1y*Y2bR=&v_hXI}^#H$<_(;zX)ZWfZU7{Xq*L z&IF~6fU_A>7Pptp6Ot#fwcViA)zzt}vDC^M7f^K09 z%xdlt6&WvaqgxyHOKr#_PaL+UE`-aY} z+&$gxJ>84^!smnhFxdWCS4~@2Q&S58z80N&PHmYVjO_%4-5CL=xu2spYo?!RW@h5* zLjyPOClePV6Z>PJ>G2wiA;VisO#?S~r#D;5*;bw0lr7JWsd}k`tJAjL)UR}vt2o;Gu*biKEVtTZ*hV5cgVCclEa9Fqw zF}<~haTVkgGzCFh0mZ;XgHUhjaoJZt@@ zuB;R0&7bw|5ir1mPi|Rd2kcB%k=p0XYhAD1uSegCt?%YWJ&?sfUmpo^4l_56Nc8dx znY@oI<`w{*uLYf05Lb$@y*FQEK~Uy-%XMjUdz(vyNj(qJB#VZ^GEb znPcqDi28AODCeK?W4$Y_CcmnLssot^4`ISxAepm6dz@Duf?y{$GNt_h=w+jdQbFtm z*g*$=qHcUhmEZ2huBIg6%}%Vkl7-oHP>1jOC73mguMC-zL+E{f|6156nQVoe|HHC>U31k>7N41m5VzwFau955^S^A%g*zlv&or5r{`z?FGUR@4mf zAAYJ#dU4-!m&wi1{&NR=WHXh|XmNP8-j#B%AMhVEJL4HQ>RgD{GdwNc#qFha9n0}b z`N=%p1?0|m-mNAY_nTQvODaI+q1DAPP+pc-W4y&pVT`Y}%;h*B8-wtcFZ_FoH6q~a zQ7BUwS4obO^^J~D0EUiXE;FMbP`?{&DA|(eMmu(zMTGsEV*Vh-x#HgeCz~Nk>l06# z-|hh~7B@K!Ho;7tie!fTo&HI$j3p`)SPFdzQXP8whnGQBA{6xc|MwK9)oot9A zFA+IYXNb#?PvFen&G31HKqq4djg+mhrrK@1=dO$iJgj=j(5U9BIn99Aqq0$?Tv>ct z7IK|d_wGx?Y)d=?Sz0IEl&(SC)|w6eOm}&CZvE%latK00y+8hkQ@&5FFmpK75V@LAYE)t^d+5lle@n3R5uhN8TYObJK|k-`U;b(y^sj zW5r~!Qth|~WXpDI$ki_GD0%tY43i7#q-fro8HN0rB;OfR zDA?Ksr=VL^4Z%1Pk+F-mE~Wx|;>PPw{#xUQrM5ky&$zKuAj((9d5nf&%pD@9A5N!r zFi`w)K3Rk(QR4}HjV=5+(0{sOxUcIYbK(?yk%DGn+ZhQIAU<9XVH8f`WEs`V-_}hR zRKbKvB>HBzv{}X4N`qErMCPH|s!NXnM~1UdKMj{B$usJVt0|tQ>~K@A?8d4)$z59F zFRh&=!MJjbDcS)H`79iW%J5VP+exf=0rxVpAFX8}jg#Ac0b>(S|0+I%Dz3G}hW_B1G1a2e4IySj|r0mJ7O z&02$>QOb0|PdUd_Ls9%8*m}a1;Tqhmkn9iZCkEZ_c>2{}UX{nTprXxl^XQ|5n-ig& z8RRHCL+{NJ(baC5sWZc1jB1@0#OFmI)Cr}WSWS$>5_eSSICZNHMzLZCBsQ3#mXq$W z#3v55k5VM~A0lCRx2ep| z0GoLHREwRQsyCW*ojHZNUWI^fKna1H{>#2xV+6k+P3EK`G zS|qH*dIA4YQ53f%NNxsuf_(tPi_7aYP!J7tqGL85O{@rr)5U+|-+z zWsv-S*{E)p$H(I%+NHB+mmx0q*>kQgx5@9v+2tkiCIfLp)%-mHw=2NTqi1)*V@$uA zCE{p9u+GiP%W9X8Immvy>%Vw6wE9xt`L!O?(picM$mun}Q_6Ewo{Nj6-rVr|dTS_0 z2*Czq+fNN3-Z@MfMH$4<|Hij-i07CE5IXVy)0;8t9Q~dJLq0`no0t2 zHf#9#=y5TMD8p9_ZNQ~5W<0T*m7A@Lt&?Mzg~Y78oBIPW{7o>y@_PI@*?%4xSpoLl zVD{4Px~R&N*ZV8v;W^Ywkhz!TwYg=6$?%t;uagHmTbGBQ*U$H14wSvIJKXG#UUn}J zwob3d{UO?}O$H~I)^}bk17sSeogP0R0HO7e@j&}w$4-8j(#ud_Dmp{7Acy$=J1jWN z>CtETqHb5>5Wo|CO>}I<4(5yu8GxzEmXr{jGMq#dOeH=ExWq#Y0?W+13uiGWA}&-I zHaw6XG(&e78KAX+sL)(-G!SrWY4OGq*1EAuS_S?B$@g~sx>R8c4CYH;INB^VCGL~V z+T_@XAVlVh>sBk7Qw=xWbFoPAgw?gU4_iI{&>8qyS@PkioWe+ z|5-5pyjcZmA8fFJj3BhTX{_NG+v&OO+xd09?fP$3;|4%S_fOkctE*<}b9k?z6)I^J zkFX<++uD`HL=|Fc1%hBRD8+ZAIi(;Nv)j@JIjzZ|P?#RR7ucv73~xOzL7(x#-92n{ zC&>yM6Hvdi_kw*laIPPTZSL*vZ$%<&DJR2ZhNHT!jV+Muf^cg-IEvD#Au#d88+vxkpT z^qS$i{jwzmL%P}C&a`+nS*!s&)1*Yojwes~`KE|PTh^P)A{V&UOtPT;k3bYvv6Tpno$r^N|gpMzFSAds^;T&~WyJ7Htea-;%5y z0EM*fT25_;u3C={Z7-rp_aVe#@^xjlBMWyBrRt7i8QXvqdIQT}`yKgP5;}o+AK;?XbJFI??B13>Tni$J6diLf8d>kz|2J8Q!VsWNI?Zfw7h| z*p{@2Hmx@{?FpQi*mS?b>4x@m2gL*2TVvw(dbc!zmE5>dmAFfwshJz~dtJECH&f5x z1?L8Kff>rQcm+{4d2^WypW?ng&-@*Qykt@wr4$9ASLPz@LnHyMf9+E z&Yr(l+Kkuged85!vb}@OA7j8CRYohkrCh3<^CDD_E$j8V7m6K5+`#V z8Gz!HwBNe;D;KEkDp`}PYI9jvb_-oRc)*=+V^7meF|<#=gUM;j`dIxA{0N$-PD$`9 zq4Sv;;c0>BA4a|KK}Q|=~aA?!rZVgvn>t){hi zKuU9e7|D*{72dzNq)AZH0FXAb2E)3J5FveE{af}wSN+n-799;;Yt`cm11uDJFomzY$g znVfL7(S+@IrFiPnD)*Nu&aZ;6{hK#bx@x0YWyl03_fq|EqL!ae?H6vAGx;ztNN^s5 zWid;@hkb%FW`Ls^0@Wz~z4a@u>`QCoIWDlO3j+Q6tfD@QF|vl2qM^M8-EZ>6o<~Ak zBSR=Pe+RokGUk=*3zAhWw?cN+AAtxL+iMUKPl5I~q}3z~_+v8gcdiAQqo1@E8;#1< zrZG>zM&H(W7*(#v@IpVBU8QCln67bX9mg{pK49+-sa;3B>}vfvyf%N46D~314T@wn zfs1aZ+tLx6H$erX_#;4$kP=BUh*i1{ zsRhil69OY8Xkip49&nyKb9z@t^VUs5a@xU^d6)0Z+Vr*TjqS*kg`)k2AddCgQV^+G zUJP3n=by12Le)9|!(|4&&?59H-jDOr2ew$gE@n>-(ak!^yf|$y_W6S{c^KzkBi#$v zSidZ0FB+s3);Zp**uR{yUrh+CJ->vqKB$B{fC2AHH^@EmazwOjKUC)PoZORm#vI1$ z{(*c~#;peuVX+XS&pady%I(#_xfOsYp;jS74`o;iCI-n$xOi^|vX-!^M$Rb26WV`+ zb5q8=Ad8~OexWHWHx~JX^m9z+)RI1gJ-fDcYIW?k@BwZ~v z%@#tS{$GuX%4Xk|+sxHgCBuojU&->WhzAbUjpzO@ol5nVW^d3)gZ6!e1tRj(5szQ- zm!-w?Noan5J@VbAlD&TWDu0Q;()%DPVbg)U<%$jr4D}MgCrQ=rut?fu2`RrPiE59S>*H=3D5xl?}-R82-Q(bXa zi~rM5(X(d;=9Psz@KeSG32q;_2fyOdtPM(ew_i2i$OU8H$_rnJB~l0f5K^!aC|Kcs zmfu(j54*}*r40@q`9&r-$f)XvAhivI$c&Bu$>RI~f{dfZhts7#vgK07FZD^|a;Ap5 zMMu)rIvZ$FE(Q+*+pfq6lVw6MtC{X_Q1Y&6)v5vze15{N?iw!S4>W(%D171N(rl?b z)NHS^Ry|=x*twLdlIYlkiTfu=i(t^dZO7%MxRksm&_u3lM=<+RSn6vWw~DnfI?!!L zM--^&bANuH1RYjG=JTN(-OE!oLFkZvld^KDNg$N-*k88*0ivRvcx@8*l5hxCcUNX> zMY~;;zIUYl< z*PzT5mLK6Kc-eUhm)cqBv%SDw6RI^ep^oLYD|p-cf@I#RV}Pp|F(zw6nCZX0oa%3C z{#g8!P)%@UW6n35*hg`~9XZ4v&nlzR^YVTYMi!pey|ZT`8e@{i>q)U&%K`d%bLyRR z?5e=>xdVUUVWW8GRh>5$M`&jZo+4QmYyK}tM?%N_(>*wtfTf8`E1TzmYH$fXs^E&L z^HAfw`XcWB4S5Oxj2R}kY7PcNW`zY$RLPUe1_7F#%j3+^=+Du5pyRiSTjmX}n>&cD zq^dW{%4+d|{CkZ4F8}&)EX7LDtMQ;);#`lLXe?BG$W0Iav*i zar&~=14G^^7WQG?ys&vyFug&*H_967jkA51%A#q+|Iw~oA-fF|ol`iiG)gZ#lZo}I z;*|O!g6%Z}@$idgw3a>1KDAC0rc?GMVtH$L!cK)dpH67&iv*9A^QR^jWyt^$H%!+H z!I?j?X4DvX=qFC!7~`8yA2G!lYZB6{uV$C2VFJduIRW)KzK()IZSm*U1bF*#+SZUn zouISlqNOjirDEgFea0$wC9QDwaGWpqN+|MjGR?oCPpwi|G3GwRQsqMk4L~is`IqJ| zYjr8yiL}F5tH}ELP*=_h9v~F**;VJ0sAv|##X3MuEpjz&K%0&DQ}#ovp#nXyj$YRp zT|KrPapu-l)Zd2h#}WeRP67`Vj*Un+DXZ&x-qmOq6NvETd@7bHI|@k(ojvCM<=hvr zQrKP}X>GCBy*02d=8e{HBmRbNwcF0lfy5Jq=amtD-pB*DSXj@1PvDG!AjZ^z;}qr{ z99C-=!syQ!pI{vd6{HNVymkg^YEa=|F1z4|gP)-}ER2~CEmnvcnc0+XG;$Xi=w89Z z%>nEfo`KtBbDre5rs!M~MX$xEH-)HwUI1|w`^203{5OPt6&mb;E(8}m3ZE?ZdZ^I| zxyZ(Z<^Ba9mz=3X2zPJ~Co8cD&9Jo#iO~Q$j5TFk!f9X>cz^903`%@ZSa`NZO+^sx zO#m5nidftU-C7`64QH!juF#k}1UsJCNps&tB;!x~2XC7nH_EsurRxTdME(gUv`;BS zN7W)sKP9W75@kOzE)JVyIIPH@JQSN9jRI+c+uF%1V^a?>G|s~iAgDBdFmdO^)-3iv z4g%%F-Hcsg*ybFwr)M`^-gzOm`Z`ofjiOCcp}>j$P{$%duGGG9|lH=h}VC7zR49yRs)1W49q;tS2LyL0wG9aM- zW(wux^#4-~WTp-#EDJ#V+Ro{0@ER&27^DL=s)mvuiogz9UcY!60o-F@R|qOEgQ$Q= zIm_11ME1XOe-ol^N^AKHyD~=8VXA|0D`1^l!Zd+PtZ_AD%n2^g`m0G?kccivdgx|~ zXn5(f6CD#!5mO9WsK_)>6yvd`38~^3qG6G_m>2wdi!D>T(|iC`jUOF&3Ah_GyHcyA zjQ{#mCK*r#mhDTiGDd)+!+^#`DWtP#kf*gqX+#(*v?Xf(Uc#v$zDdJrub7aq>E{*t$!Hp__;vC=Z25Zv4NOT)9|=W{;er?cSgW6hHqS5p&00|m$yzbO=#715r& zFuQ6&NCR}h<>V85U7!%995^ygpe*u3cFu9!?nwG%a}eT?t!k`6=KCTEDl5`*$8rAX zi+JW*Y80nPfPJuiY@Yj6?~Z)8{$!e)9bpv$31b^OPY7$Y&~@3*m$E!Dipz>(zGd>M zf$F<`U?zUagTXJ`g#9GB=9R3@ORan5_|}cp3h0Dloncprjxr+IfL#l$)K$wBLG9r7 z3*Rt`y z38Dqxc19j$=G5|EK7ng{OL;+xBCy>o+xWt+NEyLU+q4VjbfwDKSKldHbGI={+YNoR zwV-;-4uH7Ou`>tW1PUPb1+g39}IU^eYNI&F4y z@f-j01l_NOy>WN8aGlC%Y(QP7MF%fCM`zK?M+n5dr{2|^UuVx*JGAhzsg=UjeC?q9 z>nbX1f=y%n2PZ45hP^&DW$+}!ZW+wr#<57}a;mEMIa&9!bWzrLbxat;C6lk=t~)JO zLIi?MOkZ)(!y{HJ9m11b&a$1b$Vqtrx7&DP`RJm9YH($wDwX7roX3mgW zi6^aNp6FTmSnSbif?dtpITAbPF|VRB%~Y9M$;6;cO=tck(Vz-7@~aq|i)9;-%loJ$ zjY@54Da=|5gQaSZkX*Xyup04N&43Lm8BKsvVDWcp3AwIr+3yO;(;7ma*8&Cf-2>Y) zD|U)inFrHv4|!*W2m2gXtW2BTA2!k^8IP4c@L{x=kVlZgcoDKl^R4) zDr>Dj9-KlOAzWsE<(WmvTgx9s^@;wpqkhM~tO^KjL8CJ(iYrb!dsBMlven8p)kxf~ zwTbFkDAX+Wp}d{PFjkY6Wk?v*A7b>$2bJfEvGXNh@t_b-D%g_R7!8w*(STPYUvC{L zT1MKBS#*qCCq7BfVi-CZ*xtV}w#Z2%6YK&-Q6m`#_7gOt=@+aq#;d}UR1Oqs7dGrqeCO?r$gRZ(#kF)(iM0jBfv^@L=6%kKIwb|Q zZa8a+J40s@9!svcskjcNqY6celRL5Ju@g$L#lqIerYuP9?!k;|`N&em*SN9P`BK+< z>S28^sCxDC<;ywh;I<_#IPC1mk`t%LI2;{racU9M!I6IGkct^|uwTZl$9#&{HAJ&$ zO#)brt0i$LNf}&3K1mDpXt^J3)G6<|LfI!IY5_!fpf)kdYxOvpI$f~qq1o2zL6>Ju zx{+0ba?6_${*xU!!eF{tkbKAydl0^bAMsUqe|+>_oe$u#|jQ z&<@vGBPgxK-@d1C0VC(sDXpXhA(BGf6!hG^!)@{}KToazxEF|M!E z5?a)|Ohp7%!mI}1D)9M-n;GmaRy=b)#I zq2xU{j>nbs5E~c2B$KAy447$w1>|U6>bN>KrAtE%OuqCc!PKh3Yef zjvCe~5$CcevU^IeNaBsW8w%Xk(;i!d#1l1FJ`yuhODW4n1O=*%x)MuFbsb!WLUfsS8KRq}WYVoag4Sbok31FzUT;BIje3*`Ar0 zY5Pwu@N=T|{Y#<%J2>8;YCp(uTive`JsIJcn#6*Bv$8xxa z)hZuy4yb5=u5YuT+!dWh+pZ?2Eg~|=67f%QVrMgj5dn<=WD^cyDg<>X$ez6T>DY!CL2v!wgq%!_Cyy)8rek8*dA+w@eG+-qN!-d zNlihWyk2%UAD5pibNk0yRnlsiSj9rw$rGZz{~@j0{_5#M^?^mb3n2DrK7^Okt`{ZvI=X_xFiU zl_Xs^=5^w>&DB7>xC*Rq82tg4Oly_4z%x!J8gXHdk|wC|&8D`is5H)Qxtwy;$V#(B z2^*X>;*R$wl~UgGmKvM-yj>Jn&qf4Qt6l=Y$1d%rE{B9)egVCQr%>-Dv#IgSdN{*G zWBV#x`z<}X&eN45L7#$63D$0%=6%JT%1XjbUii@yEaXFtFm$P_BZ9(!TS6Wo#r<(A@GEu#GuR5X z$6#=zzYcbJakl03HvXeWE6#I7Odb&fgAH;6-Bw3RXG1S1Oy-hFG?Ql;vWjF{nJ%VE zs$>Db(%BD0*VW7q{pl zHiS9{X`T{*sN6hD4+*O4PlR|{DQLey1#W6CVU|#`ibT3dMJ7F}fR}tqU(k_qFkGQh zK$|<)&MqBnB{(=g!_F~AUTOtz{(Sh?_8nkRJKK*sW(cNx=@in>70cry7l2T-(!;3K zWO5pszz6e=H99*Ka5b0c9`Bm=+NGi6?IRv_8Twca8;2_U!VFcLvCSI6I_=pi*)s0D z95b#hi7%H$yX9V@BbGN(@g$5gzS{0MOtMJcd*#7r2Y14AKY5_7!X2Aq4eg|Uc` z07p=L#6F1y=`5BJO>t~lHae4o(tRGD_OT_&z>L|Z`v_-AD(_XVE)q!*K(e546%h|; z8rA$)%IE!#YO%J#fXDAg`rRV82on=FH8QHs$zA$Sp+7WLxd)htGigx#DHQMcq7NJBmfRra*&F$_ zX>dVq# zh8L1njBvKWyHm!a4rijolV}9*fP%YF?VVJii=v}JD%UGrNfJ!M zA*8<=VN%ZnVE>jXXmWeO5xUFlyG-zMTvKCL)TKg(YO(_Z;$V~++D(mJof0OP+4%f2 z)MlhLQH%gz7P;$>w}hPw&msz?NOe&DmbHvk8KO>!X;!%hy~?K7RgoxeYrmZRf7w}` zE;OF z@8e|P3u_s(w{*slsIUwhnK^7z`ERr;#!19qpwUXp5E&5pT8m@p2!cAQa77}u=Jn~% zMFvK>Zv+d-cCaloUlKg}1x|9$;W?7_m9B0hB!!@;q$aWb-+ssL2RztJ!qi<&qJool* ztt0nu1MNag2Kiba{ezDy@<|!Ti&qcjBPyR1Qe%;7%15*rEJKM^???UaaNS?2_ptN` zd2n$y3X=Mv^Z+@A=&a@`{~YP@6z(9ZC*!*yo`m`#|3S4%++hb6x+ELdC#>O-&6n@a2VStnwh>CCT&`k?vqD^A8B3}me&AHn%I^zC4-0gNY3lI(lp?OrSaE)8XAtgaL z%ilRR%0>GZZNxIez*}uA=;obQ;4YXmG@k0h=@qfc)62P1E5Tb`2EE|w6V)9wx7X^2 zA7ZG&8GnKr&irqF>X0>RluLxs@W<&rq}zJOAhO9=P|8kasKY0Tjt55;PosG85H_5J zh91hCvT&R@XF7e1Z{F0%)RCi}Nr`#oyck-*?n*3HG7?3M0{>s40x5)~kfBHZG)Dtw zhaVmLHidWa;f+8r2>3BgHK;LbRCMVAWL<6;vM&5KUN?;U?0W^it5{AGJ*rKlK9-Fv z39#nf=Q_FPs&&=1X$@nU`|$pry#v`TdMAC=KA}J^MGf z6`MWRmTVF`Z~0bsOxHQ{Q5(18O+!lO3oFWG_i|`$NPVGLrSS%a2x8$EY+^NA9dSGr zSkmhl@XvT+i#l+>bmLQ^KoYv%uD!)S;(9$&RnpcrVuDc7i)T!{coLc846)XPBv>)Q zdZNn$CM-A^Bs+Ca+6hOI>62svK}nERh6KkGi4Bx3O__x2zz%d~@!J^{$KwxtTRP}L z>uRanVh-G!f2qhhWbhza#Jy^H)3|JkiFuApmtcN64g*c=)$lXi7tZCVf+*!wQ^I#)xuL0c%q$Lj^elnT^oO=+iEZ zvb3NtsxY*hHepPiI^E0WKVrGR$qucEJ#TohnG8`<3%sVz5mdH-^j%OxY$(Xi+~kyY z&QLwNVQe@p>hhYN!5mpz@dvh0%NL1EB2ys!*ArDF@{v3t^i%%@5J0Zjr2e~sI$q+D zoWy45&lUYKx=qE*gH#ni3mWGr$>+EsI!$ zR+cMS3T_ug7qR`-jF??{VI_TAhAr8a?A4T;=qWVWWX+3s3T$_ob);jX?OgI!|E<7( zt$Vh^0wm>jkVy6o-K`(#x0L$Jt-#sb)`PlI_x03704$dX_N`E5UDw_$EMG&WdS;03 zdx>~vPH2vueHhj)5(b>`j=mtO+8tm|BW7 zNulgVR3=oNdU)1+X=WV^s&H}=1`-R0&dl{w4G(sn!&p_`5$+5MejN#>n- zImRNB&8mi;`{XtfPJ2O9ZB28ZCO;Eak`DBE`S(&tH?c4ZSO^MO8)N&zn#KLb1N-TC zxgUzaW76Ek;AlLy?Tci9xhz;=3fk1uGa06C?)9UUrZ(Sb>oguOrY~Itx|Y9^0ZJI# zfStcG8FRI&OW}HA7VH5nE#dy1!J+v4I24z=D4&JzS>icb%uTWM;WA_}`H;rOr6@O& zBZXYk?yk12S=bmOq;#ocYRDQqCE85#ILKAhJOzK8*+BSV|2(r#K=6S(Ytf7cm(#lg zMlRj-bt{#bG_0#=X^M#9tfU|{xWSpB>*JsZDl9ePvs{xgH8_wuyE<`3zIluJ;>PI< zlJZnFVM7pIV5=L4U&$Ys;S8m~S)~q=)OgnpMOh~GuXGLxlLUACG@6mGwDOw`f8{NwK8spRGTd6asp2$&{y1NYt_K980Vy*Hz@ z;*sG4Q^U8q!KpV6K-OWi!DZ%f5Nt$(u0crfr+^hb2k7Sq8p2 z9!6|D&@c=g?~vGG-Q8&!Ma?o&J742UQtogd(>oicigGYMm`CduGn6rp?Kz^xsuuE> z4#EF?G^iBTEqkSK3WU7URj4rlndR)`WW`?G#cMhX*+y$39{Nuoq>E%nG~eA?gvl||e}{#yc7$-MtU=5D2^O&W0;q^#e$b_{uP zzR!TTYcM~_I38{6vyqCbvvpXnxcD1bd1*?7^-NTZf1*E6p3w|KrwzqS5^$Y7g*Qax z(~aaL@_I)yY6R>4BuIrX5pk3j921VXtYnzj1x27ffi3ngbKJ^$H5t`I#6Ze`PQ2LL z>=gE*QABs0q_!=hQJTT|+8{TXGS)SteN;RKgh_PMNc4}eT3|#tu!@Y9Oa%%(Zj!dqk9(kX)ssXDc}A5HZXstt{MY5m_>G z0=WbN9BN!XDj+o3iV^Y5vb7$P+1J=^s&ZY_Aa#}amnT+g_`edyJ_(vs-kAxY%O$yk zSx^lrPA>MQf>HUuKfwc3x*WMR2vQG9HSQ%Q81W{#h(zK$K$4Z-;gL;EY~zdN`X}hN z&Y`YJ<=#*Gz($A!{dtgxj%P3Vitu#n(o0#8VM*fOen=&tCa-$_2*EkuKtk^c)B(t6 zh{NJ@H-AH%THaAF`^P^7ygKE6NACH}=az2@`OM!J=qx3ao$ucBJL4U$HNDcdVGNE* zJ4vyZa3l6nQdR|@Nq8G0XEl_cK!QI0!QT!n8C%YhHxZtrYW|IrP%ogPB?l`Dys)9) z0Cew0mdYK|Gzq}y6HxR#LV1q0DNYba=Puge3CQ0%gmygFI6ocpYl`@HpMZ?-i5>-E zh{#`}MDp-pHw}Yh{7D~GV8$NAaNG>Ch{3m#L3-Mzl$s|+aW>XLntx-9Q)WT9RyHtb zv_iVeM_fb)bdf}8*7C?sNuYoi4(Ak(WpY=dI)Nb>c4h#@)nO&B*zp<3&JAzQeA=dC8V|Q!bmf zLpoRZo3Rw6Q*9#H))ZYIM&QG}uVVV?H(-9Aen#*5%rMUwof_u1oySaIP6LNC@>el<|c6-eDa(R4QzKx+;9@_O_>{yPI z#SusH@f&JO0JP;b>cl}t;X9oCUuQhrN#h);Ne;m>;j!shQ`z;KvEAa9-==L)xhT2N zXPgXlQoDm^Ce1iJb8g5;^{r9mE@{Js8X~|!Sk;Fh#!2}{!@Y}Z=+-3*2-{e~gEFkd zwdJ*96s*{xd981wYScJUn(x03y|1K@2dAR`uCK_KrCtADY@9=wC;^jY%eGzh$~Irw zwr$(CZQHhO+qP|M-akFF=&yU3+ss8q+&K3fpT9#r_EfhlY;s=l(p>Dr!;*af9Lh z4!20%$Gs6@`QSc!cd8HNUORR59USQumOfHhMvazjYPJ<`Z@7Qn;eI}=w&Y#W+np%6 zj5e8^aeiAJc+9XynU%t-^!igW3rE^5|xRdBsM6LclNLjTT3k5WfPXxAo2f$AtI zmeKV~wL#$bN~oKtppaUSmsWDPit73M@y&-3`Pl7CGi`_zaj054 zJmbA*V)+E41?ZX#d@i4J>%D8l8}WLj;E`22(;>{R6r_xZmYyK_RI=attLR6Iuf7l7 zo?w0|Bn6h~uiT(`hcauJZ zG-UWCCEN>B4KIEtL`b9EZNa}xJuQzQcOjwvBRO^O8N?m9%e-&(yFWO({T=EV>$0v0 z;PBA`tf&ygP4 z8DZ^p3-!Hi99n?9c=*Gf)0bG=LwQMd6K1Vg-+IG|Y;S_(t{c_T+A~J+sL5FsiwB*I z72|hK1pwv5sD!PxgdB5+3`WH9HGR2g+V6z+R<>~TvDu%Xo3a$T8+9>D(Q{Taeoe!| z@?X@ubAsE_caI{IG7L%%{(Z^maERkpY!50!}J0Wx>={$K$#z7Ou)-@Nn6yr}QO7`Qy>pKkBd zRXKaKlb zKDo}r)t=WD`G=f+1GA%6e73FHfokli6D3o^-70IQ$Wl3Hce4%P1{)oQr5xqV)3 zK98(^55Dhr_UCtgc30I?M>V1(ad3Jzd)Ytj?Cio!qy0*X6@F8%8zUPJ;Y&L|&GPgc zw3xpddD*teHEsAFqtef*aj5Pem1!H~@6=Z}W6=}H;>VoI81It<^w34h)pkiutdO|a zyp#1O#vfWPHgS^$y>!j`2?$z6T40zF=tn>Fjw$!%x>itFn8=}h*S*`?YM8(5Ojry>4xNHW2>>Ts? z(~4KIttuB(ssve+XE6^iJ!E<8tRQMoR z%<_w8JL=^zQ)*%=$fEEQb}+LacM$Qz?)P|jKAZZT;vQf?W0Z)2(#_)2ap!}V>7L$@ z7Il_=X=$lt+)2z`8jJSqnhg^ZqA)k!TItXmSt z%gEG3)AA7@@6dhX=yQawRLeXXgFAM*C>8V?(rU_vI;MXtKEre;bCjt&nl*UF4_(4J z>RuHBuAavgA;ViQfODRW7Mj>r*B_rscxr+ywlckBiIwRdb{<(gY$e&3g^jNb5| zi*1#MBzk1liBx!%u$_r_TGcYnywF(?R0QWrB}FaKhoi=N?Q{P=-$3Qio;3)#xvJlW zqU;6cPY^z7YwlLIy7pn;)QZ7vCzj;8R<*jgnGo(Y4qRBj?%4pMCfRP<=|%EPzbV(S zb}c{=*`liG+^34i&YSC!vT29_b(g8VW*diK$|}Z7vj;dfbw0Ue)jjqsMaivUzOTVj z&76U&dpM{9_zLtB2O0CV683zlh~2B@kDqB{Hc2Gtq$H@|)0qbo zcTxRWl$%g;qn9evCfZRqT~#%YVzWh zLN^shk+U4lzY?}$OZ>fjLEY^cuAQVUR7yPXL2e?-6^Nw>>z$*VMZnO|-#_wiXK;m? zZdej>R>rB+@Ah6%6?8GVoCQR#_Jgn3)Y1CiIPiis1wpzisc3+HO7B_u>L5n+iVN@T8mIQ=Wxx7Ko@dRFlT zuSFfpw9ZL&(Zv*3<}rPkwHE&2klYhR7>`mN?2V|SL!hSEwOE`NKL?)-VlqDY5r6i; z#o9WyW)%-)dQ*}}$-+ouS`|6ANSs{NxH|jeEt33s)Z#OMK?|Ny9%X-`HkXhn*&4Cc z_rjKTz7m8k6{O;M_~UiDi-yIU(z2ok!%Xe6zkV=c1jSCY?wg+91zKLai?$sZ+Q{!2 z;2BuBi?&iaDRth+I*b>iJtM@w{E=@pCc}u@gjT0!c_@+yt`dFoGm@f z1Veccw(ZFGdlEUqm^l$B0)z^*PApDT@MGVb$@9|&c=v( zdTbD5lexsg&#eqIN0Biou2?ZADO@jvN7CCLWYNy)M!nMTQeyDETH*N)+ z;fP2%PqE&XgNyH{%P6PryU+S8>H^hc9 zo{}bdqC&7c8nmyGOsx;S%v;hT+U&!G0I>^JU}B}R)GN+LPH9V{CAEo}xMe`iT*4q5 z$5gPYie-|5n%-;vujFP`!;(AoW9Is?=TCZ_@=n{wzzp_fRMgvB_&JxfvI)=;6jwVj z(ooLNn}K!UD`WDLGM63%m{jQmM*5RQi+d2W6O4<^THizqR!by^iGl?BJ3~!pIu^Y^ zxHa8(4X0T#^=8WX*sfad!qS*Tf~Y>km>#PgBHBxlVl7OCIz{tbW%U0j{9Q|CR6ag}@ zGwf6lkbtmu0My5a=itz!yZ%8X|A8?i(lUq>T%6l`G(XS)qKA@*3?#?!)F*27GNpvo z=s`&L^v}mC2FaYo^j`;Kld((A_9!AtI}HpeyVRxN^{x_U5rSTMFdq7cC4vV^u_R{+ z{_c(mV=!4x{^Teue90h5<1<|6G$4u-fWld&oy)`eV3vIi*oAoOv~ps+We0Xz!2*4& zv@Uc^O0#wwU#1-6KFm7vwfqvl6-*8lYflh(!Q7}YizWFa*Wo|mJxZGmgHrqJ^bC^j z(6H=%O|gFl2e#W*WFS#@Ccm$O=Kl19gk;8Nl+n2pj_XS#{o>&*M!195t^A{URJsn1 zRXv9z7pN*CDIDE~BW+JmXRweMf@}ohHLSb|rVC;cA+Jr-CrH&&jQa@paUVhqkPIRF z!09V>M!;9*w+yVSC#tK&V?2exc&Yh!d7d219;gz1*z?jf>FF=gl8lKE18Io5i0z2} zJ@^);omORRC30C&a$QY<(BKKXB))&@ISw!AC{bSzt7>jME-X|;T05$3Jla~BGy8f; zdCU39jR}I7`9XN}F9_1iF_Ms3*>^E!NWSMV*rcP*VkJ}&wz@Lu0t0Rn?4DgbW?z97_&zwSi` zUskZYv;>C>bXca04W?McH?g_1Y7}ue-*C`k*TDl^)ygdaJzoxpW>-H_qbEuLFNzr3 zCyp>Cl|aZ+R<@nhY}*9C!=i42I~8fv(b^nCbX-OmRA4v>6?wO*e>R(lB2L0NEXG4O zK+QtQ$cFWO0UT49w=UWB+o4PYlUvr|5NeZ^ zUiIA}Lt@7Mrn+|nus2R|ZJa+fz;%LoQ9>B*my{5~o3}J{i{WFd<*n1de5xB=X-QI} z(u#M|x7=(SQb`t}k^>P<4|)J*Bfe^Qcq4&0SQpP65ScL806KxkdpgFuhdO_N#7PLA z2O1LxL|fez=l&_2nnWbC*f5sRdw8WxWx36DBiQT*GimD|imV2*J`j{l(X7*~JMzpV z=mgiUpr$qzmj=j)bXr~Ems4KlxY{@-Z5=oCx^Y~A>>%q4ogFM{x>UR!!1m#la+Ja% zp2!MHW2tf;v>THygY8zN@W4y2`{34TR1^mgOD&6WCkaki#q}@8&NP#-2rR#9n~GfB z2TVg!2;{Yvt1?OIT9v(cb%}3|h^+E%j;u}&#j};1BsGz>STVvoBf;A z9v%4p%G$jgabE4yd|sVYqPiWFJ@};3lYWk?L91skZ4M1UJeC1XiXE1?*wto^b2*=V z^oRhKDadF`W`q1C(!_7gruk9_(b z96`*Z*$6Yr5!PtXNudAv#&<(G6$~qnoK&+#rnqhxq%mMttkJ>#$D^vD6Ga8{!G3h^ z1@09ayJ=Kn(kaY3DrI^r<6yM}z{ZD^{dx(ia>+%0^m0f2AD-dDW8LoX8q5c+q6*3q z>XHMV)PhE{Zfuj@mui$mw32@ctUA*YK2W26(7>#92`ZSZ?eIhpzJi}w-4}I@<%&R* z27ZR1FQwNh4i4F^k#f~u)tRK|M>)bYWk#di4(db~O)W;u?M9WUGuVRF@PtBQTO=uN zt0_+2mvHCSm9lpm0O+T3w^bW2O@oi)Kd|V)d<{UVPce86$Dw3Lzq8c!xjpzH0jjCm zgQYpDnXqg~;iz}nx*5|1E>twR-oe=DwPMq!tZLU*GMz@a7;#)oURZVN(B!BPBHy82 z+)l!aGSBSS1H9aW&3}H5yVlMO_-!*dOytuqSAHH)fQOqKt8NVnD1w}29JQu`mk$W} zPG}{EK~CwV`Ibjr2!)sU{Y^{Sqs4H~;llbo9=gtiE}KLDas=taozAW!Wp*vY#8V%8 zC6DVHRqjk6Bt(1_%~^jaA!jMOMS@99Ix5nbzNma1c)fciBi}gAhv`-j3h^~GCvsI` zHehK>eLIEAoxYIThmRONGk6q8a9wJ9wH(YXV~C43mtroY zq$-CF!09t02GYKpLm1`isP~_c3h|Q z1~(Oomgh_&hFYprrIBLt>C~&eyh(`~_U%`Br;uhq1fOQhH?~Id4$fKc*Ld`0RnV24 zQo{zN5blIHeC_&S7le1=mrWJoXKFsIF#ZgI!D4Nr6NyiG=z?}t(^Y-&@RP`YQ{+SZ ztXxA!xKXI=gk>1Xy_v?P`&%sQUNSxDFdwY#K5IHQ_e@s;22dSnI7GVYq0RjpF_>eL_Gm@v9Mm{lfz2$Rf;)o^8AjUw+Q&c5|3 zpolZW%yBeoX7lNTwKa$X-prncl8wr5;+PU{>K{hWniy{mpt)<^=}Ky~L6@On8CGd+YuE zRNaE0vq?6_0ROnu&k0TKk>`e%*(XbHE?RgcWN&jk_;PXf4CM?#*h>75M`TSLOQo0> zJ9m*YJ#mK~GV=90n|tnz&<(F;t8VH?gQYoiw1Gv$??&FuuWjJxjzBJ~Y08;Js`|Wz zZNOK8Sv|^A#9!FK*fQzn{~ira z@)M|&!2jy#BH_GN*%WK&9amfm+s0rO8%T=_fyvVkm4PdzX{G^Hr&&T8aWus<)7|6g+eYtH-cF$Q1sq@FeYx^D0V(h7sOaeE>F8|w zUA%vn|HMa^zDw`{Kwk|lttO(E+WgVtY`8vrVfgws{9L3^s+KkzrB3oWh>Qbm+2&%> z0l?5n5bA8<#(=5fpGLBuwb9*x52}_JY!qdph;H5*V!5ExxuVK||0YNRXS!3`?e;7U zC8F=^8BLdje_K0+1jf>T=v?aMrH>IxyQ62=91O$WJ8K0d+J_I9#(3$wSEF`!h$Gd0 ztOjfuDVitoLAv`Gw)-??N0ZqYt&dU*PW4U|w}mgW++ET05ZeoFMR?rniyNokTR7xm z=!N)Pa!G>gmg$@DB8Kc6<8w5IXX-$LjIIjxcM2mNw;BPhUw#u z1Wf(#g4t)<0c%ae`YIun!WyBkakO^nZ<)g+V2rn*W-MsroE*2G?XkB_a&rh2-X zDk@DZCw`Em+R(%M@=78^ma{qd_VU=8vPMVBrcdbx`4>=n;0fXhx7-C z1A1DFL@mROaD;?xnv)(2H=fWVRuaypq>ND&O(a_)r!k2NQQkkKi39)mFbMA?2Ff08 zxUK@U8etv2t{41JOz9CM!}N^E{7^b}JO81axAzxLx91|b#Fa{-xb>yjAuwdq4ktcD z+!&a06}Iui#niI&<0MvS(OAyCM?1M#iNBect9;Q{ZqKTgk@6PZ=sP!0tGeMIyS}J&Cp- z`Z2`Y-kh%UpSb&Q=laIG&$s=MX+r4u`yP7(shNMNVcf0Eb}a>f;S6`L?a%Iv`rN}b zLckD!ptavHg}ZGJ*QB~7)Z|p{+qVYBuRWVTFgss8CF;ghs?Wp&J@&2S=AmJHW{HqV z(7p7Y)VCf56`dw7#;+3piLGai3U^e!6U+v0d4;^Uj!r$p+8=qDE{?QG z)%hNtk>U*;o?{DUC;O-j<7zO1{Pk?Wyt2Ay@arV_X1;!=_Q-=Yfe}09qoNYyF`ozd zp78L+QZ^rv>8{Beuj<(C)n)%MU33`o1SH_sQiay?>5g2{qI#z1Ias$iI@4l~w3&~M zJr07OcjB~@m)~op@lO?mjT&OfqBU#t4juzV5%EO8>W+RE;$Q;hnaIm!mQx;n&_AY9 zAe)05TgwYW|<|-YII+}BaOYL!t!3q1O~_kF^&_ZpCK%UpUa%>N28cWWmuYq z7dIhAMK+ce9G09#5Fh{-`W%M?YL)LVrttKH6bIjHYoX2H#>niNFgL2z=+&~1zbm>xe05Q!GkBgq%rgIzmTtlXEX)%&u4jqQdYrtKB1z||Q zMR6J}|3>5VlMm)_%9UjI#0uW3$0cl{$im-ytsd*kOy9RTdIrt}T`8i4g=qQEk;1rF zVcb5@N(LH%#dv3eurFgpM>5|R8Vq-o(UFE5q3>A#I7=i+FO{%s()W$_fFJ?`U+1tf z>sU-Aq7B!w46mdiMzsSi-?KaI>*ra8j3&XJhTmMk za0#ysKO0gy%YekW0%FC&npOP1_HMkz@2PzRzUv@4+L$yBM_cj^P^U6tCYL7#*u!Bd zC0o)k=F*OOP4Dj|((1FaGm+(<@|@1EefVHi`uOigDS;W-q;r;pO=qYRfsv}qiBF_7 zKSrQ*nJnu>e;p#J5k|AQhQ(Gh@Q_yqWs}6YR0>tGl37tGOgzq#eNF|7;Aay!Y^aT- zftW(aG=AlA?n$`z0U*Z)(|l7`R0gh}wa&ON8xB&LhZPQ_T&C&1lEL4z5KWCXJj(6tM zbfy-_G@aL2&P*O~PfCqei5h0WmY~6@)*QmEa3{hqP3~frsm{eK1ua#!`BTa62$iir zz*0!cjTBh+6#7=S?#E*B^Wg05PW|>Rmdk+>pfuP46Rd=mbDo!YMSq%-M%=27RSL?1 zrHY1Ebqt)!4%sy0iAPqvrm_Oolpqe9FK(3xd3xwzGr8f1Rz#}8_;f7IZ%q5uzc1gv zXDJyBY=%}XsSF$siI57_s=ZL;X5`a(o9QG{im@R-S+h?Nx@doSs>|D73)0GkrdYm; zeq5w>D6=Q3fH{o~U1N;b&mt(j-FXCh`PdV-%nz?Rb8FpVl&i#9_CnGaM*!d`&a9pV z)m#&k=8?*IJk|>_(h6P+vVS1M9m)Qu@i+UNDl(mJrZ)Cjv}>? z{e^6rhphP`95DEeI9iYip&PGVz{$5@lZ18-+zSP``BW5O41h0)6{t%$tU$^^05W!b zLKDous8TQJ!yjq3z_;A%3|m?;7ZZuBSzLx$$fcqUN5`olX3NPNkDRszn$`*bA^#&q zQ1_IriIO?tte5uel4q=r)|_T3nk2+5`vw`9v(Q*B8zb3yrB@FQ3{O%j<4*+sYgG9W z?G3Ax_h!9d`=V(O0H{BV9Rb7{OtPmMJ)KhtuY`uwct@67UsE|rGbiOUwLS@s7C^2L zJyq;hzEpS$NPeEMLD~!%`qUz|PXn%$w0o`eR?*ZR>13V-^uJ3IkvthRHBLqXr}-6` z;DZqyTpWk5hJ8g-YDplN^Ohel0Y1RH-+cN?!SmSF0tfYmBiVI&Qv>Zr8>XZ2u}{HU zVWNTtsC`~J&dC|j>sd= zL?#-d%jo?j%~;9%N)=MMMT%lsLt(I(3mgYw3kzY3kb-TPQZ>P|NqCV<*pdss1w-jh zqj)!|LM^mHZK{|>rr0$~QA?^=g-Fp3N|A*makFsI4q8!5ikO7@N@swQbIjye&T_S` z&pVse?w#>i;6b+P@>c>nol>rLU_Qi#V?3R73{CEZ5 z(=%_ta~tZz&SQ-M&qkd`oJoRr`}vj$r2OpJUF@&wZkI|8smvoHOK{TSiZ*6_|ky(3*A zPK@3^X1KdXOVEu(HgoY}Uz|AS&t*~{z8K}&VM zy{GHnIy8AZ?{=2+0>Jt@V!501fohD4Nt+#_aDF;u{aq-QqkQn0mHW+>_Zu@aE2Dv5 zL#Iqa|FzL(#R4arsoFWmWz)Q0=kDOM9uzsE-0A!Rs9}2T_%{0fOyXsOr^C#{j2h9q zzRTZ2EOo9QAT%P-oK(b_%w1=lN7Ie(=$fif<`-|nL*D^4Hxd39JCKo?9HHwi$9;27 z1a^LzljOEgfr2FQY>lhQ!U=O5 z=Ta?oA4`L++=hXMsWLg@jT!?Xn~F z@5|i&-k)_*wgI`obh<}7-y)zy=nFV^kKa6~s%kNIfil_6sE=d|g)Bw!4ffdKz@J}t z^IYrO->2iVws{+`aSe(7Rvb@?nhTWoaa%z(keIB*8ua5?2q2lB}#ju7?ORLOur2xqi~4 zG|6)k-GKwaZIJf(;^(!q=b!W$BCLXCak8Z$)UV~SXk|DB)}NQE)nbJJr8mMz(P}WHbORg2 zYt{2q@N%>D9tHwT+*)2qk<5uV1$i*TFt^IO;-2f~!y|Y#FgNGn`tdVldnYTGLs3(3 ztz7>1F*I(zZmu4j?iQ8d30d~-oIM?OctI=AJAAi!owpnNF~8n??)GssW@up)kR(ge z!XRC@cY>Qi^@JrzL~A%Rr?b|3h!*B>wK`)rG$bq0@iQw*E3JTNlh5giTx_yGx7JHi3u1jGWkcRNgT{)4m`yuZZ#C9Ig$~J zCj!We4{KPGBFIlV$G@hv3A{#A7l49gPc0n7uZM_Hkf~08?rVH^uy?k4JXc3qHs{^T&wF`$Fn6?e zbaG$pfHjLTnHP@H3)@qbSzIuQtbs5(+;4v$4tobhgH-6TIbd(5lq zb|BoOM&gL|)>@9UL_e{|atuj?)m)6?YRK2zjD*fL3(X=%m3Rvn%*{h^(xzyK6@6zn zW%k7W3G;&whTyP9D(d3^0<_eS(%L$g*|^@QH~LokqdXjcH15SW0L_>mIq1Fhy##hn zOf2jVjVKp7oKSKR6oXhA9e3qNE%5#5V(3{M6YIZ^b^`oWAQNiG^C58hmB)LR~H zSCI>Ge{NM$MH=2W>{gND_PopTnP!&URVyHPa^2C!dW$=qjf`+@j}NgCo^XtP`z(Zw z!BXKCHW$6~a7M?t*JRz3jACyRzdp2aWPZUR{j~`Dd{KbmMvWbV>+|m_kIF zXzM|y4I)={$rzat)tE>TnJA&cFK8s~_!wFiyR1If)NCQFM`m1*bVE2#g1nt1#iC4W zq*#g&jaLFq?x`RopUl{VLL4cBwx1^SQPI9DN*{p%GUL*LoTg7F(93?TLT3b}a;vm& zPAvk*pt`i-Ohv+}XH7uvjLmr{a|~Dd!ZAgx!vM<;(&K$W)hhAAhGu$w>Y>K6FoPkn zpjmANsO68^@eWjRMn*%18H=X)xI{Hf@t6X&bt_j_A>OjYd@}C|Ui7e<9j!@})*iRP2FP=m9b7LV)iID=}Vwn1v zdWO)XJwMADvnb&lO4$lwpo?0T(R1c*0LE=b&9h%kL+^g*N`bYHg1ku~q~J2ee)@^> z{U#+MG!Ry5=R4x_ZRl7_hM)#r48K_^`E+8yC;^ZLLNvm=)T~2Hbj929x$SnIK7RU>EevG6!W7k{nH{DM{%uYDFo@ zs^ZL~vgUyxREl%b(2@?aohkRoNDO2VsoN?Lv%i4V5y)zpzh9#LD>PeF_dZCNZ^R*b z;s2CSW5khyd26b0^|*>bk3?RE6&_tEhe+&-9hZHgeo}KX>-NQM%G#KYA(MDjlIF9K z%3cMOvwmRQoF^e}sF?~pyZ-MLg>O*++uq`!ie8ImFcJ3rcjak)q>Q3LH1Y1vyJS=1 zQm+Ph*B)A1o2rkMJ?#;~9nT7pCBCdf5z3efWF-^Bc%N{Kw9)FWQ+1FI@qy7xoHWI3 z_7#n>&hO_$qDB&xOp9FHk$Oj3HUjqvot0z4_JOWvLzkyaCEB3fStekaZmIhg_I7QP zq~dTS9Sjhap9CCKLa{znC2M%AFP}WR{+V8&VChH{L|q#-5d1m=bW|C~l|J2mDL0?! z=!*%qr#J;80x2M223M!NDrFnyY@HiGms~oR!8e*!DU%&QfNn zSUIN!?M0ucfXw%lQDcORk)+}c;seqh_0}A;xt3sI7)W22M}V_CUUyVD>Tj<&BX{I_f%f~>oiA{8&P`zT>73e=c3k1|>M^<)=w`f6 zHTx!M4*>~mqsGE!0X4%vubo|0y3S)^*`}gsO6fhUJmK!XeX3&1gK^R=2{{ z>g-ScEH|$Hd<0)PUj2Cr*7hCxP3=AX{SErv^&D9BXGarQ_Ly4j5j(MFAMPcsu50b| zT-(~TzP{~){EPYh0sZ@&I*v1Qw~m2KIw|u{WtoeCvb|*y_fQLvCG;kcHUiFx#AM9% zPYO8G5W5`u)q0q++hVOheG%yS3-sT|mW|IF$0%$7fHNNe0QCQP7ckMc|F4Nf`%-07 zqTvUZSJ?Jc3Z}6>%73fPjgEj=pAHCyowHF@1+9{%*&0;6x{L(pmg$l25$F_frfaRd z!e%^y275g&D4ppjfabZlqk~pgxmBZ+xBD-r)vu4w(8qkA;pr{BObonw^NyL<8kZIG zzq@{YZ;y2^r@pjhcC+9O6`MoxY`4haS(Ao-)kbWAUgtapTQYwB!VYGj$@`Ug6uB-52>! zMEj!8FXRZyCYQFJ%VJ*9SZf1tjh|dfgd35CHs8X@EKZUg?PJkg^ByMktAConS-KH7 zRaXb}eZzwV3Q#>6?bmKJLCBv_ae*~NxtS;7YJxA2jifZHaR*!h{VZR^okHpb}O{}Z_G`+IZN z)V<5Y!};$mjq0Xa#eTO^46c1fCRRo^Homa>l&eQ?Xn0(<7Dh^v%etyrZ+qyoFuB6d z%E-LT(#6*7WOx7N@pgV%wO%>SK{`aLjg^h5q22BE@rHS|K$}r4`PuxQtek9oMGqC7 zpO$q5mEzNCz1GE_Mi4{^2 zrNT_DXfN!Wg1g(alYK@pcrC5+d_bj5mX_3i?HX3mfGUEd_@?4fAN>-&!u%vv!_AM* zmgZ*V7p5{D*@+yo3!JD&2qC1ZNF3(8(%FiZM9h?le=IQ}6R|VB#p4de|7D45d^hwb zpxhupWr~TYS5*n4;DpebO^QcX(-^CMJ})LNf<%a1U9rRacdx}vgYfaL|K7kha@X|f zgV5vC->*yJ-h2{PwTRVSSmojd)hr@c5f$Vk@Ic_4#m%LG1(re3D zkhy+lRxA1(zJ%5Dt(bU_y=ato1M$mh+Ie~}IW!%3E(5}Q*65Z-ZeE$+HUF&vS#Cr8 z%DHRAGFv)|_CvIf#XYRHD8c)(+eCxfS~9Ek(6W|4-Ub6lK4^a&qrBqASO#C9An3ch z%62l4MM;>)n&S@kZtSI5{w%sCXJld;_0h1-x9<>qO=LAq3C|E8aBmJCnNGe(LH7$% z*^m$Et!{N7Xb8qXmiX&%=9$1*a!2*(>;f9-uB6eQqrGy*3%7|iF$cJTdhXTi3{aJ$ ziwG+|8~tG)S|t=38b9c{q}E()J7q&~n67a&PDoUa?&4lKD?DcMWgseM&pUf=34MmSyvzyzB1ge|w1 zTHmbJ#F^=-lfCM%4wZ$xmx|!IDdsu&8Pz!T-RWAguYga5`sDz;-oqI2OjkEz2oRA!5xU=SibjHNEI)ST=CkEwb)*cpVouYT1X;{oyI!D@=7 z1M((re%iPL^Tl0BRKs60zid>3{duOWCc2*q8s-h0yuS?Oi4Y@2FB=ha#rUriDvM6+ z9;fR?ge$1a(}2OKgy!W`wlStDaIY3l)Qu&CNHwk1EG&xii-IG9M7<$S+zq;9J(0FW zdQ9YB*s=E&Hu2=uerz-+bDXCWM(=|xei+1ppr{LpHTyFWA|*>~sEQy6?sRE-(ogze zCBPz+p#&)E@;JGsqpg)rfC*f=aG9;jLm~D&(r3B2f;L1%K-QBqc@8;v^45_&f;@$` z3TQ^Qj?Jz@aENUgoOvTQw#G*8p496=F6EcUAkTG+Y-}e03-q?V^nHaMskk@@Pyk7N zVs_x{BYv^{*nT=l;{GjBg#67+A!ZMl{vysTi)^4gjgS>c0@WN9Av?prjeWLY(V5jz zy0mxW6w%(ym~v1!q4=_HKag%N#YQ**?zT>U-XaNhXi9WP2OFrkW#KBTB~kh$D>z_# z%VR}YW$5pQoB=W+@HW0y~SQej4LSc;wI zjsLWW^4K zp;HUssy9sQ)cl%9B#WAvPW?wsE+-YD*X}gcRI)iNDNgEt)?6|+G42b8uo(Ql9{;OR z&MjkH-C{uzx0d|8obAK@BA$ZRKXxQXS)N`rHJpZ4vuKBLir5qm?1N0l%^q5PCK+=mR^V32B7iP z`)f!TYQm!(AA|*y$eP32>OoD32jdo2IE$&t*SmzPH!BDDN$1Rv5EC`72!PK$5xpgd z!2i@XLkd>4OBP4ZZBxwc^QkxB^zPI!X^SG4c9T=Q?OmBS;Pk|GeU=G|R#J*?2+H5P{@-np~?!Xn3JlF+Emn94jGKJ&;B$vzM36j6=5$eGlS z#($dsXUQ}0nn2=-VoKfc2u{>!fvQtJ36wJa(*M<&NUB~Cr4VS(si6x&pwzZ62=@I< zA&8Bzsg7vzHodn2>I`xMSX_Qd0L0bS+O_k2q#7#16&DegvQWTTt%)rHzdp&>Ew``- z8|Vxq%G}DGe{_q zu=d>=&T&uSJG?SlHqbUw4ePlqm2A%t;|4!fKJT&muum!j14yp_E01( ze*oY)->?<{Ls$(ZAd-a#o*W3+i+~kNa@rmTAty_@;jyQ&sD=2=4n2|QPn+7?eG71L zn7gJkJT0)7llN;`2;+e@D@=83b>UU-GxAb=Cdz;CtG_E{qPHh=?+I;YRhz2Ype%@y zyoREChOp_pa~Lscv8~8+LLs$u4Uofd^c`A0nmbeJ%v8^dtc(r5a$wQu8}1{O87`S+ zm1TJ_AH}s_$$QtKe=;crIAkVS3swX*X`3X}Y!CzWaDiqJSzG2E-}AC(QuccRQ{+l9 z$K%D>-go9LBlQitZTqcTr1Zc7aAy?2ExgUSz4J5L_gXN3%18H#W}kl^;EATibaMiq zd2p|D3hR8K4LLaETg;JGV{g>YQ`5d~996C@81xF?zHZ&b15bI7WP$m=rqpqr^L&6A zg;}?<&3+R|1Fb|cHC)v$RXs`%jwH~Z34aVH0DuWvn2|BGMMy{797A-a-|<6q?CDcZ z0B-|$xsOzgv#Sn24hr}QT44rkMacEA;w!HiflJt=peiKbb@uFRA0eBo1<;8?SEwQqa8iL{9LCT3e%1d5W zg3I7-e}EU50B}oST2V@*NR@^|1lQIUeWzP3c&jbztx5RVnif1J4 z{zISH1tIkmXZRF-2T4VW%)6_XYFU!1{-c9Y+$%mvg1DQMHpYg+e1L1`DeVw~8%o*v zF2?F8*bgY&dOZ-|%QlP2iM67p;woc8hy?1dAA~hNV(s~cw}Fm!kfd2g*ZVZi**=AR zozjhj?AbhcCX4Fnh$Vd6o-N3OnA438)ebvLJx!eF>|paCQFZaKv0KU+E<1`&Q(6nJ z$wM(xE(rjeK8C_DPXxY67i)w$70#jYlGtQ@-M0+(Zmm$Gbk^T(y{jrNfe`axpmtTK zZeSVv`K8{l4bL*g((?h zsOmE3_*O8Fn1-(>fv|dbT1bWzc(Rc4>U2J}Rz0xjF{=x|T5xEzG`86ADmy*R#9fE* zZ0dI@l|N}co_O!TtQcuYYRt7r!AvU~Hrk}el2!39eo1Lg38pNM`m;lV`||d89)JK0 z>bU*cey`0Kz`ECIaLlInH_S6tW?q-}GT{c8irQQxje90J3;vqc*0frO=WPCkpkFyxdxrDQMP`N~ z!Lm%2R?`*rYf5B(-_>_q<2+;Zxuf5%dqdQ_KXhD2>V&+8524E`npPL|;+~Y=42jC= zFgLHJQ&ajgRr6X>XoNTft)PVHEsSxL2KjY}6!Ql>Nf9co(3Kl{AWW4p?J9#7k#PPT zP=e`KBO5t8s*v%23KwNCaui8AsfR}0PYtFu>o&Fu!6X1%QjKGY(?T~+c#@jfoeIG% zM`CHBUSC_`XTwEW?wuSR69GzaU<`Jf5sgn=fiJQHwp=OOH8g{p%c!^JnrU2c&aWA`H71{7 zp%MT`VE+ko>*h8~-tQ1`l`5){XJ`uOPZTT3lkJX!Z@On7-k*7!@*xzw!*Zkn3?g2o z+OikuemaPPE=;MGW-{%2W@1yKA53(@Pb|^gALJ$tdJ!oH!ndqn-Yk-nTr`y{$d~%; zLo|CK^CT)Y-D}gFznn3VlWu&G3LM*B7rTMEH|NDgU%VpspIp=HQy{HL&+vdTPZUH6 zlOX#8Wq}TJ<3lLC(G9GFbZ%)`&YlU#GJ-gIa0w0#5(U~BITDorf7m+5?o6OH+rCLE zwr$(CZQHhO+qP|0Y}oG={6+W%e+e7KX%;|hx*d2-*ZGL!bdJ@o3YF3Bf%0HdSES$sYJfb=Ayk3g-4kyII z2~O0zNfbfWo=lgV)WlmUvN{((chgT>mfBSmDJ)>)@g7Uxy@u0%e=0D7RX~pZxd_-- z8A(UXHoblfUO>Xnm6PFp$2c%wtj<%%JQBtUsQqB=g#^FQcn9#9P(=#`0KbmY*@6%I z-eWHOL6sLOAq33EAhl;YCQOhV6?ZCUjJq(o4Co{N9#&NC-*)F=6lk9;mVf~sjGk}3 z0Kw@eM!N6NCR%zlMS=7DRmYG>*{v+;p)l`x_;~Z!0Hnj>0nxe4X_qwW<^(%8KRniu zuT7fnn_x(!?hQ{V`$|rU1zMra!hNPjLO0h8(s_=MnUhs%gJ(do*JDt}kC% z!mL0)_1icQfRHbuI~U?UDEk=J{}R0MC}y_wbhG$6kDPmZDx9hFetT)9QPYi}Qu3q{ zx&js^W)UsycWl~FLnV`U6img6nIxR5)gEs;+yu~b$yVTEt@wOAND|oUWaWPFsl(3Q zrC-XXgd;Des#txZ4)a;p#p7r3&m`re*c&tOY31L~VnMXE<2}Ucp_=zKmf(>_zXju0 z=eSJUf;p$+(G%v~ZLjw$&%fth${gBr+|R4;?_jrl?VjeBq@TLzx#mVa!Q806h>MaQ z6NFe9U5B--T$+Jz2LBqQs(ISSWcGo~C{r_kPLjfDXR}v9<@7Xt;^>X`)*^8Z&5&%5 zB)1||4ZT1zOIq)|y`n>u;!JfSgm{{Ph)faQ74uh16i#9Mwj&}3kh+D{b*xfY zzk8u`f-06gXo;t7a883u=XE)f2u-LR7>upP_(B(~*mRU02_mq-Pgw#QpDko8lgPMI(`6}=evuJE3jU6Y-V!v9 z%jO#J?FsPEFnnYY2vJ!*!qYOJYmw@p!SleNZb-N1G`6@=Kkl5O|X`gZ}7FZ#%YHC8jRlkJ$?| zgy5x-!e}m!Mmz3a-kv|G4TobckFHgv2(1S}i`zxC1v+OHElKPtXmGWD>+c*93&z`% zMwo(gE2m5Xaq$ous+5uw-=K9mp4dtTl*|4%xvF$r=F%ge9@42;k%0NpO(Z0Heqn`> z9uU&0`^8}C1V&crZ-#^4?+W47=`Wz2IXxkQiz(_9K$Tw|fq4y;rvoh~jQ|7`-!Gd5 za0~n!^VryCaOyn>{IP&h5DYQEwFkj-%YvsN$p0G3r@P=-8S*u5s;aGl!5i!shRuN7 zP_Ved?39ljIGB(L7DkCD%i$DeG=Src(F8VaHDW?Af6&qEsT3gh(Y_AO=sdRxqee=Y z%sRo$^#!T92%)-riAK_yvWaM~TAcwH-GR@b5Y`*ZHp zLc!`~i^WYfp^?`^#Cc%98Pw>t?nQAvqKP@YkMoxg{R<&nS%FkJea~1_j7P8YGHN%s z?b{jES3HP?i8l-b*TMN2GZTkd0Hm70l}-xQHd|3r|A4Azb}yT(GUXTO9?Y|b1$U6e z2J-LJAYEY-d&XF}H5ZL!YWbx_R-r%|pcW*dWfA*UsTNFHe=oYCwjRk7=mq2u5ZsCf zWwUK9peJ(T>O048%$;{wZG?Y;{Yq*uxSL`_%h;;>!@ouPYjR%Q-Ih}4#T66S<7Y4_ z916WA)KAOL%!B-)=B2eU`(6F{Xs`|Y5A?AQaB}hVt*f&Y$821j9=!T(1 ztJfCaQ9pif{{cQ6+??A!&LXn!ZHvLk_vCXC^b@cFVtygi4eFyYuTu z{qoWb{~`6g5^RX^ZH`U1=hPp~PE*mJ3}_{5uLssB_LR}&Ps$sr(D9EixkJ5|oYV|O zE286AeGqrg`8$>B=-W!W>19C^sAtX35bpX|+T^aYmE-j09N*A7eNtN0Z89|F8U&78 zP^N);=Zjom65Mps=hJonP?NZKrQ0)!z7?RyiKvKi+9v{$+RL;mT3Nl$ZVuKO~ddz zep&i04Sj#jJq>%iI>xAx?>G3{{2z<}!lMG=ZB@us4i@i+$A2d00q7p91*j5Q(w>2? z&CU{O5znQ~h6k21{Kq|~jOK;)uk)jW>aXW-ac9z_@8(^6-mcNp_THAXH77rC>qYFM z1@xb!zcm>2HT%Bm#Lns07jiW6uRRs;>VDszSfRgl+k{5=I`e7%WQrd8uIg6u&JH|o zQdtWQjc|x9Ju)MJjzbRwMq3!b^9zd5;og5AzA|jUrryTu{rxb3BKho5`s6%DC#wn& zoLDBISFF!!ubA`hd1^T&VX1X_4IATpLl`D{b_4DGzzR1M8@D=By$cf5uf6|eDkxNZ zf1Zq1(imM@KLH&~ovQ2g<$d`B_|F5L0b72{#=m0Ev*!QF9hsZjS=yPHdi>87569Yh z>-T;07qx?ocT=`ROR}|hqp7Khrd*1n&05;+=7b3g17u_iWe}Obl+~@Dg1`8WhC5L2 zV-bnAUx_j~@nItXE!r38mwV6q(12v|SNsy0qsbm|Ir@iqEg0iu+1q7Tm?d5d{P-S! zYadSH^(2~Q_%eE(ONTt~k_{gAWe7%G%0QeY99Zz=zHVqZZ?mu2uA%+bI*(kMX`Ssz zTAQM_eti|=?Ry^mbJ=6_*Y}tA4o(1MC0ER5Mu17aq*Q}w=8nkCb>6~$uSqXWdpHnt zM5(iPn2G!1FrHiF@w@^<2l|BFn|W|XC;3jUOX-*u?Vg$YpRr!q&}M15aDLyqtWGgu z&pLlup_6M&Avj(#oY4}&oCMdGn79O$&NNII|IVfk)y5Eph+if~I4Zk3RmFPvWdu>) zNNQ(bO!~f}_LX%18a;&VEF=w+==xb>*Ir$&PTQQEBV(K!#6MAn5FH%c7dg2(E$$4; zRT-u1mMio#&CUVptR*d~Zyim!)wVe}xGVB=*}3%O;I&7=CwuM^>0Uvm2;sHGW~ZNr zcMYJfrBHR^bzkN0o~NIG4r>-dfTWmWG$`vneAVX`myxwhW<^5O#D=T0BSMEJQyHvY zZb^XKAVNokpZnti=t)lVGFD|rkyw)8;0-4d|31gOlg1Z#heL`L1WPcA4$`1dqwka)!VKAGj3>S$5g@<6E zQk$qWm5nj9D=u=Rv$+G+1pNhCUtEuos!EC~v=^jp+P2%~MH$KxJP?CiiC(8n?Ag}9 zYo{PmffSg91bX>3{fZI=Bj7av`p>>3%hmP{i~4~bb$>_93bva?#FC zi-^A(FZn)Y_`5;Yd#q)yX~K8sy-iw#w15zR=d$Gn-$4btLNk$5!qUGy^Aze47}_JxCDaEr|&H3of9e!>lT?Tk$V-Chm>ZP6obO;n#|1-XCO0^WXF zU{iRr&HT=C)#0a!dXvEa&xX1Rk+qO zSeb=AkjZ7lK=aa(i{7S!hML>!!2PhU!eZsCO739upv}e=qa*z>_oVYN{6PcX z%?51wH}KZcYOEjJ(akM*-7l&okQJNl#Er!#6$t*C;1?88ICiUV{7V#e(fXs#xk$Pc z8J#7C?WS@rCaW%ES8&dAH6?4MfIrN+?6A9W5owJHS$E3Oq+%~h;L*GLU@N9U;!$BH zLvR8Z(Bu%;VBM)EAp194KQorrENP!~$BLWi{=xy=eaD;_hpP189?@DSS*NZZni59B ztzT3eFmr&lqDzv`w9P&dCyoO^)F|c!`0KYH!tqm&pwB#m_b6<^#Wyn2R%S9wb72JC z2OUdKsa{CZ$SJ#{&X$=UQ`RxVWhcl(SPt=GT$l{a7P5s#P$qWFpkuv`jQ zF-fXjLY)S>Nns4h2H|b{H;3RPnS>MuJw~93J^~(+RLifvg)bvTqbZc$RLGVfMBF5E zB~=BR{GSFzuha)E%Vr`RzV-FZZCqC-GOsvhWGXI3p-~o2Nh;=N`H$Hxs0PYpgk%Qe z$6?-qHYD8A9hmFnU;>*XH^Q~HF&|LH-l%sVD2&JC^LMUSb)m5B+m#CekX)hb206c} zKvt#(F`$0}EjIU4h=s6ja9x*^c|7|fM1(0Vz{l`@b9HHbB8fbVQSA9?_;sew4C z!5HScBhb1fn41>BEiEt?H6YjA;1ty08m2}U;FcJu8xvSNwy5m7OBDv12cqY^D->Ch zZm@3r#pqyrewA=dtAGdjLbbZ007+bp6GJ9^oGj^G^Y1L_uW~VS{kGX$uX?@AJBZd+ zbF34eU&mEJz_nlGC3B8zJFQR2&8ryty~&x3Y87k@#=q}ik0LaBVye6Z%o8X9^*+FfSsfACddkv)_4Lso& zEnd75Q#9mLQHjzQ<8s12>^!F%QVP52P#-=>x>Y4BV9c0^iCk*I$)Qba^3)hMf0koU zjevcS$M;E;H=i23x+(nO3jC+$gN}>ssiUBS#X)10q_RQ&SQ=8{GAw$*WmChhTM2fiu9Oyv=o5I`_JWr!hG74aU=+P#WDvxiY2D8&mSR#CzE^ zH;Yqa_RiMa;ZPhTzCjOx{NL`vz%MbKO7g7{+20bFWbuvVG>ap=WP-v^AIt6BK(K*i zN2GK0v8M{ih&P?n&8#<6oS^4p+HyFm)Go1sa`kATp4J#ln4x~+J(LR?f240Z!k9^Z zk^{>Z_cH&S^2&yL&QpKa4Nm0Y*(^ z7G8G-*iCvzyS$^5F(T=!Sx3;QN%r`KEF2XUdn4hwYaCn|p7Whko~(X2nk_yX)=Ws#H^F_mQhzYV^=?`vNO` z_|*UBhD7edZkCS1HTk9|o0s(tYfbuS*3>-?=oyATzBO6*K}rZ=;9_9TMNa2XYmjJD zPCNRc0xkGJ{J}&xn0I+@9G(phF`iS;XRc6Of{)S0DuXP_4DN+vk=M&FeT=y>(ou$A zumhhnY8F!d-|NRNj4+=6Sz*G_-1?+(e<)0}aS}GyEZ^eQm<*h!d)6?4UK0^%3X;Ew zk!r8n)q4)6Tj667#ce?-v<5sXJrwte6Ho#6lNO5Px?GcG8rP~GfFA#cW_Yw(EqtSi zjLyW@Zwi6chi`hGw$WVU!A~UU7=#Xr&c)b#0lgP1|1)&&J}KQ(+7s#a_PFQ)iOy#{ zq=ZmH-Djhf_;9P9nF`Nxsj-^`m`*6NNQx(UiPKZJ14)d}0QGGNlJ(f{Z?X;(PR~?L zB8>oj?}HCKU7jK19PDS?pHj`~Qo4AkldXv<@8Yz%p8e{QVhR37vGa?YX_BEQAK) zXZ8{x&FtKXeGkK4M;5JYQ(!~Z7w~uMY7?8-!s1yI2TI>HrN!+xlvSxHfI9pr;%9`% zY)}N^yV!uj1{M;ox-2?dUpqt`xHZKGbUl*6F*$1dWJ573(j}*nETe;TUqbJWCv1}l zs~k$8TKb=Iu?VDx_mg1w2#jzFhGAlo!W9#GplD$XmyS{KC7b1U;QsagYpRkr4bj}J&y(i-PK+2R1>eqU%o_uPa>S&fe9nYgdFy3DPoQZrdH|a1Z3hf z?iXlVa(Q2Q1_5g6Fdrgbbvn_AVdfToVg6nO!qH6>>sB=JWR6^lxg#i}{uN`j?9uNd z0L0a2#sv%kR&bv|l1Q+|hp#eTIaXo7V7iB$gy_*qvy(xYKw^iV@mFnZ2gpNItY1Mz zB0IQ5@}KAstkbLtDwq~hh`w+FzQgm zpKum~2 z#Bp2B0pP-6T^Q>#4A}}>aYKiR0b&3C6cbem%&vOH88QJz&KXUNI|;$=6r4=^9x#cZ zl?R*sTXa{73GGsm_YxqAv}~<9{)%UY;`LI}h~ms60~3ywxeomc6ZIuum9BDeT4OI7 zca{T#XkW@pi0sg!s35rY)Q?Tbevf|Nf92ZA}Nhcy0@1{~Gi; z7OGVKqjBIERR39yb&4~>FW^aj4%@eJWu1)Za0e++MIZi0K=|_2@Vr0Th|BqG8T8j+ z8oh9ffY|lo5ib*LWEDh3ehYv=Zzx`*->tiqODb0MC?vx2V?Ls^nd`NdOK)-eP&-Lo+$HP|bwY&W|AHn(O>E5pY z_YQbqnE`DjHPX*ZP1#6h2~hc>1tb9t0L7x-C)O z8AiP$wr-BAR(F2=zAXF1N0k*h{Bjsmdu!$8%3iQS>cD|XrIJKY*p3nhMHmgqXd)eH zLbs-O)HL{RE@u&)Fd)N+EQgp$NLK1J5j_c7Z33RgmRWnfmSU4Kv6E!?ylaNau}CT( z^#2(Nyi5NjMj`+J0@DEi;{VO`nwz?~SlXHEJG&SUvYVVl{>X3 zwPIs#ODzBL5r^AR-tzP4tedx!laB#n5`ywC;e@nSTYL+B1N{A)=$%X;i9{qx-(k5n z_iVoqqJRMdh75X`{vQ4w4({&Hx6cUJ;>5ApJsy8or}2wxxzqCNA-U6$W9a9X^mP7S zqwHB@H%uq}DG924@7&o1%brmUUkuR!xpHQ__}u%p;z7aF2iH@$LR%x}kM%evTU>Uh z%N;G-`1fvX&)!}7fabTW4dx{rD&{tJlWSP8fch$Lb zOI>rseGA3nz1&m)tQZ56F0AB>5T?k+;mc=QcDt)`c%wERQ^qGdKVGgLo=#p)u3m2L zkv_Uj{+H3^KmXcOb@iILc{=xN@qr6gZOfUxcs+dqUuwgs@u{{sv$B`R^To=*+sML7 zyg9v`oIcK%Z}?M3`AEy?{X}(fbu|0f7`ei#gF_b6wLYGXj&4qmr?1B^T$UmRG+6ED ztvXE8F&K+XMrPdSG7+C2LbQWn&-%-qr@-M}YM~wF!dD4_e)L48JUU=zQ%F2vA|zN(e#CFtqH#1k`1zz=-`=<9ZKnCz>}W?!5G294C)=41p;JC+UqQhK=;4x&?#f@oDa&ogc_DmQLz57-uCU+lWAO+&o_ zCR_@ZN70O+oIFv^U>=rhLHSE(V>V{mY@8JV5jV+~kJAKrxerNMu=mI74&9Pmzi@>^ zmCiwbOB90_-Ft&q7RavAX3~3m?8)f#h1kW7E7C@vd+jzPAJ3-qG3}+NPVfzc0iVHP zbA4UMZFIvf>A{W}_?0KcPN2aW5f=0TNcw=+1LlkFGf0A0J%}x>w`l>Oe$b_(a(Wxv z&Rzbk76-u)sP?*rpdbIzY2x2!hF3CfBO`&%5K!8&tK_Yd9N29%yDmmsukJ;p8PD)0Scj!^;NOG zjvQ*?BCb-_X#aev)q^gb;548t6~5$!9=a-Ckd)4MJXnPi`qv8`G*{&wM-&ihS5};| zx|422(v*sSY}ILXatm&uD3P`%%Yal2&LPvyYyb~p6_j?u!9gf z379T@%_13P#ZaKbM`-X`LrDP{x2U13WUG2N9p@jmT0^4Mw8%9nHC?swTe&cx_~NG3 z1=284)3D5S{BOv&W5fc@8xwcV$Bi>3tB{NV+Lj8+##&?vjwzMQ7l^5WJmZ%Ep~Sk# zoUdYi3twe;>VB+nWWlBZVT)pFiy%0R5Z|C*K(Ig4h=oAHtMO@>2Nm<`1tkGc;*_B|J@0v*w&H-q1CTgx=0OF4AR<*j zf~a$&g39r+;sHMOG}3vL=y@kxT_}>DXO3cp{C(~k1nH}t zJ2P~fL1WpgP%Rr4ifx1WA!Z^2aeIABlyJgxZlGp|#p2PVD-lClw75x(%8}QL zQN`$0P{H)2^+kO$D@_edG93iNz|iiU)=ueYD7EYU{RG_KRC*|r4SnN{F2k}N9>@Z? z4C7)IKBZPJ3uYaW3HxfOfiFBN^y?IhI01n8yI>CM+~qWQutVQBkV5K@04|qnzclv6 zs~1V(Iqi!Pi{J7gnech%puE>5tmy3t)FVR`Q!a28L=P1SgO2CsE~+n{w<1tX4;+%i zjco$CB}uT$=zD?6rbH6%^VNO+sf1_9!8Lq7(w@f zJ3|OCy?&(cJ%7zDBjkCA7y!t7ID;l*1q0+Pzy*c`B$x3IVs=Fa>|3qDHb|N)xeaD* znkugTI=-#t6A6_9!j92$D=Dyz?5pTA=9^lD+S06HX@ds&CY~=jZlgA$m5Z=o<2=9` z@`2WhIW2E&J_^tx$QL7^OOPn2g87}+%cv+LKA3N-dK9T?cnudE$2b(p1iXsf?`%N| z!1b5gbe5;q&<`p^DU;)=B%Tl31{Y6^{Y@X7!6Q)Mya=t@*f9B8x%~<3wy9+hta`6Z zxu3djIn|VG;EOp3b1v~qrs#k%7s7~naLVl<02g#A8)+|usenpWZAJzyQST{mZQrwI zopr4VoCMCER3d{U;f{jESYuFC?kVCS=ABW%_UAc? z!lmbO&SC{cUle9bF~e0iR>0^5Gp_9{0U?Z|?Y{2&14}S@HWW-b9c#c8ji(()sq|c; z^t=@MJB~Rjxkz^De)Otk3xkm%|E|J7rx-opbR^+DMP+Dk&DU-4QvfZ*3p2<{Z4YRN_nv9`^V-StFgtS2EQ#vyFr&D-9rjyq_kp} zT%UAGyGWF6b=+T}BSRK59X%M*WW2;<$~YTy)sL{$WoT;{v`cl4M7%WC;-x+A=ztw? zDrp2N0XrL0(q&@2DRm-f~2_Cc3{kZXXN8tXeV z8Bm50MG5BUo`WoSQ6jqSds0g^%GY=z zabNl!6Ukt|-k93n3J1&q?RefHqQxTd2C0fpU{%?o66#GHr-mMb;x>iKLJ++{RG- zyvU@z=3f1X$g(rr;yMV?-Nd-&)GV{8^%a4Utq>N&>*e|bVu}LKuZX4eU<}>mh|xNb zh2$^DyTDSi-?wyjuu7C(+O%Y`;MT##S*($t!lA#`y*hh(dIoexd_b+ifdU$81B135NTXQ7&1@DLV?PZy8?$!TLEa~ zG`0kWOoEZlagJa|XGPw8Kh{u_lfO{C>kKzkl9m@EQkw25!VXXt7p`m0-l7O9R=U0kk~i=@Go-ZHPYaxjkBhY!-*^zteJBEWGO(GkNPkiV>sbA zat{Ynp=ef~NU2giYl8T4)0{mjsmAX^H4mzSl3xdVlL~KSfG z98axJgQTI|BB)gByD8Y#1lX3D*=9F!De~nX>lxt~lfxY5S#M?{Pc2H>Nvan+`7%2s zj}OQeQ9cwY;DO{blxRaG!~~QUV)KO5NrBH_zKEQ2f+_ergxDbSxk9C4uy8Ul)j4!* zV^!TjG^@=jXx53O6nF6>5iZ%S0SrvhiE;&2@xstOLm?up5hWFgjmCewk%}}~Gs6@a zHBw_KmA?yWlY1YmRY1LgF&3)Po{t&a}DmlX$j}$QCX;F~LA(e{CbzrHSC5s;HdZDYD z>k%KD8jBi5*t(rfoX;M{_;Z?etzg(u4o}ML5Ot^7W)rGnmQ6KdJ~lThPMC#0$5#8? zmE=P!F?!k>CfRh5^bTL6^7ASa&(Kk5#5O=#AIgc9U@%DI#!OPcSFp~=fX(&}xXtg{ z<>QJKuJLUPL+Mn0`!UB*k=O8&=@59N@k#$DrKu$B6CpI`%LiVk{c)oK+mGm zerYwg9Mw5;Ck8Cgm6UM64RHPZYrAfozYQEuZKIv*=rtiyhY&OT8<8?>Dvd;`R&t=0 z9uhQgQb_vIC}ex{A!?Qh-w&)&qg!oU1_FpYQbkvALiK}4wgviH$^rdJXgmeTaE#i0 zHMxkZu1#A8VCc&12l(kfD~M=m8qH`mSmBP2Kg@+j@PQuIhL{&CKxL~75Y7IAV9XCl zg7&z8ly7gf{w<|c>`_dW@t5KArJ=JLbn*azw`{!(|S0GlP+9HIiwKE zUG3@H<~`g}=O(p>7fM(^HQnCM4lTT{;*nt`{#cLnj`l>)L>2mcF>$|F_98Nj4P-z) z475WL-IwcwRdo=KTLuKQm9rxT;1ZIw^&Uq1_*Ft!ORd|)CR-sMG*3Wla8ySy1uoT5 zpdwaMuN#V3koznq2M)Gl%yXO-i=3$vjfNe~qG8gU*0h1NUpt=!SL;nl+*375=AbUP zDo0N5MKZtj_Sk9AQ+wBe+O6YI?eHqyReF&N}OUo{Cx@2xLCC}F*T;D0A(FW zLU#U$$0th)7z(}cF7ld0{Rsq&dDG=*1dnSN*q#_xi20R9kl4hFS8k0rk8A<4`a4V0 z)uesqxH+E^@T!&;%W3%TO0*paP94*hf z>vs0eElhOs!jQwk`f`g3j>a(I5C)0P)iR43%w!&#Q%Kxe^*Zk1rJo%SPq{}A$4GD1 zBjWLF1KM)3mh!V4>fsAR#kyH0C`{QjEOoTEnxkCgAq*6eqTIsI?Ziu)1@fX1B&(nD zMt!44E&c-}{3e1sy14n<8TF$s&#pV`^2lxd2(7CCS^+OTNSN*yua*nfHlcb&re|QQ zTU*}_)!L`({&eMpr^ghqt>y5r2hULsMMg?(5^Slvld* zO@K64V;{u!TDCqtdnpt*7PDwrN;b8dV}uZyzd~lJEY0>1 z*X`NWvA$WDPD2{^a{R3wzRVcwl+n6yj4nV2n&ZRj+xqlSLO7DQSJ{q@T z-dt!t!{z?agnPvjXn2kv`w<&Jde(QR(tBzqV@d1GC?HyCSZVA_dKNKcb_^cU7Y+Yg z6Ix;z{+b!I*|U${P1idGg56Ce^8P{W2?ePzfQZUTebY>ZE+NT#zoRzEE3t>R+%y^q z+)JwT6u^9RtiB-z)3CwCJ*#=QCuM)w(1v4j1Ifrz@E~@2dAKf#t-Wu&kaP568@QsN znwQ~GGIaVVm@qHswITzO#3oU@A|no|!2i`p6!CdjKtk$PMYPeJ$#%q2Zj>cr!BUzp z%PV9Z*;a5r{lRj|KBOhvyqj@%z%U96Yg#+Z2{f9eKz)-hZRi zbMK34iVr?*r!T_Ii&f=qf)*o8gUS=W21YTX!#xK5RzRNpH8j_TZ4=@a)KjT&9X&{NuH?{5wYB_qX^O3Rc@Zn@b@Hh0aObEOrma2Z>V?a-Zibub~A&M%ErO2 zw6QuKT$j1aDgz^QFS;mD``McGQyeVCFx&HnX*9zwkur>6z265Eq6~Llg_sM;e~LFU zJteau`dDJWPD3oE*i_y;Ht9-Rj~RzEdEVWKL4qI3(4ASPB+N!wCZ$v-JYY#zEQGjR zJ2t6BpSvatHRhc07MhUiFCol7k?9CCm@n2@^+5s*>vgMXSh2r)VIGvIqJ?eQ0Um02 z|4sB&zPmq(p$fr6ixad@CJ78ruAeD8m6|hWQY1}o57ekM}Ve0zi*C@FYh zH51?9VDX3G2IXheCA0IqSp_Q{FR`{GszBTvewb>nUIvY#z?M)PWkq{;^85SQ>)F%7 z`N0CJUqNflo4VE|Oe#33B~*VMS$@D%2R zmxlKeT~{dU_4d_{Y1^2C{B)YC{Xr@D3XN-mH3(}M<0zgH3RO2#vW%u5?SN6r1i03` z#abR#@>3WXx0c!Bwv#h&WHvl`Kqw- zQRpIn#!b$4<~rx4{d~te6WM5)9BS4EFSz3}qi#YfV9xlVotnN54%)8@tqNU*BztZ+}L+(Gs1;Qo0Vf zlEKGHlK^k?df+}VkD`%)p6c8LQFL<@e~mS>##BGu_J__Gp#<_p0 z%aHqi6M}rd?$@D&k#J=HC;>Sq}- z1UPi}GU4yCu-3PT|3d66Em$(AX!<~^7g%(~?zVHLhq{-)BR>=XXYBXc-#p1rgS#1C_*L&N4J-t|J zd8AdIhlM5Xu3Pt$UbOoH3d^!nADmtzEWIN;VTfXgbvjV{q-3@m8aSDFpyE;UKTAKz z^gEIIB3xB}_LE{rec3Qpf z<>KZ0{G2=u98YXeeryd~CI_F^$I-d{GZ!RFKYyORUOgdl^Z)+M!}*n$o1>T4m_N7M zZN`tw%g_J%9U<^p+AP$LX|&j`;@Aja#2)94)m=Sj=w?+78{Cn6K3$ZLL2%B37IrLb zL=mt}b#$L*oj6~i#PA`42c_TdA7{}BIE&k8@pgbMcYiUconJSVLr)ZMq;rWW6|n)c!BY3y2rU^Y0K7xvI+Q7cM3h($LqkDUrZ)XV)Erqj>U zX;3FGx1RQ9e#3Bg4*qHq;{FsKTEI8JiBxckvGy@}?#YH7`iYIztvQ|VfWUa1ez3?; zsw4M3T68&6~8H4^b0I$xM zr+eE+>9k~<=diZv_Wg|edZ*S_Kv!3J!LZ#!LBZN+7fd;i= zKg94Z{IR*dyJIa6tAU8b*#|^b*%qjBi}koj09zriq?W3m+Ro1;rVd|g@`>kDCicvz z%!=|TdV3CjHh{o=-0svL3mWK_Nh?U2b$Bp=0vzg*5l40=730B?;Fqm;(@;L3`(6A9 z2?XwuZ|4;0a}n-i-^4j>x$fns)Ty#JjcxVpBm>0_LuM-@n033fdB(pu>g_mZrmVN* z^4Y0&8`>)uG{A;;GvUe84Dm{^z*qEg#eTh{$9|uJ*~6}$&i`iEwnp;Tw9UMQh`B*I z%1xh&C!M$UlR_Q2KJgL7-S!y$+1wLEW@9ilne8lftra@SjdGOA1*E$+3(f3>lcwP! z4AsrH=U)y6lz?uf$n(2QK2>0aUih0(tt-vq+)Fl1VZ!K&8ug&5^{H=oxZ(uFVogP9I3gAhfWMV1a3UjQp-Ym3p_Ek5i>s;LT$A-?ngnmi{azOy5dW zzOvUsYUzv^6YKw+wl5u*?ftdsGpWGHWE0UyskO1&;gVr}9PBL3)ht@f0HWMNs{*jW z^fjcbxpE4t24^n2JX=Afh1D6(9cpi4<@{H(l+*0074f0Kh-NDZs&# z#fp}Jj)RVo-o)OR{{O2OXn#3vu_66%dqJgiL|;{kYPoF=w@qXFmpYE&S01 zKqfX&1c3=mb+*ATqyGl~-Mdc9N1=#RsUcaG-o+_zl?-re#P8Fk0kr*Hxj$Zb+S|)1 z(11gJJh?vDI7f8S%6nOQPCUic$1l#;*U!%{Z0m}w8 zwP0})-=aH8boV{xrLeOVwZqqL61faxa`_Z1E8q6~^(U7b{ zHvAJw0)&l3g6YNV_s#3Y2Gbbv@+dvVY&`uMQU)?C*>nrRDHjm|nFtrfd49{yE@UJL zLXXYg2dlXcCd45Tq@jlObyGV`ffdAL#7U0mkTv1-Lfbo5w!&zP&^d+^*teILucL#n zldq?@t<$UM-)VvWWi&?c_*L|;pHtoCu}Fd?#Co|oIlE4K{T{B+-m+>QL8f-s*l*O*X^gc^w5_T}XZ?UUtM@kcKnOu3-aX;L8HR8mbQ%&m zSo#N`XhbZ6mqXvf>ygV0SkGNnE=$|8)PoOw@_9urd zq?=bR?zVX#u0Jd>5EqD?L?odikO8uYa>9?i8J$N!wV~>f&;&@^sB%e4X$YQ&HM*{Dv$cYTB~_ZCZ+=tVi!XW=BID)x7NIXyTYO%1r_*Lf%RxTZzAp) zjtn?JoyPcOG+J^LMCC+Gb6#9D9vRXJ<5-x-*m0j=q&(-=AeG>KNAGoKpKEHNSUf>< zr22D0%f^ke_#m*55{iBmZ!!|cHOJgx4r7X}ZQL`7t}&uYlyV1`QSM4tv`fhemB|y# zg9K4993_aAVfysiWC}el@bqiboC4g&`l#^ugJh+|`VWBp(j!_I&})>L1Vi6Pbdbba zRq(6?v8GRBc+dgtYIbwO89oWlhAN+Dpp2%zX(Tp(UUxEL71WpLz+Px6U`-kBYpVr-P1XSMPJuKv@_qBu)U&2r%ZXJ`VWhVG!kd_!oEE%;Ck_{X{ z-jqv!F&eUv8L7%OOdde`O1yK(n}Ej0mBaj=&zi|nS+qH9~O&MIklaT63Ha@fY+^7x_>842>Tv?)|5$%K)+5OPTf6?_$ z(XoYX*JfpoajqgK_?8l#@Io_k(% z&YI5HH8KtHB)Y(I5SMtbBTi`ybz9t}LIzs}Y;a2bpCZlA<=<8?*8QGor557h9yMzi zgK!IO$iq9XS;g}F!m&?4Kut=;{OT8IlD-}s5YO{K2g-pWWR&*)bmKuN3&fu)s7 z<}1|s=?tdsD1!NYcT+>a^C_cbnEF`T^;Sk|!|c%BhXz=3+TyVvb0X~HCtTnC^#olu}Hh&l_%{#}O3?N8J zwCjt9lY=~uyk)TO8jlTeQ~n%H)h-t&TmNgF{sT$rNcYTmJ}AEsY-PiW7*WP#(+PX4 z{@7gP1yW5~%w&823CX-1J!Sd}|A{W)+6ypc6dmx!45{=MHC+d`3)ePm&*Ds=@Mlh3 zs=yD|Dz6ets#tkoAVqmf9MqUh$$i9gSyr)FSm=YH+pz*%=d}Qag=0EM4T$h{!1@GS zbiZB+BgUzqIw$#L2=B^~=6E6qTksp30&5j3jiq+EdoH%`iHk7AQQbdIFKq#YX}YOxTUZ@fS^ zy8HX()kL)C!r~nKa&LNKrRO%gY`+T4f0C3V@8({@+%94Yy)pcZ6@;nTxNiM@w+zkQ zq7@|!rVU-9YD`r9C=15rq$MOa6Y)i&z$e~A+OtXR39)+7Gz?o&e5kmzqFzYlJ(kPc zqdqtfXBnMM`8*7yzBcnGO37l*q-J}`r}>jl@DZ3)UREu^2-x~w88?RMj-(NYyUNNc z2Qxc|R66R?-31i9FFSD{NB?L)BT}aFPmJYCotds0&3HEXZ{?G3bzYb`dME&)IAYL% z6e_4`2lMHzqn$pR*kJYqo_NpiPVX|5GK#$dsxMLbhy+L!e!37QL|jOi3THAj@5(}X zp!(PnyFx+9McbKt**8|ms;jb=vd*+MnU(yHyzsxvi-!44#zJfV9MJQVR**C<+L%pg zpd8~M%eIx0VM!{*DJ^#kBIvMa)HG+oA4B9-`nm+HP+R6oi>Fg?l?)P}h|(#qYCTq_ zGy4#L+BoJszTvsyoI!{K%vpaTafDld7su)pl`3a?~%^Hr9EXDSKH_X%rM zG-%bx<8)x8S!ecW#sg!|SXzs?F6m!3rPbS-8BoO73{I=>fR zt<*Z+=OW@2FB6VO;}k1X6EextvxBJ|OGUTpv<`tb>Pchaq8$cJ{Mtt&PZBe{ccGV8 zE?ayVqMLuJDisR;aw&pzx&-0l>^O6sv54sjx5yIm3Rd-CVE=u-v6!4LTYWqHgti)- z_fPC!!tZ1U!tJEgF2y9&8>yiaemMRxLVLhNt#J|x1DLd%NeEGLmE%l_H3=#ef9dPJ z=w_MEe8cnvv;l$wk>i?dVw}oxCX30hlqx_!q%1dQ|1OfyKTE=~`ZL&aMB;6&C9|1x z+;$_^WrN?C3*3VC$)aD+Zj$ovoH=$D^_}nnmgx%P3t<^oQfNZ68%l&2w(&?bK+7dk zs=g+X^P!|PR05|ZrWt)GmK#`t3K4>U-WZVGJt17M@O-{r`4l9DyVFRRKk|nG5SK9U ze?4z;Ism*`27IEIm_x`a=AQ4zC9^=RGsC5J*y0c#5)GYZ4J#Ehdl~e9Bj`p+6+HA8 zwn~@xUooqP1%}N0oBl4G*L)hC-YnMjm@b#sK4z=rbAVEo>o?tcB^SSGhV((;#Y9|o zzg@vJ-K#FaRK`f?0<|Uq@pm6C)CUaM0+q4 zwoTF)|J&M+g2E0hTWqLxjru--7L?1P>Gap>z(eF`tD>78&mQ?q@u7YGMm+c;{~b$h zP&=JU<8^mSqr2*gh~H~-se!*it1@YoBZS2Cwf}q&vmIjK7zr$!8ot+`O5!Dek*D=@ zdD-J%d!{DUds4r?f$+M^Ga3%8T*m(;+1=@WId$wgEI|ZTDsFtU|IbYkSV9K%A7mgP z03Q(0PlWRS-W2^1)|Qt4bzc8xTf`CnpKTGG1Li1d#Df?ebFERK;U5W>WA(p(wkDLx z4*kW&1*MT$1T>c}CO|J?H$XSKR-8bIL?jbw-0EDMRLaxBCGuuXS>E=SiB?{&8a(FS z5A$S1Z&%1MmxoRMq3SQ6Jir-3 zD%P`Pw(2=Vk6Xgc(3oSSU3ma*&O$Q2^}_yzk^P~YmfC$$E}M&6T{xRIV*$=I1-(%Hn+BdhN>G?rHSzKMy>X z1|A^yx%c!P6F!NJu(|{C(CYJ@$uEsZJJ7vnr)VfK;spWgl-RJfzXy1RYqHW9+i$FkPH;H!%6KmHiHVvA-ClqLlA^r zh+6;iva$Dj(TRzM%wIfV?a1I@Gf4ir0kEb}L6i6%Mdp;hSW=(co7T79$_i0R* znG{$jZJ$#{a#t2W7vJj3nX+izpe-@1?uRd=r@%+9?HcNL0?0zicu1=Xo@FK=gI&rc9D&}6P@jOk3Rl;r$REa%s?z%m~#fnxmv;B-9j5mZrbsg*1Ix=5f& zc(Lr<-(5LlY!F2Cx?ps}DD{}XaDuEYx?U(MClfoK{MEq;DRiM9b)7n2Z5q$+Czi_o zG=W7_URvT~l)r&b9U12yh(}PB!%Y=LMNkyVzw<{a2OAuj9X)8I1&B;6Hga$&0m~9a zma`Z_LDMW3?Rh7b$hs?mLZwH|T4QI8)qE(1)wg>g71ZYLSqvN#<>53y+~koU}` zf`$Qsc>Xc>VEh@fLwS~$c=!W=vNuOFTFmlEi47CP76^i8LRw_DMytSqr4XsGplj263``P5r%Hl^i)V=~ZY$6P!`QeN z2Rr#(Kgbe6>0dn;S+uy5jQ*IW0i7C7mdJtOM1NFo!bqL0RsW>w-zcNd!(0*$7B@TKN!v34u1ITX`vQPUfCISt7GBNLM&UQYI&I zQ^O;-@%28q=ll|tw_{`gVJ}unYv#v$<{$5pKB-GD^G`K*(`z&b{JLOE$K-Dg9*L!j zGR_$a*l)7Fb6(5T zQE(KtGk8zGZ2oQR8xFH$4`b({mF*jc68UiOem^xW z%&%VzGk^X>>!DhFYR%;j3H0w>y@+@}RO@5zeTKK&^pWr2MDNn6O5Qt}411<*jTd5h z2lreex+ihLI_Aw}I2cJFzeCp2zrPN$XDZ1tCh`Dt+hJDvRK~c<4m;c z1->a^#RobPj)Ig#SG`S~G`6VDV8s-_MuFf)P`%r^defQu*HQV21RsCb9$Ry`{%X+~ z8DW5>E~FbdvAQ6IF;#=GuzuUMgnL5zjSEP0%E?zo+<%fP6YLEgK?+!*hs>Akp`A0v zH^nbn7>pu)Rmj6*{^cjp^MU$sH5UOvF*KMtEIT%-VJ_vEWOR=-uuwx*LE21CLNkd+ zape8=H!E{#%56qDEC!I@#odOmW>zszYdiuBvzk4Xs61TMh#{Ct5}TNI9yp~eBqpw? zqooZ&-_mOTBw&pUw>1W`g*AAKV=HJnNuRad7PJNbGOxD)W{Uh+^;kvB|&etFvWbsYEGY2deheMKJ=&EBNn<+`jJOz>0r=+;E0IMkt|8TAYDNbUY(3SlC}NKoKCc6_*2 zIZU6Ta|l7yfs)0s`HT4?QEsm;$ZW5aadtsoB>8yr>q*A!bVqRcP2q~`fsaa+yfTo5 zp#Y?KKc*YGGMK}Jw96CTTozl*0Na%h)vdO<)QPU_m|^I%RgX^eB^63_6=OjNu-r%} z-AF=rJVEkna`#qZG1aUtAZWMga*vWOvgde>O^fQm7$7g<@v9T7H_a zp8??2^%Si2B+7qqz}cMM-qCgIz^IN4&0oh73=p6|EJ16C7q2*TI815L6wAC0_#7y& zuET((S2~SqH`b;ApUsKQ-et*CqLh|yxlRXk{p|Yy5=29?quN}7X=9Pw zN!~8S>@>FROfjO1XzcjUv z*vq1b2nXi)0x>iD*^+x;#nNk3*6x`~iqC$0@(F~<7x-v#lUtU+nCQA?>-+#5E8yyWsLp>b-h`re_7{AF9>`9u5o^RbXC zUhZqH?V=%idQANqQP4bvF|Ac*8J=piC=*1I%+X~Dt*+OY zkQKPVAUlXO2pg0FS`9WV!;<1OvrZI{sBc}m01~)EyP*LC)jvk!ny8-gTH8orqzo6a zd-~Vq1&G8_feA>hRn^6UfQy}bcjob0DFERsPqlRct`%zW=Dj&Rt8e`k`Ol)?3NZP1 zPhR1&y8GbdW6pJFU8!`z-@J##-IV;#K<7E+^U?45VZ}YEL#x4nXRwN}tYu`#>*z41 zy86AG*S-sXHj8UBS_hLyFk+Y%?J_y)m+4;G7lo5*a zZFu|B=fex`Hp<^Zfldj_UVn@BR1E^wX>#n7k0(w!k+FT|yBbmqzxa0ZIALD3`f-(K zG}oVpq9?FA_dP$`b=I61Ur+8a?N`xBb5LMJp)mYx&8NpsNs>Zd9Gf46|8pke?#RqN3l0P{ z@pIt(@0pB^zPXWsqXV6pwKMPpIP)6`k&>;cnYAm?&j}-IM;m)LB4;Ce2QwRMA|@CT zB6)k8|NNedNZyS_^5?Vaf4}1*`ajb5&k1EyBO-evE4}{+;Pyl|#zc;$W)4KgW|l@o zKko)MR<`y=4h}|!L{1JrmU|+1OEY~s7}vE%jZy1;4y4X^YOo0{BWY01M_Q=4D6j-; z(6rO?9&w|LFj5$_L!&P5@N$g#Xo^xfVzVWsTJ>s-;N(Ukw=DUO@eE~o)g=Ymx@u0h z0G{w}VdZP}YmvBbJu*9+_k_O%hjeS1*I8=d3WxHD3Rhbm?<^RS!}hFf<+TakETE6T_ZI0&6oaNaT?;O}9`ek^ z&gltU(kisKk@=9EnuRpe0)JQ}G~9sx?zbRH)PYhejiB^Wr7z*skYKrh5iZD|{6Y56 zI<<@hQ_3FFwzdj7`N5j@kCAl-2UjeUQwTaw82>UJOi=|o*moHxMRR$Ln^Kl;b>eZd zQ2C8vwATcTc&=;87<+y?p9Q+m)CN9;R z?$^*bi-G*%v&?umD=QK?G8IdB{QAzx6aiP_6gqB>4#J65X*nPSB9Ac`;?V}J3VK-C zUE4gR4mKHcm>t%c%}MgE+ZW2cg;>*D;y(<;Z?KJV{)=8|okhpSfH_pm%QEP2G8_Q< zU>4{osn2k2?PfK?Lrg|IQv}!YnxdnWRQ&vR2c%=S6U|YRPklamJb;skWatItD(Z2o zI8UF8Ma7h$uAjrJK%f2;UBY`Y8jCel*Ixo#B0|k{zrMAMC|CnK%%~;jrh!^D^WPM@ z!K>*+Zl?2YRHlt>7sv#I3j8QWs3}JzV<~djzme`n2Oy(avCiJr-WBc<1Xa;Bz5L;F z94NVGFRV`Ej~5l2m66G(ea^~$8ztybx$md|0#0%;&X|&4X5&>t7ExgA;4?T3v3xA(sI-KB*f{*E7utk2+5t-GW zoLy?0*FULy{W}_oQKdDqRY88bg>?7m`6C{0Bry#J?D`j!Lr|K%FKAgEP`yjKL$wl= z9R*aum@gerU*@v+!38S7;0d0+__ZMSX|a_!U)$z>NnCxj4V0WUB~YX(=|t81}vksm|#c^UFJgU1|kGdt?oO=Nyv zx0xL|vj-%A@hv26)s1?AvXm%JkGS}kR#5BVmeBIRU;Yf!v`7O&WCrJdeO{en`)MDK zq!t1#sRXJAw-}+6OO>^hac?boyo{Z<8ywrujsUP?n<(FDt<6&U%|uW(86$sQ(fZ7k zQ?LfDUjVZlodM%wuhPGwaoV271h`}ZZMZPz3h6wi^!CD z>7~Q&wjxz)&C^PVFJn~Rm{^6^d#6f!ALe%z({%W(X;o)KR7-fxGmL+gj=+my;f_1L z^RA1tnN84jb~hVMs0Ud)=VS8k!vx93!q6MpG!Ju1{r2B$Yxc|>LY8YvGtL=bz_TtY z$LOvw<<*`d#c_!J7EDj;HLd(Aq|l*3ZThEV`|Mt{iZ_6SC;40Fv3BJ?YWnq<=U_wh z3xD#qvr>jEq(M*0sa%%ukMfLtYr6gok?C8*7V|Naf{nh+`upFgQ5z}Z=_6a~6(2|! zwCqL-2g?_UlI-WDraxPr<7jHA>oCXLGOyfg=7pML=}aZiUxXSe8I}L@MzH9y2=V{} z1cU(q0wVn1Hv(HXS}PmF{|C3E`Co2n-8&rK0se3ZP+72SVJU&VnvCj%H{N*YElrm1 zCmt&NGoJ_6T+)8$_lo|ZmVn~(x-1LX=hNCb8*2shYge>Co?z|oSIxc+Z^n%<1v$fZ z?v7qMTtHMeoqXKA_S%XAlwMvxcltS5GC{ElNHL_D2ogq|+A){te922V7PmI7&UgvQ6lgRpURbs|+zgUQ82OnC2Gl_DWJ^eM zR2NRQhpzrCLC5@xGV)@ODgn=OGuMdc6IZk20HG%>&8bTowfp3T)%p*zBrV$D#hE}H zvkUrMA1X-ybPC#$`XU@CX3P10^j>6fGLarS*c>}oVg$_@Q36Gogb_%I-&~O!E6g-4 zW4!J8)OzeHs7|ENpauB6@J7zp^@j-?=7gp<3agOyA$uO^XoQ3B;gtZM{x}kP)}O(C zc*E@fDtvQ-r+rbA1+{f>;_Enjh739Db=LCe&U~x&aEf|xhqwSe<}Ox9<2}vJcA=|o z???Qc-x@S~J{H1C5TD^3)VS{8?&#v=KHtr}ESWE@@Y!x9`1@>ACEM2GM>`+6_PSvs zCMvZ|(GWF7q-bM3FztdZV=`r;*Z9qo0H+rR6I1-~+c2;tHd9O=KZBE0JUCVQb_D%S zQ`KaE*^p3C02td42Mt_3G=1So3zBX$AwEjbp*EZ{lfH5dL>xx6-*5kSlArUVZIdciZtS>blG8(Q2R2<_!LQek&aui=ty)3T4Jaw3suj zs4S{}MG3f|Nex6r0vRbWmmzq+Fj0Ihs>Ho)Rn*wy8fp};5OHoz3TM?lK2+mkx59rz zVTwR!KW%qBz0yC5qajyim3ogr0fbXx?ANm^m<=(xqc$-B7h$qK%;JflL%0Vr6B4-+ z6)un&4CIX^yQO8`M2}Eq0Y#+}gvudl;MS7g|EHMQ;gFBOlM7_`rAm$go;2Zrg-h_ZS- zjmA914%GnU&bAXRpoM4a!H3pyCyvC2GF`$xQ#nf1JUE0!EKFr$+pN#c1B5}j$gMwYtkE9t$5j;L32TfvR53<)YTu~XQ zAJit=~8xaZm5zBauG0|2Ao5*sqnxKdz zV@%ASG>kUQSyQVF(^`E)S7}5y5;7>E8ufrNfb~an2|s-7D=c8BLw&79-Y_4!zwnuK zED`m#I3;l6*-Y!=C}^P_KhJ`4A~b$BGMxS}kTEv#+>47$gLL`Ev~A3n+9p9HK{7d! zoK2LzggODQFC(B@ReDu~P~sl+XQcqUL4(5D-emV&EC)oGx&1@0jRyx(*)7%K$x8cH ztWc>C093vPP^kUB{xN3R@BpXnT3*(5N-K0rBTmn z6E5?B+Jwy#E-c7{0ZDt_!$Uss{CHn3SXd-Iju)<&12;FUy@>en3?K+QDDh+gRVylw zSk%Oqfl(>5ggLLAw_!eiWnvv9>(vkA(z6RT62Vgsf{TY#Z!eI7mXtCoI*WoQMi3AD zRM&3J!zgIG_oRu4BDh(oXbS%jr(HggAb(N(Ua_nJD%4Q6b2BmcJe#&};*XBk<1kjQ z%)Kt=D&b?7)n3-9%2Xt;6aC^8sDjq0o@@_|skxasF9jN{wM0ynS13Fn%IuamL01n} zUl3Iurmih}V;UrbrJ3iohO#`ou60FkGY(9!$1d6Wi%W(GGp^**YF8szsKm6co9g+b zkbl4ugL?p~SKH-FfyVyl^Y9Eqd#bE!aDq%b;>%hc6I(UQ-Gf1k(d)GZIQ1By2o;sQcLF^6t9Z2~f0NSUw>Doc z#-yvj8u3b#XXPZ8)d7jVgGvH;Cy@e#oGd-=(5oNY2|m-(#IZxpi|&>`lu-1Xp3Wi_ z^4{jf8R&a&nTU~0YDxJ9&Z(S&8Pem6vI>qg!ESvjX=hW z?b2{w?s6oFEI}A}c#|1!>SGhM}rLp*LveD75} zUmZzqOZ3J3EB|}^(&6FtV;k85if%XgET_|Qu7d20aG7bxde}|*nneFt;O)BTI5)@8 zY1gT-8cbCuD^}!;Xv0%nsg3DzV*K7RqAC0$j;@JUSn(uE06XW0mmRQ#S)0uNz~pm$K0PjVr}wjPu7d+Fju>d0A=QOieGXFmCU0;|uVPrIqp zdvWU-`YrVsR$3JZ=X^^}C4bp%Vn2tjc=_l!d~kd3U9Wf295-gDBF>fucBWR0nS*tq zh?I(bOLZ>lPs%3_P7|M`3QFBtekf#(?2m!D#;z75%yrYYZcUV@fe7WK4c;-&{Q-m6 zrvy!lV2yF{Q{fQgrTc^K<$L+3d~&&41udF0CF#8klR(Z`Y{i<4*{kU0Y-dO2b5UUb zkq<9Fc9*bQ%;$qoz=jNpv`;wBRnF1-h7q&;p-XY}{M>-NS2X1Oy1nwL(Yx@M4M%mR z2==(&t4}+~gj2>eUsk3w#J`x=3`vBFBL1W;>Dsr0W!ZthP+jJQU&boG3YuY0MbM{N zS*NAb#&XQq>;O2BOatunI#osbY{BVd?<0JiW5&_JEvg`zA^Epj$h?&O~FH|j>$L7xwpPV}lwhc@l!!-btilnJIed#2&PIcx5^ zIbAO-b{8LVrKI%Sv3WxEBJuxG31h2lV^g!*!}qG@s&tM>#Xf!bmZi-({q|zwPd~=~ zRQKF&ZeL&Up_%>qf6**1vHJsHKQzmh+z-+6|MkGkVy#o0xFRdQ4ZTaGh%IoZUSI!RH0l$5v#DjkUS+`8zDVDGQK&JMxOlgyPw z74A;4@ao<0p^=EF1kCwrHD4*!+Zl zoa4d&^`!mW54~EmEWw`VcMu-o`xfosYF>$?AC(BxjBmn}wvs^mM_8!X<7xxY3~!mG z#ipEFc}Y2KURiN*!e=w=P2@uT@?*?-Ut)pO7jzblbaw?#F({(w#ejFlzt7ON&;bhv zFPr@rJOQYBZL*+rF2_5VA-xI}jW69)hsW909AM0Z;~q?!*0i}`XM--b;adkqP`vXo zL4Mu-{-C#K^SbBd9J6kmjvg(q6+0{`CA{P)YSlZu-w79<*lsLH{}1b;gn=z$S!3~7 zkQd&L@U#oP>|EFo3KE7Y@aS?Z@I$&_6?bGLIFb;Q?d?xL1gEzr=MoWj=A4yd3*q#D9yMLbdYzKKP!N(8 zeXt^#=e^ZT^fVQimf09U!mhS=e4Z+z^j_bf8DoM1Nf1#$4R{h3)y0&A(owA)mGK`k zCk3S1H*{#1Z(=gm5HgNJF@!cM6buR8<}R5e*-;`JTh64se zwyiHK)-O-{n_^4uShN-UH{kl@_wP?(K5r)H@a;>6Sv_}hkGA#Q!tFV5nHc5L>m*=h`+6V$k zE?7xI2MKiUzWC|#NYkq|e|MAQ-+*<*Ags3v0btAG9Ry!X{R@7%Y@OAio@(+ z^%T}79FyLe$6H7%TQXFOcmh(|^Hwz*0yv@~xv3GtETJ>hLrC;ZS_TVN(InTeILd3- zuPMw8y~Wq8-_Z?Q^g6Ohv-lwxu~A>3qVhuGqM4}ztb2sxW)}(>f@GwMwP-dL65h)p zJfj-n@J`WmIY z?%8$?3$|L&SJ;Fw5c9o4F>*=$cftz$>uFqPO0F?76D{6%>7t;t2Ldt`!%v1lv-}=& zdC<+M2v0FbY)jIpN^1O)i~dCT0KH^lRw`Im&R;>#SWoQ@qS0l;r5A&-wS<|?M!+#6MuqO&bh}hAtw$!b|x;fxFI^zEYV!&V_oH1ln z$cZ+QQ@{-3?OyQ`d|Ki~>RHp=$V|y6I^Xg03l&P)H`IucY=If`Gs@ z;^53;{cONi{>ZKYclZGbEBJ0#|MhNftzE4U-)n3^M3L>)Ig{Yh)zA^$g2!#}ou=Da zRiG1t@soOH=y#bXliLk8H#4NcH47(#zyHy^Da9xSa_Oy%2G6nQUFW%^dII>VMtkjc zRny3%MZiJ$O(rZ-dfJ%&s(7QYNYBDbb;x6HTJcT=8;fXIN;2P=vtUed{WQH%f(4M9 z9p$d|oFrK2v)R(dI+Uy2?7<*T9e|ZV9C5>oH}7F*riNkiQtmV@PSQ-?U9tkpL%9c5~%G6&=*ui-QTEL3hP|(>hE{*v=NDCXZa1YhZWi_Kq zb*LG8HE@ofp`A%>9^^_d;pZrVU#ErVPCN(B)FQB?jvNXx6dY^>^hMdvUC%aaOt z3_c_}hYLGRx$-STMlCQ{Iz*XaMu+xye1&b9IHeFnk8-=!8m=gsR_?Q;$>=V9C0yR311yDZ~Gn*D)`hmB;CwUYbOigrHFPkb62KJWOHG3J>bjtH&`j* zNXJ^~fgGY!oL#p1<&nVI@+2(@y2npGr|JT$__YF|mhfNq;y&g@jU4Xj4!?YD0~LHp z;F|t@nb-nm+Lh5t87)V zmJHA!eirZ`iRyHu8$OsfN4J9^zNkCOL_-kt0_!^ga$popS5?s$X}MK28XBlTfLdgb zYk(M-`MzgKSQ{p>ozbU4j~@S8aD0#n_L=pTO3?VJtsaWv0D6VKt;XL(VDA1sJ6m0E{U!v91u6iA*v4lC&;Y;B#|RZ z{Y%!9t;MsusZHnK&8AD0JS(|p;!!J902Ju)V?P&J+vED83jPEsA%yZEvXpM=#Zb|V zTP*YX!o7Pyq1L)>wWIBR-7nQ4jd#zLvTKePNUY|T;y=~MAN~$A0h|nQafYn9p6LmG;Yy$V#3dcWavfg4&ykWQ=ZkkprEjT_NsAfuxA>SU3-qDg|?H;ULC#o zrlK&_U#b@OF^)+&gp?fvD2NliAG)FJ`An@hJR}eh=L#b9mCWJsyGYG?ZKC`~Di14E z6>M;>q`h^NA0(Xac}X8jdQ^n$sb zx2cHu?rf*JM$Ee?{)JE1HMPtx*etjd!nv#A)j8cvU<6KsR84u`_^8&{GMS*S(F zqPLEzbff}AskWR~fef#)NB6&sgb`(27+kU=Ip`u&B+H;7!LE->K7kf!boS34`&7aQ zEl?xW$RvL|ZW%>(4^R!zcRby^pof@fnY^^M-FWC|m!#_*Ftu03I_A`KU^%qi(GtJk zY4pqY)7}IV=cIhFs$qGvpTp>=p>|b~eWaH|cW1B$R#QXjs3Cl@s{R~HuwUP5&;+6M z-JR~w7D?}oW0fWLnAers5)o_dY%5zPzDX5?T6Y*Of)qHr$pal&a;t1Cne5>VNvg-K z8N~|`UL8~puI_dp5hh68x?4cM=IlMp32J6_)8Nq;Gt8tPJ13me$F@sNo)&a}y{6V~ za_5=g4GW%jb4+{+qPBtUC2Y3bzx(QR5FHG{zTD?IZXu>HPWYu`zQt|vO%*^W9fvX^ zE;ntZX&(fGls97@p^UX!8z#nICKAq^A8<@6yjQPz;lSHh^n@LNsjwCGCTbOHld|ef zUH4h~54nzjZWvoR^W5cZNZH$tU5r825!ZtjWilS}H663VJ(~_2aN`ACKd8vr+%D}> zB;(AX3Z@0`eP9dlanPP%a|E+)c=Ecuz+_8$s6u$DeMf-WhVur4oH?aO@!3t+f%J$L z1BMo95ZDIt#D(i(g$uch8#R}Xj5KYDCUzACt5sp`=YWKPj?|=$P9Q`j^(Cr682va9 z5fZUI!~V*R6EVUlLlX)mdl*EQ7kO|C2yQq~K?$37N;UHeHLoxOb@cldk)xf6Q-Xz6sZcdeB{K#XoTyuK-M@D`t1!`$4Te?E9=SSR+m z*2+pCM0}oFyGCS<_|1iQ(v&03i4rGh(hIs^%H2q2hsR%eo+>o49b7(BY?@&_xr{}= zQnAH?7b7vGdq}+R?5J-UJ)Uz;>>mAM3&w`U$Y4^#s8v>&mD1rF5U8P5GwRkvpDGU2 zggg%o-Sbn-kURrqA}2*t3V-EiL}B0zR#J`vrcHYUEUw6N(fN( zUito-{&36kI021gVelj7OwMTz)*-DShwAFjWfI0@`9ipQRusn|FArM=RaWYl%nHp|YcRDK;nT#AyHs0|utiY2Q$9-30Pg2d}o(ROdBd{-Fj za0<8hVVl`awafms6i9y~0^J?vLgML#-&g&>6?p6lDu@gXAz0GKIimIhn#iPn*p^vE zT2@teTUl1wSXRdIV!fSYf8Tt+#xN35ftwU*6}?V(MbEYz64Y0;`b%~T^!|r@^R`J_ zY@2CrxWIiX(oSYyBl+eXBCAo#~vh2}uB-Ao>GfC+RNtTDnuNU3u!o6azVY(|_KZp|s-UtQ%XW9!!PC7@^hdU~C2Q6G}YV^;3GzY|?Zb>bQqMw-Ga7 zm6CiM`s{-WwmM@e zE)+dR#fw3%6&UPT0B$=&`9exvH(Q4<<2suKexx

_AWo#zp$RnhtKtNge(RVjUWqHg7 z+a~RXe%Q+VRGq^<@k4 zs#&vaqf@ z91+lS3K$0<-rcdEss&u`TDQVmLU*JLasS}{d84Q#9n&P&o-+=wofh9L*UdR@P`yoU9TZ z#wlu1Xkz|y$BLWvRlZt2UCE|C6)u)EjmBMQsJC}yk79$E7gxDq+S)&Unt%{$<|k;m~dTJDxWFC_Fxxnafdu?I~2GGNaY*m&w9# zRMVM=v`P@=GCL$o!ekG71J(ZdtdZo;F5wB63cd}6B`fs6-+I=fws^PCskb&^zIr|# zSJP&@U)04N_X%R>ErA~5yQ zhmnS;4M^*oLp_Pe322Az{Xvnx1eZ7Y6K*f^4MmC5)$aPs)Jl+L2V={kYL?LR&<|9r zKi7kK6&7Ero~!Y^Df5pSm&mjkkBLy>#uiZ$5?=Asve~~lC@n#-?L2mQnF+zum|V-af|CCkW#;qHxZ^tZ zunX|HXuL5llEsc)y!# z3&6lNOqmNWtRoB_gwGAuRG_V!koJjn=zo zpRC$UG==`$nJ;Qg(Xn|qoq6L7eDshTgU=?0iq>W3G~|aJHyd0YIjAwGYadJW`wbdO zl*i(ZGp8nv?&1n=9OW!fVh!K(*VZIqS`XsV50bYgwhB{De@hX}S$v9LJ`Et7O|MU7|J21p3l$>X1^ z0g2D_p<^hwy8FyWIhmL5xHd3qR$x&4k(i2QC0({R;Mm+hv*jJ+ z$r%cb8sv}RzxJg_x1cq3^z_&Z42JjBeno+iG-$8o*i)*53hyjWK6&aIYd7Neo=tfp zK)V!#_B6930HyOzTkN}!x_fb`Mli*`5VX_UwlX~2-+>WxLO|c|m?M9H_G#rye3{o~ zj^w2;2IEUdlmG5k^7h;})yJoxu_hBfm;G9l8VRluMA&gElVkurEcMWzG0`3)Z)Id} zy|>z@Re6YcaOM?*!Dz@ZURhJPx!CGYr6pk0vgZ?XEP#8u@WMlSs4GL}oq5GCd=1j{ zpm=F)~BmDoVy(@qEUH@zP zGh0RdQ>_E>eJQvhmIV)4swW9?4O2$af;?Kz0$Lo}BYf(?2%f9im7nA~_Xbe_6jxet zllVYfG@Va|iL-C4-@Ai{<&QT@L&@{o;U9!Afh(?ty}_}_YGjp?8mU)$eUQ*N$STm8 z$j$4eLFxh}0G?G-PWpUpfmlC&EIGpp#BH%qzY@?_W)t4TV>?eW(kC!5>K&m}KRs}i zBMio31DG1r&R%Qqc}x|?$T&YkaA#n?f!@j|JJJC9D@+QKN^LX|I1dyv>(eU@; zl*K4ef(?!j3xQ?D``tD~Z;Nj0`@Wy}WbH>+IMI{9Hrb_%n|&+}>ZS(uD@s2Q`p<`n z0%gHEpt`T+!>~J3vsW*MP?9dmgUxh6SW7GN;x^6)h&q<>otV}53R6C#J!J%O-bqu& zv`n$Ay39gI{^U1JNJLrR5L04=<4GT#?&9YOAi&5dG49-fBW(=}Sfu%$P-EIH(ppz0 zimanSi$-&zE$lSWRqDfO{Twg}Ej z^+KP*2GS#D<*D)*9lVb+s?3?&rAqm3p z1y}j$3YJwSo_Q&k@CG=3t^iVr_+$*O*7*vTe+fe1aR)lw?-`?h`POdTaTQGr@vG*H{ZzajDmewu zPcaT0u1%0|M`Wkkr`B*D1S`ISE1b_P2(t$eh^m0UGJ6bTHzGU`U#^~S`>nC$ZtKd z%n#M1X~6v%zK!gR^5e3a%UJm@!z(Sdw)tlHcKM0|5Y2EQozbi?WJ!v&h8%L9D@GD6 z@C2CET5p87&P*sV6ra-2CH^_i(`o5@G-YrEB3+8Y3`u>-cCo)V1gwuS(T@CQjA;Ak z(2}*vA!4Ail;4Fqx&X1{Mnss@-bDwptazgilw$mX49mVCf!+in`6M-} z6RlX%j`~MT-FUK4V+y*_7sOS*$qY+qW=F623Xv4}SO>S1chB8#nG|=2hJ~YXZ5UV_ zz{2`^M4x%0RWhRVG@@}#CDg?gN@jLZMX_*l92GnT0{JtyL$qHHJWkUKvvWz#E&-~? zOYjlWUMxFbZ_Y7HwhXFf+$Q?1YBl&2Ba^AW{dG=eET70Ke!{J?{}FB#_;0P(zpbFD z&FB35zeJK^nCw1jfTACWJ^rjJitus7c9?S^@P$yKgr75nS5}7tZrl( zq307S%(C53)3)s0)fhwe*#p;8xB$Z^loN}C1_vSFs7QAab*3jXf*#IyD*pK_Mk`0R zA@k17vjJ+NpU*!yU!AIg_qe&=mqnjiGrfOG4kRQ034%mi&8H_4Y;ir}SQWhdM{a4w z+VmOvj0o|6Wd8~MKgEn0_|K^Qm#bqmIUN0CV04r?D_@# z8rj;$kXM^(>@s(lwweC5Ze7!{JT#?IahAxtY)F>kV8F9ja2AoMFly z{V-q5A6x`Zd?#8bT1Q%E-d&9yU?pLUA~SfQc?ZnyJ`tuRH*?n$kjCOpt=U zVq_-+by>CCS$L3lNOgUvFoI|9djK?=d{OeSY={eci({U;>*TDi^jZw)T2l(7ae!mB?XxQ59$v?w5TGy`s;#mEUkq& z^h9V(FuK^URIijLMG~*fM#9ro+ju7TAXYTH zB(z_`E|V@G&lW=oZxFkLWvLV2l{wE^9Jo{bG#9#WFpxlMrh!#z^&meTXfOd*hA-l1 z-Urm<^YO3-5&X&-pP8QkD)J(ViSXF*v4w<3PJ~wXbp!kj_hs63-%3@9*Jj_d#H&>s zkvUh}>KJA8Io*wNs<#IlrV0D5P0J2GC>U_(R=oJ2z@x(_Mk;cdzt%+L9R&w$T2cIT zfJlB1!+w!xT=5Zkg})<_J^LPP_+s3%>DR#LQrjZY z;Eri<$u!KbkkpkfHmF!WB0*8IJKVZ(#z>|3kWeV52Vm*`2W!jXjJziJ^N{%X6w~;7 z*4F>qrf2!DO`jnB=^^l`r?#ffs-~^q#7_-{|AM%@D;p^UTO^^XaGg^uosd9Iw1N7j zfd9SjpQLSb>G&V1C#Gia9o-&16T^pVJboystTvV-)+%l+C3M23`V@`rdL#Y!4=y5_f;T`_~S=I!FG%RifbYf2Esr%>`Y=)ucEt3EAu zfPpsZxN@!>5{6Q-{!5sUEF)}`PE(++deAVRu?3^aY7B+l4D`v>97eMKS^eK8l#|<2SLxz&jvlD(ugkT7 zABGmrog<5kqCPiBlc^ zq9z`|1~o;*GYEUf-%C@Y0el`(b*&{Ab>UKW$rUJmAG>Bk7yq^*U7uNKr19~OQ|xO9 zJh9>D36}A>sDHnpV*jmEOw!Hc-^%7s^%OTtlmB`C$H_~7WkMX>YFyLRtyua7oKFmG z4k&X+aulQvgNb-Nvn!=y(l&O>Cm2`m4J7D6Ad?%=X2|a8^yrU;?!3&MX81yKMsY;8 zB3YYhzyMApiDXz630E1Kg1q|H%R+Rb4<0_l*XAgzskAXic! zA9Sv5yQuq?LGwuJ+;8!su~@CJ%$=t9No8~+zPK%;<~3Ak3y)*@^qZ5V8;b zk+bJ+2xa&w7M}@IZlM1G^qP4*{w;g!^&egOQr4Hj_s@7{{l~n)@ZX8}Ka)g}7PJ@Y zBGyOOq?vg=79lM$Eny5Y+wXnELGal)FccY3a{pF|#mNg3mii79b2B2rGVv-c>ng1Z z(W;hWwMwXZ=;8=*Shdf46?`^7wJulZ{~SF|o?)?*Q+MmXUru^%J#IdBoq99u)J+%T7giqz84j!sa~7dRyGBG>4u66}l1I8$LFy0zgqMb`&>3lns1D4F>M$6I z43vg3A$f}ODi1V>ycUNYMgVidQp152Vc$?3Gs7OkT6G8Th&O3=<&aQ_;Uk{3k$fq3 zZ;9XKU0OxN`Vx#(9Mp!_7;1YoZPe!j?8)1Y01+xlRy6@CmbpWS{2*ry1;8ki>zkPrH8 zYtSzGZ5+@ps%>r1F1l?V(0Ab;cktH|zYpZAEl~x8Zn^+;U^^7|4hLYG@iqL5KL}O? z<~m>ykbq<5kbv{MuLg6|t)CYdyo1Mq>;D4>y$>JMrG1C;39ld0PGi7Dygw4`$bix$ zZLIcNKPv=7KP|@-5QN}0XNN1~5Za*~2fZH_@L}?lij&e)V8;pZei2gC`RaHD1_EPO z8vN5v10uIIAB5kz{}cBL+1ptLFtBtEUD5Z4g0<%2V0-wBJ#qB=uFrsd7yxs3NJ3(v z9RhIF@`m9kYP}|}S~FE1({Qvjk*cn~LF`(XwyR~0Fb=}ec({@IR`(%uWI@@t=EJ?8 zg}4F7;qb07LQ{dT^D{Ib)S^sMxs-Y|*%#`qe)Qv;}BevRI74dK9gAO8$+JEj@q zgiF8VRUyvOjS>Y*r7h20v(8% zsxS2c0(YO6~VgIBT*_JcLw-FBPP((9@f}1K&(% zn|H&JHwSawjkAMW6*9cJueKy`W?3illnz+!|T zKn^A0I90n!(XE{WX1$NsJPM+;PxJe{L`Xi)QG@b>(C zwL}U9R&26$o2|d>PNWFiF%7M($dSW>tyg!Ml%uIxyY#0x<16YI;TZA^Wf7LazJKk- zFCs4$Ezp`MJ!YERG!RiWGLtPZOLNt#)P@_r_banv+AYV;jBIyROC6yZ65X^*_#XCh z#8l?_^*f=J$$8xvHsNMbTjxp0dq!)Sn3Y7b=65&;%Btl~ZQvwgU7(bODL!DPtq25|>YmPr`xvx7#)q9~l zFY9H5veqm|rJ%s$xccK3%&SFqrQJkEu?>-H9&#D#B<<K@0$Sy^eJHi*MUv-*l79CI|8B4kZ34d5Z@@)Xc0Rb7pvunMuSz9#uA&nj!0jUo(~i4l6&WN+t)JI$7{wN2MsX- z$Md-6w43bIdV*-&Do$Ksp!8ai{x!Ny|{dM29|AcCy=`lJ&KM zK*2=2O{FuSeM3ocaxQo2BZVP(y^N~BV!23K%{@mOcb3LlvQic8y9lQw>6O+<7ep^wI@Kwn# z2g3qdj7Rj~&af9uOp9~!KP)azmGNUE9;c1rR&r7BtZttI)s{viXzD0R4bS*etET>| zu~yx>j@F(dWz>}C8Of~7i#hFH#+s7WVAxISQRG#(W-0lucA11y(~ht1cR@E)p0i4C zlj$zZYnbvc)f&~ObG~W2W7DSKhq}dpns<-o`Qx-H3Yk^dR_|e#FxHkBYYZ&TEtdA& zrFJs8cVz*Ycq<%6`a=}?W0EQ@%hhGp7S-Cz8+$G5%7#b{| zt!pe&%qC043`ygdNLj(o)Hqjp1{r8?Z;S%~<%2L4XIrI01dRpSzbWv`hmU7LJ>vOQ zr$Z}4EbNz48D$b^PTtggc{@U0Cij34WjTNgNd*}@MZ;rZU}NYQ9+I`2o(K2 zTtsW3ly1X&+?TZc2JrAMTHu(3H4=$ypi3N=LJ{-fSnHHrX@1HF zx0uY_iGoexTdnLs6;YEMA6vwR4M6Rl^E^UNXR9N5z#b};-jp5W5=xTt!d^d@K=!cM z6kOm1IHXq8C@~fC_-NP&gb7CpMKqWl*?VRiQemfKnS==>*x<^R25!k9l7*SrBSJrj zC-0`GBbzLi&mJXOy4taez)TPuM*_H;O^3L4I!BW{A#yHj(2eCl< z9LsyVKwAMyHm&t1Z!yyQT2@T=)#a5%U6nv7g*;8Jq|-|7v~1}J3D-#zlMqT|Of15f7Y{)6(5S0YsT@%RJRo_c*o zXIx++Onci>F0(b!wNKgyc{zrb;k_=pP`>P$(H|w(LWAQ$7!C~!S$4~r*ClTUSq3hV z5{+14Vym6>#0UmI9Bu9N$Iw-@SwlNGr+$kcaQdKsKEpj=zbK=Yx0bYNa268_*gCY3_tW6yT*vYbA1=@fn;cH#lK24(g+s^3?;cfJQ zsHnQZ-nNG9eBjTz2*?H<%UP;a-6U#c-O#;!1k|o7#`!u@Tm{)!a0R*akOFK!fV=1x zkpLXS_H919!I}~Qz`^vR?+{o4kS_AWGsV`93?P{WHRusb4IkSXD_k^$^^043x(oN= zU09WsV)Cx(xtlUXts{V?A{T)cUvGlzZ&Kr$y zk=Ob|bre>KbxNpZjF;d(8skwnl6)69-CvSDy=W>P+g*Uz$-_#DgJ{55^F#_bq(Fh9 z1EXq@-g1Fh{BWm<)~05Gwuz#roI*Ko2M_Q~5-@8CTgS(i2hfRI&A@yoMzflW&9K2{ z?IS|-XMp4-z_c3)ZnqdYP)LEf{@fUN3z{dj>UQD!Mc;)MYqJsysL7_=W75YxMkUbicY#m`b%XLWO->$Jo&$>C^fQ zDs3%Ix@(Hj5!^?Figb;NQp5&ptfLPr!$XDiv1IiVA&1c!p~msqZE| zbI$~5y#-~zpdZi6v;i&%TosGkesIRyJi=e<@Ljsh51{;^PuQKq%$sdnYQXi@&GLOg zi8YZ9CavsCge-|)GpVan`*leEX`S^AD(Z$0=WY=D1ez6|QpdT=jw(@beBOY398sDEfMTR|~qvpd!5DYVI|v!h3>-eTL+$?5$M@}-OyDQ zyoOxHMJlAkZ3D{uK$Qs$Qv=j_;A58gZUclG*@ZUiPn!*H*g?vKfy4pFlnb76gX;`5 z4*hP}D7CP^`9&t4o^644?vOrBx&@T&P_f6x2!L60>=cjOw@e%?3fGdFqS@58M7yUW zaO;JC$dTdcbJu>*8g4^qNLz^b9AA@Fx2~}yUd_mR4CxIo#6_*`nxuXR^`mXV-e)H9 z((VjTs}TtB`QQq&bK9gMec|hGXn}F;Kn=k+Xakk(p9+5ebR_;=-s5+6GW*K|-BADg zO#EL5JF4bxh%3n40YRp40|@X9z~F-Pc(iJPOyqh}bf5+SR#;XjhGvY(!}B|J!;vB^ zG7BX5OwtQ15Yp{@>xI&&u6Zz$1X^bJPJS%X&%~Sa-n!jiXWPxkQAC>YhrKcEwnyFX zR~^qAp4*e1AKOqVKPrcLkvpaP){(h#cM2f0NBa+K)ucN!phN_K=nRFS^hAz6{M zlJwZ-Klmr*La@~ zFU2hiFZU*Ot|XrnpO!EAPOLA_cjM>)1y7_Q;1=zUAO%mr2ryUP-o0B+xT0H4IPK0p zMR({vMfOOCawbYWqnpx@I#Qn0Hsjr51ZQGu2sPAiRwkJ*phI9Dk; z7e%6P9N63jR!xyUG)LvgeOy6V+#!*nn!3#LPToUx-&+zfd&lyY6}wU$d5J9iN=p3e zZhqYqFj9`>n%_pJICYfO1*#my;>aTg8qV0xBPUHK=4Kh@YI6p#4sTLNYY2Ck4=-1k zS{9sju{^n=dm^J)Sg=J}3y{q0B1(zLHW2aAG8Sd63Ys=X6XKu0Huths`@^nC`sRui z7!e(`!#`C9qSoj3E?kDOJ;ko;BK?{76(FDj_F<3NEjV?iz4|^4sRi4$!{ zw_egOO7ZodRYvNtr=o=vVAGa^_lk|pCix0urY%isL^9HK`23F0LKH>jmnI9BCx!$J zd+TDLEyV}WnKB%@eyDfSF25kE|jRMVbElH(xezN3y;3ForFQcN_XRn`_8$EJ2;q%+O`zRffRSAQ zTpQ1an~-o3JbLn+B~uk!Z#}AHv9={-#~brGBlR_+Lq{ozsm3=#Zz$1yjQR1LXe}NJ z4HiwGeoG@5k<>(W+~k@V5KR&tHDv^@B3aR!B?(-+;qANUOHE0%2GyIaV4I=ZBe_70 z?HGOUmSXI4(6<8Al2%I}cDq%E)uZ7y46F_j9W@JeSwcv!I0ijzu4lCakwnR&=TiKqX6tFfZWr8ClA5c0K z65LAhAEc?Jx&~u#@k+V6#joz!f*%#Xd`yXO-!7?@a%MV3efg&j%cz^U#H8^u>IpW-sQZ zEL~P{(CX3Ms=V4Ao^poIi~d(6TY`!&o>yrN{O?};DCdIKjyK5U zoWb}AwJl-Q<`{$fv@TP4E{G*p*gn2jvh*3Z?&aL*Kf;`KxSOdtk4}Up@Yg#R8q-z# zy0ac<%yJDl_*Q(3S9{?i)#gX#Q*6E@4|-4@b;#d$CF7QuD>F>$j6_h;{Bm0%HTr_X z@6vCzKy7=;0~w~typukPd1mLSx6+p>{HwJ_As3m}*H!m{?e|h95$VFm}j0!;~k=x((Z+Ixp`H zq_Q)dbBGTxNM7gp#^SJvu|xiy3}av^6W6bOi_`|?g=Qtw9(+jucw#= znBum0xNE`&H&t!d??7)e;;hTU`ya4-^cxW)dHd?~Z(QbGzGzixjVm(@QnMDkUqJ(n zL%eNP5T7B$M35J5r+^CTEX+%R9U4FmmtWw5(OIluliryuFjCVN`GT0deQ6JY!y;3G>c1YF89d1e5Q;_M$QTV}Gc}9#T zh`2kpZvh{-KA{+*@)4q?_RtmRYEa@QgNTI(;#mFmYHpCvyXG6RtMZw>1#DwH2eMAd z{>-aaP;^krCFsAk-gHM&1yafYT(*=*@Ecs)NY$Z&SRAxle<%3))rt;dvKF*QzK};N z-dz0on-{5)`W~*)NAS@<;SoDr-vW za)>I7#tOCS1SY>zEEIvVHU;Xs80T?x%|IsaS_zdO!D$ZcL{NB!1Wf`i;Lm;i_C4=0 zm+m%h&X4A}pN7PdOLWTw-~@Z(iwSB-(2S4>p_FQTB7Fq$kPc!Dz)%^+y4T)S^rJx< zZ5K&*b+v(-MzInu35;}rOX0OI%75~vvos5A6j(hBpR+)D5q0vmlqrvcmzdfbmY6|7 zVd4w}8T6xX+yCOHKVkYXSQDbs83X5>U66{!W5iu+f; zk;mYgw%hmlr`A87I)5wFAGQvrwEBO~%+i{>xYIg+qyK6D>JR&ORR1fQNebViV*GI2 zm(iRZ=OR%lL>4r7;%Wg#UfaaA@Wh1sAW;Hd=UHWy)P~Q6B_o3~2)Kd3A3u003Qdqf z5N96*vlf=T%Vn9JU*C^Ge2F{n==TkmM;vi!&q{NbeZhLd>?2v|qs09rEcIo3ErW9Hv;5^xoSx z&3v0IEdXd3H{U3a^*HL2+?B`}@yE2a&JYq)(Og!~$hDwj1*YI?#4N~i8*(efx;Adm zYo#m>;{5HPx&mBk>*MffJSLT)BoYM;*GRGKz0c%2)?kDpA+Gz)((m;>3bNDFjrGGo z#wMd1IkCgp0gR;V4oNQ*Ohm633({mDyO)s(V3<7;`i2|_=7t!mg;4c3B4%U6;Oq8jC;zbNSPh1Wxc)>q>ovp=m{F36M<&gK^(p~Z58O@ofs-|DwpoGP~$ z?w63tWCyv={yTMg7uIKT7suv^E+P%sV}??* zKj~-tRE^O^h%$pNu_KZy_STM#G~3q`7-^EvPNDryWe-)=-7J)lRIlOdDeHGjd5t)S zdjx9SNhA+JeF*K58#PI&LyDfDg{UszZ^bfi?e@_jih>cXfOEj5<-{D%UQHHzf zoXp|NkOWu8z#rH##4Q`8Ix+o&<#D8}_e`erW*PEYN6r${rI2llMkI{ZN@CLYjB!Mr z5~%{V3=@d)9-?mVe(&=HSJ3+Ib27ifL-fBeiS!F)it<)CKX| z&BS;i{{Xoo0OyYf2(J`Xdho&eKzpD1Mz-J|k~s>4Shg$t^4&FhCZo?}?~FnR6|7ru zI#4jaUyO2Tz`Rt^zHs^ec+zdWvNZM?>d11)bdEwpFJK>+?piP1uq-Wql|Q2m!(v7U zu%w3@X_~fed;#aK4?^o>Gnb*^wPsjH>zC<{ZTCuHU_TmH-bF}vA26B7t zLJE7qgQbJgVd7!vVeDbs8ssru8R+P}d4bkZV za@t7X=vG8hfIp7b5>ZpS)E*6_a~XMJwH(J~o7005QBhzpLbOiToM}6`njrH}4x$ugF}=>h z`Sk+T6K(rkB@KRR<3a%2&r+?wS% zj7g6TdAO@dEXjzK?E2t5+?O8Je7wU=l*RMG!!6PJJp3|r2q z5Z?0-MzbRNBDu9e3}~{13Bwsz6w72%^Pj-9lJkT0&cPmDpY%feg8EBs7-`UOntd-q z$N%U#gz|rRYX7y|Bx$N*DkF{j0gIOcSESihpa}woWCC6bZ4|&56et2I^7Sp6mEaT) zrlU(6)G~VoucpCudfg+v846i57EGx;EcodI&KutTY$RaDrla$9Jjv73HIwZB zBO$ecg$hNz{3yB#1&JIkhPT;yG@q!M__$NhxC$YndVT#6AoYm`$8fauQjxQ^vWDU?Vsi&>e5>OPo9zTE1) zz~+`dYxF$`_f&Cu1<9#vdq>Rrkw$J80lOT6&%kVyFeE0>0n!u5j;&l`ukheYM5i8pN9VOSM0&u9 z=AMkk01s2jM`7NzxnN(!$7n?7%`IHC*O=1XI)rK~Wy5E=XOMOa16RXT72vy<-(kFB zbL`!u8Gdy}oV-VhSH|75+s;+RycRe}*U8dWQmZF!A+xc-k|V(PuoV zApU~l<(@1*+Xl6Dqb8Xf*uD1gDupY4$`8>c_NF8OM*P5VDoDObo3|pCBY6q{(IxJI zS5zZ$B3j6);2|c!i@(WJ00dhVIHHB{f_@qYtwLlM?<5K8Nqoa&E|cC!)hNDl3I_%I z8P6lojU+3(yT%jb`OZv}mHP?&b+}^Ypy8|ga=v9y>(Oz7NRgM|i@E+IvCU*`)-#~vraa%kzm{Ptf7%0FmqS^K-An)e^= z30eQ0pd3t{t-tHt2Yc%uKN-SPkHijc9DxwZNKKtH3DwG}bHp?5MatWaYJ zWW=+L5Z3Vcxbte(3u|%|g^j`rwM9sNXHsI#PL^VJ4fcyrHE*P6QdI@0UAqFNYjmM5 zoo7L+U+^{9S#=)M!7)EhPo^BZdAA&&yq{N6czM8L2SqtVoV8ZyiwreJ-Z^YeTFVSY zMyRnHs?7Q<0qJqMP9FLBbNCj)l!cH9ozI%(+e=uQHDj(pzi1@Z~( zzyO(dlH2w5qD^Cr?ob1z+&24Bx~=pAPoulZ>M7t++F_PJPlh?;m_k#9yMbM!y}*h4 zBScBVwddDed~TGq8PTh1$V{^}D{IJf46g$-V`xsoMGG^6Weu0RUZ=ZgWRe)9aqPzG zHTF4roOY+V&ewqmaCp>C0-I865Hqq{(5k9V#wU>1RH8_iSUuZX)~gKdVr)tmuvejI zPl~OYIn{}8l5hx^#hkm;L$r8a*DBX%!qd3r7AYbOj7%m%k>qhtc*`oQCQH4m*p!W8 zY)iFS98XaugX5sD?~*vX)wc=wK^>~7PQ}x`da`>i;-7#tm{qAwzD`N-v{;z@mcs{5 zmNxB_pBoOxT&i!Ney{wOKBZ`Hl2!1f6}vh|GOithT4Ul%xGB!G{u zFYgQe>o;b7XbmKDdwk_s1-SDTc;tHcMeA}~>yp^AdiNpuS76!V%ATygSI__BE>>9R5@vpyMhjJ5vdLy z?jj;{$3s?Sotna z@?ajNhyh50aL)TwL3C33Qi{NiixFJdqH%9G=I7xmnhoJAe(0lAK5MfW#~>5@MSGHB|UI}+~56va>Fm!G7$2iV&7Tl#fl#~%z{ z;t~}^jO1x!Bf-7Ds6h)|_yR}KaaddOQCe$ia?32o*MNdVpEL!-bWSF9f+SXUio~(< z&e3wovqezM+sooVO0}*fubk1oQ2$CX$*ZP2U*F2z=pU8)UtQ<_Qttm}iun(zMRo$x zpAU(O^(+SX9saP0UX=z7W%KhHP#G@1F zj=`uN{R7$?G>|{*W5NwuJHw;_>#T%PU5m?yP|X&ha9u~2tAPF+#u(9- z+qe&!&3{T)&*IY89zlK9x9r^#651zN2yX;w?F2>~Pq9f9@M3LXo}R6LX-qbO`Ru$R zayq-t_{dm{z_yt34f9ge9hrO51{+qn1dF+5&E@wPm3-kgw!36_pmuP#J2@XY1CReH zOC*&TfB=t$Dx3~zm<~`s4&pmQz+{r~Cz0Xz4+&L)z{#_x5plQ9bU(UN%WW0wz@Aqg zr(BlJY-kpdtJ(u^SH|G4yBh~zgI6}!Y6Zif*5!kzjYF!5MX8BHw5_GuQW!y@5t)=W zAR)6{rpPNkR;&znl^gO@7#9=$Bp$!eZi>#s&2F}&3jc{jONiLCMf`0_<9Kdr#5Pxw zjRogCN~vN4D~oULff^cfaF2Mm2US+QQG)RNniL9Fs1^UFw8zsUUx~*Wa6}qa(1Z8} zMvj@9lDni2?JYJJ-?6-Rf}a;7wwleSkyy| z`g!^<25av2gogrfcSOUu^Fa|9Lat7$%d^wJK0N;X*eTFATp7-e^2P*XIfF%0!9Lhw zBg;L=2V5C~UDjT#9gcloYk>x#HQUZuv<8D#u3QKhBxK&&^fT2K(%WG(yQTs#_iF(0^FkCtDSDAGD5qV1V8Ex&h z9{1FoV)!~Qt(Di`tjG4R3^(TQ8WwHkQ}I_AjY3DzC4B+MPVFx}<;w1IQ>dUCOIWzjDf_)WQ?(HznHk&My2 zY+pcM$l=FU^U2Ln{Xbu?wywFZxlJ9PQhw2D1IWc7GH4hP85bIzFhHY^5E~I;#oMX% zFCb*c-Bbo16LHbz>Gne*=p^362NL=u?)n6a1cxAvs!iU60`lS*nW^N`mFSTR0u6LQ z7b58KqbJjC49phlLE`AC_0`7fBM(@BD5|$%)%{SSGF6+Xjo#h;L1KtLkp9EJRaO|4 zg^mb~L)mCN~sPrru=unh|LD0_qI|!wz4R>QlSwswTo@wcQs9npw9u?w;`%g9KxNVi>)M3 z!4uuubsXs1g7Rx-o?Xt`q{F6}#tK^E;(X_mr;g*_Gy~Yh-5%fe(!0 zh>eNaEA{5<6g*73*LB8Pd=n{#b6IONNvTM&V7JqmVJ1dRfEUlz92kHWv>Zu_u8ZD@09bY669I$S9c28-_ba%9DOf1kU3`*CS!=FWCTX~bv?=M# ztuUdadis{2sM_`AIix95#554Y3FRCof{;>ODQ#bq52}`pF zq=qMpAEP`7hCI3(vrn5nN>4~oH*KSn2MjUZ%gnt@%&%GRz3krCh$%ZLQY0@5p6crd znr<%XE3?&@k7irrPi1+)S+%&N%;MYK5`R-l18b;n1(AY#nyCWOiu=}w9%BOUUzx4r z9eDv{BXL3?YyKV#m}2qlG-@7%{UH@2R3KQ2h_{bV<+Mvd1O|pF=clsHZ$MOCg2d{9 zuDBmiiaxk;YhoY1Zj)h6n+qx{BuwNH z=hB3l7p0*Z>pJ`Lh$n&zK|(V3gQnz_@&+=S$~|2&n=P1@A)zxe#w(T!i{QxDTWdK- zm=Z?EjSyAN-bZNDG~qLl={cgjx(%r{5=kbu9Hk0NpbPUhwZzvQW($sb*Dc_hsk8b% zJ%*`R+dT%>F7KsdPn`3dDY?;w5?QpIxMHgU6B*xBT3%D3G{S#f9rBs+{%@eEM38rlPhGILmFLR zF$mCPFg$KsI1om{-A#d*lVF3m*q775&(HE#tTh>3payI66J_GkNJ`!;y$NU+h?4w8 zd2U2*z3+H#hAWSMJD>>&`4GjFtQSB-kATp9%2FY_ct&A^kY^8A@fi%#*W zR}_p}mgOBLmVIoCLdJ(C7nG4xypkIpo*}RoEZjW=p8l1>dv2b->lf%>2=A!fgWE4C ze|E~jcR1;RcteX`WR%=DbeO!GY6nyFcF{Q*mXen6E-Wu-x; z`MbXMdz61&1YttGQlB`uk}qGQ1CdNQgZp~UF|njWW?e|Z^}J5UP+MF&8uLuQ^dz{` zH~(#si$Q^1nw{xUnJ^BlnprS^FbLIC2MeG(irHai-D7sWWUu0*&zrlLo3R56m7Z zh|4jqHJPxsY!gh%*Wu_4c9l$w7oif3A#@fiMVS;M^OFku6QD235FE6_ zpG2R!in>pu;-dOTiFG_ed~sWb4!yL5gn3qTBTS0bh~ERWwMP*(ao08Fn4-?R97jXE z)d)%}?e;Y9pAE)8S3NbK&qYu_`oGbV@11P>L+Z5C?zfZ>v|=yafvUt>i8nwHyoopI zfw*!ZW@_t#+9q3BIOat7|xXJVZhbFlRc@ch0CIt9~D7G(i_yv$% z(C@}~9U;3m!2!^?@$kK1Xa>-(^iGTqbVTSp0qYPMjY($EX&&h@?F7z9Hvm>** z(Gn-nX%3Tq!?QCtaSloax_$Iqqc`9K3(z_GZYq7MH?96Pdxp^8=BXGs3U={9J<%b^ zUO=*BFC;Q#Y(owu_FDYWt|4@#da3S&onB#wBDdrqwEO#z)B8RksNWr#H9MjHw!5|d zwv1k2vJ@{g?=LtwayOdE@7UdiJI^nr{=7TrAioY~zXnDAOvZf$?bf_x`uFTNKz{j& zB7Y<-eFe!fcxm+Y+@|`o-e!KAS*rQUk@g~2+9K1Ds%zID5+p%lCsV4$}5tE-zJLVFdY|PenCVVuXs7z zmn3sCO^l}7TN|=u%{GmNAt7_r^p^muChuI6%SR}*0hi2W?U2A_!Z|I@Iy0m*%*&X; z^zA{TnNQ3%VW+3(9+Y4>aG-Z5KN#J2s2BKXL^UTG*EoO*L32=C?PQb1wB27ENJ%ag z0?YckI%j4UT{Ho5RX`2ym8Q!SB9vFvm6Xn<%rQsXR*Y*iW~@TLMq@xk!J@xrt%}?> zo4&U*JC>|fA-cMRu^H zFOp~zunQ=q?r@5LDu=B}%_4#|c}t%(t4>?9rOgakyijCadCFH+?Au!~RVcRc=Oi`- z4eE_6imjcC*vxX0E}O##tTAOW%v-GquW?r9ilhJ@!D-r|=dA;)^Nqdw@=7LCa6LTC zbE@S6b#s>()==G8nF5LNZ?P<7=OlVjv7;FJLE!2_*?tYo`A-w<-A9uuDGKd6@ zgp)-Hl-;p2q}p>wMh1u(V22b(8_WWb)%V6hCN!fhcuylmiPNHnza}#o44y^1ZA+d| zcIeFz{ImxK(os#Kx|+(v$Aj5F0#E|;0kAe>Z42n?8fR33T=nX3!Kh( z`GHVW+npDOI-n*Kp}aR%_rW?}g$~{qK$4#>AAkub1Z8hSU;$z?IHeYlv={%>a6_=<9tACVm)sMOM+CgDIf3Cp>=}T<@PQ&1m@VCfzD%Nsl@2Um` zRouB?&|TKB*DrP0Ys4ABur?9=5eyMN+(~?@Gqhn(N3$m=V5J%?me@TD6}vcr!KuX2 zMA8GFRcuEPuE8E3RVnCJY=RIot2Q+k=ySj;d8I)ilyZceE#^RD_CU~`Lp~W0E@GA# zj-*g%9ds*){24H3=sEex{94=4(7{y--}DolzITxHp(3eG2`X8>gS@t)$ly`}SzB_! zm7OG3K!c0~&LrneH(}3GpI&UGmW~rJBihgAQYW(ZIX@jIecmc30iYNH!8;Oc7AV@@ zVtFpUs;-|Y_Pn3KSi_udKmPOu_Yi&TuG{Uhm19_F3dn3Zu|+4&#ddS+-;|$@`iYE7ijK)vr3pT8y@kZRmXzq#1zp_U zK7<6<-3&F$CytO!od?&xARmL^#lz_KWsmGG#&ylabdjmBt@ZXVl(vPY`8?CvglF+j z;c*R)<=LRIX7_#E)*nVu5q;F(qj-G%`b$c%*UT*6e^)p$fBsWZ@84C<|A&;YbFlpe zS2>v*J4%jG(Ts{$E=tQxN{&g1Q;AbIE7ZHsT#Cf`zWQIaSwSm(M@Jcb>;D#~|2=|{q-v?8^$qHVWgNf`#>7eRP2-w`n(XbB zic~(=%MbL$?~etR;=o_i+5toiX=8IG@mpMG;<2!(T;?IDidp6ElAX4NyN>v{EfJ%rKrAH3ZV1Ro-OnSM(Ios=6x2wSn2u)rLNR-#>L2;PJnM+jT-m!QBF z;!eWd6a*i_U2TXS>V6I^{nn&imY?CXL$d>LhO=nF=uW}1ZgS8;c`^9Ib{$|0(eAZ? z0XT`Z0SaD;_+xhdyF>#cXjc!p&|ql!&<8bGKsau5Gjop#@$eKQ*N;aT+zBCXh+#%JdL?#)*v#IZvnf> z_C48%_l3PQg8S;lH>eH6MHirH>s&x{5^foAVs9a@QtbV9m4nrTBU)vLukBJIxA!9> zd3O>RzD93wuHtSAGRWd zQ-=A;DQe9wp#IL$|>e)!Zya4O(Do9B#Xty zq$Znf-uB0184U@~ygGqBy$z9c)Hdh*GloF$;itE;qO-R!i zW{g$OZ!adgk|KXCciu_-vh)#d<7NFhUDw~!>5}Q(^s6nzSd@?F{3|AlcrEGbnsvtF z{twH{MJqwh+`Ja*LE?Ee*IlLKsgfd08|Jl5m3scM<1_-Z`EBde57^o$r&~1v80X>O z;C+jPVKGweCL@{YFd&oarLQZlpee{Xlm+%m-WqM3G`VxJle2Gz9Ka*9#h?I^M(i{6l$Rd`{i zp{&Bq#88gs6ypqfKQAJWQ~=Z(RCgWa!uFN0(_1RqtD#Gi(~uiNMcxyMj0xqW_JPW6 zROagQk`5Z3o z$J+b^xW4-X=yb%cDmC3-U09x8KOPy$5$8H9C51H=W zMy*ViQ%-Qbgb(-fLlrWPs+t6}l{0*AvBAZZ4G{(V)$l%XO}OA`ng^S_40Z(Bq@}W= zY}0~^1&Ps{DpRE3<>$ zlIU0O>5q22mo}jnq$!pHUlBh2s#KwcdnSaJbaQRK*67NWjHj5jC;Yp~^!YXNMqrY< zl4VNhjYM@d0nl8gje~Yz&66te7*b37K6bGKw7pMsB@xoP`y&V|511kybI6VYTD=7} zNO6PQW@@?+YQG3I_F&~EKlMr`yH6w+DMAHbK>(mf>;3!dHXVNZMS4G&i+I}7a`V~(gE7y4f7$o*EY8HB%n3&)k%9VRFdN?{Dez34rxM{S1@L&GcSBb>7^7@H8niz;;`C(={z@B19V> zkkZvsUpEJwfeF~JF3CSbc>$-(|MUlc9+mF=hg zTSm^6VCC9xF}Ki*FMvH~1c^5S^;CWzX%p^au+hEw_mu|2qxH;d8&-;FrGHQ|WRKNT zPqN#u!8kPw>GzP`0(+p|zyEYz#=P=Su74|=U-17)BmDbs#Q)G2(UX^f9u!2Ks?N9i z4H={_#4!uhMH>|=EaZMQdMZ*D!%|wizoF~#1A(j!jl&E{SW;O3dJH9?e|T(sY+XW7 zlP-DNVv5+Q%_;1emW6(ZG@XO0u9cm5K@D%Tn#2gdQ^u%cb@r-c1~&biT^thrN}GDe zYA7V}Zy7e%xq1tVV^=2){U*85W~caOv!nu7bymZR@5|QrblJ^(&|gr34|SvUhVX%Z z_U&v^Vy4T#HP6C-R7Gh2Rz?5E4D+x5F8?Rz*7x5zw`~6<=k^1TR5ab~VyzIN_>aKa zUIjD9$ks9n5|DgnKRY3LD=EK3e95%L?!s&a3NohHH zBNM>fZ6JtZ5f9ktrlH~q>XuHz?cqjk$ z+1WYU+n6eRYfmGW@6O}@w#?eDKHKA%VnWMeT7+Q*5qr|K za&wKOAlYw>@ZcrRi-4Ac9TU}#Aq^qHhPtg7r!M+w=pc)f4e_vXH|tcTT{ly5VOFf1 zy}{gJo=n`;Ca*!$D(#$2%)QcL$#o($V}gn#SFJruUy-F6Qba{u7h5$NL2Fq>r;(Z- zR~3;_CbL3)ss)mc%x^Q3wAc^F%3?5I8ccJ}DuZM+)A*2tje1%zA4H6T%1jv+Y)ne& zBvc3ic7=Ki?OZx0Jm11sACaKGyGL!@qh3$9TwTX?%|1B;%_tgO6k*pjPrH`&^YDy# z2Xj`^f8MdrVup1iiHITxd9`2LmRi%v#$N8AER^a|l0=0$T&7rSi>X$s28QirP1uPz z+7-bZPG=F^<@k@urfRJ$6`kueW7gD}m$j?6x>dcq*uzdO%l=A$4;@0K2*IvmVLd%wO~uwW#2|U4@{L@HQK#?&!JYJz$~Rs zeuzfB+ht>|b(@otj9HVLkjCw72^oyC>t1aQ%^@1}DPa{aNivoyjPLSSUPZ-^w z{CM=t`H=BU`JnME`LGI%`M?Uy`Ox`uKG;43S#NsdKIA?~_gXh2(>i-H8FOJW(_4n= zuecrEqwx_u8a4ee(-i%3N2oq{_lVu&KD@f-{P1lV@uBr5{NTNLo`}7Po&-~mU0{{42CVYe`lK-|IVE;$oA=ZCcV*j}? zDVrMoU!LcTkEf5S7_z_Xv$wrIIm4P@01;HVVFoE-gb;|R2qfY-7}PLOifj^)xDyl0 z-i^4DYPXpC08BjFc@)MtDJ}iZzAs?FtM<9E`n>U~ZtUzxW9540k16wsAAjF2L7%{j zo4c#K>-VPin0>G07BZIUJ%CP4P?fBv8=`815_e6K0v&vcS7(VySWvAkvJhy)N zl{&Y6)l=}e{M09QR>x(h>YhCp!WDnfQ}Z~N+o1gZk$bn`K9*ZRQR<<%|Mp#C1m#jkkQe(`mNd$;ml$HlLD)_&>rlKU%n_Ilnkeb&GJ zp2zhQB6qj+UXQy^=lJ>Toj-TC{{A<&zsRxwf>4AtDl;}EeFo@t9{GN~YkFB^2_0zqni7@k~+;vSinOjEoesUWl1DcQYp8Ac=+(mg8y>Uja`FROy&SOOX1~)(_pE1_P zr8NEu%E42ZL`Uo~2Bl`hG=LmL1NB{YaMgqrE`!2PSt#g@uAufjg=usy!g$qW9Vvi0 zc}?L~vKov)5Flq^J{`e^GyrkXTdo81uN(k4g~!IDyym37`yg{$Fu;z2)_e*I-(`@B zbRIg%1?+7W3U1lPkOIZ$5|rSaup`0{LkMA@9rTO*WdJIcwLNhK=>-xSDho0T>Kzmh zG#hf3!Y0w5D)5A`V(fXv#617+tV%wkQ}kpVysss;UW7?ge@8Su|(h(bp|Y{_sS z8PE^%mA5cmL%eVihIn*q@S>t{|@<(iCgRA{2DM7D^zLotJ9J1KB>0J4A_FYBW}yg%iT}}fq)_)@e1FN4lIGrA$AMi5D&S{DdzV0DDja#P{Ekt6cd2*`R7DHzx!<5%pvjPN zM4V}c)B}#7p`c?ZIf9C!jOhhYhDZYw;P%HDIPbK6odfPSbl|&FW0<^y6xiL_`6f5M ztUm<|Y;GVt*$0$BDUegJ-W8U4`xQ7|M0|1v2#$uJ5(Y>}cZ*psnR>?n@%JjQpP+q< z0EIglSigjL_7@dhj+dqGWx(7`H0&oWlSdpTuY$8d!6u9k&;W99-*`Rai;Pe8fI*oh z!#!&-L6NHg(X0L;2Ie~({{WyqSuou4Dj#Zxw9Fo>@KvhK5cC6h^d7E|P@wRx8n*EE9&(l= zBzTUQad1C@^vv}RDNM2~OQ;pKsua1zF`ek>Mkp*}ihyyLwx3=;ib6}X0usNy&qik@|N!l#$0TU2L~1Irj| zC9N3l8zkdKofobtB#ZMR8FUzCLHZ-G>$wX9i+zU&c?tb8jw|qV)`w7rx!vIkzgGW{kjs?5n_&IOe5t$)whh8u5{_7|1*N-MOZ{P z@`<*NxC6YNV30GY6Lbpe2gs30LS_=qL;XF3$Y+qC(D*eBdu<+LRp9q{lFcMAII%U> z-hx;$qBBK`G_5%!r@QcoWv^?mRPW=SlsQTaFWN95> zyx7ZLLzXyfY&u=%Z-wZ3+m({VmEMASstW}9b~;M$DjbD-j!=k% zM1=xdw4`tf!nrfuPUR+S^vera2yODzw)PgndO>d-H@MR^wK$o#!An+(VgrfV{MA`v zZJC0ZcwTKD?f2oIdG)oOVs*?V_3bQ}RtTeMa}lo7Fd+$V;w7ENE!_SMZ1YFfne=RW zG>(zOr5==>C~2vSgQa~9d|=+`8DR2C&N5S#Taj3i>VL@)3;FEPjg!ce2dq@kS>j58bS z8B6;*;Sjm=qj8*#vsEwHI`2zwvR;Y1YqVlmnc13$rmE?GXUAh_vQP-gWh8%nSTuhW zmKhcVPbYYVACh--m+X_c?$vA$>oFPoG9tFg8XkSIKSao=eHu*1_kj#E)W;jRnseqE z^@&{7=V6nE+2Y`&;P?wWs&1{HeA{UhYyEA@`)Vh63!!d%%95omc81~@(Km4OdFD6c z{_wwLVh7(!Kc~))!a2nLSRumEnx^tGM%1*!zesfeFy!ip1X(<fBVjXzvmMJ-q^y|mg;si4~HXXSuay32s@gBA>$-}#p2q6J!~sg z|0sf%aN%*pZFU#Lm}@e`=r0Fjf=;^So&ENliLWZNDSzBVk=;>~cbCIaG8i+PTDIuY z1!3_`(^8lHCefy_Qql~xk`qlThIqGCY5c3mB;TI}wOFR`au<+o?k~Z60Rka06bkvg z@j&^^1jn-$gN}g?OtZO0YMM}W>de(%N_gayWXRE)vBv)Ot{iG^oaSEWQm!|ASa$xV-$8&nyNKxsLp9I)pxhmc_m_)5(aIQr{wC! zy%{^-5>PdESGC2kPby-K1~J>4O3g%7B~3ERE+nC+XsBvtOUJGp3d1f^EB2`6*2aDs z;(6M)oEprDFxuf#K?zE@g#77@rx)2xJWGo?AJ(wL^swze*eWpq$Y zUCE}%GhyKoS^asGLZ02qMpkN?rj{98{5WDkol4gFGN^1VZ8dS1+mw=qYn|biV!~o7 zvig>mIO8|99>Z&?v~h8_nH7bMo7r1YQ&v+_w=35u$^Jwits1@7?2<<&i@Sh2DQFBv zXFY*ttFAs*_G>p@$f+;$wAGa?`s}Y>Z_DOLv(@?JIr^H|>jqU>Q*P@b>4`j{#64q; zFZOy90T{cjw=%lMOx@^KbX*ccv3Y7e5}$6CLRc2Izk{iF!^*DggPAA@)6wIMncgo` znhIzuaFA+7x9|H;$rFmQC)+{0Q+C-~FnZzUez%=z39CuqsJ0w?3+F1YXW8Bna&NSA zohQ)l>FNO+ql)b=iQR^Rik{uV*w&;W zxb6rZolWiVDQ0!8Vcp}SA5crKXc}Hylp17PAmfh;9IKIB5~xD8m;0fYlwWroFcmUs zZ#BRstzL{v8Emu64))=<-Ltv8h%Y?qmCOZKjU^Y!DViEJ>CWm39_*(au}W%B(Pa6= z;Z`Ti-lR)*X#bS`4%@73WosU>!sQ9RP#Sb&BW z>BI`*dv30Uy;x~P&HkML(Di#%W~gO%EvTuiMl?D*f+4^SI@yDH-A>3;qs=8u@i>gO zRx*@l=%DJZ)mX0PuDHfBf-H7@Ez;?AkkbfK30X^s+HJZi-wqjxR%(X(6d4a^6(vVW znm~)1ORJ4(!3HK=R_G!RQGrUl(h>noR>h6Tw#~n^zL|`7Y7*vV@CnH}s6%i)$2ze_ zL9oNMB()HE2FmnfUX~r$q9?gmA@m%u~MolUcuex~1BY<0R>l1Fp!`GD+=De;9 zOqfiq^{_P;50Ck%5D++T<1bUdX1HlVXwoHutGS0jQp?TB(+~we1Q}AbmN>{=qVA9v z8;#$UEKjUJpiTQ6XSg`4a#zu%^)s-CR35k~GGUfG{x@SYUf zPLs8sp?+o0E(eBYtWafh@b_2~Hf)o}MhpG#mFAV2EsBL=Pz-ExrYHgZwC)u9NLtF? zn^ zQpS6*&>}!`>qqAMKA-JlsO@>lm_O}Fd-Udt$HND>*!5GwFqYX@4(gAzBxsiFUxW63_PINQ9~zNydn6Bt^<(Z<7V<0ZW5t8 zp%5!(H5)Mtiz^^-r8*qM2_Ws`BAa=avPGZl!7nB!Ga)a{A?CY7*ht|abtx%fM9oJH z`;JLQ7Ctd2`lF?-x^3BO5@h)zgq;jVh3}(n}!6+K-D#BHDj2O9`%JHH*gIJVme&br?U(RIu_%sS=I0tKG?aemLQ!1Y~+G3xwtvCL5)8Y+L^n7)^IH*{;r zbI;}sn*&NraF;Jm!yx!3^*csMo(9>ga?g#tjjG4gvUavXej}Z1z#Id>*Svo1-S;@~ ztFL+f(t6Tg4_SHC03G$dckmi)8i0278|X5 zI1$}ItMIAc(E=b;?KHa%r>?h#ubN6-hU1Vhq^PQuB&*W749g~Vm~z9EwTesZ+FPxn z$5bu33WLcDG2uu>iXi-D!aGwDxg3JbXI?&Sw$tfZn3z67_sW||(5YG|vg5o&5 zHYYrL-cTyzhi3z*I&kvD>llb0wLD<+##Ui&n7e`N2dZ!2al_*G;9ih-g8lp7Zb*Cr z*x!j%06*Dr`Hc?t>TSMbInV|$Z18<1sP|~vaC^ln`;6dj5LQ723LgaS3)5Z?=M8iW zE0Y$lqp|YHBYXOXGkvnyTv!*!aaX$e9E5UWnc+~6`uC3Ss8(;7B+K`*DC`+@Q3d$H zRMVnT{GCVwNijyG2Ej?r(eupU@B(QwOlF8}9NyPyx!k(g+(>sMkcak+ESfWKgU9v( zkJ~(gN3`8ix6H@I(_oUXAqWbD_np7?HH#Gr`ed5I;+n$F@*smDX<`WYP;>bGIS{+ zE}Mvih3&T|3umLGP-vG5u|o&U;x<)grG^9m>p{C-ZDWnE48UKis06JR-%iOj80G;3 z*NcK%0~7WudEwL)dkn3t0)e}Sc&XxvI+JQ;hxoF%g z6!DD@Uz!Ghc*r7-W##nLcDE(K4APM^cbhs9G##xkyd>M;_NuGoz{ zECd7DZ7xB08#cL?U)MYd7sKe_S7tWtvXvOc;7{*8?rro_VEL#^S~QUg8BtAf4B003 zqX$s{fga=JQahY=Wbq*<#LkCNW>k}k<>d00AT^qOBLA%c!^+p!>sbE4oyEJk;Rnw9 zGw)8^`R(qIy1)+B=LW+KrF{RrUbXJmm*$RF^z~(1tGLWW@#2g*VPuPmt?K zkoxQdxGFI==IJZ2$Ykn{aEi{m^7zWz$NqIAe}g_KgRa8hZiWdo=Z>3!hwm zyU1Uyq1=o8-#2nS!oS(_?2anNv`!lxJ5G%36N!8$Jh~=z5C$2k>o?#}RqY&ckl5>B zlC~#O(500_4!dekiGk}Uxy#(wI%A7U{Puq(Ux_UAj<#-8uZpiXlY*+@Ki)v9v;}{6 zn#2O>w`-6kIwy)OFcn7QW7Dw6u_&KTqIGvw@sq%laWK+>=PE;+K>sLVS3%hdZ z8udEq+2?FLOx7O!^uY3ZhalZqVpVl_N5V9pT~tD1w~Zz+jEa8&%)7i4_77X?2+2DC zIMC7mzF{TrvnoV(rgjGBhB{~GIbq+byF*(O3A+?`546(zik@NRK7waA<8B+;Jb@u! za%Af@dTQKANmuuXPEzD{N8TDMb3O@@Rwyf;XH*ba>fjz6)eMhYAX+*xqtZ(tqeALX z&=zb7)VngVSIRmGBVZbxtmqj^;sO?M{kYLeZ2sEdAsD1ghsLn7_K=5KW=(g zdY3IVrbL*$YPX__O7{=mE2NN0@o1sl20$r$^D!ni}& z(|*ExPy2wVGw-# z4_diNoSu_tW_i(cV<(uS-9H*aWVxp zD2zFPlcu*)Invi*N{KNlO{2!<{cMM%{wqUM7&njKfp<8 ze-S@qc$YmS@k_g-63}?%*e!YQ=cwlH6O=o`sHFF6%$i_OYkoC7lk`jF4zpKrer>hF z|ElHAwAU!V!&xTwE9OqVR+V|BJHz$+r9I+bzIw-_O3)_*ccW*8)2G9ie68Mo$9InZyX0S;8GPNn}C#f@Qu;O;N*1Gj#bP4~N=7!;~^DDit`f~5u8i`j1MBBaTmYaWSuc$pIHbI1oBpOx%y)L6LY><*MVL0D9&Gt(h3HbfxDuBb~KBvM`Mj0q5YUsLYrsxNv}`4^^SNn#~XFL5TUn1T<0ph zSELU7-2!RfKwOm@NZK8E{d7v@Cy~;2fRW}l9LIb%Gcd=#_q`OQ6V2jES8Yj5*4!`1 zo7trreri5xIV+88k`w>i!oTIjBeekONqQds_t1=ygtOQ+Zj6aP5Qy{H6D6IryYfNN= z?&?si3CzRfu|t$LsAr6|O(!49R|NC~O&}^t!b_P2NYIQdXJL5a?E+N@e^N9xy<_>? zeLr1uq4VcR27q&ehbMddBRtO>4k>fP$Uow!lu%C4hc)|5KPz zHpHky6G0*xE`dQl{9qW15n>A3k$u*Ko_x*c{DX5)wyiuf>_qy?ENtwMG^U{tq~;cQ z`83opLS}xRVW_n(z{U}+<3ymj@x|=`<%fhY~g{U29+PpjE;QJ zULgAU*+vol=@}hh2=DBwlh-#1*8yPr&kd^EP;&eFH_^X2?_u&X0TazsR};aj6Cv>$ zI_=l|yaqnc4K_ZPn3Nl`NwS>;tz##F9q}{$dW8!*zp&Fp)V9%gz`7;)ZpQZOcH)Z= zaFvJKE-|A?=BrtM!vBEBU{Ix__cd^Wzj+ge-`%)*LSCNm93|iHPLw3diNW%Uyxu1>$vEi+4g~z`u=?p8 zmv#~fYX=%W+RADULQ}!=1kHe6jxtXkf>E4TNc`yoi$(>(GR}zhYc5I@hQicd*vEY| zO>+Hkbw^D0D2|(>9w-gttWLRZIG$l1hw2xYZIYeP-}`(g=+{Fi`yd|bJ`o((Bpxb0 z!2W3YgKPUNu=56kbNkk1QTCc)FA!6xyG^vW3@w1(^RgHVTzaw9!M8s`kP0?i@&7V^`zHkz7NY zBiP4{mA5o93Ag6<7jU1yQ#Smg=XoD)H>891r4Ea~BQ%W0DwZO}%rwXOGz_?SuHOpTBu%$@!s#yT&QY_aM^fzRE zi&v3!3O+EFk(2ZY$oF!4IgE`cC+xfrY)K_;hfDSGf-m~{{e?hn68z>Fd)x`;nJ*^&PKAZ&pdktk2&xJs73DV3 zxT=02(WG@kc{jXD$`GT9CyE)BeW6?r989XaVdNeCODck4>>UzlYP=)K+$j5^)$i5` zrA|4f9Q-v%*s^EuMF~^a=H&`4Q!-?cFsDNu#T5U2v=4ZMH*eHa2-HdpCsnXGWjsb9 zxuBn_6!ag8HmAWJv?)L8uKOOKv4U?rUk&UX)kWN73X_5hj!X{JD3_Vlx*AD%)lVi= znbl+&^IWma?!=%3TVzPQ+z9YQAg(0R-wM5KKt>Y6m{{;s*+3RmF<{jp0(E9Wl+F+_ z%i%+PR^Y~}h6<%;!cwi^bIRfZ*v=>+tBQ?sxxud&NtmqZ+iE-^>2I62HM)wg^`C0F zLYipUFmTU6=U03r)hL9^xS@JBqB4Tei5Qrhn6yr%c#I+rgjRGi`J*ntJB`475U4H} zYalPq4tRo8tI&hW9Ju@8hQ$7l`Lp}3+!WL7cJTt3ulC=WD3pueR2irMlI0ewC zmVaS33%u&RkR`PTfW}YgfCVe=FayqTQ{3r0mOUx2eZ~i$ELySS)U6f2qUSxVZeh<1 z%f8uc6*7j2+6WZY`WA}m!YGR%XV99}?BgG?F(1<7rYX32wScsF53R`$%HDSBXcx`- z>mnKZQuboF7a}daP`3u`OTqvzo#5j)zYCxECb6bZ3TnO3s84yg;!J_*F-?{$Q7p>j zC-0)&a7rLi90qcnIia^FlTfDKQ1ho}?1h9S#K82knZnG;8Rm9&2I($OiDGva9AAAm zUHZzrXzAaeVO?0v%P@{CI*OGTL$Hs?8P#^7p=YLw_4@GTRT_oJ^#if|xH1uAT46c+ zN{lUqz$l^ykKoJCMe8Y4>bO|=1XvF49Ar6tC~U5r>|d(Z6T;Si-CoZf1fDmaFCt!7 z)6b;_32eFgJlp-^*wBR_#-EY$jh=icB*7rx&6HwSLhW+A3DJh_Rj00?*Hr#uIDKOO~6aQuT%AI&jKp7tl` zV*ctpQEtL{0lfzIlhvy`}_kuOus2vIIHY$i8jy!R$*o;sybcr3h>G%wm#VA;U z?4!V6TSy`nIi!Xbtt+GORTN{7T=>;^PLeR+cHR}8R)iXV+=w$=tU1m1pi!N6p^=_w zInRAGili!~IU&-34cucNpZe!FW&OlBccsDl?yh7XT$l51CeyL?E5AK>LB$Nf$WN4|R72}C3wW4UC zt&K3XX4&AZ#k}ld`wK#e6Q*8U?6W&HSB%gvhU7K~8{e*sf6}BzTX#jL?0o#+YXP*bcWsm8|`cts@fiW#mCF=C5s`pc5PZ&wzPvTm3YvnSuxCiWNvsqcf8JPXftyHJx;jD-FOp=H{ukK;-2F#p$f2HEh72>1L zz|h-bIcZMa)50HX`*X3pL6(J7PSQM@B~(90)cAN+PHsO*Agn}ggL#44Vf>o^$uu5D zoB;(>n>sd+Wrdee3uaV<9yFni8Px*EsB#{X-12ClhQs90gMObhLJ4Qua=^C(t&+#9 z0$Z(UA~=)GRIDEvExGynGS3&jM)7yXQ%1LDtu3HRFC@kOH35{sux*&nskL_8i=_9^qul#y?^x|ez=7vr-(3f}zVct3%$Z0n`{uUOKqe64>yz z`)aH6W~@@uqUL`NK<*p&EkGR>!jE|W!pDw&`S7dZn$Z$ImYlnAIb}icYLw*R1`9gF zhVhOeW*lKcFX25n@?`*;17ukGEaC79?6KhSw8klLa^i5u9b$e^d@w}XoZ_bQdKu37 zqP^{*GC==o$9WPU^;FS5Z@tDA51!2~W~Q_lx*PhLUR?w6*$pwf4R?$e!m1&K;B)>P zF0_ zhqgF}oKl`s;#5f~9JVQ$Nq1qf%=*foHpQ^1DMReiRVCea#j+_WYc?scQl2Z~6DcVg zHYs~aPA3?xStL{p8d>$F?abq=#HMreP(#*2SnYlPv6$3rTk^ zvCN{%c}+@-WQPT@%$mx1ElN$Khfp{pWK{9%lvGJ~4l%;tNyrc0zi-<49MC`P=(_fH z_#E8we|q4>f&4YAJ_yA}Kw)H=3q~O#AaMd>HL6A-A3)~>ZyPe{fcZU~576{Lx{r8v z+-X-FSIrm=8@#mv&gm}`L8=ZpwE>$pv^FB=A&A;&bs#ZZg7raDzuGE}JPWf4K(f8VX^snoi@lpypjj$X zhz}V%($XTCDl(kynIQeB=LAWMnlkL{z}I$kfBy0D`)>2=pYS0*kYjljsvke5RsNIR z|KCJNk#Cpyw`Kfa&P*vg2Uiyr7bjCAThjl1Ld@P(xk>g4ONywY`M+vu za9nX1L6vR@CRfGCsfmz^$WdY@LPD@meAWhH-A=U^TTjMrK`y|^2_wlzH8NfbW9BNO zfrMjjXPmv}PCs%5p5I?yUugokt;p-LJB;?rBDGN3oyH3&nTeURf&&|QQR)A9(jI!! zYGw>~jE977WwbGdH-$#|s5y=V#a2TfO1Ew}KIL>Bzu2GQT*mFhrJ*gRo>hw)(8#K1 z;mtj|_8`6V^e<&^ZY=CIy(j27`f_-cBv^v-XdC@qW|;f<&1of8=+D|rbN3Uaa%KHTcxF96#N3q+PZV%$zUbU_Rpq(@Hx_U7LDb{;+Rew~JD{^l! zolELE{_%QL6Sz8?kKFFp$awX}f`<80Kc+g__kSFvdi%>~jfc*15;pr=@%ol(5VY8^ z*V*g5@pzA(BM?Op+A?M`**Q_U2`aLdlzoO1Bg8AR9PO`4i|HGrrw$%5jq&&RI;?Mr z!5;G?aXL(gNLMVb6{R8mPu&G1tp=bes$93LGzL7x$VK_*MSg z&K~%_`@wa)dj)*Rw38@J^QEAP3+@3*(%n-2Zr36aY{HM_*=E;LmABk!RY0SgZ*1}DO3y})5v5LX2JP`m_- zC1E%s($b_fGlksT80emSm5~T8^gP4-ga(xXlN@@COzn()Q-FD4_E_W=msRL(!^TIs)a2iPSwi{H51@1k0?eiU*XV z6nO~1)XE!amA);|&Y));^N1TiKUt(8+6xi6rQE zr~_KXYdq*(3pkTDu@}kff{V0Zx&NH9Sy+);DDWxxek%901}tyo9amEr@YsYo8r2#p zn#BsHXLF!vsj9>m&7y!7R|%6?jk1oFlH-p{EH4D%JU7=s<|W%qV!CG0zcO&7G}f-L zt48l7bVI|?$|Wm{nQyC7J%V54q-Y65*%Smr7ym>dLPZF!4}PymM*p!U{@+;yS35h? ze@77N;HshfG3cO)nOXyvNU)lQNs`Q8az;aG$qG-0AW5<^H4dDhK7e6YS9?`eOO4me zc1(A~&F=RyC>3FtpF|O?l68J@bdpeLl~6 z|1f@$4lN}v9aP_|kgV>|8R4V8Qgh6YdQ;=A+Gj#@rruGy?TunU>#hQH1g)WV*8$+b z?5Q~m0VP3LVCAUYr2tznE-+89X6Pfd?rK0w&@*a|l0zH5K))p4W<&lSp@{GhHLMk+ zk*WxZoe8EfrrAqr*h+-8#b`hnx^6WQZa;sR4%?kO$`BD8#vVFcZTD6bA2$6c9QJC{ zacf8Bz_OuXeVA)a6Z{%WjP(oAC?+}UzLVgP*8>LtS$B*OD+xB48(%PO4|3=l^Rxd( zB23`a19yo#um6%eZ`j_IH`M>c1Ka)71H5nbme!wttok@{;TG6#>6W-&f6yM&ul~SP z_S6GR!qw-8##o=nv&g&Nm;~-63UAGU3hwHyec3J6)$%RDcHu$aOKRNQtvQ|nA<2)x zjIzu81o7utDH_YE6H(>^uB(>4lAe;EIN`}@rWq>o-(H^!NSk@qCA>LMZ<0mH?93Qh z%*YZN2|nx18{8q1czi0UIm}jzt%jMD7-*XX6Z_Cn%cVyL{I2=?t+#V{dWt$dxju zTmHGy4pKWKrhHy?0O3rC3&-mY{0k@PJnCuniHjO(0(tIwT?rXSqnL_GCHdRBrVzib zP^)szV`Awx2F;TbyX1A#^#Tq4uk#>OcayQvcU@LBa?~mm_y)LdL>*$p&WR>?H#C4IX z7wBz}Xl-r~7$X>U@i~3N$H35{)Msi5P^{?{^pnXaMFw4@a0?5-=3V<%Egp`k6$%YH z1$NGh@?JV%R$jRwq*r7ZjFVSjX^rzNw$}lcrc#z-0$2!UDkO{FbUsZ|OKP_-ed9eO z49l_?ZB>gI#Zqz$?z!JfPh48=s&~1_CmlpTS5I-gXrMVVPC2)tdFa$Fc$ zy>yn2rfwb?(NI!$4L>z$*K(FT(kJU#_iz~851!`Sb2L;l)6A6FU)>K*N;9)iz`%ZH z$r70&Kj$S)Q+3rB^_Pi+((T;GnyD&EDoMkj0YRjVe)Q{;M3dJTfjYTw?3^pX9 zw8gzf7+W6(2D4yIBey-{@^zh#xk8&(%~>9lWjU!LWh_yx3SS$*u@&%Jpbvh&Cx=Cc zTj+%zl9nfitzs*1_m%;GB5&#+T$w&fwLn|bj~x%CZdC6$2DuH^BfM~f+Y2_&rp5Z78qsU=Zt^6^{j~sSD)QC4~@X?JTWtX7NTG{11a}Z1a~`ZIKWi-i+|vh4Ho+4 zj>^Ue#dMj6Eia3Y<85`^$c|qJwQ|_lSx&E;K9nZ0fL(XEva02Bh{zKyb9{48jm!IV zxwfY*5UEhOP;N0i)at35JM-|R}3`;#3vYzI0nBFK3fb{XIO2W`pGyPQ@B23m_D%#1$V5#1H_whknoXg<~IzT z8(CGQc_1R2+*;(PC+k?HxW=E`Viv#bm4dX-G0klc94LZYo{xd+e+~sV_J#ITg8%qI z4fmfEIOe}KkgJokz0<$KbpEA#QdO<(Q6#<@gpVXUuzwa3LO6ktwY#)Hjxtli%E-b3 zh{#G?G#%9{{b&ag8;fVKhwV=oEK2?MU<_V#;AD~~wK=`W&2*mQ zb2c~M_4jxd2w?S#J21m--ya`!y-Z))^wY<0tWMP?)7&h;5|!RHZLS^c$L)oT+gS?6O6(wC(aW zXF2t-J*2ft-kO%;*~K{5ZO_-@xpP|4`D1~n~9i0ZlZ6CJ1x^< zUy*)Wy@Z4_tfHj9%y~v$R>=)wN+l@b(|nYik&ILIUDl>W@&#V z#WQZ8?TsA%xXaruGdHXJ3AmWXDEGc>HeWZLyYJG55LKFuDZx9iu#Iv|q;np;I0Vzu zle2YfNHqJZEwbq&F7xLF(MxKJW3HU05lj>382%2RI`Q~fY2vu?*~ss<>YVW!NyBZ{ zXtSZztNU%+pR8`LsBqS)ujwEod;5Pe_Ras9cS)4pLC3Z`9ox2T+qRu_Y}*~U)t*36?Tc=K)bL+gkea8NpbvWjfae|L??tIO0Euu*~Cq%%N zRg;D;hH*F}&eFfw{Hh39rqo9}Vi`1;Xw$1f|B3KdL$bBTvLW*I@ErR(kpI0M{`ZEy zu$iUN|3iQ&Qc22oRS?}PtQ?PGL?*LE$d_Nh=Sl|qktATJj!bGrHl7heDt!yt)KJ42 z*>F^pehPSu_O35hBjxL<63!L?OuL)lOxjDvkio{#>kDjh(bZUUwh)@C$9Sv!`fxtm zXj3px0kxuiHBS?5@bW$tZpOF za962rL}O&lK@`x@6h&MAZmE2fGF|m>FyhuhPXIwmBuKUX9F$_xbHeW$LiI^s~)GO!zv&CI;Ywq zg<|uXy^`ZtQKu-Jt3plL%ke5Kc{xYiNhe4XSSY=jJMbCRknF(%mLGn~3npdu^K)9G zxCXVX&|_FAUOCDW#}6=6CUJ~wRLpM6rOR%a%~Q_ifk0gR0OfngbbkiP6dk$@REup;%Iok{#=v%YZn=3L_ank%JF5IDM?c(hR2X zpea?H@6)teR4&!MpzjxW>R#X89!)Y6OXs!=f0XvT<#;-lBV{`7;$T`5(=11YajSO= zeo>OcTfcb4yt+&A!O_D~P(S|xcQ-f=B9K=zWTqyc`SG8hp8oD4OOKLNc$<8C8dZ}X zS+Z9{NC7tb4P^TVN5v;Z50!n?`Lrq`w6RiPwPqg(`Dd_MFko1&7aVs;%(nU8ouCHQ z1$@C=vs@EN#%&d14o;g&ZY)vBg7JJO-D_~58R4yUV;}~UHXIZ?1;-w50y2s)EqP@3 zH3}_N_sv+To*dw;_>=?dR83&iiiDKb08%E0HaCO^3>6q{WJw za!`4=l)fPeT~T-sW}p8qq~%2~;N3YAL4=6VgDpDp=MxI zn@k~2rr2{xboPZ+@qrUAKG*Yzas7>!b>@2(yC~Yi3ym}0@x}sj#PiKFb`5m17AIv3 z7>Qp{l)EE(+fa~IpgF=l71G7+qM~d5^Er>IX#+pyhQKNgjI;+FuPr#o*vAwKAF>LT zdu_LfLSB3a&b1m0kaf)hL9^Y9%U#>CUcn`m^R2Pu!O1=O=5CZ+KO~8hzcYT<5lS`l4SFw>JOF z_A*dK!xDQQ`0}%l2 z0>N3h0u_M}gGxh5LU2}WlkG8aM*0OYpYw25$Q%^1Y&RHQVk%RN77wa}gA*6J!#(5pvw0;;3gT>1Ivnbv4!c`ro5WTkf- z60#rpC~Pi}mS0*dn`jlP%?vPSyG!;*6H209=KKp|Z}T&qN{T%A`-bOEG}o2NCX`NA zWxXX&>eaa^e`qLl+cOpG%z6WKm%&scy7ea%|KQ}dsk)z_PAmSIk~s}gqK#H8lu0eH z+f~dGAO4&^>NEemuGTL|U@EAxjE^0q1F>s6>X(E6di&<(^NuTv6V>V}bYsDw*&kpl zFlHB{Khraj+$7$ZfsRi&{Gq4Qo1x}3n<-3WZR(+v4#mqa`1lptTpJwOOm}K_P#;8- zhm!7Pj9K(W3vrmhm@iY3(2SC%MFX5s6B)bp83LR4+cMBQSYI|r?bq%6Q2S_!P%?bD z!Dvh)jF;(}&S@@VGtlVw^MipgRSRF7{zG9Y9%DoyNUdkyAbog2mIz%bKHTD1l`=V- zB9>qQQMvS|?0V#YLX{(t#Np+gSnx*3|6Qf%y+sDiWc+$Z)^{d=gWEK5Z7-l!kur71zhY!h_2KS!tk;`ZUIGP2 z+&4Y-5pI#Sp#ITOHu(w%LV0IFx>i*Rom}}8>Vq-}3FeS8Tn*Q|%55Rk?f2f<|aVw~ao61QMR=zQ*3RQrA-baBrCQ7bixO3cXRKy5kWg$0MteB9-Zi;L2Zi zp!N^S*kDq9)%Co~J#=cdQj8T#JK8?0tt(pNHx)D%F4sAA(8=T?^n(76Xn3 zIg9RSx(Q9f0qT~cG!cqkLHc70RTUw;!Rj$5!F?~UJxuFT{YLHn2KRjGjfpPl0SP+g zv^=f8^^AR^eW4=zR&qy7Zf|3EOM=!q-XyLgmchb|CC! zB_aA+7JK0GRu)0j>s0tC8p?Nr$6wYeRV*#|Eg{~Zw?}hEt^mJBgvy#O0A9M%Ic?RIBi(j2@4=gQDS7HbKr^q}*&ML^A;3 zQw`Mjq0!c;i_{-)|LMEx0dfU5Y#O8Y?_y;qsz51GFA6)K&!)WRove0WUlKv z!58;cMndy``Cg%$Av6j%6v}x6C@$|(?ktbSvY#lz-idK&p4Rv7Yhm<~9x=+5NSn@( zZ8J`3tvp9BzqC4qAUc0yHC?!}Y`q{}F9)d+`l-DIJ)tygDzz_Cj4{jITiiM@)>B-% zQ%bSk89JQuX${c)@_lrbe8bh3JKe{i{LS^G|sCjddu{@*Ki& z=*0q@l^*Ha=jlU&_57Mb_8z;-bG`S@V3HNQ{k(cMPe|P19bDcG>CNA^muwX1jDqFA z;zQE0edyk((ud14$+A8&ej5V;_IzUeLZfvz2ncJS64M#&o>4jLz;aq`FQh@(@gdmV_mqRJ53eOm#@u=<2i5p)K~^Gl`)dy91F> zQj8&xcIQ2vTO)x!{OvaC;?K5pkL@m(Y$g}&mlK|!w{Cd6kVy2o^l|j^^uk7(iLps^ zsJ|nuL2>U$uW}W917i3)Y_U9#5D&^|W0`GD$*FhRXy1FF0U>(4LfB}EntY1=Om`(i zTNz=e==xVD(`^=)bdJZk?EB(L-9?ADK#pSB8Y~SF304UU@)FYh zIT22cNZn4a^S^-_juW6P$G z=g>#(6Qooen$kJ8rwEm0MvBbA3FIWo3m=*OHChCrcgNWM5=5hNNOw!wqp%zbo zh`5*o|g9;SiL%h@(S2`A)zr)q!T5y;Ph>*#>4Tp zHWFK}rYI_>NGP2aJI@;^?2 zR`j^}mu6t#WF*~nWh4&(Qp-_cPU!GErFDQG{MoM=uQ*tGJxN087sYi)rK29OBLNhf zIQf!vhsz8$&*a1yDS|kwa%* zSuX*#gFWThQOFiosdUUwm!?)l?8-B7#Hl*mumgf%e8N`Cw0+FkgGwriCjpwDWGIX? zNv2e?Zs#R=@#Fc#Az^5VmTo!%*$1Sj*!jcGF8D{|Pt~>Yona%@)wh;%&p&Lh0q@;V z*(Ofp4t?&Rk1Xs5X35`9-+#IEt^lUik^YJo2d;$c@mEpc`x-_4_jnQit86Hk{bj-V zuZR&!+sN2{p#%SN-n5F*`O7GzfAW4(x;*LOQV1OJ7i0JPw=fz%nOr!m_RAH*+6WH% z2(JovNTBm}IF$AE0NvaDwdP=kPQ8i6$3a-$-%!3myb4>ATWet5fL(My*$YQJ2$xJV z#CAd7s2L-8cEHU3pV!SD3xPy(mS2hk2V#hr3W+1j#wZf6@cn6-qq5}->QPdK6HDI0 z_)j^@OMw~1l)GFM=lMj8pIIq`p_;5cO1T&q$fN=3{vYRKV~g=zd{hv)2%z9;V-r8b^F=TzI*)@rhLO!2I)DUE(_Sl~YnyT|37^86J;t{-}4 zS6>kd{M#7P{nr@&M@;|saQ<7>(c1Y3tn**mj?}_G{$Ik5Qj++>KVrAW*8DOQ4?W@4 z;&k%Cv@P%?DX6EaScQsYO4x{{Jo72?$UhJOC0?cyP$U}_GCbn^659O~E0RyiHR2Pc*`k1Y+-+xaT ztpE18{}aqrF>}G%NAVVpV|s8!Bw>bC2G{)=UIsdw4?+$GDTgS#L-Yq;1Ojpsq7c~d=u_}rzFsBh}Joj+h)AhD5}Gz0*abvw+y z9!y6~TD_QSMLAN;E7clXf4a0V*##GT^SihsqaMJ|{%Z7o#hCT0ui@K%nT=M-=*sG< zZWVXgs8?$x@E^o4H2x2J9Mf7VM%J zI<*CnIJHIPcFnSHbTvisbS+$?QsUS^wfTcNwuRH-=!Ev`=$QGw2a4JTNX- zR4JF?9H#ui+K+XzQ)962t`Mchh}}9ADUocIf1Yb#JX@F{$P%o@S^9?|F`V=?_)OlpxLHb3v%FJL;b{wdE1q*J;3T5Ff~2h{9QFqENfx6+(!>&K9MW}ThLLZi!xtPD4P%=mQRQqnLiN#!z zXn=$rh|W8@Gh~`B7D3=nN!E5+T6mE`u%7-6G_C(Z!Jc+Ws-5u zO{EUFw0KUT`WrO|d(#N*@|D5h^72VLC+aavr>xZL9yC4E%e`VX_`!Ff{XiPHKO*SM2bGGrra_<(?9+pNU`sg2Gv3^7 z2RE0S;pmRm8}>1oaQA=@2u!AB1|HwR?YqR7Z=Q z>8G$XgY8*>S#E!~ca$J>|K528PCwlC*hGxP6ps!UpXn6K2Es56N&VQs*`=Y$lkJ93 zs@Bn`n%=a5^-Kb!FlLWg?f`Cx{$}*73= zdBxxcKC@pMFQ~}pMen7^XHdWPDG?~i4HW{fBk=Pyke9hew~LRNI#P-?f;OlcM+AL= zYt9t~ZUN{uR*%Vn3+Ekn{CtEFXj<^6J*-wv7p3*h^+mk|UoP1kW4I1^O3(R?V>`cz zIx#{*OhsA`EM~yLG!EoQZy|=GiF0X3^fW5u3&(ty7ef;}LPW6N{nljvdKYj}71c$y z07Y#l@kMq&Ok4o7|M)p~4|K!WjxbLUVC6Dr*k!=S7{3>U0}|bEhP1Mpm;sxBOuA|m zb(Pr{AN7+Oxp-R_+ejRDZoSJlF5!HiuT&QxtAg(N&!sY#Cl{QJ2Ph9vDO38~t!s zW_Su5=Kx zojMgK-~K_`Ng|c6uKAjoJpJtu6Zv0PpQ4eym6^4k0x*nFPYcs<)2V%XLSQ_ebY(B@~wn#bD>!8}>RR`3T` zhCyj3#jk$r9|YtzuG;IX9y^FDtsVo`rj2ze*8Mxi9b3{(4O8~Ohz-dq(BoqR zakVUhBkLeY8Z5QIDD1eLY$h7FD`ohmL?aYI)?rKeRSqpB7{D9b3R&&cdM1xa2)2$^ z>*{R`_lk|s<|p{9`z{TulRiWAIHT~p{1Q+ex(vC+Y@XsQPLB}*Q!$CCh-m|&)N(`PkREXW${eHcfHan5t<)+=XK~FrHyHu9 zob1QAN~0s*sHlA+#)+^9qwsIwMXJr=?td7s_gY8D5l?1YdbgpA4k+0rQhaAKreO%I zrJK}G4hW5e&FMo)>afRi)wZ?aG0JtD4xJdJ3T#bNd38dJRRVTR)pi!*9p)QudYsXS z>f%_av1v;9iX8Weq~Vn$0$G0Don=UDE9%a=uVm)k9vN20jLEWUT52)et^SfVo5}3% z6ZFiDnbc^Ym{Fu&-+l^eryQ*S`HKUdL zrEU=J%aKpLf5&8jj-8L-$@m27`;y#`O#5>G`i7Ym#`S!rOAyyX zscP0!mWAz%q?5^HOi?t)v#79s7$`k~6t#;8^%ed_i?HXW292o*x#}Z?j1gh0pOrCio`JBnS|;j;KlwHJ!+Pom%G~te z%wRK-?aFVAhHeYW1YVDn%z%{OnC`TveYo4hA$bTG8xW;0p_a*RoM%a=RTo?RNWAkx z1I%jl;4BLoWd@D1TSj|-Pm#k?snBtRKKHjAac8TMyhmmF7V}SE>4U#c_u9nCe4^C{ ze0RCfKLj2!OwQX@azwk{Q>_akW>CTXb191e<}JLKqvVKMeafval59&^9Tb$>E5zC1 z*DiF{8Xe@}CD$BMk)|`%6okV-5rRQCS+Ow}cDetIt zcW;aQwqQ(ZHy{1_t9`Z0PW&EZ-}Xz-?l)`(@q^k9+;o%!YY-pwE>N6damZk4>lIuJ?u+h? zQgj^oBEgCTRV`;j&;SPc*aga2=V7%#ao%N9LJ4*jJFF0#`Dca<06f{7$)>wtHd;c+?k{POVxvSqL;I4d~X*AfZ?MYP2HHTxm$ zv6mH?=hFzLVnh#HU$+QycAM7v#< zE&dHBz_0!pNxzhQtr^+ZJcs#6)6BC!nUPZxk9Tu4&7%>~r$D|Nl2a|{AByNXwpD&n z1OVtB{>~~Aj2jTA3>1#DPn;2%36*5`x*LuGdKUDk2Ua|;_z@h#RO(#37_#mm(%kq6 zNiaI^6lOHiMWtz(`vTy}AG3dlieO1umP&dgCTGh#)Ee=g2m~q zuSo*4XQsvxSRPH!m3k)!MKtkO*a~te3-L8$<_LzyfG_eBB8WahGyESo81&9R~EKw0B)3U5O_l1)Ny;YcU-}*`nZ+~}?v^$1_=(HA*WHu3T)2bbf)u_mb zq_T6}35(H+lDeSFm>GYnoRENVqf^Xq&|Icvv@Csp5;A^~eB*8m#Pd^yxQ?Mkn^19- zqb2uQqfuOmYW77|v4PzyLMH($)od@CXLYXHKgFtxvGZL;}{$=3kn6 z8C$h~7ORe6M($8icF#JSjVFNT-VIzYWhtTGOoFS*4@IiH;yw984a~Y^xd)oNL!5pv z7hP^off>Jtkb4bKdUa!a?PDE?uuxV}k0j#cnv(V+!LFW7u?(DQqR>A6ljm&w`Lg(x zH`9OXcKUDL*Z-b7{}U=ug>cnWLiwwjg2H9#5wZp$dT_Cnu$L^365XES0w+P9!D$wG;N*@w4d0ZNMo*1i7e-H&-bY4HonAP`H`(53#y8d8Nyay&-c81~Kiha|_GTk_B}8fJr6u)g zE4d}8QuP+*01>L&Vkj>2W)ZDDIW&(s$Y}LrIVIfsk-So_(t$i>55;nk^6O&u=$5{` z1-Ci(=$5%WYKq^tG@m8>mR$f4Z5rG3E-s);6m+zw;;umuZtV~$*ZM9C5NjmY+AcAg zFGe&F9_mv78ij2h8pXA^?;_;dxe$sran~~WcKLvoc3HOu$OW=1pvy~pZrj**6?EgG zpNMtm3!5Ok`r*q0*(SD?33Q3*+St_&bcy!#rz;pl+O=9*QX^OeeC^j20?op%6OhgA zj(nTuR!WygTa#n}%a*b)2gnBg*ZkTG2=7XtAj;kC7|ZDmSpg^;%~f#=0HlNFD!GNl z{t;Mt{KSHb{FEE@smniz@ur~CHQOQ&3IM9_)bn=o(pPJ)#udqI$EDu z#X?`2#Y8_EhS#z>e}{ArTbf>7*my=r88Y2`P-2-(GH|=tF&e#EDx*J#DczphX)lD zp-iCvWsJ!;3mb`^aC5uD`k`4~OqEbTL-ENz7a=otDuT39K!s!0*0T$lV{PC* zR5#$nvqbFBmeCl4ijQs62%A1-D^Q>ir7_)%MhcVpETIK4RZV$JT^`X(U1~&xE@M*&k1tH6Co&nCWrFmy8LiP6ac%>-$j6NGx`i)s*7Q=@@bzvsWGZ z8V{L&G2hmIy7=hZ3;Nk539-Mf#X3Ht2}cSv7*GK0C{6oeUQ$p@*l==Gf5`P84NHj! zY#6CCQ#iJECoEDOE*)!%#?PAAdw3QSs)poSl>|W#_M;_ClZR$2Il9@cD8u80r!WqT zqV?9ydi9{Vc!Q&|TE#A=?a%kP^4pJxYRBS?8ggG64;J>G?1=6{W=2lqE4;uT$`3LY zIJ&P0`P5?_Cmpk)GoCvSukQFKIj&ya;XuKv7h36J_$=uROy;*>4le@|;(e*?Bp6 zgJ4vAO@2Yk95@0=D=n%rt9n6BBert|Qi&-;ya_|L7zAO?K1=H%@@I-)djE++UF5)$ z8&qpja>K8U_%>%Bt1x8K>h>m#6bjRnA&>Q}RS!Y^OemgSlNhHzB3`<=lbtPC3}`v{ zo_&a$@FHR1dEq3c=>d(q$f~J-C>B--OYZAw!$N1A{>gqXA` zV+HJ;{imt?nz16QXMnHeS{9Lpjm{i^|zaDo`HZCTnF9hQ)bj( zd{&XJqyjLg1G&r1u-yY?IA9wuO%(!duI(b^%U-FW z*#fdw7`+dkAvZ5t;9HC0Y6AASHjdb$S_Z*{5O6 zY$Va(7ZsYVJ8~6mW%Y^<9Ykxa5nC0jj9*)x-8Dp**z7FBx^9UfHZJjF_?}NFMf425 zjvmJg`N1w7xlW44bItO|=tSk);NC~X+hFsJ1G$giNOt~>PVHmjHk8?ZG~IK7ptW9I z8s7I))*iKRI=JLbH#Bj;JCSs@X~jVgh|+@Bnx?R~GZoUrR*-D1>}LB7O!~!sb%x7v-H}{}a6R1T zV@L7x#NOwJ1=#I7+5=3>o5RZwDDxL-0_pUNbR6q~TYR$a#u=QvL@MQ$=W8-e$@WYw zOBY3srScQ4(hlDF*5kO$`wIIFTG}SIOKaqLN-_P{r_{ryDIFbI?HCKa3uPjgW5d2vcvB~lfpCf@m6_401^derne z_qt+86dJG8Ojr2JTmI0FekCCy!b~{YcQe}wci}hsoCKu%?)X?a`aCH{t11FMM{fs ziWblCk^03R?c+1W`{=WRVN$UCx+&8&%|vBZ=jT6qos?v8o0gYP5 zk1_Lkhx>FLPJE;1egb}j`9Qt5)+l0Ugfn@}YCqhFZ+;xCwAuXrt6qL?fk@H7y;A?5pYNwD#fkQOA=*E_E1A#r=#p4GAQ+IQgVntlT>#$Pi5r+HdV_;o z*QAa^#j+?!8h1En%w=vD6v|H4IP-w35~(YL7=H;Rw8xStS`06+ZB9kcT0`OAj#Xfu zZ3C`ldEm7qw#zCHVZ+p99!?uG+Jf$H_j7gvK|nfSPN;ycfL8L|N_*AYtz~joP!PZO z=`#r(H7Lkj;F%}vjW)+Fe<+>WYo4Z8GaF~SwvL}5bS6NyKyl_sbi#n%f!_wKB*8{$ zF6wxBApmoZf!i#c(I`IYyF1fe#<26$r;~KPRu{gJ5ZaYCXcd<; zX;$pct?D-doIbcmRfsoSlA$g}JM`G-7Tmlybi>saxqyYM?YS(Md>)39gb)j*fN4pI zzAYf66=^C)^h?$ z0ubb`iy%c#x~;{=C#n1`PHAiEif+n_#q2eUFgaK~FI`in>ZN?GcB<|r>~}L7=DCLhQ$KaSt6A3^ z=fDe_GV^*1=TX>4@CJAV-&(|SQF`TB@`rDpBwpsUFkIstsypNgz2Sme@&$f)HS#{j6)DZ;0Rq|t#WOOvaX;9hteVKPB zxMdHmisrG~hq=Vs*nadpKqO=uGy#=;d!&8$4f4s^q3L2M5tQCqf>;Xj8=o;x-XczRocT$*IALNc}sMMJ*l*bbB!g}ouYVm29CV`Ulv^t{2 zhonlrshfqh*qm(;7)AJ?Y(W_EG=?9+M6Wb?`q%ny}LrhF~B$Rsy%3PISC#G@blFG}7 z`}X~V)F~+?1-)%-qQmXh_~Cr;aFyd5oD~!pY8onR_)rgXj-h{*0fB#R7j*zHu#vC9 z9Ef@`VvY2p@09&ZEpPCM$J{onyZ8_CtW((^Xj}?ZnkTYFDgkRwjLIM<2;^Rcn^Xq^ zC~$kUx^-WI1>GIdsMn(FrUHP^lI-z4d3up3VRA9{j!UG{AGbgl(rw+0%J!0`mMDiM z8rKtS_}TgN(r&MgrE1Aa$(kYhTyzl{-VmP7AxkaL<^wPj<^yRkKn;Mgc}Noy&&eA# zI~`eDnZ=>zbEca~XoLq=RAzde3_QL!Mz%FW+3kvyPW}&+1)Jb!MA;ui^b}_{!V8`M zh`gCM7BtOQxf=QU{rAYT|Epa6XXO8Xuwg5?(SKvZlZ>3|xL)xs;I+CceYyX{hKoaR z;$73E_amqlbIvQPc7SX6%)|Tq;B-MfA;EoBe_pVX?i-m+c#U&8l-*torZ;{ATC=a( z`@)CQ{P*c8BQVh!Xsva7y7I$;f?yOisYN(72ZZ2ii``Rt>w?{Yp_(rA5L7syjH~=C z0&-SvgrhhD%*#7yvci8Nw%}Nrl+2tf%{E?=mDjw2s1vB3+L5yDmru9M+{iV~CYT8@ z5?m<1f{?>9JI;)!2!u_L>+R_1`?cb@%qDpjj~(nG74$w`{lTA&n)%gYg$iyYjB-D` zeqcv@D&JVk#XpR!As>(-z8)rUuy@)@&NZ~)Dp(AiZ^_MSK1`@k{n7409piK<@%SCmgho;NTOcH%2 zgnM}FKBn4l2i_;Ft)HTNsVbg8I2)vB$|f z&g1_G)XzWZ;|{dy?hLugGAxn6n&b|>&!Bk(qgtfv@B#trWxPOBJf$Kz%Q80Q;;rX= z)q{PStGNsj7DUf^ST&xnOSiFA`a3oIufsc@IU%t6l|Czfix2;sP4*ut^v~Na^9v%z z>PPv|p>*57Ux`gz;;V*$wd;yCIQ&IS3UY-{Lflc6iO$zOG_#@|Lqt$621JxUB_B1z z?<;RQs`Rt2iON@1nhyTYS{-(~sYLW$J{6UeuYJSRqjC+^=!it-nq;=gT^F2Xy+lCSs7>-)w?LbUb|4KUq=T?1;dZ`VMnEc1Yq|Om9mTmo zhb5|Ew@TQ7t4i3B=Rl1m#eF8A7jkleG~k*vS4JqKc9~(AsntM>rK+J?KrE%Ifik74 zk&_UbM7FhEndA$)f?EneXi%O4MztM;w*Fn^T5YQLfHsS!dt|?mEfLQU6}0} z{nbsfM!=JcSHB)0Jymr1xA=U6L9*ao)wk4q4;kOEJ7=$0e0{btU~CLDx9ngpw@_d% zA=`on8QY>aEuMIlKN`Dy3N$oj1k3nEkQ>L<7eT+62d*8j2KQR1ctwlr}sskrqC^@`^ku zSo{sfc!QrKhvW^I7Jie5;QBIvF zw;Rj}a!n$6ow6JkT;ifyh-7n(vr)x0u63#OUAlqgnD#7M6F8^_4xzS`66dcyd?g=yQ zM3s74tTi30DvL&9huzB2Sar&zO>AX;k#tdbqEufpkv5M_a9n1vPG44n(95Qs@i;eX z0p`cYgOpP(XgT$i60K2 zAq1KJ5h)DDb|*xKv0G9FeqW2_WtFxi<`@T#HI=jhA98;1HWpoYti3cZJlwm-s%}!7 zRJM(J6fyYHB)J&JnHq25UNG#N;{eh(#dnY&saLOz4#$Xe_t7P^T&Rs~OR=OJOAR6& z6E`LO(ghA*Pm@JQt_sSwI?0W^e?IuV&9F2*bX z&4f#UFuf*=3IIO%##01~2eMs^tV@Jh3DM{G=Dnh?EaWU2PiY1EwPuq+yi>-X(}+nI zf4XO!NfCw^FXbnnQoa#2d?yhb?}0MFaPo(N8?eVVc4Sy*X)O7nLAjR1SVE&jW)T!$ zFgC>o2G!8NX9*!?t+$EFAenjTTkm-abkH5yxr5Td@s?7;HkI|qW=gfX}8SrL`1)f*tLuX%qg|O0l(6+ z2E}tCsUAL_^^5J&4$5At4-rsR^a2o%+7UprUaP#JNzMj17vBgajFOVoI4@08RkAZy z*@9ZQ%8^v}b-1G#9&bBHcA+yu%`e{5T(rmOzTujT;(gb_Vw0D-YJVf^Fyn!t$>Iaf z+Q*o@%}91#fI8|AXB4^kWuEByvT370lKdZLS^L&`JiR(75-r}8m2XNGy}vPk#+sDo(r^JX{)~3 zY68>52FOc$6$JHF3u@^UNp%-bq>BCfI#YDFRlGL70MPp`>+)qi$&`BfDP^USrV(d1 z6N_(GhQ=ROnNlTU+W?-^1+!693Wxw=70InRJIfpPp>!5s`%e(^H|tl{mBHz=af;GD zf7r@2g>;>aNsgvtGG3t%Q}gb-ebI35h=*EDH~31LT$J86HUX>nGkW z8}Y34Om#8A;%#YFb;zR&y9*Dk%L(H^RIj>Ki@{Nk3G6&eY*DHuw_6i1Ss8R0M`Q-} zcoQjI)!s74tUC~%`ZANa-l^Jy>8@B3v0EFjs^V^nqC#S6VQi|G@lxvKdHssryG2uK z$Xzvxyj4U62kba>Y9ny7NGUC)wiUOdt5-{SDJcy{la@N(~5 zBgJ33zGYBl-#jf7>)bvy%W@xGGweA=t%zhc=S}p;9SY?&$PE@V_{^0=vP+0Kh&Tw3 zhE<)yauQw|CN3V>7~nB0Cu=bT6`D6%L1O_hFiRhM4(*9-7b1@;4b8fADnWQrII>l@3Xe_{H1y`27H-toHR}YpW>Pexy-N=HQ5d#jdiF(IbM22Q`BY7!YhS7UC;yfs>M?s3t zi%zSK>?o~o1W&dHnQWB`xf$a1e&5b)!1knBf0$BKwIq$AG0lL4d-?fiyd0AKLXdSv z z>wlC_CPrFgYwK$*SbODVlpZ^`B<2H z8Ijp(&+ANu+WQtOO|KvaGd=1JB&2~ET7o|jk~6aN7*hWf(Qv2X4pcLCOoXb*2~)Lk z#>_7W7DtjZxthhvIiTR4Q37We6g{r6rlq3wK6%Jxpk)FzGK>@aQ`AZ}vdcP3(3lRs z1p{OMAh-gt_qkw%I1Enz!Nk!q-r!traJ_81iqxT+KiIb|lTg#5oJ8h;%bbAP@Q)|x+{EunY8-pkjn?XVV5Bx| zUAUcuCOFbkc*bPpYy=}@VZ|ck8VV^I+JW((@`=tdo8qQpYyPb1GVI9r5m?)15Wf`! z_fI36?=Rma5N!dydi%Q~>Whlz+3Bs&RY~t~=D8H1c}P)>>sRzDj^@yq8?Pi&&Tt=p zn=(El`@ihYo}mGbg^`I(Z$g_Kl%a(GkxJo{3-k=&G<-L8qtm?I_Kc5lxp|M<$h7*) zVpLj-Ybce#g5cq#9n^|P&+1!wD1UWBdK;kP2)_Jifqocr{0x3Jt^CiKcW=Jwb z_zO1906?Gsh91zcPD(fS=D0sQBEV1!g9F~?OheI|j{V=(N`vg>!xb?EUtY<3KKh9L zGb(?tiQ>a>C_eWDED|hujBzR$7d)bD%rk7zctiGh9qg^o{)#+!1}~dqomU0ZXHQ73 z!j>i(EtHHiIdYp&;r8pWk>oh;(Z!K zHd6D#-cSGrV0k2X>B!B}=#MRDwz1!+`Qwdg+Ky$~hI9hcN&M)RuFqJ}wqK6>jm}-4Pq_%F(?_b4 z(FKLYbkuZxfiL!1>&u?2o?hbpI%wWEyJy8J4)(73GxVT4GTJcClYs|VM6t6eT1R?$ zxYEr&g8!7M2+8GI%!%d!KzkKFkuyG}8|%^rKL%6K@h}j_ZDE8Z0IB&ARZ9Q=wP`6BWLL67TQ3fA7?djvlh$Lc}tNPeKFv=Nk z(=b1$+?M;ow;%~)$DG8O98Vj99 zljpT0zVFI~4mDan_QYrJ*^Sz&4bNA#RPGwqO;pnnG>V(Z|0u1|c5E zWe;e}Uvr9zAlb9F69jb_j+eCY9?XY|E= zv(w2zQg=y?pW_FT^c26cd2ElY?e(PbeqT#kgt++d@j={*%7w%x0bp;$75>FKoCRy= zTV+jB=lhrUx#$SUci;hLFL7EQ3j_esnvPU2(?Ct z4%G;J(Y_E2MK!2{?3NV0qZTk>CdE%KY~x4{LH@Q;Z!;(elbb?96FjgYnKhUM^f6Lk zhQfRxcOasI*kFrLi9qJnq6`o|r8rQoMpGpqx@t&6U|b$#-qx#GW0Rov-X+# z-h10Ascg}yt!(!ZY4$Rgjc1e_W44zkdv(8i>Qm(w>=tJS&lB&S_ty)Yfmiprd!CThFYbxA-UOS?V%vm2Fk( z*@SBu086K#@Ax3V*cy(ONeByQiLE-WYnK4U;<+hk9vwUj*_0Xj`<1g$kw141sg>-eX3C2#g)b*oxs zYtX&}q+|WOJKYkqo9iY@4h8&3CW#dvQ%+9h42QdM4q64LQDxZvlTglB5fd!~^=P3x zZd#7K2r#H!i8$DVUw)4P!|ZCX9Q*p_6bV8 z&brC08P_AGM2k;~rI((zG)SS;m~Q=vKXUj@JxVU!L{I~26)(f(6@SR-WC+x(8s0)( z-6(8!B=2A)FUQf8c-|Hs;Z@40W`g1KEeGg!h7u#~t%Gxr-sFgClq{YgMVoonXR>38 z$mda@Vr|gkHWX^XTfhP3!YK$K9L}(3E)9B>a6vupC-f5?^{;&GDa5yZjowy8#qW#75|iHT6creaYM#%`H#iK zDAA`GbDSS9@Fouy)aLQGhCyxzu;iD$lR9pb&M8U;lC<%S=*id>y^>{F0!N|i48%8i zP*0oyJj0pnOmvTz=-DXx6LZkcXujTTewkUYZ@PeAtN=XySqaR?Q1o0J(-}CrbA-Sv z{n@B#uy3#cp2?gS#*^esHvE0AOM!Q}NL?bl3nHzS!v)aEAe1sY9&ZHH>EPzW{n!u1 z{=XHpd8e6`L1z~o+DXM^wh%8jOVL=f)>MKAb_S7CXNF>DkOtZqFOu_6{=c2^^#5({ zJQ_VGh4BOv^n(!eLmYIIjo1+*z&DZ)&vdql?z|d3SC$^_oG#FtCSYeghxPAJnF4wG zbDtPbU|`)AqGyXRABbSzzF z?Y5a?DoSQxXJ$)k`13`aJgL*>lWRctvJ`IWTLc;BrSOti6y0$Vf3?T<{wYkx?{~Z) z2UflAB}slz63r{C(R?XNMnli#qfd6Le#;~D^r;4L?$cjILEpfCr*fpX{Q28&nj5ug zZQ4nkXlk*{s4(fg|!{lYY{`MK6LD<{ac)MxfcFCw!aQqCk;P z?gpCNST%OlV1_Ya2dmzW4K{%ols!&(J(I1-OIui$$#x=e3Ya3c4t&FQ=IzqJp~^G7 z!2_v*mf+8=m&rTb=d|U&qgBnVl;l?bV0s@6NTpD%H`xw@oTRZ>a7)hWb%-|nBz?FJ zp6o-(DaTfGY(@oonPT{Cr@(l88+`=tpUA47KvVjSxpGY6oGjY^yOmBK{b32Y{XG9~ zLR}#XV}d`z^;Lvo%})kwLOg=QbJV|3TY&s{`cJnGS^jVxM92z6e|X*B7Lh-OiiPlg zsRM4{Mh+6|aR)^9A@W%Dlttd&O6Ev-(R(lN-Ui8W8Q<)rcx^X}*+7`t$BX%oSS7xH zlbryx?NRvtbT=3U3X;3Gp~rtOBSpFf?g9M$!VuSCtdN6QSz3K)5Eh0C70I;*gqRC^ z@&b|)>~9aAzp2)MLiz*y=+M3c-e#DPj(m$^XqYb-c@Nm2B_2%73+S{;<|?6{)R!am zZhlH?e!mYgW!fTDD$+4z$u^KQ;pTRqHQZU5wt#wGp->#v&FDYFkIJEkH^m@zRUph# z94#6 zmBp$^dhJh+K*Skk$U#Ryfj-$cc`zD3jdvW4Q6*jjvpe!187h5X^=YxrhUD65Z#8CY zI7Nez*mQI>BQ4GF=HHby@rUf6bPYjG;w29 z@i&SIj3QVFULktJJj@fyFsJfvolNn+HCfO_@+J$*aaH1yi8By-frS&5mrGkQF0N<(}vW9q4un*Mhvf-r9LpJ1R3>Bs_q#W&hkhp2i%s=|LZdF z%&cYmg$Do_=KXJ4_y6l2urmLrdvMhasjn>3O0V5bk`r?S0Bc}JZ;lN@Bmf(1&UQX# zbH=|`6i7@A3mHu8dSjw67c86nvVn<$W0sA9;eu@{YUUnGh;J$A?@tZ36<7|Ec0VE- zqXiMZb(>vzK|_7`c|x=M+q;^RwNW+0;c~H5QK{7SXlsGbDHx86t3qN-Wq^nEewyd% z?g_iO6V-+8X8)!dbGdi7UShJmLMeCrM{eHH%0WRyq{Lk3=pQ}}%1R?-nZYO3#ls}R zOIlIUo3hU6c;4-Tv9;$x_s+v~`sQ^caPzg6=QgJ3hU-=b()MO-E3H zjm#y>{m1lThMJ0Y*fWs+@Nz$mNwK0c=B6?2GJ zD-4}GqdtIdbHN1aP!uT6)3A)T$GG4l{`nd#NQp~74LG<&v%5{aah+X_=-lB#WMMX!9`B?&*n_{{U%TEQx8uz=I+Whj zWmF7nHIfaGF@bJsnoWCoIhIIA-Id2W;)yyHthZQ}K1JjApY^J?;+ zvkHEOq?Tvz(7MVtLEp>IcgX5d^U^bEmu|gakHhsRzD4h9B&M~y+6@@E1GAl{cL_Rt z^Ndw;qrm|}1!+y-V4`d&KT^*TV~d5yjuKyuu@z{lsObHO{7H0JcJhNj8qoMFw{peB zeTtsE@df^3chL|dPQpWZfMV>&Yk^>7n(5>l3k5FZAWS9BqtCM-zPZnuXFFz9_`>Dq zYc2rzM-#o&V40W9mF3Z$BD6%P@e4!uBQGy>X9F>-w>Z;?ona}gu(H0-2&dH%tJseXpKQ!7cQ!QkimBX_2Vdl#(c_7;PaE(X&u7MnpBR;X zhBxR{HR5SA;MqKYxIZ;l7ujLbtWf?7chJ@ra8~TBAo~pY5?3(;?J_BmO4e|xsWLg_ z7A7|)-U8G- zPXFiVHTwb3q+bqzFot=VNAe880J2lmKiG1RxUMpxHe{wf6b?^%lQ}Ss1O_G$Q~M#_ zEY*l=bSPLqP{*fA7nEE%hnZLTQq4>S!99`T5Ix@nR=uE4Ux1a`;EOIZ2xD1wrF9E? z4yb`&SDqIs{_r6{{QTuTTe6iyS$9kiqER#+ep2*!%3gF7bb9_N3k@@T`{6V23)X&U zbr3NDHMu&hT;6ahTCf86lY`m!xJXhK2En-3QB`a>;&`#6z8=VEQAw!*I9#}+PGVRhm$%O z^c`kQLSV(fMzclbjuk7U`Y@LcgEM@5=Ip)SixDa0*xclgIl^CzPe&$9rdZ=EBu)P< z_J9_I^36Um#x-Vruf_(c+Cn=m{W*fqr9F;&Og3SyJa~6P&s{5be^) zM<4*OV~*_Nsc_-C04X*YASS=pj+!fj6RJhPB%j*h={r{@5RvRK6SM=ar}X^N;n%#K ze0o)yrPf6;VP`vAMqix%1xhFDW}68{i%!R`?>=E$0-tIgFrvyqY+irn1dS>`VH2n6 zPvn%@NMoOg_|`#caFQwm?&195%9@7=@mvJ&NMLMB`dI#6?)AS;Ek3v z!!5^ZewCxm{Q2O0U<540qYSo5!RoeeC0Qu3@cU`;lw}9P`TT{)M;lc5M{j^WzvhbZ zFnf1eLWtEbxvk70fCK(L!dSF*dv5=(<2$(pXKox0-jr=H&PMtBH;Svao8{#j2%advL0z) zh;v+%v3E^JS49r8=-49+J~ z52m+Xsf>ev#4A;11v4vdFkk~!;F_P}dSFobX?qRmDVZMnuYFw9@o1=DV$ehTJXDX< z#N~QP;3C)B4lK!y@@&3aPJhB5D%NGNK(t}B(v+aQVPNfk5lQ)upXbqP?HI4~d?@QX z`95#szc+iK%Jk!Na@!9)`)@lQgv_tf`g-7c^MgE3jk=5AsNU6d+B!3k!s#YmxdG&?=_yd%Vb$+=nM;KaQq{ddVI&zjhBtARe z&Nn{}8&H=QHM1_W?SJL?>|tDeeV|o*aN&{i{>G!>3jAVb7iTgpZAcqKHl($M+-Gj= zIx*r0D(Q@0e157_fBaCt!IX3}FJ}30snT$8G&V(f5mF)dX3U3=ZICyTD;=$W(nVkC z3=6P)DxsgUbCnU@2)c*f1bFgPVG(u9@GQj*8+0(fC`tCKwiq%7wZMCPztXg$( zC4`FJX3Xcx_I^v*xjjsz*W`~UoNSPSBSg$-Z7MK&$h|UdH#cdJ?^vYa#C({p^f*e& zAtwF-?cMK$?J~GZeI!8emf%IFFA(q9>F40Cfclf{A6fzC1G)s;mxNG&k=xtDe?wm+ z4FN?`UL_=($ckTIxu(`)j@AHm2=dnyNuJprJo=tMDnZwe>L6#Ac869!PN{pURC^xw zgX!!gw9~HdM>B3cG9)6hOLI0X*nYt?q}U4U0ew>QB^@1_2!RwU2mjWzOAwqeUj!U_R4lk6ky25@&m-7iL9fO50rL+B zDUPoosH!gp;iH3rx`Y{I?UM5mp)sh*3q_=2$No)4_<~eO^7Mc>;AcpD2czT9lb%;I zGQ%0%0UO-f@VSljq-Dt)OWMM@P|KF$7#G}yoyZiU!$=c@4nDT24tnMb-^aB zWGhu5OyF08jRW#`qvzePVeu}(-tQ(l_Sj#9-!%O%2;4XyE4vD*ZK-zLmNHru5IHzc z*#hwR_dA7D47Uf)?$98FCF`P3Wlg<16gwNDY+S~%g0%Sw|=pt+T#8E%N*PL)X?=)gmT z%nLMTXO0@~I{0g6ZE&Aod_57j>9Qb*Gzko`a~ZIa#!O;V#} zM<+R*PBLx_pSZjjl@<_n|5dA{Z9u6x*- zU^Ec8{iL+CA8p8ruX5v+Z z4m2=CUDSpINx{JzaQW?B==(`~HDEF}6ZR()VB>Yk6d1H^Bcw2APnOChyhwNX-FODf ziD7aK%_B@rdn>*2z{Wu_RLxUYJ;C58MD{xem|Z!r!kN7{&y8w$_FNiq(~hTMH-wod@t$e z#bG2{7}f0oeTsZ7?$!G%QR$DXRA`$S`}fl7FSr$$sI#6lW(nDg9zNm)+-^{j4YF78 zlI;ll#4u8vJqbxC`+o$_h@_nY>b3{erG4eReT^{^k_id90O)Q9;yLu4oWd$DlUCB9 zF{I;SlvgLwpX+I5nw~@(xo%%DtD0_(#Ds*im3mJSG8u@Vx1DK-gE54sw2&m-Vu|j+ zIV{hnn-eC{57Ikv%1$(fsNga0M4dqFw@YGkz#}2{x=cW8N z&kk-yi823yU3d6OYrePm#ung8BO;$A_rc@2qsU47^wQmEsYEgq^tRY)f^Od^swF&` zB)^Fdi482w2Kp=!nXUQjdkM0&i#B?V&2I3e_<_0jC@p~7141KHPw|~-T)Q=hs1Fn( z#7I6gVZnwUHe{$yiw6y_6^)8}AbWph)U~e-Z`_}RJLmc7eB3*T2QP@JA_M)A1AfqO zPlj-PFv$Xh?^vOwXAr?B9R8;$_OH({7E_7ldSxV?^xx$p_|2P5@k^+2l5Gcc7A zhn5s8Vc7AHsAC;GU(X)D)P~peMw3{)RHFuuE__2~Mm6j*zu=Xk8|#8P(76eccRnb> z6WfLF&f_LS*~gA-nm=GbxMh2%^f@$hVD&L2L+2m{m8|R@e6&7}lQym|oad?kyq5K3x0_ga2fUAG>Dj^a!hg6oLUaU!2S^Xa5{4W@MPl06CdpojDl-{?Q~-!w`osRbt(NM zQ?fYq#zc|W#!j9D#7VY)_eo!Sx>r(|6A+5=0hV!Px&;4faO7{3j zYNT8gYeD;9UY&rz>4D6f;lDXV$yzsmrR3TG9=EXae8n5d0l=0~^0U}oa2fI5FjX?1 zfo6Y+G6VikQB(f8O%nC4G#m{?^&tZ9btgba_Z}ATFlO&sd14Fw0WrDSSjIASeA{V;yz56PM3TwHW&sQ zbKwX51a06~JKBW;5_onSg+epBd>Sk^<9){I1Zwr$jX!8(!5r2uItTh&E8pEUC)p~| z0ly9lMa+k}rMdOLEakhV`X=tIUIL)-r^cIXx}G zkx+Kgp%Y=zJ}erKZA6ZWbQ`@LuP%1{$!j6$B`e(our;@TWLVrRMInB$V~qVNsN?;8 zAOyXHincu3ft#a#c1ThBqqdQ_@b3A$E?y;Y0ATOT9emP^LCS4&(8*@1_y}bgx?9yM zen6Ep#DH-;ot_X4@c~oekb)N={&1R>yStNMGWaJH1WRkzG_FY0KgY`PSJNkZ$PrFt zTTIaP>=k}aI!8I+Vx{BR83RAkcrGOH_zAhgqn*6)gj&Qc$TT)BCh&w}Z-onye&WG8 z3lL)ZfBYd=90lzqQpwNqO#Z#s$8~#2K5ie@i*7Win&2G_uAaG2ELaOr@@~`Y=D#Cz z9r)A7q*ZZ#FxaM*ef4n3HJGAE6Qst0Q0Xks?Xzx#iEwL*-T=274+twQ29pO;?}LbX zGT2+@9a$PC!-{N|pLqzf{7`nTU3QL&qy>}2R=ve!srz2Ng~G(!*Ls@Nhd@4 zErAa6bABtcJi7-8UM%-kpy87dG7m8np%yxK9)uSFB;UcNjRGxTq=vRBQ(&x2&1o)A z_Q$+Vy;WJJ@T1HBd0S#8KE&;~4g|jkwQj>e!NAXlwSfA&B3-n&qL7X(!lTS2(t<1l zO^5i8aJLpWz9F&Z;S*LqI&h+6u$9H6@%6q-5m3z~sQ2NKa z$pB>rq&~+{J=eQ)o4THt5=-t+cyx-WR~nIuT}o{N(%c%r!SE#T zH>ii}1!dpEF0-s&H84bC*`|@27E$ICw_JeRyMUp;VaiQ%T7VW!aB^k%py1n8*O@Q= zuGG6KZw>W3V>%MAs#8BEf#Y}4El+=N9^{PM+lU%C9; z(Kik<4qw0WFS-u7G{edMP7FNPrq>2OY@t}vOB3xbvC-_c?pS`pN51&wU-;+~&cLrU zndF$C*jR3nTOJ$cfqau7= ze@TBz#TKy_=!C~Eo9B&1=O=Vd-T({oqJ|7cwPGGrGbAAAa@+cET+SB%fFHkpNrYn>sc7l^c)ER z0Lp4345dJK>!^ivR!FU;_T<_cqdKHwS(9wW#Ld^-&!}&-&;kzq&%kYD^&MYCcpXTq z2(-WEfSi;d9-qw{ z>F2!K6oSlj!}N9YqAkLbOX(ECdVrWInQBdU_CGO}N{JEO-=KxHsVuV`C}oJoHVMElH+g5 z*C$hr2Z~dTB!+pzPr-0cjrjbK+jn7OURb@>Q@p9&erceX$)~WKKd+y!uf=WZKQD}F z&R_v^O{E>yWmoLU@AvfkTwF=nV?SQ&KYoto&595f0WUzfA*Aa8UddXw1tMf87p#KB zvJyo)54!rjTgJ8gU_whd)?VUH&DIRw!j`7Um~gWMtr?;nI{l4d*tUr2 zAY}k-TDKj7u)X2E2Xy(ok(5fgQqvImI=>lDD8C*4%LfTX7AR4F0KO zP8+!yKr7E&ZD|E=dATW1pt2I9FW#n9jDsz$1!0j1CvImVz8NTfO0>ZGR6?|HjeRb? zyl@Rc_ze$ER$%Edi5TNN0AYPYB1XI*$7v7rBSUEj=2VYyvWJ`=R6`H`x-`1(%c(Q} zSyoJr=NsT7du_{2hVX5K&%Veyx7`KU-WT_HT5Nwfii(4Trto=o8q+qXsLosIy*{m6C0nog0r8m7K=su zFV3qv44gkf{wzBxdZ>^cXiPNX*JAX9VyXdx>6|IH5@~bDOi*DAI#JyE3|?GhYfNrH zXdEMY_gz;3sA!ITrH%Yy1#4Yl#0Fv9LY$tS#MtG1GCSl%!l63e+0afMOCVjD*2jUM zm~y`HQtWIe=#b2PbVLi%;eAgSa&NnfZfoW$Oq`s}>pM8@@~IBk;j7{k8ll~C*i1}IYPJd}VjYpM0Y z1LeU#@$_jV(s9!A*`w$DFx2u(NX{^!t@l(FwcGr&*vD-A+q{Q>upg$~Yv3Q7*bB98 z2d;sCP^ww=S#y+lj~>u49*?RNb9VBWm-0RGLvzeVH@`hNkB_|g+dR2bHp`H6!S46I z=gD6lf701*%#;ZtCZxXV?7P=BBc-e4=4w}jA;ct>@K%=htE7mta7jiI_qw(bb3 z7kxgIw_tz(@Pvzy%9$m6VX_imH<#Ujz%KCD?jc$M`%*vJz3V{xqg>v|-SbhS<$b+i zPCtB`dDLq|U+Gc|`E#v?szBa{moQxe&7kQrF)giwvx84T-CqVDsoYEVWAK7T@v2%#_hbqAWEW28X5F3y=AfP;8uQ zatZ{631ZRI6UgoS5EgQ7hB9j8F+NdIV0|q9LJWLmXnY`B7PBJ+SlRP%0fI@p(9U?( z$rgLzAK3TXVZ~AY5>c!5Q5$&7}s7? z;#YX5H@o5Hr8(I=_pO)u5#<1q?{_r(ibv7~(Yya9MR^7r%d@Tg4b4>|$#KHh&Ov$~ zkt?f+g+k~(#sMGw&Y;dTt>h9Mz0p0U6RViNWnvyA;P=WwyHrmaPP{c6pA;IOG=Bif z?aC(Il_Llk`m5W_#xcc7xxkDh=fh%N(i~*EEXg5n;RaWOydu@rY9RJ*+G7x&rlF_h z3i1d-TF=ebIzRL7i<6?PS38*S0D}dq0WRb66g&Mkk5?=7x$dP<*AZ7)A@OMuAbTte zv*K0&yUchLJ$%s#qyq<-6w{1NROsAIHq!iUCDjM`>jiGx=OuMX9(|yGj6A3$t)%r1wkHGyeuCS0F$)(I}~_K5KFs`Fq#fl)_D zY_x2_O*L~GgxK&CFBR!l9d`&xz)jZWMZrDmixyF~=!NNiTCcQ2r!zCdhT+Q<^d_S? z>=4)nd3jCpjT4H>4J)UF+W z%iH!-4*rrl}wb9;}eyeXlJxp=*3siXv%dirXi$3ym}2Mn0BdJMAURyM3{0o*xb5y!!U(4~&*qu##u+V$p!F zSAdTd49bdx1wfBg*IY#C05T_!%|Is;|;i(ryu5J4B5#KJ|bAdrRNXSA)y z;b)SK;b(uvYrv8XV96HnWJ~1a%T%BQtU&ur2YW%%8q7eZU35gW&?Z(OSr*G{;paAu z0;i+bVQt~1lf)zgP*UB0|0Dq_wVNXFB(Wg_SqPYdR(K?T0)MG7#jQYV6xfMQv-Qkp z?;FF)w2pEb@`W2XDJQf zbJ0#*vfZ@`VW^K^N%(xVwk_l+tDCa|9%5S@@ktA2JhteeuTDzxxEc;t6`{5?7`1&| zI~biHdODUUo7Raro?Se-b()NfXFLgtZbff-ox=ACz6H&w(ltY)Fo~ zG6280%zNyFr=%bDrap!0xqsWfF~o4!#rUM%@vw! zY`kOH8Za4{ylJr$cLW*Lve84*R==l51?`8Xb_GD0Qe-`XZgY7hYP=Gjn!tJ>ffSwb z5Bm4#=;ly92MN$AUHaLcSjlTU?_vJb!~BzBpeIB1XpOw0&!wLPO{!yhDG-TX#%kwy z)j)z?>vwgkwB5UO_qzCbZLl^=v|T`0wqpLVQVj2!|2B&Dw4=+Pld;KbsvA=3%Q8_y z`wdN93$C2arV=_BXCy$~t`1%`V_bV$D61u@NmeM?+y8@f{@AIHU$*iTjErX*fB5~=JOzM;rLUyT$t zbdhVJ>OKSyutKAH`S$=Ax6vLN0*nTK_52I&-aNKd`tEkl?Qy!ZB43@$K9ou85*r6k zNWUkcPSkZB+|ww1rD6I=C4uTOF*R7y>M^01;uYfy$msA68KGHN#h>xEt(FmWFx{Ov zTK^d)@WSQoPTZEMV{?`FV|kOI`zI--MjJ5NCWK0`QfV?v?o$ z?@q+4*~r2T24~Sbz@KJ(&}1mFhV-3I50!-)L2aFim5J6Ab*y3 z%k~nPK=i2?EkP?A1`7V*jLFo{Jfbc9^K&p+Z*GH}p5fD!cPHgzT1AVJ=a+!?q?TP+ zapeNu6V*Pro(%)pi6y3=^VA?JX*xzPf!Mdvg}DMf@sBzo3?8;!JkLq(7l#ohdWwRDYpHN4vOEE_8kx%#UK~PZ+{pdQ z^J{=54sJRR-Z{p5KzG01ypPS5YD=pDXRMn_$`UqkaPMg*Xs82^!ywJ5UP(#o2lCj9 zx9^DEX|q~DzowkdS?(wVPDRr6GR<5%dt$}NQHB{TJfJw6->6Gy#};8&{%p~ezW2=` zCOzq&l<^V0doJm5 zzk4p~5xy&t*Q2|G6&BpyiXj{CH)(|itAhnKC55^Ng>)t%?5q4(!M()_=baT#liTbL zQ#Sb#=rE})F3c(X;zBlm$dX*eXwI;lotV%1r+c^s)VOmDd#6KrckndjQqspU0Jg98 zq?opA$=I%N*74RRP}D5&;oHB6{B%IJGh6_Yn+H_4&H~H5 z?hSKpSyH)F7hT{bscI^E;FlIm)|4f03rEZY(ZrJF^eq#}E`37TY{BoyuzW(dmy*ty zmjcQx#Y=xfyFtuyVh>1kXVJTtq+$l+4~<>as^toxBwn)9HQ~K zEqroH?Y-DH9j1ieYhJ{XtE|AGhd&Ky@!}Yvnbf)kPc=SNBjEX+eN_X?5m25qIVx&` z4XouUcZQ}5uTuYnk_(j`g2Wpj_mrPH<`3tDq`y<&v~*Q$0lcRGdIfsb0}72Wgni}Y z_XM@Mob?}m z7UG!tr;fNTmB{%J8MCOBQQd5#X`m_9yhq9@v%G1-nueNvY$~+!Y57Uhb8OVl<87*y z0^IKeN{r>%Igd|Q9*k{ zcL=#USXI^S&DPagUz4^AZK2F#dki+y80en9;+@RxONGAX8^b9-m)-~q!Avi0O0nV; zOg)YPRZiK@oY7}fF)6kqa8iBwOIQi&VUZcoUV++t3tmd5tG?Kf(RvlikjhakG4Hv_ zl5jCTOOJM%Hy6FOmly|C1edk)9h)iQg2f}EBhzr;q9cRJu+Qt$rL^6tOH?ic5NfVT zEflOaPx~^DqNtXnD{ZLYiCHw8^vapV39|rmGG8DN&a0arAzIcUcj4^0Y2*Mtd6t}Y zUW?o;faM=rd>fdsPj<%s50!@wxSUikn8wY8YkEgowb#_v>CJ>J;5y`HXwAk9l#qk4 zPSXt=1AEzy6RKkD11orq!7cjI)r=CBOd*12C+RS>8N5N7jSCQ+BxnPEvSwMdC_-p8 zYhoE2WCj*Ztk#85q=lE-Sn;PetRi+C{fO{4%XzUD{6#GTOp;7JB(5<7%Jb1ENumsTJR$O1PVFEo3fp|A2lWb+0(TeS0yLcD3_|!ItVDkur zB8&5!8;)Y6U6cuur!jXeGIQ6u3$cYr6k`^kK0!I&?6Vbo5cKY~hs5CP!-&_dgIF!_ zZa{x0#Ch^M4FD4Wsc-JT2`0yN42<61gOLLiyWH9bCJUe6>puwd+d#NyG3eKf5KH;B zN->wPG~sE|ig}f?_ZeNQDB9&Z{upP(p77Xh;@WmEN8(Hah8W2CFuXq{AGq=1Z0$jz z>~}5EY~k(n)?{XpDAjh@p%A-ag#X`3V=-FK;RoilJ4 zSi79zfZj$xN0jDXB?j_qLIRr}hqWT8l;m+=L8_9{yvqVD^ErhRW?u zrtsH{Ez!7ALt~PgB65sd)db_Qm?A2jOy_Tg=bvN@>VZ;-DI#(d#P7~@ycnS5 znyENZzzL_RxQl?sgm(oac~*M9nZpUslhGU16?#9ckE--uoNtn;x_`Iyeos>n8TH;d zZ)fDIER^Cf6<0-167znJF`|DmYGXG%_0>*NdzO)xYvZis@;{WlV|1m_wk;als@S&e zRBYQ$D#nT{w(X>1+qR90ZQFX;d!KjCednIL+q-ME_5JzQk3QPyqt7uh=8z~lLZL_# zNXY)p*d6u#5XiQn^h_g|W=q7Ilrii|!olB(ek`!p9|r#Yexjz}4>#7|u%JRRz&mPW zr4fl`gm21(&S$mv_Qo8xn}bHFFa={$PhEjjA37Lm#AJ!2++ze$lj~a~yIuTx34*y4Px$r>Q*~V9byKv1+n9bQL$$dy`3kbO zoQ0zaQ|?E7IZK^N(x1P8oy02nr1IA>SgEs^8wEc-{sLaxyrl)%2ld@eA>&~@>;|#l zm6*njMkK6t{cDwR{0W6~BjNXZOI3~|?%>W}Zq~R$B~K$EGOd+0j&k`ter2M#kqT|$ zYlG%j9%9R@qvu6euBMIEUe44|jP*b);40S4_i2xE2h&u78HyMF z1W9YJ#zk$~gr%Z@Z6I4*%BL?fH&ww%J0b!fiBvYGLmb$bqPK_A^UHU@MoY^S4G<<( z&n=P1bO`=!d2&DiB@-PzSM>~$h>P8d!}RG4q1V&Ti>#~$qpt-yg^g|t71dTUh1f;f zqVE#1Ajk3!fx!B@${*0WQm-_flp9ndtw9|Q?eBWWFvBQbE}ES%HS>#BpnunCN8H7T z_HyeIE*XI`MtaDZYr?i{5U?SGWP!lke>nOiZPNrvx~hiTRD=%?b_vh;Nu#g5s$RIZ zlE~WvM)XrpPpn z!$&8m{B7izFkE}4#2_NkA2z9rOh-soJkGn(vW!d_2W4~Chv``57rGEOIfj2Dp9Qz;SX&uL2699U7`0#1a>0toxMijn~GiGFrsQ7Z^EoC=|8H-q;GvY&m#_yhl6YKr_ezyI?d5uE*jrsl@K4 zJd;5j)eDYLrU|Jik8pY>5B7b`-^x(`k3Yt>5##+xSgyzgy3CsaZ|z?OZUK@R6vC7E zE&DFvR9B6N7KC&qqpNe#hrR)>- zO0XPbl$eyvu zwE{oYzZ|)Ks|Visj~+a%j}_k5dw4#@Lp$T;Faw^rb-MhaN?ehn;vcChm#mA>3UMcX zl3HbNb-I*CrS!n0-Ig1wNU*}M-1XNU5~NNJeB_i>F88*CBbGJ<^u-Qmz*{5N=(aY) z_FNw)1Pj}fqp_Kr#zilW7MQg< z;tPt>_)~7NOFG{0zrnxvX<1p52>A z%@RGDOQDNk4@T?>6mNToYP|(cSqItAgA7y>4kh+0APmQ?wtgdyTC=`7WLa^}9IaSW zZG5E1XMs2Nuo+%TzB*n0BNPk@{0hCP58kA;xU{2fovbhH0}4+FLHym!nlee-xb${A z7TxM1!me*(6_nS!OymQRe1tj`4>5>PIXjf>Jj*=Gh~|+y_dFgazun1DUuU^mH%Pbb zM{vOGea}&#-kLRF>`msWOFjip_7DCD+I#sQNV$MZBr}z(BgTs+%JJ*5o|5T)@0+!e zt`W~yVsGGFEK#cJmayz8Dh8Ll$C{oIqU5^?guoxy`+WC01m{FETLVlL2|!LH&IpeI z?^SpVvAMR_+Fr0s6Ib_zcI3l8FBZ-z&&|7{J0emaYOJ@@+3L!0L+GQfX;n2GZ5Dv!x?R*!gGJ%Ecia)4FgDOzP>v3z! zL8%YBL8_s~AQ}F!ia{g8L^?=nM8Sv3lLiMNGAtvI=)2-Sn!!SqhoFtFB4uic?@pSDHF~Wp2bumnhVMu6Y`8Y!B)v|J7=9I zFH^|S2wMFqqLHzTStz@3DXb;75mUPhtVA@?aN8_=Gt1|Tz_gezq2Te6h1y5f+`5kq zHkf4U=Eck=KWto<+M8n_X`$V$1WeT>otR6 zO}QQ-uB>0t3h_cL1Wq0HrZd)BNqtm6$);M8ZB<5lQ{@^Cjv-|E?^W*V%Ij9aU;p}I zdUS>_B%Ul4P>j$j!`VjKbUYNGK)RWLUzRH3^&^j6&LDrr{^-ir8YI(4FHjf#!0*Ai zVeSE`zY3KR#Efdr1?|A0no)dg0%|%jr5v5@&+HN3+|mc?y!L_Jm@em&On8^fKibKj zM!%=d1>Tzf*@~sl3?4{1@CuV?7b#S@)=tir6Y9ENK1SLAa<1&E9Jz%Bssm@^)S7%2 zPo>YJU>JPL{75Cg3w#Q-Zw%u*eLcZj5e(TGHIor+eW+-0nIX>dE}{C&JvM{H%kyIk z))kC+@4ldR(B1$+TU*!sC~UB6I^{GktUFQ19ywemUVTef@lL;OAuZKQ@6 zJl~iG;xTgTjnn=b&-RLdrt!=iT+=x^R(c;h z^e9x&DaoDP*D)b)7(1lZ1{_k1b4$uCrEqByTrp7L?-dh;zXx*eI-d+0VcIFEPC{}M zVp4Jq6iB>8)TI4gL&o`@u*-uVwc`nOUgx)iM`R)uA6%rP_QuXCvR)Ok0v~*FAh^3J zd7qaXq(|)loNt1waA|btl6DW#pz0ofVlmktL{MPNF$`#3WHWROXRZs z7-2(3%PMwanwND3T)$i%<1=AVz#tGG1A$*yrpmHL1*BSmy9K2bt$aQY4Mp~eDG?N2 zoQCJDJ#jrS=-h@_{DzmG78RpZnu|8YHs8p zjBfRyo#rMz&I!WmRO=6ZQ$g63)kl?cdTlsu;>D#$JVzPG{e32kr{$#p55V8|3>R+x z9O@Lb+q^uo-p*A}zQ&-D4J^wRjkvq~Hg(enINMTie<6wu{UG;`PBnkD%zMZFok zl{_iQ@e0%i^JzwEyLVoz0GG^X0lvfx5u?z4L}l@A`Z? zRyR?<1IEpRzMQza(5j$|Ar8@ql&N3%ImTgE&&4!P*33*4F`rIjEkOkniQKy`O)JvER+a@s*<`|RoBU*j zmOlQ6PLa*=sT8DQof*xd)pqO$-^7zfj!wK2;PMZZe1zMGrB@08E{7heYy zu3Y*#ce3JsT9bx!tgTg{M>V?9<(ioie{xIHbq!r!qN`I&8%p}OE>FVFZu1pDyb1B142ZAiWX)gJL8wOqbH9AKZ`nX8K}m3o zuD|-7!;SBFCg@t7v8Wy&FV$tWvAsoXCkhFduhjwZRr!ANM+HC0{JMT{P$dnP#54T~ zGY{BN^9>g!8k<*IbcBC9C}BZ(9Gk19#mog9I9U1zhyAj~yHCI*7p~5AbzbUPBgi^G zp;};OB4Az9vk;M~M3a3WK*7#}nHBH(Uf>pRf%~MNP2tq+0b;iKTGr4#72LR?F1Y?P z7=44yl(tIAKIqww%BS!04$~N(#XTF@?1ySC*qvc^-jX3k9xk~vlG<}ym326R>An5e zUJxfWxRG?Oi7=0hoDs$vO8giUwASA~kR56G;}v5b!+3*#R>7%FSE@v_5ZG4aF1J*$ zNSEse@w~wOxi|v~P8 z#eGNJwW_L(!qhvMmKsh3R$IWOuec62)SCV*oIs~z*GH1?s`FeO@cof>3E?pG+)ndJ zlSrJ{z8qc58R`o`b}p!JOk(B86khBzU+R~6OX zY~LK-{^aoFaM{(fk+p8KciFri^*gzhq{R~tP;3ui0-UwHaRPGoE)g!99G;XfeFisJ z)@Ap*v6y4qQ=Gox6gxD>w&Pz`?Opb-dndO?t$Sy*6F5EPc&}2_ZAr^AYdvqt&}}`8r8B|B|EL|hY~u-OxN=i2N|2`rDQYFQ*G)Q(v_7_GtkhZ zj8oH86pAkXJrE_cwMHI@$n$hhvOuS)JV+8HQ>$IDnXH9aV_?wUr@F0ZN>&B(!Uxo9 zba}{l`Lwo|tlH33RUk!#i)s<)J<^m_E-I<27D6hmi;h=D0Q(*I;w}Jh9uE+xuauks~BesP8G`TvJ8byUZ!z@>B5dM3Rv`D<&HUZyA!i^7T zRH4_R`Jo*VF*aVyl#ABK+ScOeuBzGEK9KI};?Q~|b!JqWmpYf&r-=qpmZxCvWyoHndj3L<|&S7p$u&ud8^WID+- ziSLzO=dvv#xg0~9Uhz_kR-FtOTrJW4XK4xVy5w>OE6E`aVD|Ngnp%sYvmX2a$S^f= z_@CEmEP^P2X;QoYBY-SWdYxu0d_#!3?mbiPQdO1F$JZ&AJIB^eYpIjj($ zBT{NR)HKeN@oVs;A9hC`JtQMHhja*rFUW7$yM4cENGvYE?6wziB+PdDmU-LKKx`iv z7eXmWCF-ZH<~;!=3{cMf&5yxhU0^;<&DwQcB1bRA--n`iKircgQd_wL`3{TsY?z?9 zkRhE_O*MlsK@iJ@ZiZWMR66R6B-TU^m+I3SYCe)8-v@ zv5bc~b?B*=HfB$zW(>w}d=Z~Lk~8XBI`d4_R5eBA$XMZCl3V%EFC9xlVK|6V+s_nUZizdP^s*}Jw<2rL!VUTS4f!KQsTFe!3}El=qM{^tq)m)=zTG$8pXD$cZ0Gih1S8ZV5IAU_j*M4J zwBP>7ufpa#JiWIbGK*|~Y&&BamLEw^_3n>ie=EQhzm!b?KT0gE>ad)@GoQ$0^PF0D z2j|#wJbMRkrlY0RQqkGf>nLdWby9bGfS#n<>{%viIXI?Te%0bMx!Qh7{xvV~_pZzG z&((|N(Qk00uKngqkga3fMr8fT>36YY{U0}rpA&X96%XAR?_Y>sQ&;nIUseb+f!zX6 zsDr@i*XnO9-G&Iy>fK*(zuzB&>`W+`FU<2NZlV#sbiDfAo+4*|(&<}SExDhaUbO-` zulQelzP>MM=e@rs!NK=FPl@?DU6Ox4Z0`1S+`G#0cuf=FeH)hj{d%XL?fF^#d+pth z;r(MUrKzLCW$jVjwc~a5EJ~Iq18}i9UkBL2c|*kIy*E4CA+hWHm}3(@^zxsX5z3BZq5o7+FyZCb)wQmsob2=UAv;F4l z%JxgZTe!>Z^$Yd>MXX5=~y;ydijKujVB+Q>sBqvczI;rfl4prMI%+k@%9{0hI_DUG<2v&e-_Y zzTSRZOO@muhUyGqLxhdthX%*rum?k31s}G*K@SG*v?6BcVe!QUp?|+Gf{T2EO4Frw zFB2gWK5UhyUPnf*C&L&kugHz7C%wc7(?CofW==)CGvKCM>PoptSU5MP9}r>Q_n%OJ z2seU9BLG4s1os8D1tO>cG6A9xglGKulY~D&62(jC#OG1Vk*zf@R(zBtLGsPyxr($K zO`;u0U%-7`ay9F^GvwWh+A;@b%j)(gzU6d4wS*5(R)c&<_0Q}Hii?+P@6cbFLaXLI z0PvR&nK^#cF2salv$#jiYwtkb4JQn|pWQ#5;IX26RgJ@P9KyJfj9A1g$U7 z#Y4k>5f-hD!o|ykx|5}ezoN#UleZ2CO4uQ$NqV5piIdhFb`g22CuEEQ3DcsaiO>e# z(~!u4xZGEh&i^5fR}#Iwv0C@l9?c3J?g2NU({sQXDCOY{HAk-9d-<5LVve{cGSPr` z*v9H~VQLK~2MLalmSn1<{SrXy(E9;F?UqfL!Zfvbf=d<_QLs|oOqEvTvmG14z0Dw35M z10qeB#Ys?N+JIA-5n?5T4KEfd@^L@f=2#RZ;O%v(^#YWhOnHn1YQkxdy1@0#1pSfy zGiXids#r-V>>aM*di9TlSz7N?PT?{V!KwCWBwWHsM8Q}ckx($(PYvN21p_e%qUSEO z_cWe&)do#tDEIviHIbah*GJDa?+eE_zKgT9E9-6n^HZ^BEcZswtA@~Qfbbsm3WtpO8&YJWC!QN@>cEBPC*rf#<1mhPHCo}XoBLz6it zMcP2LW-_Xyc@dzc34=;E|_aimFtpDRp?Ck_}S1%s&4(EC`BpJ}5}I4r4rDtvJ9mW+&L+i8}) zKWLA`ptqT#WJ{#vnBo)T!Qz?evkNB9*H+tSbjg49n3nnUi!RZxr?!7)WfM1(J4vJG%T8-lh- zMFQ#52FcH_NqBsdfav#CZ6`KrISNbyk#ernTkOJoeBNJ*I2Mwk2Ng zw%lbtg?DZjzp*SD$9nlD-1{{$?+b}$_Epap?6rCn+%8P((1|2PAm1}de)rzPWq;afw`1tx0WL6T9oFD0rZ~R~_bGzjyqH5>k+ZQcA7!@S z9o}qBd`|1S#^o;w5U*2#F;GvgPUh3rZLMUjWT2uh)-D1E1o@&8IO#`XBAiuM-*agt zL9j`I;024^`mXV(iS}vSDi99md?33yehlG8+vO)?0@Ct#SSY`J?zcP*2ENI3Zrp!CypgMr-;Es6> z%vkN5C&>0++=1kU_IZs+Z&)1g{cgbT1-pi~(jR)97O$RD5AQ@wyh>=d%flAqH6~F5 zTN#kykEmoB=uX(!`E^mN2fa=}t0dbVnP=g`W5eM5nU)3en~T*cF8YU3-x>KOVkRS? zS74?gb;$F+z<7BWE0$3BA`ePhe8lFiE`H8=s-rN4cZT5)xtri&pHGv#-UH_+IEI9I zkLJen_l$Q+=49$I3;?j#jI|!3v=BsqL;j#H+z{+gdBq3K=4e5CKbRl|^Upe<1?vF? zKpPyxGTuj*i6rEbL2sSTQ^kN@q4^d$eI@8Lg%*`?ifx&odPykza6UV$IuD`G zs!~bm^v)l*9TWz_(D)=zE78|QRUdC}$BcFh4oG8$b%~~mC}O5iguBQ&lGH^|og&7_ zI8ORc^}2Ai0KLM!a!MBP96~ICV2mzRYs$K6nsM!8a-X@-Im(9g+ZhItodp9TbmUe{ z5I`wo8*pT)oq$-0{BVJ@L-(-puy{yD$R4#Ylk~r;nks#W34S=@Rtz3x zu-wqXsnfy4uNPd50esM$jEJ8l^MEeyb@jrReq?a`Y=$jMHB;?#p8+j;)B$hX= z#>m-+v7Zw_cFMPFceEiU7>E0mLFLhT+1<=U-gU1#fTqo_2R@*!XM4sZA?tb9rUeZ- z3N1uug|T%;gvtQV=Gj_e(E{p1!{B zM+uN@a*7XZ$2>6X_6-m8FrxQ%DtU1me$?L9aHW3XW5^*K9_?)MuJ`nNyS2oID7$e+ zg+PJ<@9p^Uwlrr4L4u#{?QDzvqC!MGF!T!vxv}@egVlC-Lq$Y9+L_eDI9wALnCLI} zzrMD&-|BY4gWW=|$O)Ogwa59MqdqT*&eJ^ia(2MxAH+nrd&uG77C)QGb-;hOWMwGs z0q|qD;tX%NWhW!W{f$WxT^cxJyW>1B-h7{R@&11AacLD3sUqpwtUP-M{9UBHK2Fpzk zIcpDJX9Rbm0=TacKA-bKZwxOw>zE;h`s*6|-9IkBMy9`<@sDp_zBr}94uZwXi>;u( z6Vsix#WR18r~2WvH_xsujW-h>)ju|XMFjxrzMaSGM{(tk(i}aeq4#y~6Z6C7??t%% z-*=_*IoJ~Xo>K0oe@OEb z$9yhQ3M0_h;A_4&m7RL*BaZDMdu^7x2ew+bw1>A71Ik;pr#6%>F&vwH+h5ij9XPVu zOV?$0Z1peGTQf#$PLty`+^y!r64IW~9%oJA7{^%ETfZ#?9e8H$6*VY*-2p!h zxF!`5eO8OJyry0HnbL3Us3V<6h1A7&AnLa1f28ep9n_`Xp8zS}8l1#6yksY&F8%rB zG>SNpw_;{y?C5i#=t^U-Z$D`R(@qc66G`=|T_ z)Nwmy!sVW9AN2YtpnU=4_Q1hR+4<}1`^pXB^^)E1enr?8h<>(f6}rvF7e>PBn3`i| zXY4qCt?D&w%50!*R^$||J*b2b#V{@epO8Fbp^~!*2#(?k)2_8&IT(AOf2K z9De!c2sz-=^tQJzf3X0ewsXn(uMvAq8ZxqnJ87SbaPyBHjrj>gIi?AHA>aJPZ}1yj zXA15J4bUUmHGXFPJ#S5H(P{&^9qkb9a=qKK#c$wPjnu6eS5?wn$6x&IMgD{TPlV8y)A&? z#R2)<#lM>3Sen@Nk51UjGmCI04ZLSksO9lI${kumt%ZdN~IiY=e!M6TrzlBvbHs|0mII#Jq5%YHG?Kg&<2|oDqiJ zdWUlbhgIOL;!{h3j)6l32Pu%|3&pRDVnhfE~tnHcm(XtpEH-#G8UuMuOA97pd= znP|Hi?hDOhpE>%u$ODF6LR(+{Igq#l{#_#6?%szds-%mC)rdD^(wG9Ohwv6y4me(C zl+%s4Et$J5Q2096N1PhB#?WH=hfK!!sJlnTpB)e<7j)&SkHDQ@FiYcLh#g~RGBZ~q ztTD#AWlSe2d9rP0_HWSl>QSA1wWvDL2>2dRZ}i7&M4vuVp~6wE&<7`~nZ>mT--Wwu zkNX46K3oRmd(#e5!$lmq_r71jZiv{WENs6@GT06Q<}H__qU&#h4*MtB!)!R~{MgTQ zi52xR#;+hbG?3YzpbzUy#-%P$E>O3`!luMIr}upM&t5c=p4j`147-f4o4`D5(C(7uA>Ca!1}OE zvU(V!yogw#@YmQc+N7R3UC%u^fNk#*y5`M4ga}^8SF_v0v5RgJZ{4Haz@+L*G(+Lb z9=S5pL$FH`OQJXzKiy4!ika#ZnTVO0>R}X+;jP5+wURF#1jDhBOj@vP%CNCa{$kmZ zS;JbZsdt^4S}W46iD)vF=2?|$tg93rx2$XqM0Melg0YoeTJkohmn+mbd>GnJ3G7#v8=43J$WdNwqeX z?2|>!NzKh1?ASY%b?@lDmFn)R+ni+W=5tZW9gxe2OgX$lA=Ji ztbgj9p)x0F)z1TW8uJppomW`R%F4i6DLcb1Jz9m1DNVetGyC|+-ER1mXEx3JcgVn_ z2VghZP-UxZ%O-bm^ajSF4}0e5_a8kuZj?GPFxUJ#gNSZI?>d6p*#4O-ff) zz$1Z&rIXEe9H%bwYknJBXzh*MhTZ)95?Pmnarq4*Mhf(sT-aa#Y;R3~b7JoJ-IrML z-!>M<{h#>zKXW%MMfS8vsJ-!S9L-;yk}5YgNlrCL#;C{2NW6(^?+Sn?On5Q#D$MkR zpW`{zYTFxog&|xdTt#Ruy2GC$)vrrQ`UQ8B-d@dBPY8FT<@;Nn!3#1>>V?IxSf=u4 zAF~4Z(gX}y4qpY#F6H|^>c}DuWO)zqgnxBlB7#2McmSN;^(xrqFrqVl%#p5#_LFNy zXV77_uniokb&;k~>mxT9IkI;uq_;rXng>|C3>Jcw@wgZf?EAFFq~@hAa&AmOOI1;b zbPeN*MS!vTlS3N2h7Vs|LB1aRb^WN)t-xd84hIOyD<%ilW;WKZa_3WB)a&-W5|ibJ zfF0*u$a|x$Ws2C_?^fh;u^8*Q;s@2Ate9|85mF34Xx*8>r8yZhVPRQ^jek)BXOWl_ z+Ckl0NEmb5N#MP0gfIB|BGZI@uCa-cD$bR_4XU{B#3q5&t7XuN-uY>c-}T)tyvY$_ zZpq`?f6w@W+~gI8F8HzaGkxpr8M%|;p$(>FF47L?)P(v-;UNtvw;RXQd+0to`nqRP(%jUk!Zk3p73i4lH+#{UaW-?ol9mBq_#0fS}Sg`QEA zpfg(A`nLy4q$bIett=sWItBY$C&j4?eMVeXRQ_HYKZ-rFO8TFYG^w$tGE$`_ZfVCy z|I~FtPuvl=EE*7aMsE0bXwkI=f4bOUl_};-jsx3Hd+6q00KcZ4+8~vi4>hY=t#j9}o@ag~_g4s<2Zs?D>S-v+b93R%;-*;UNxf`wg^|x>Ik#8=Q!#8xVu>X*DeRh@w;y6=f# znykai{d2+IuFduNmQkso#Y@8Azw%wgPr#zTjn7ff8L|5dm}=VT4M2~+?(PtLghV_0 zDbwnw!EyvUbL1X`gT&nT17bQ%XTYC=&`D)|fZ5aGSe4Y6P%2%>gB95!IglEv+QLvr~H@LpCiTw+u! zfmEWACL0ZgN`yO&KCL!-hz3z}$j!r?3ma6Umq(qU0bQb8A)g5g5wPYmNAZ4dtjpc5fc$w8MM_yCTTQ};5~r`Gqf3Np8s7}^C%CV4>?;}w zktZsP+pZ76sz#N#yDXt%p0b-0mQY5a00PUmfZoQlRzDneQUvaed5%C}P&@|a+YMSO zY+*C4_+#NBt{o*9K?c*fpC;V)#*CBjVpH-EowNP<;8kX=YsCDF#BDwi1M1Qar|`jV z=1*|sCKgz1{pR)%3&yfVqinSU`J|Rmx$x>{#cZus1*@ENW2>CU&qADS&~E__jJIDZ z@VpQliB72`V(|C#Up-5a6+KijCcD55Ja9VQL+87E^+b|Xi3JEQ-J{uPELv?9o8V@H zUUHue^vy(^CO2+SYH4rK+l#Il2*8$ZQ95B{6D=B*t>FzpB^t^~L$n9QKXFfVLv0poam=wenf4rCSc0bN`JWp+-S{q!~{E+!qmtW-{WRK6W|B&kA#e zyQIH}Mcom`h02=$CMQ-yp|Z)!2R6)inOx(arvY2_p7>@V^VAt|1lQ58vS#tvte$6! z--bl}{CNG6?;<%N1jiZPid?Qky1+~lA@Tb7^j$YlLS|fXU>Qc82CxZ>n5M7Xs9IQ#)K58 zF9UYt6KfwJlsWi-NKDaf??Gdg&sE$PkVk*Gb3Z7DwTD1g*EO7 zM23g1?J+FS{@3Giu>0CVB#I$~5ygDN1n(+}1$x@lf|2NwWrZqS| zYLGS^saPq-jk2R9v95sB_Uy+BPwnE^^j9GiNhRr-_-rEe~4YvZ>^2mLnxs z0_=Hp_rq#>RD>x?a`k~;AOKd8`eCwX<|I{mqPg`btSa?ZF4|;C&Ww3D{;NdTbuUsN zls5fw5n)DshY8|oE0A;Z#j-9Dsx-2AX$u-PxD3T|(oFq%=Pad+3Pt7D)@07!sW?8S zMY&#F4yi}(3z5%0UYa{^K->vJ>n(Tvp`;Dk!s!(RG~TfF9NOyDy3hz+g0}nE(jt^8 z!kb8Sx|mzA=GlVzuIFtb>5fbSX8Mr3!*}J))(YVMwY1~l7^<;wg8hV%zn@R?_De#y zZ<6B$crK!`aVSh_kqBkXXL@7oXQ_*Lh@zQF6 ztGp$5`FM-O7Qamm6c2X9*q`{0 z_0-ASx9M5_cv)m${FaOtjne}}vd-zbB)Usj!w+EyRUn__6C`U|ew%OCwPKH8_oMTgT3Jt&n8~=PIw@JJ;|nSzbYb1JLeZ#=jhP& zI(^#!<*5*Sryi~MSyFcAJIe~^m?$yDNb?SL8xmPH!QQ?+(cxiE>Csnk7cv;L{Rsu;58F#@dXHCKK0c9sq$_YP1~AYvsA05WyZli<+4zgx5%J&u zIkGp3Z96%D0Re?W|2y46`ER=QFTsjZQFKE67A%apO_O?!0xr2g4&7} zVPb0#l3KL<#}|j>^tP8n8}P<>1TUz(Tze7KXyz7;EM3g~U)OdeN4Ck>8IkE}nV#KO z*`Aj#Q=OiApEqZAKxQ{w{ljoMj1`O(%oR*3SeMLRWA-tBh7`;-%$E&Xrmr=>Z7Pf% z;KIYc9g{iZ^oQk+-nGe6+nQtYm&B8qRB{s+$2NC~-05I8!-k**<*-*w8z0kQx#r(g zJ}mOnPGzyjQHP`5Df44CgZByrz0)j6FQle(I$;2N*JW#HhVYL9)aYpDo~;)?_+D@mt8SDbrC7Ct}Xh7?crM5#el z#AwtS3f`C|8qIp+G5PweED>8TB0zkT91lQC{+E{u&8&R!;30{p2%_`Q#alZOuaRML zF_^)vc+b|F(W7L|@;Vkn8M&b|TC47DUxAp&qBInk0fz0=ah z?&DINcEMyE9krzXPDwPM^U@FaSyHdO5bJ{U_9;N;=^4{WBeV=cM`+tL0knWZm#Sq% z0HF=Ok7|Ln3i<*gn?yP~mKq3i-htB+@5uVq*o!NO?jcWnf&U{Re>gG=1AqbnN&nk)1MNSaQM$?lga z>AwQkMM+V9UIFd%5~fp|4h~a&ZNmq=-7c-tVP5Nbkiq%sQrRa*qriPl^r={v2W z<}S)!efO~WUaKkR>}<$Y=9@Rg7FRKsfRFbFgaMvvv`aK3E)Fgt7rq1Ufm?DPr6bHO z%%Rc@Q@Oz=P=BN!SRDlGIwPaMWdImR^cjq8y2enRzcx%V+B&-fVmUc&qU^f(=zwvO z&2Ci~g;b@1pJAIu8D6-tVQb%;!D2?p90CoTR&B#DovZRrtz+SRJUc zDb#f$ALRoeX;2H{WU{Y<}e3MrfTW)(_c=) zX)aKFtgldVh=E}^w%da(c-DAFEM88oI3GAmK@JbxRQVK%(n{iwuV`Bh&;6K!3c;1o zmC>#7d#cLO9D@c?1lXYS3(EwHNtcK=pNJMZGaB^6@(u+$-Iqh-+qqe5MA$8b4$V`~ zlf0ob(*rm;pIlr)NAYXj2R;gEQ5)u!QRK0odv>HR7pY=DoNJA*F72Oa_;5Ju8-JSa zeKBr=!?&nA5K!>yUS3$+XK8p1ur5FVMQs3M_rcvka{y^qWF^H<| z_qn`%mF9^|6)}kD{Q=$@R+0dqHnUv08Vv7HA9gVaatJX9Bm)ye69Z!dh+0eyKT=ds z*l&c8P(ax<&W_*6V!nU>XT(I`Z~v8Miy8jX&-#{|6 zw@7XluSg|qJAKGCA^G{fJj$iihakPg(r z!;Y2Ib6oRT!m=k+>!RjZKS20Uk*XxmLU$veb(5tisQCikFjgVkMA0{KmO@3uz5J8ns4K#|E>Ax|7E_5t+BJEoh{@4(Pzl;vj4l|c{ z=Q{S|C*FBM|3|O=9q{+hloEpgcCHl6GI)5k|M(RNCj%D=N7yscH_|iI182lk10qQd zf$>kTWx>W_JAZ?U`?s+1{x{f6q&CKqlB7=d{~f;VH{V?UK|Fdf2n=|6S^573vBMRJ6+Bj=G;WBao{@?Y^P z?SA$%n8ozEW%h77RDb1_MJe+kEB|p-)2qP2k?XPoxQy_cS9p}AB#tCFF z0+3^_!2%92yKDP@uHer=#(yM#8#VHOY1n^EVE+e6fc+mNK?Ye?hUkBB7|FkN*#A$8 zb&3eQx_!fL{J(_G^53cTpK8YNKR`G7$;uRjiS@i8br&zqe0LZ55T1bQgo1_D;(sR{ z5wE<#?4*Q+rGW|y<4BE#1vb+N;^Mz}*8fn?#LURV$m|P50!spl1IiLQ*aJhFhNl*sNor{T_S3>x0P&32ftRTW!=HG6iuzo0b^sUFsb?%0cO3sj4h6lSC{H~Q< zdbBFPxLKmY%Aq&PIk%=Li;KTEZ4pj1ZKS{yqdSRcmaV4te^Nxcm{bzK0x1Q- zYNFQbPl|>^gv|=VigW|R65jrQn0w2hI`IJ+U+x+zuXaZXj0jl-ylStB<@%5J0TCH<(sTmJRkJpb;orqw3P9r*4U z+EII}sU9ypE|ru~a^O;_p+v?L{ks=$V>bQ>@|7wdUT8jX4%sv0gFmJokWbVv%x^V7 z*grzMnE&a0^2TnaR^l!$jxK-PoM;t!M^t5W{%R!)Y^XI6+2pxE1`TX<9l>vscv$u} z?0IQg1;>4Z%zVxx%%Z^bq2w#i7T5hIO0aOsjOCDU!7;DR+y-$hhbjL14$tZKi-VVk z$7A@9?(505)TZJ?2(TxpRnjet+bO|o%2HCWEyPB`!q|rxawY~s62cJma0G=IBY~r7 zXmQH%k_v+udTP!X<+Nj3Dz(6hh)n9zWn?>WaSYEm^RRumsmLtj;|+%x5^ZV?-KRW* zTDNxX6_jt3{iitYo38Th(k`V+K4UfN-0pX!Uw44b+7%uM8*_9(Q|+Y87=7y>*#^xI z7vJnR8*tYt5?lR~6B+1O#hT>SZqHeQJ+q*w&Pe*z78~AAI(}*WA%@LL{=&&k7lWJq z_02l_RVVKpc&y@tS3G^bY~9OmqtEgZIB;R_xX;_3eDWFQK)SI7xFmk)WiI`AM0nxx zof1PTY2lW$wyd^n)^t~&rBJbOxa%z4zU#->76A*1K1n0M3PJrOd@vpFlzz+-1jV)BC<}jr{i^AdO>fl%-aN46AFI$`e`@!S_^06L@E?s# zbmHc(MrOF%6K`Dw&Tcn!Oe_EaCs*#70U2#JJx(qn9rvozEmc`X=Ts-~MP5KkE-PMa zx7)}=%B&{map84_za$sMKkFw zv=rl3up$#VpL)jhFj{(WkCP8QXucIipl>o&S7J z6ikE@@yVBKARZ^=U@C*?eYD2+5zJaX(JI?_fR!6EqqHbd<)aq2AKqpc8v3guXb%=QbGpad!cVn*|ZU z_ZSOzdHWK5>v@$6RwXJkvq(TXmpm)Y{cQO>p?K2w{cdAKf&ohs=iDAHS_A5kQS&tM zY6ooEK-3ij9AlmxhwZu@xCq(9mG{iDTQ2l&m~y_YARVvWTO0FpC7N=EYLA3fG!B^7 z8Bok_ffE|ryinc7rO^R~SmX>_Z890Ts+8a9pMgt7Ildv8>?;tTD1F}}{eQL7ywS*{Kr>`9ctRhS#t3 z+f7p{b!u$!)yFaYn2RWf$q{79*k!_*-ih92j|UE47Jpp&)>sO<|6>gH+&lKh9c=bE zQ4It#sA6VoBS;)zhjxRb-moh@*CpRJkoA&5yPQL`WCwOUg=?Eqbb>ycg+9Y$vDKY^ z`l`CZrAX#<($T%ODoKpsXRO$c`VG@f&T^0vyBcjPd6S;v6YIJ~n4>%9s)7>@Hqh=k zvcRH}6qhPOp6g7D0J4iZE{!O~Q}L`5ENB){H}V_c$$7@E#l(#m9z z@+fB)(m>od8RE6`OeH_ALYH$Sfz)jdiUbVNCVd`Egk!JkuCq!{&LG?XFYHo(g|o%d z(BQH~+yrdVlo{?k1K{c)2tR$@M+|2(>ckU`{DsK@-za$l6)d>v+RJ`|N9;8VxJmwHIWcF$w zAWU^rVcrxr@xW{~I6!fh=u#S*05loL*KH`)WnJ320+MKIb%pLjwQu^*n=fA+L0q(y z<5g>Oce)}%X$i};RIaqdyYQs-PJj`;$P%hXEg2&8!Q4YCN>2R2@#t;LNfKcFxDLbx z#&EkNa#kZI68(H{2%fWQKmYpNB?u5`F2_pU2Jo3B);2A!8NP2lc51e8P`eA>XJQ+O zPX7dB$SQ}2MaC$P+eYtnD98t^h#gWqNZ?QfUA4b9HyQ5`xpFCz!%h|LpB{;$CZzpV zvHADKc~O=J=2aWD=c6ys<=0NqA+msDU1NJswI;ZQDEK*A+Bde;K6TU-vsp1X2*4Rh z*^lNLhr9D8DY#6#B3f&6Y^%m>=#_sEc--~3WQ2^Vr?F4v?D8-Qmx|2mcBLs)J|yf< zuv?Hjced3`yL-ty<`JIs$SfYllb>Zifj{`{H3&Yune<(K8@8EN@1#bHEMT@&v~#iQ zrSHzOOq#a*q;Ty^EjZ9^4a#cl?hyyLw!z7e(f!?0cFFqXYft4!ko}{YjHFZQlunT> zWT%#w(=678g&A~B7_ZyJ{GR%;9Y$Tq1XbZtQRMy{?9--^lgBxPub~qKaQes>!Tp>e zDfVdU0BzqV;cZpX{SiQaKYl>!TJe$wDn=@Hn=wxW(@T*X@@K~msXH-m!r|7#bz_b0KTk@ zCR%AI#zSU$y}`_>=6aG6Y4d%0an9g8I9wr2v8t@+TYTWyZEQgy>iZ!H^BATSntE@4 z1^X^|Q@@U3Qj5ehV*PtvK3!{F*$Kp=9GT~^2E!mH!#g4v-Y=gQftB>fHdVQcAztik z+!xDK04mnr%RC(UU@-7rPEiIenaW}f2d$JkiBn%=JDDA?p~N%??3f-YuiGx>tP`50 zfo$e}zll<$M1z@A%lQn2?{JB?{jl>nE8k#Q4|I2?bS#Qc3{p}Eqb&o6DZanQrkiPP z8qdgE(Zn%Hv=BExqP7CQn=fT7TOtcEwYpcckQ0@U)O}HOmzakHZZauMVq9B8t#lq* zhxQXEPXN_5%FGJ!OMu^O0UtEfL>n9Gj%wx&yk+PVd5kC}LD~!ASV> z?a^iuQ_bHpTu0r;jcI<@ePJEfVJVVv@4W@X+y`-(O1sq0f|M&TDvd`7t)AeUSlf{zav-kjHXiYlZTEa=T#VHNyILO$+)=5Qq!b2D>` zKZo-F0t^UyKnZw2`FTKrGQOch8iY`YP`_{@gh^%zD2{)F75oM{I5Q_YCQy0bP^!b{ zq&K+JFK41dt$S|0a3Mc#pb$S7Mo~!FM#d(?z}dw_fs_nf)5*_T+2yk?rBn<@Z7y+1 zLVN%geHoH+S>Fh8V4tS+#Q3x-i-lR7IL4GT1EYrY)HJ9}V44sU3n)Jrm>eJw&;dAo zgWh%I%h)VV3`2oI?YF9mFu^+eA#NdHm|0+0FIj8Y zm|2~`-m3^@z>yOPm5Juge<;jec~ar6P|OMusYMrtz)(g@B!pw( z%px(63nLekXa1U6XEZuz;xBg_(&06eI244mC5qGCgpIaq-tA~fS)aHvU44%PLU4+q zq)}UBqB|H90gTW`z^Skv5uH~Ks49qY2WG6`PCXJdlY3e&QQs1HE}4&HRN6uf;x|A6 z++DGk^<2t{T0M@u97-~`2EX1ya~qG`<9~s*R#xZ?H=;%&(9DvIti7y$&_c~Od@>;| z9uCI|;=Cr^#Pp>)knw3vx*zHz1VkB)Dm@~&vt+~blJ^UeVB^vxtl|P&IP4U?85sbJ zNqDfE+jH-w(C~|h%L=9BD#X!;k``0>_;!2cQ5S0{kidW8BN7T{gJ3!x-g5E%iN=6% z!Y_a=aOdL(zJy4Zb446s_W^znoOU$K^_lRvFh7DCn#@nd77(k;vEZQy@B7q`TxbXZ zUux~i^iUUGMD*n`C&CN2yOANz0HbyYqBYFy_=XP#ZKw-ueiL8{kp2@YS!THluqn9O z4)_#F9O&$Ax!d}a% z?=e!1B=YG}S2r;5Nbe|7G1&%RJ8JaZ$!HyQ3H$1k*^lTVuKfm@GjNLQRo={V_m3v} ztGoI$GXG(cPZ}G)Xa}zwDR-Q*f;0?DI##Pv^XSxuD!E~ENmjN`B-{N;7|o-_`s=5_ zNhoSRV%i6WZ$WGidqGH34@SZ)HWug>S_qN@$|T!C>MXsZ9zKp*E#1L*1suQ# zFGqEh>7cl38sJ5}km;bh>KcGX{X#BqYs?YzFcSvTj|NbDs)357V`nMOCH#TTZqzM; z_5*tegT404#7y5z1UTH1iJCAT5Tju?H_+Wntk)Y~r3N&X_Nd1cj&r9bP$3w@%~-6L z^0YcMLK;{A7{-oa%QE5~y<`{7+D7kZ15C5>4Lh|F-bk?)Vjp~S25yUgy`Y#_*d?<9 zdol87&s0KicUoOa(4j?UZm{Ews{xGDxAlW{%HrlDFl-;C@RXL!^GJT;Xy{ElfRIur zI29zRp}^)Z$X10$Sj@u9%HS9Mw*8Wv)T5-m9btM-=(60j7*YlZt=|>YSml|YozH$Q z{UIc~I>wM^g|t3BjIYaeI0osw*^bHKfN#H#IbY1tn~pLKvRWTWnPO=+(3VRPo7v{dHI!zlSX;@P#_X2CN3Qf?##JMvlH6TNRgJ50RL@(Z zKKAMnZ1U@p@%V%eaA`fcz2nXTR86v6)LTwwNYRp~1pko03k0fsIG%vw z;7VPR8)gvPXE9Ev8CS484mo63t^Yo-;iBB&{m%3Vc7p};Xp*&|k3}wEu_>jE%0@3+ z7jBj!f9cU$x7lt(8~dh-#J!%EC+}VOBUzLE3i@0fAAE(r0nTD!#wnxX1tPPF))@UD3wu%OR#58mk<)mr3#M7XMae`->LuJH@t^P zX{%qCGJ;6y$<{hXp^4I(J^D#bqty%}{!9*$l}1kmd1YwNgVLArn5-^wG&EgJPld#V zA#p9O!gcf=X?xmwJNG5L&439g!c_oY({n>oVEJWYJE*{H02D66UNsipth6coUDAT)ZTvHSwhqnaz;&L=d*I$7iY|?# z`QwtxqqfZmd{~P0NVdwc<$}&P6<5Xgy!W$JNFiMko=244j=hFLvV`DyU>Q5_Z#P90 zK_}k_pMN*zwSgfk70vQ9o0_{AeeV{@WOWC5?s)Y2yYA3Za|gM51Ef&l|EWg9`!AWz z(ahb{?azw#&+%ht>Z2x_<{M@T9#9j7=U!I2g~|evO!0AN4h{w*@?*%V30J6DenBEm zmK^nJXJh5t&j0#eo}UHY-i|OjBNSqMCEY%T2c2)Y{gf}){p61y zIh!vaO3}tlUDrFdOkFoSADH|ucZPkuYBGuviavwDEuj-fQea`w+iHK-w$TctZ=`pp z|4x6RwN%euV`sE7QWtK{HjYNGSZ8OvGB(#YH+GFa&Y8lQaz?LM-%x)_EyjtfodNf@ z`U-=L6?ytwNu&Gh9=|{E?Wra`@n!kiy`ypm3QGMs zuq8qlTge%(`g2RrPgtYS*B{>mcIZDDZb}>)TJF~% znG*?TmswvSEMbsuFKkNTX!d>tr@ggHPJgCl zZppGw8fPhOcHq{$)!ds?h4y?&vXMZ5cu;>vprr<^{0#Lzl^C8yoP}OJQdQ+c zoh2}IH+1r~*Kct&d3{&Os7D056c=A(dY{<;M4a4)%`XX{tLVXF;qVNVAbXFp253G( zNzDyAvnn1f;3U||^S!mr!91CPww|#a(8>SF*s5eU@RCR#{edG$phuQatc=VdL%+#* z_Hm9Cdk&2%H*f{5Un(bSHW+*s+|==`7FBj%yD3>u_KXNYtHAVfcVf>pvP{xFat3FY zRL&Ut{`AJ$Y)rG#XuEJYCvY*bScVSLIumm(by?9c>?W`y<`{Z(G0i#cj-JW_m&As@ z%Htz$HE)xm>w0ZxX(b#G?_2qIXW#g(Hlv`S9|x;QG3%V8t|5{SA1A$})>5xGC5Xa6nt17Ynz9VigMQOZ~Cjf=dPgJP8BNmftw!$vF9hjP+ z2g~qj&@@1tOpBJmU&P{2>QucW%S`rozrO4{6ea_qa2bT*$83`p69{;UyM~FWTyurlL&1xKG{kXH9kPeXhZG;wN0qU|$M7b$HZ|wz6hNp$wBo#gawLUTB$KpHl91 zZANmcOuS0n+R7%uBu|vY?awRNz;=CudPZFfa>O&gLIN4GcyG-3ZVFN!>?aW!>LU(j zyJ|()4xWot9ERP%*I2lZ1hdS?pmXGZ=n(~$^yeIZx9b0hq|q--5E|!>pQi|UzqQf3 z{d*)tltEFt^OjrX{;_HKi*u>u@>@m!LXIK)gSq4Zst_fz0hysj??S;N17`y#LuLit=o@%rVNLb)^?(%e2(c?=A&^h? z!2K>E#)+;DgWiOwdi(sno&QCMe?h|x-ndt95_mun{vU-f{kQy{-$Fo;M*X76AQ+i# zLE6rTW`MCHg*Z+Ojr2_Pzy!+)L9^5|zkxJ0%*-Ixh}4s?`2p8kpbY+xqM-ewD8j+8zle-9QGgDk&L^dUkHGv(`DJXJG#u=J zIM^dvOiT$V4=7CHp!fiA8CMZ&Mq~uFVW?j`px+2wA$mn^%$qF#%)a_-4*up}Uhn_7 zfd7qo#rej^{98%R-#jZq=HH3|>b5VmXypE6#2*EqA1vds{%@SV-v7br8(t5laC|#$ z*gt9?`|W@KnA)qFyI33BS$i9s*qQ&$p?{M|g3nTZkTZ^wyw zY*BK5J6PmD9v%6wqdVIFd1QArS=V_D^cS<0t{5c|m|#-9Tu2>!JY=P8#2FD2V)k&= zEMv+IX0EAdGKmyD?J;ryFK3O(r*mD$G*An#!ZSduy5)m1;83TJyJ=;e~Z2E!`Q6PTJ_>R@K}4PGV*lzoXNKTkWa}+ z8tvdaJDNW2o`*)azN?-C_Eg6DBi`ZKPRa0@)|DMAX!8Z`>v$|>;&c4ETm6{5!_@n8 z72;9y={x~ug_K+I@!IEcxCFHG1f?3^RTr`4TRy`ESe$S)F(xjbarc;JIwhyUN;LKI zmHdTC{ed5M|6s7UV`5FRbHu7{ujVIDvj7clptsZK6E+UGMZ9g5DzB7N ze`*)mrc(IOuSgO-SyU;N96O=q^y1#)4m`NVPUHF3=6QxF*@N~c1}%~dU~8FUz!)s< zE_Eymaqng@%j~TF!mCvKT6G8L{EiAC?E*aEl1*++qa$skWR&k%_~!)TDTU__Co?9`B2m7{8R5^x}0$D)w0S(9sR{%lj49 zDZk%?8~ejb7Sp%EO~gOGZXo^Zf;yR-{%3Fzb+mUfc6cLW{1Ig+6y%Y5m5{R*ZOWF0 zKV^a=;(9vAbHk$UysQdbXLu*8K);Sj?`%OoD4XcEkJ?#Sc(UbOoFEtlrXYL)S{59C zRB|`&C&doh&Nh-y*7$kC)`TY97tuayMvk;AON}b*)MT9<;*tkf zO%R!RiIx}p@-EEWi^6L^`&IB8TAP0t|F}}P*hXZ@&8-`zP7y8vHhvlP&VgPOhec-s z_~IK(}urK*ME(Q)!d!z%>S%7qSf_P@zk;X z6uLYT?ZARXHb|Fr=t-pUdNpiw8~4Zag=&a1bOy~`Vb)gcb}UjsUttA>(w)8Y*f2{! z&EAdS_=kFaJxcbBxfpezj$5#1{gGWi{eJ2u`=*16s{8fF4HJljay$#Asmf4N6b~#N zsz7`V?Y1gxjx@FOgGygUa5cFV>Jr5f>XE%FNF?>_o}W`YNu&uaeW+#&_Bk()QxZd>08qi+ zrl~mbz>C*?kFfvJ`8r=$-=D;lbOY4U(0|dr!9DRudzKqQ{b}_b1?JW+L0WkUx(^y; zw#<)mtvXwC&A2kuR?bWRN(-{vJ#11mpgA`ej)-a_tL0G!7ybE49ch#J$~$dB-Ky56 zDNBUju!2A>uD9O8m%n#l{an7L3t%-QOYm#mb(xO zIz8Vr^ha&o|I-PA6fQ7bx6 zK5Vk*<^@Q6d{;M#VoA|+nV&}zz_V3 z#|}t{V|cxBdVr6jJyA!j5#ns3Z=jU*7P9XTuYC|pze)ZfNFY(^`=AWOx%;aaCjKKP zk}RG9Au8I9O8bSJxNl;gE~L)vOIV%1KP~9)B?!$c&gj4EDW_?Q6^;+cP@6fqkMR50 zCt$!^S*)Tb;Bx!mG1Z(oHt_mU(%f=%j#1=w)U11Ob7iiNjokA}@|-52NPNr?M!$F> zqz0E(Y{5x*tQ19^b^vZRutGvzn>k5^q>6S^818Hvi2I75;)JBR_`o*Oex$LhNK!sV zGIh4y0n#skjhWjPnkG|!@9cp^!9j`s0e)14B#%-zB0R;UPg4$Nu&_Gq5(7qo zoY6=yyY*(KRJ$e4^CgIA%4VlyW<(^(qx>R{2OvKMJ<{AHpSdr${eLxu-{>nB1>W#R zKbZeT^!c+@h}N)GU6VlPKS_;385*HfQ_5({3$K;egONuQo(t9#?eB-k7&s1s)5TUM zX|EH@dkrR2CjLSR-r+ z^e+Wz;XCux2axEHd?=S8ADJM1D7I`tDL~CZGd{Y5Vu0d--Et1ZA5iwoGXh~4hl#fe zuvafZxrkiYa76+A>rn$U;kGQ~4|ZUht$8bqfW`uXM~|?UzDZ&(ZIpZFp~=E9@%62P zt}w6*3@^v<`xLJW&|)$A-Iy?pb&M+>E+l8&0Wu#_k->&A`E~Rwh9MlS43ad$EOV(M z=tIs$dk)7^3ySWRrn2PI+WN~n`(ulcEx*7MibbWHqR+<2>eU&K+Pd{>?KaUI>_n`3 zQqxUU*&Ozj`B%s1^JrA<=5q{3dqA5qiaC51|W0aG#JIC_2R_ZT1a5v|0idyCje8*FOd%3@UK{h)cNh~|T{2|J&FyI#X0&o4=D^{9C>P5k#>%#Jyb-f#; zk-hO`;d&Z{=8j;uk7SR`?KSxW@E*sHf@`ZycF)H{InS&8_QPy2+pyPKtjxs~&$o-X z=Tb;Ro4MGwJz4oj2=ce6L`?aDp8ILI4lV(On|LPxK4u5&WGU?R^6k zKOdFECHjsNP0Xh}=d#le!EvOo)QLai8O$SZm{2xNRa>#bu&}5xDcHr7sg3JsC`d!m zg%RnLdGZ^~&4gpd>}5hpY0qTj98yiQHnr7uTn-hHMSiqU48H=A*N02-{lkw4QC*%P z^IZ;Ep6;CEgN%^?t~H)SEa@6VJC*#;q>*|oWo{VB!Rw1^a_ksr*(QN>9`7Sy6o)oM4t9MTaYzz*@c6(8T_HI1G9{hkNC-oxOuPBPNlLpqz=PsL zl?T)#Fb# z7W{rHxGps()m}|F0HiG(EgUDh7MugkgHAwYZdK2*Bp4*8IH%-F2bv`MBR9GavZNql zJx}j}8NPGJ_(3N(@X`-ono!;(0wEf4cNqlA$p?g%a#^m9r?0qh2Ts83CO(29%cMTn zB)z~qMuS^t+bX$0Jw{-V0^^IbusBXVtwYTt;84vc=7x5Or6fOKo;HJD<@wXS z+P-3?*k{Z&+gx+sCdix#dg~QwdvY~d+<|?!tV8cOf34J8uGj|S{^P3-B8#x5GO_oV zOT`^dz~!UFB06@P2~Tmi`}p%w5<@BdRT1z|VLK~)X%9QQ7iZ2fQ3`l zE~cJD=5-RGpmF;!0(Y|bSPjwyqd&d#42RR0@<)o$98$}kpN38*((Jc3sZ*j_H=#@z z{G+qR&+k2ZZJ8=^Y`qj3IX*O-aK6`t0N%Ti@B6nVdI)boX4$ngCd{(a;#rKgxmGVU zbY|=~$jJBGr#qAhsJ9ta;bm_a*ex)nJvS7qwsIL3l-bNPEkxNiUci#?omJQ`$%p1> zfNkW%0%Xi)Ilff37j9vBX`E}BX#TXWRu!l*YfCZK-F(#93sT9x>LYVTNr zfT&?tAkKsMvpFZMCr#0fzr=pgI_aY7{S@m4eqmc9o`9iHtLbufYmEZ5V<6(Nz`H#t z6p08V&UZ1YJ{9iI5x3<8f>I@B81u+!CGEa@kdFfUB-#PxW(5S~vV3T437W!TT|N7K z&&%M~yO)Ad7{_zR?-agFsEUo7#>TG=brD28>-YqMg9#>lSo4JT^$gP!>T}PEhaKMU zBMlkGM#`VV=?Z;9QxRU+7qWR5)`luI38AbBUbaF^+euFT{oP{tY`tPU{l(Vzq~#$$ z1FlxJk=S{D=VGcUmvu=CJjaeK^Q1bTtNhQMnfDe-#>esps)IyfLS)^lY0nCL8T8YF zk~>`wO%H(ZYjvGeLIDxFIWjXg?<6}7&4wDnc~b9g%*EY|d?fUSw5967Mo|7Tq+?iL zCCx@*`Tdo$I)RF!u29sk1J?ErT-15;Su?;KG7lnY`v)8ooo_4#YXg2zWT;Wh+)d#v z67|h_8^bv|ff)1N#T6xua+)j~>T~t1|COWjm+AL%k)*g~q(Dq>%Y)?|!(>7%K zgLaxMbQtv&+@u)wDlFp>WGe9%p6KH`2^jTHR@&3n1^R<}nk-Zpt`*$07_KTT!x06) zzoAWxpT8~rS#lfp>$iSN6B44RU?SjK(D^WHZ(4^WQs$(jo6@d7W1S}8kS0z0Y9KG{ zpUEim4IPEd#|53TIgS>+IgSy3MWi@4q&be^)aX{Z=UH69)alIeS4eb|t2>#03yE}a z|5V>m{%4>4MqaZs|NWqtNz&Xw`i&I-3(b?Ww*3AHGA|Z!g(BcXfFi2f za@$!m=Xbymi}CQON(=(I^Mo(N*`nvr!3y^^KS>>ARMK+;4c3oLaXis8Tlg-e)rID2bGB$z@lV?T9#M-Vs^E9)HMj`nl=SH)%Mse52x1 zTX2q4+RA+;k?L0VA-vWsCnabB1|!Km*ydTkNQS$Y1^GLAt0~cOnER7@QQJWL2?8-WsCi- zmmq{P_=e>+!~vH-4PDv|t3WJ{SV$!%tXNi~5DR3(mR!;Z8@Wja)}Ga@*qW!1pUSet zc}6{K5(_0#gGcE~+|IK$?X$~5MpKGt$ek_B@4wZg4ld3CefSMRla9!Cr@zVg@Q;AQyT5UXHyM9 z1{9)aJfI+PtWDq!g|gS|0FgOgy*}E)ASXZD&7|i!&(Euyc`Oj~^#kk!?Cq8j=kQ(D zVT_n^{4d-RJVV@7u2F|^;*N+sr|Kq#h-ch|LEAMkfV&f`eozX;Z!<>Um}^Q1-gyJE zdQd1Lpnyu8P7>j+M_f*XklT{fD&$&5rs=0vF=70z|| zOYn_Yw%xMDvcR?%Ygu1IntS-dBCV+FxEA*94?p}>mv!aUGD&j^D4bO>{ug zW`(bb(cctJQah-SA77&j%M;8F1;bg~nypxHezvFj3w2j!-}ir}05fKi8rRmV1LLZC zXRrY$rj?qfj@K}>*AcdHRAq>l7;t+~Xvrk6A%N$&6(?4ZsR^zE{nOhqXwDKlWSbC)bVYQeKZb44OsK7L#_11{ zxi@au=|Wc$U;%oOyfp~=zIe1L3| zUcH!YTxXt`B@0MvKX zy;Pjf2>kwgnUVx4pWx!#!;XiynELl<9mzjmoE zi`Z6KG^gnaKP9$V@hLMG>Z0cY)92nX>D>yQkEYc2xu;c}mPaqQnX`YY7K`gX>MO$@61U=eFU$So&Hw(Dh_HXeyul ziH+m@y3Svv;hR^eO_Y1ccn4dSB?p5A#_Wg`2aAny>0eeuNI*f>y!*NLoP|#;$R?;rRJvJD-bl?tL$yf7hF2FP`D`*_J0eg7mGs?koacX4O;Y|=y#RpM;iu(ETlOSw*jshf!|M}h(d5D zzr1Pa-T$NA-Tt^W?s1(CeM*3k*vEFEyburSu#h}a6qN#cb0cVVk#9nVTL)vgJ@#PI zWBu-1pkCC$;DHoowG6(8___l4ulFE&9n^f(?C2SAv2coT3?ZC(_X25y61rz;Gdi5% zL{L*nK|EBZBJ!)lIB4H)E}2;0c1fe@&~?toEcVVl-f5VS-Chj(P~clnUr!iDo^V(u zX2pv#b~ykITwy-}+E;)$4SOHSHVe0{vG$Q(9V!&WHp1Tj;MAel^W5{*fDuO?qs}F7 z1fwu4MBT<&bQ96H0c}M4Ri{B~+fC2EY3FTz`TyiYZh!fZyHdMC|J&9Z1WGL8xL`?0 zNXU?RqgE1en?O=+nfRD0V|9J&VJXq9M z)|cKVWL(NjsyDnlfoJb-LTkGcT<0>1)=m*hvrRjt@gz^B(Vl7+Uk#BHlLO0j^|gqA z&+bQs&;AdfB)5Ua_od5<208A9lSW^vTeNn%^~dprzj2E!)j@9Zqf(QqQ?12!;_D9%lFFA4P~GBhJK!?YSzGn^9ZC93F9g z-}CGeQp9%2+e3`;`+fbQ)#7|l-&AM!kE)}2`^Fz*>i=G|skssE%Y#@}bxZ;LhT(&IjrhiZCTR!~mJIB4m3Xr^NoX?XTs z9~OSDK-f@IxO4oe*{Ukc;QehNTeP|7ph#3WEWBQ8 zJ9~1Hn~ps%c~=JfuSXCbw3HG@`Bc`^o%gna-CqSkqL@dVlXlablmSE?Mw3M~I$CM1*GYIYo`)N7~I57Gij z^&CRhvvE|tA8{?Ae*8cQcBm1I9f4s$&PaBLp*jMIhaN#i;yNW!{_4vT>KOe<8sn-N@AIV;)MxgD-S;f2v1N?RrA9YR+og5Xu+l^)mO z?Sn7Zuh&r)y&$vp&S{lN#FUy#nOm3vagoZ?6j>GgY`Cc_%{fA*&AqAF6}vex^?cCD zb-~}6e&zr29wHI13aC0jdxVdwP%eYgH%edVE9Dk?-Hfw!di3W?IS@RxX7!(i`Zq7_5ng+^ z*nPqrZ607Y=bVrX=Pp^MXkmbW8$0T4t(Zz9?U7NhK6NF6uPB-|lpUs9Nq!o!I&HqJ z@A#Z$$UDw=wo#@z&Jx)>(t-1^8*>HZlcb-ImuFk-!>o;?!q^6^5?H13&UqLoL(Ogq zs{AxstbO}X`#nUE&ronC<%(Dd6Z63>GAh?KvN=f88hlX_4&nI4I~Z8Mc2IxCZ#_+m z-Gy5M?y5|4+<5UI*=10*;H^Ptll*m zy5nf@l;ygy@-oJhLffqg)*1~{TV8UCcemew%mFA~aXEXW9E|*IlBnBC^YL2W^h)O= zVpfdoa4cXo5iWQ0R7^Hhmj29{iXnW^oJy7?lbBr>i5V^BBR#&~OM9YjjZfq<$o(Z` z5FSNyWJ<=XPUxcA%J|V`wVNFWo_0S#k(NqR_CqS=K`*UN{4R}nDGVf>As!JQ!1L)E zqXM7Vo+U|I9>z0CoKTEox-2I8l{I>p~U9MROX!B$9J?tr;Y~5zf*p{jY}&Y z{Wyw8c_P2Yd!A6_^Fw3O&0j&jGrfz6>^^5fzO9IBI@FBXR9g~Xhi?7lIQTMF7qUFU zRN}69`LmHs4r1fryLW4ABLfS^fsrX3_C66bgZh}MzaK>LT3EA`9#0$5Y(($bnWBL48255 z%MnQH8@^FMAp@JReRN4Z+#=oHxJ`8&8bQ0s(AO!>s~#=ZCaNaQ!{@84_f)|qXzgp? z?_$KRybi0Ds~8K&BuMhEcogYx@C6b_0n>S+$)qBc{jB;M^W5Ypb7uTfnZ#P^D=2VCWkMQJ&!VwHXHWNdOhOt7hsWS<7w z0~6mHsAgk@nbq!p4t^Tu`Ia?egL9$Z^4_4n8!02Q!K_xj_@ypuje55$s3GcY92We8 zlMkm^+u}l;=l>(^t=r<@lC|Fif?IHRcMb0D?iSqLJ;54i+}(n^ySux)yIUYY;B=mu z*>kRa_B`)?uk!)=kM32yYTZ?L{mMN)!W+iiWF0cK12bmSxk-`Ey0{{;o{inKM&InF ziw1s~4=e6_!)P(#9vsT)9?L0|hrF<_>S4gyY{UvfUR2+t3Wo}XhDw>ZwsxH+E$dDN z%0315ucfD4Iz(JkP>;(xcYBbIeR$aY#yE9*$VA*H6ef%^U+IOilPdQzM(G`qI2T|# z9~~2;sADpZElV#uxHwb?J+cxt4i`P#QcJf+k=GeTcu=xdJP1*YP>YOfU8_mC_io3q zKtM|B;bOYaSDy2}O8&{Jl7UKKqy88tYl_i}o$ylAa7L`WJxXDtKTYdg*&zdP>S2X( zp@tRe&~(_pjLK2H%a2)kcd;JA%^UVGZzsX1cFeDQPch#9qyB4+{|Rgk&f@bg$hf_b@WzZ$iuR%K&8+6%F9^}Q4Oxg%)^=q@ zOl)HI>*GX8!xuPe?pW9{uq9f74aw8K33yDxLfB#%&t*3n8Ixy-GmBbH7?$l}3LN7k z+QGX91Ue6FnUc@zQ%uQ}RwfuON?KT+!SC~Fl#fr8#09qbCy@0?-o3?J0akAvPF|k1B=zMVDAb=ATXJN7U zteCr-7N3@$rlCAOOyr-7W(WIKO5Ci5RW~5kzle;m$JE%wz{uc}FowZ03&`Mm&05U@ zGWdRkbYq$r=!Q%%G}#8NBZ0^J>v3{)<)=CW`UL=g`}|`0``_pfK*QG3%hcpQOhN_v zQ^}qId8lw~;i+Ukb6BYb0mWtlv@f_#C^)%OdrXXGF54QsD=J)VFLssJT5)y%%s`Ra zMBBbcW^LkvQMCkx9^E@ky0cvQUiv<6xVNz0y?|EmfED-CGj=}iLNWT??nYn=c;AF$ zc6m9vec~-P&|(MkEFKLB81>n&-Zxa@2jpdt53^(!08BrQ2`3HE$!J<>5twR?-eQe8 zC2=Puf`+6J=*l$J>v(%+`)dKF^fIht+({3hL8*E)BFyMIHjM6Yt0Z(9?Hd3P;YoBr zzddJ{)u+^ma8v09Vpa7#WbC#TSasb91P?8J!qHL16D#+)zH!N|)n{tA6s%kv zOxUhP#I)tVJA~tYxp2eD67Q>tF|$0_pCFlcIlmRx;owH-7+4k(jg_1ZNIMg?+@zdn z5^u;9XDe*b60gH&fljA3G$TWC$ELAoF~&QH7*`GNnJyP1;;^zn-Yt~T&8FSNE3h$E zZla>P=K!YC9^S}erySbj1z}w&xUUb>D#TF4vBCwV&f_3B^0WvqpfGQ)yxj+4O!+l@ zFk>a*roDL#uXK`9kWb@z%1ZGBXLZ^fgn`68Fy!v$dXp^}R}vlH0A#r+G1TzEh2!cb zt&pS#&`ll(xK&f3GD$|#Pg?MVI;v6Z`AgTc=gs6SOmmVsbG=0jQ7=yWYTN8IvU+6i zLF&dMP^vB8mcz5UpFQXPlCmB3HN^Z?1ndW+)t2I|JSDsVK<9SlXOP#59q&~9Y zMlv1Q_JHWPgS@S%DWC-0-HuSsdx@=TeOmP@Z)qQck^li4fufBt_r&W_9DUykAAgYn z0)n{6Sfi|t7iZzPviikNj2DmnN9SnccJJvgFP(Q#4El6X;FbT#JXI;FhvKG& zWlhCM>uqqeo8=X=F_bN*xc^M-03EBSMs!&XZPl6zSl&jKab+d+X}A?lr1xNN4@8d` zPIA!Mz3nolP5RdYka~TNUQ+!;i*dRjR4R&dwc(g`cCC`s>wxbitNeD7<|A?bODaTU zYlGl9fg+SGq>JW6#|Qpf#jGoZ_+1g{grA^$d^}zZDyK~8wlN6CJ8ZifB_iV}=5jEz zE(oSF34QF|alDU~#wxqZK$5TvjPIGzHIN2lEjU0LKIaVU_`o*+Dj+|DxBFA8pr#P_ zP*(FzI9h_KkpHY6mycOCt|oK4Wqw(U4js(*tDcG*pY@8#nN58N%RnboBmZT0M z1@A^Bf0AjZ%h865;T`NM(1VBsUy4R z&O%r03d#yf)sHZBMopoZ(x7@>V{EKh0`!nK&Y%@I=8PzuCG^S{+X~ z3+NWxClsqZlGT2HiHDarR2ek@$7dVjYg)^p>eh(vZ25^&;fYN8)ClM7?C_HpmSJy) zA%W&N+|Y-mmFIewF=Akgsz-pTFH`BPq)lK$M{a}J6I_p~ZLQfH9QJ3gd{09OuRP;# z1c;>^n@+F?^_+VYF((Kl2J;qmsz1+u7P z)gFv`11g+;i{3*l^}2ZGxsr5vG_hzIX=Xw4nrYH3kt$g9AffCHjB+$=B)Av&_#_?{ zTv=1Y-=O{Et%v|-v+h3*hOg6h(PJ;_r#IpayQ%oue5yGuQCacxk(y-ssv+Jf46(tUyqV<{sQ$k`ibq#AaP=6^u2weeZg*7 z=K_A6l`mHrSAnEK)2QL4SSAaAH*IVEjSzh5KAs`hvsIYUGx3-C`|?~=dmr@fc?Y$W z{`33r|2~?`?43YZ^#4yZS+K9zbdx!)?>)-z_^Nm|oF4lIb3IAmTRn|| z${)GL+YB^aVtWGMpz2tMjVAkWfCL0aBYno}kJq2Z2u3wchm&Xtv7`^4Q?Z~cgHLY7 z&txHQ^i6B~t62~{MyquzAHj1W1Y~7D(^|Xzob4<-m`k_S*C{Jtny2g{v@lhDjU#Rn zcx!nWY$We{)|pE>s@+J)#xX4SIu&lr*5fcwsiGy_BJW6wkQbFf^>Q>~2wba|LK_xI z1hWfgnim0+A2#qX%wxdJh31#8gcKn-%`cP94k0-m3w?C^<$oS3IyBViP9!4LE6c=t zI(ToF+a=@mhCQi>r^t&n)U(wk;C?LD%LTjf*o<+Qe)v; z+}I-ctf=JR+i?fTIauk}Y?vTP060)AI=_qPhpAERI$Z!6 zOHOOy_g*$+`GZy+*VoYiOO+C~zLo4G`B+B@?0x|P0U>OLLq}9$Kp{;?E_au=I~FWX ziOK7Hu+ZIr@)>h;n#I9PS}H$NUDEjcEjobDq1(dn#|tW7w^?DE=9CLmi>yMmp(Mbz zj?wV!uW~))s-K%2=#V~uhF<>jKb`1*JfwfRaQ=xNja1T+0TV)d6Q6Kxl4EVNbOVP$ zS3do6l1tm8j-+*U#@#>N3b$uB(={3p6yPT)K*LRHEh#hIopHvo#pOJCw6q5G9lzX? z%2IPiak08I&`?_fwooX^kf=lymX+dmI90PZ041CPILxA2`lak@xpKqBXX!wm$fs%r z4!mve_E3mwogvv9e#GhmKs@K9yAx)RO}qh&n9tohfM2QmF7r^#Y-BvF)w_pjSk-5$ zGk`sMBfKYpPE&)f5Wd+QFO}ow=jmdLuW{_I;YWqet;Rhi8n7&zE1Bs!>;=70ST+%#LPHr=?VT=9PC7b#eZBoYZW}H20)xN6onf%k z*|5+pCwe6dI7r+c54Kpi23JL6vrEeRFtqFqDaT>-$L?9?ddL6;&bZZ8x85rVf(%RR zJ{&32_K4g@E+Olgi#IS8*4ga!N{+~WJPi(2r6Dk8)S876-`9fh78*YhKA+b@K83;8uIh^Qpk zJ#E}r=>3i~uYpbB9l3O=!f+zC_C6t0hTYnv@5}Tf z_bE_uB$>R*6JWUk6E~+o7@4!Tc?i4z(H4oTRs3X$=8zf-Lr`}x#SmKs0GGm^apeqy z3J9mdvdgvL43X+n8^^jd$cV!>JIN9`!T%GljPKOqwxE+6_}h4;`j3~1i@npo>LmZ1 zBm*I1+(Gu|p{_Z4J#lI4kJJgZpWrQt<>t^8EoEqfg$LBpbc;sv@g-SKOq5~mnggjq z=>mu<=}5pHhMAuNzkUCfFyuXlM%E%F!Zj2>cQxMr=m{#5Y+Yn}etW+=`ud^$X~Mq> z|7p?x3h!ytpAhe<27bW8;HIWNNyNVmk6d$XcDOccpl`N+*VNL?(h#)r!2HDE#AKzr zHr5n%Oa;V-;q99pn4OsIs|_wlE{HzRN~)myO}9p)_}vwqs^1KK%~5c`@Lb*T6R5%g zC_*RjC(<*Hp7pC7wu!Zh-AiLC417>4LaOr^VE z?z*B0rf~;3K$+08Czz>{W_XnWbS#?6+5>A3nSrTl5GuwIox`xCIXDHl23?vGP?@ui z%|wym$7&*uhSs|HleXBg0)M$k&LS%Y1N#9!>U{3^&%ygc;x!hMym?)P3Fa=0SzZ$| z(PBe!-rup``AeVx_l_i|qP>gs6YTV-?$W>elGt-~m~F}Ok;GrQ4a>;Jp+olAMY2AU z4(eBq%ORY=S7brd((A#YHtFtOrtc1b7%`NtRRN}zZ|k%b?hjMK<&LHR{q@LTu;r;r zx;XAQzPO77R^J=RS@bJ$$Tf4Qbkiz=6p};|qeUCkZXaz?V_MzRG06sa@n5Ma4=dQ7 zlvY6>+YZcDy`pJDm}gEVS|s-4v&Nj#h^n}qKh~M-0y2!vvDY)K9U5QP6P%~ZdDaDr zc52_bnpDQaL-1;n!>7;HqQ@L{_%jj=^*j;A_1e6T%t-U->^)U^=7QV2?aT=1Ry-lH z*o#S#7-Wz)S&MRiEi6|cne%Auz)y_KXPmhx@XsBr^G01fNYur-6!O>wfwg`)8aS}j z?@@f4)b@&jQBaT!A8@lkCM_PSZGY8(R-fy0A;Y|ThwM4?dLmDBNBi}Q+%C16WC{#Y-)L5Z}42gLjZr8G{>`hW?_jCk>Av0UE$xDQZ902sJe4311dVL`C)y7lbZrhvG+ zv9?VS{|IS>rah-syvTz1xxmW&ms0F|bO8ZAcs55)OM&)pX1-#v`319~*ixybvf=Ro z16p4#V6EzYa-HDvdKanWVQ1jemBZgl8FL)7tL0ysp61rp(tPCb z{V@nl;TyJeyu)m7bt2cR<-F*X&`#1`DbiaCKF?t{rpOoXbN+^XQj|Da1>;AYZAgpu zcF_x_;cr5IAmCe2?&?i=VJ`_ll9Y*~r*J&FHUgH(tU>@cVCY(jbB?ZZ8>s7Z6n86k z3(%S(3K_QfALBlq7Mw(no%@7!1n^Ne^0)YarUq)+{7q4OK9XVsa8W#jjTA=C_fWhs z{KE7&$*avj$1FoAPsQG(>j_Cvy%eho4k%Y^kk_Tgb+|BpD&@V!eYhkMp zA3iYrZPDU?l`lB|@zCi=`DeZWw-i~6^^KiHfKDk4u9rIjE3Q|}*QC-E4F4g|ii#!4 zV#?A}3lmCZmopqy+XAMx0xLE(L&!HIFuKbz)f&7d8Iz%7tSdLLG}0QWO*=qO`a)L$LXzp5S{iE&)n*lReAuhW zC`c_38-T=9tU=B$;RG~+#$puIGP(`XDGh7RHEz%mOE`n8fmJnP1D5@&8A46Ky5)*p zJRZ&|XH0{#=1^N22E$|^ZMzq`V^Sd9Rar9t5q+sfVZi-Kb71QH3K9>U16j#2{PB1I z8NJJ(3b<7*F`(C8v`gTnx;`Y2?#qO}@9p9oO=%HaQ5>KfjqVPw)8k?9Sr%}zdE1z| zN-#mV_+#z&g&t32PMyt@ONep$%mtgC^ju92#^UQfy=<|i*zXFZjbr9}L}L!*0TLlx zt0>dCRtfDb6KijHoc@{%4F&u;3%LQJ%rJJqC#N4mRd7VCE zr!R-)TcQ-j=omI=wA+6ts2v@zSWJ$M!j7u;!F#kqCIjSCe-L!mA_CRx_^itZ6JKEn~G_1~z< zxz5-~D1yqXH_d|Z0)&$)g8SO7X7Yp_pnO+z2NrSp>W|rt>7~A^Oe`C_F4XI>o=A0= zP}Q96?Oa2SMa)|2cm;^>N~rYg@1z!)&q*3e|laI zqo$bh#&A5s^ZSQHUI*_20xZqk;`5x_KGup__t>&_@N(L45NgRVdicA(Fy!sO%u7v>aZCucwMyu&8)UlkVxNJ(T$^ym{x8mne+|%|V>b`RZdKp?SVJ)irycjmFEKXd9m&JC zEhadM2%QTqKv_dItRu`_*@hb5n>W&KE8~Y>g48v7kT_@Mu|#fvR^kdaiucDg@}IwW zz`O}~$lQ_=>gQN^KYs=GKmT6Qc;80dW zx1Z4`)bTP*oMTI$XtVLH;Re$brQ3p9&Zl|KWlJm!08Uhnm;yv8+jcGPV zP}akd$cOIoi7s~$h*9Cl?%CPliY0`<4Wxr{e-*yURT0FLK2|#>4X<%h5&)WVQ<>(1 z_+m8aq`8z1Y{8Rx1Zg71{7}qm3(;XH-yEywU#XYsd>&)nu~jTmuAvWLzXaoHT!rd; z#6(o(lA5+N&y@P#=HJ*5Ml*CcdlKHpMft$mZ3Vx4oZmkZa%bg|KKQ&oYACA`m7~c8 zTjgYND}H)gg`}>=!rpX+Ohdpyz!@4~99p<74h*jQs(Zr$`}TeI*XLi~e$hN1a})gN z)=({1Q4|RtdFLBtFr$L9IE1@|<$NRjBUCCp4h2X-)pC>n|A~j|pNZ$+LZtxTKS_?N zUJ(pXAJcdO8(LqGXk^)=j^>a~5zoRj5@ezbcexu7nuM>4!GVFS#p?v^74>$^^`=B{ zB{xShB8M}|^>*u<`pAFP`EuFT!g_V}&F=+Vkk8L?n-;3mZ5tP=({-B%sx#YPc)pe` zQLyI}WxoauWR^gX!I$Ba0S`|uhy`Vk;DW@0NK^VTucRU^_G;d~+Sr0XQ&bsD8IG|! zx^yis&Bkipq1mz82DD$-2|qrZX|8HYAt)nVV><0$>C<>vX4 z!&+|yI~;Ya9n)NZ^4N`LA0^yKn%jn^+Uiw%RX@7G3Q;qgNZ@tDb_i#H)CR)WOGeZ~ zlS-Ml8s`SlBHS7wc_#9DGGjMF8!+(ws4=ZXyEsUIml2vUZ}yK^f3+#K!C`b(xkGS#Q;L;&>6Uv=?=f>e-BPgyzAWvxwIW-{5*w zUwEi)%ps6=VG}%;kyK*2bTX-%H0I@jw{{k7PQD{6G0ybc%VafuY^FWHe?2qu_#*{x zW+6tT-GZ>VV6>S>5hH4OA>hP;{SVNx<7}o$vwjmA`zIs>L}t=g4JPLwDV0(w%1mWT z&?mk%+c5;#`)cS31&LY^=vrewBj3@(pYH7fi>9Vu-yEmV(X4i=Jq3kIreF!N>Q_mT zBeM=81zh{VKJx9B6W%tmv;PWFe$lC?_~zWbCRq7Gs-DvLBU%|bVs_|j_ejS_(tV9> z=#;b$wY;0%@21b*58vf1x`BVFmKYuTzV{z5m-ED4?v-#{ux2U^mMJ7gBagqu75i$k z9`Ci^3ZcCi6}C*U2-nY5DrIMRh|r{D;+Wl1r8p3QFq(T1x&6um#Iy;z*6PxV=yc;) z%Fm^foQEz}Ci7Zbg*oiThb}6z?A9}UVW_UpoN~h>#hux_;W$ty39xyEJ9&-Wi$9h2 zswokR#cI^gH|Y2F9c+^~|lVsiq2F|41s1Kw+HVj;*4an?z2;V*(;nf|o>!Z|~1qW^R+<~{jVAT>vR`E{omDLG)?he!NX%tqn%%gFmQ-fsBNQMs}M zp~3;NYwliMlgfynA&b}`rc?7(Uqyb8ael~W`CfhP;JcC^hl~kth^9w;<4<#X&dFv= zn+2{{xnz7pXg;Ig?0oJS|744gUCD9FAer(T1SkB@Maln-Okw#K2{|R~4+(j&%U|$Q zp<20&SU>$LmSPZ5rJ9mb4VCshM#ET)tcy!ZkBP8STa{a#H@$6ztNli)I*|ovH~6aa zvqVD&$EW#y3J?Yvq*4qwUphdk;?c<*(DmNp1}=#H=CEhZG1+<(j+?yRd_IWI;pe(% z_3@Uu>Vw^r7DgQ%ZBRg(Fv$P2HzJHv#@G+7S~p)y2XgM#hMK~RDJLC(*vPb+>b2!Y zS`&X+e!xMNAMqpvIypLAE%qATiP`bmU{l&LR=Oz+V$8en?cPI@{oJer>ZAu6?dvF@ z?Gxz#bxhOQi%EmH=Zn78tvC}bpVXx06B0<%X?_XKt@`>MQx$Ju4CMGZ0@cH^fqS6x z*QYg}5!50F!2!!4kgQQ1;L75Q*3jBC^;8&uI9GH;wl6t0^XBsq6Jyj@M&9FlzV-)N zgLupK9&!T+;&!{P=uOuE;F^Q#x{#<)E~SNw2Uw$}4(u(T+x$!)tW!rF4qC*`q}6(b zt3_g=*+sdH)kXzW#=uD5g#2=~+w8RoPShthl`=AGid=xL8oQ=YhYAw=nRi$U8A^Tb z;j&@cqvTzBpLPgAlto41rU`^gZ=p=e%p!&>R;dje$@xM3xN6adX`XZ}!`dhfHi*zwPS$hkt@Yd5zTRs{YR+ymj4O)E*UFnhfZx<+Y-tZA{voxj>}O6R-7Ftsgrnhu{4C^t`Zig zvRWQ6d#|@5GKb2_53zohu5N2t{y44UP=}8Ib-w_4rIG^?_*+6Xrl_GQ{RIsT;{2y? z`bDdQW?oy(Z+F@sxiJJkRU4i$=f~k1DWa#A4k4^X-mJW6;YCjhR?*?tE=-i|t=)$u zJ2#~|;`Mu+MF<2Y0PwVu{K2vlT2;s#Bqr%S;(Ys`accY0EN{!@t|0yO?$;RTYlW`z zrJ9gSu9`SkL)TTjU3tC)Zk~)?LVgpOXX97}h#?nW!5`m1l?5n`tPn@9?jqJ-U_%Fv z0qwu^)>icElg2vgVZ5eS-sSp_B^^h`VpefBbbHPZ0Tfs~V4lZfWiICD^qB zV+Fm6T;V%h!X)<046J)leT5-VT%dfZw`jGiGEv}!vb4f zihJigKR3+FhPJq?w96`C$fB1fAExv^m*+-c&OU9g+~KDa7VHCz=jvjN}x zf0lNLKOWW1VgyJXLEIM!)sy)j?n~>!^PuWt+h+DTcZM;K$21C%)a{WqVz!~2n{fLu z=xua{6)<~{J>jjYz$XtWQq&BYx&`!S_3DSwchFT+A}1|}DNjd@o_mDxLFyP0Ccj?v zW_{v;V0iv0{xm8>IdGk_e-jDv0O!= zQdfoMYC7$R;xvl}bPa?k6AVhnUi4n7_NlfTlQkm|$^guz<=2BKM2(Vy-lC2m=4o@6 zkkO_o$?h?QyS`*HhMQy$P8?uBO#lOLw(MR{r@qp1~NY2OMxiz?A=5N36 zva&|r?}AYJlnavCTfgInMV6wPQ{6lmsAB$-c*zv%K}WkcFj;r1&GXG;R~iEesQ%}r zHecUj>EJdgq#Z%9N>a*Z0`r9T;2&p-+Fm!qa$h|;Y;|VjLQV$qI^Qx!-p8am-Sb3G za$~Dn)VAmT(S&DA7?d;vDV-00>-44kU$kWYN!J2Z*!~Iemt>OPI8|1}__B&M{lO5u z3`}@{WM0^|;f##ce`W+;o&xRS#~*-y#fENc%Y8`J+w_~sYS!G_ra)=YBj9`D`aiv^ zDKGy}I_s%BAf@w;Dv0V=-fhnBbKXbE?{nFQ$?tR0r{R}CbSy#L_`K6MM@O9Gl?0h& zOSh)gOBV@BNfUI)Nf=4$AXdLy5@M2k5b99H#@M zlbpR3CU2Vqat#$&@};;#uf795CBkNH%D(sQERQLRpb7 z8|qDNzni5y2EMuUqbw77WTqE}2dBJqz{vGTY#IHMpJVUPgB|t4-%zY}DNMDSYF4hc znXZq_RVA@%&TFUXh>7B)<}kZY_o>sIdpO|)VIH|qDaAAGmKNlvX-Dq$6hPI6bwL%j z*_qi=P-|hWLA`^Xz_Rdz%Tj92_Xuh$Q>oVI^v23=%qF(0@8YEBjzhChkyBT> zc9lX1W@{XuZFOZ3u~pcPS>t`fV4{UJEh*krK5ZDMNmWgc9hv!WlCTygm}~D4o0FmA zUe!ee8G8GwFq7Ub=RFRIQUvwaq)sHQqz)`z7fjZi!Ryyfj+;d*uGQ2Tsvi0pQ5TQ$ zVB|cH31CeReCggPfjKe2GPYM;tSo z!`Y3tJ@wAgGU7PXR&G$F(IcO4w&X_3Ycg<1lL*Dub}>!I$oaxHGIkKLBRJq8-q-Ey zAgfP-9=6Z?S$= zlqOABTLfmU?ian4R;q--v}}m`Jgs_~@T3Ikb-4(W798!?pdsP2+2YtF0n)lLvY(LR zbvE?A1j(}tnA%PJkybFB0UFMDOGP#6MpI+w(qSDzOJr^jRbbx8RWj!^)RyL^ZY}*; zeY7ZT!oWv1MAH@SQiRGVpyKc6Q#MuVT|!`aS{oKdDFspPBS>SD9f+RbL8(N3pd9=`6&l%4e88s(Iv8Y<1ij_7w-BB2iWff?Ki8#?y@67r!G_F zZW(udw~Lv!ziI!zD0Pev4|7Zb(*Y!H8Cb5SkEBWU(;5{n7?)Q2nGlrKtBSD^Ds~V4 z#1%SP%a$bh8X{KTQF;JXvWGaCby&}=O8s*JQYo$G`Cag2Cvnu5?v0ib zI{5u4sjlm#c$l65;>t0|UFrDq>g|wi+Ui*F?<$qRWAdQxW z08c44__P3Pg++n}p3a^6S^O-VhjLdoNIPZEp~{tnv`PG_e`ik80$E($I*Qs_7=5S& zm7cz@P$oY#b!-W<3-$1{YNu*lpEJGORPfy^KT*jpFmcH{y|<7rA*jdPpRe$=p{#H2y~lHz7Pmi~_kToc`8TLG-uB z{f{E$e=RM?RKHZG5r6~(r(mIR5u@7BXCoaHC1<9-Wg_3Xsl-q|Ue)43eWwje6H9)A zdXm6as7eeuS=OKAdd!;ol{L5K{q61f4g7l~w?ty!{_=sxaLrK7l%XS@!!lk2n3JaL zgdRG)SRx4vEQTtYV#eU7B({W_xHL5co#pxJNFXHm7`$oTX}h}^5gi9C*p%EWRlPoi zK3yD@9g{51f(oDr;7rXAP5XH#*mbs*tTHwc-&I7*gu@PVl|UxIYa#NRYuaWT^6f7X zj%yJC&t4|i6o?k{8J%duL&io%2rM&vfqQ+|HF`!rdhL(IXMGK>C|}zez6kJpz72Zj zvKCJibbT^s zQUy@DB8%tREe-k;M1u1XfS0bor|0Ex_iaKSw&R{k&C41o)Vrn=$VP2`Jl;&69g|mL z?zq}V_xgOMnS0KsO)m+%?Yr-l(vlSJDV)rjBg-#_b_Z9lfX1{=$wST|lmGC+8oM<% zcl{OM2@qRgnu(ghnvt1(k3M7v8^`tok0E3hw%)~?mS(hAlxOpKKEC7{9s|t+7Xmo) zAInr?E%V34d->>-nzK!S6U!uY`3xHDOio%c@y?U#hd@L6^Ytw|zVir28$hCuss&>s z5+uZiA9lRCdA+&E++a+=Ro`5F_&^_tRDNQIPze5&Q3@Y(aO8nfDVa!xxm>oPxPC#) z7`43ehVgb7%4IlJV!vjtYATD1^bK~!r0AQPK9_dEucPuSKh}ABg$YXLiLaU6P|0Lob2-J-a; z%I>919xwu+TBOH{5Bni@c5+7G`_QoXqK`FFtC;JVpgw=y3p;`|ycHiqQXCfVR&J}-a-uxdQ_tJ)L zh9dTMe;5-WV7c=@J)FTxax!2_XngLkSea~HwX5~e+Qp)QcJV|(7(W`AfJsv)$fSGY zOoVUAZ^4y8K{>YXq{kYfk8;W}a9g{Z?pHiV>tmi*Um2@Dta8pP%_%L_RPA+tAT}3N zArAKW*v6zv^;tRc5WmL;p2CP?oU<0+ndNx9Y(&Tfkwnx%2sxMsF7)F_JGO!cs<0!?>zG6SF>kU%d(BCg5@pa_MGC+522}5(so=3qFw(d2C(@b& zoVIdAo(PhJBl1eCv&znZMp3s|G|zKGbA)3iA)(R=j{NM{++VOIA&V%_8ErwZLwbYX z`D2O{R5`6rp_X1rCxO|9(q=a@BNP+tLp0_H2Y=BUHqZNtBtU0_`?t6D|2RJQ-_J%G z1UJ)gGIVhG*DCl9ZBGN$1LW+H2E#j{50UF3)q;{>#OXFDh+@zm!TfW=41Pc}w%HpR zMpr|9Zokq&Q)nN^OSYXUC7Y#LP%nQdUfgW>Sszl-f#khB+3x!y>1G~nuqJTpedT+_ zaW}!S<-@x5Sj+hBX}s=(Vqi2zUKy4`P3{n&6`_ND-KejEop%d>xqK~*t8da5ZtD?- zz}T>3%HF;X&{)2f#??3NE3@^8M5t=mab<7c1aK^04;m`MVY(Nn1_II8ifcSn?AMU1D`5C{ag-n@5C4H4R*sqz+5(j55jU|IX7#L*l_~TVpW-$s*PNhrnAyf2z#U6~2)r-+Z}=%DJhLaTK1`WzNDUUlGBNj|xfMRLf%yaDBhY9l(4_KaCn_z(fQxi9~?F4O2<2uJ)aFjS#&|giGU-= z!T`)|K8UY?ue2VqS5YK19x!)kKEYS|gCnq=KuT6s>hfTBEWf^MP>mL4TM}Ru9PaS3 zPdGTdMrp?qUZz@hM;Wi88s@E_Iu)HRqzS#U9~G^!`vq1bH`5T%tDaHeK;K?gjSxc3 zi@&Mp0f|9lBT-NdPD!PUL`b#%osxzZj*|L3pV$Diiw{=@Eh{h)mZLVnxkhk@qMmv^ z!Wi9(Q#u!EQ?(gig6e)x3Ksm8WLR66GuRMAq@@@CSP?7u8|^wwmqv5=!v#PC{b%jY z(T+o~_peJc53wEojIwKtjJ#`hbuS)VKdANs2LW}ec3c_s%|I3O&5%8`&FDEa{_k_> z{GkJ|NZr<-IK_8FuDL%>YCJK$YVMG7BKKTgKe=(Q6Mv%$RosVu3aEh=wtO8_Ix9U{ zWb-BHWvUa~@@HCYb)Wyj-G+O5$_E`BMIaE4fvXY7LF0H^pyIfT%g z>C^3)c@t5FCV!i2dZS9&PwO7~1>21^7k3v!CueWAb2z))L#41o&H1A${k#XoHA$=R z<}Y398;ht9!2$DZJyt(zo~1~?XjDxHI{LfVuen}&0`D8ik7wM^vfO;3m?^>{iU}55 ze|rr+&M2mA^5j3_8`>5TL{$UTx7_%kv~{K@>bX#C!wKg5_&K71RdTt4(HIdgX082? z5MVV_v=w#_V#{!IEOon;X|cg*2xr(_4 zH{&==@ayHB%B&tLtQA4_S}}FW?AgH~L!al!ggA?(oUMYra6ASZ#>A}yRm=eRSue4I ztUfZh)zJB;4${TFrA+LOFC0AX*I=%QQr)^4i>gTM#7aneT(#jU?%XPS2}vJ+kD+zCz&RX7I-Y0s*omEKie3cuoy1(0K<B}= zY4)77nb<~pcp{}zgO%7)U~ri2JEr7EDQxK)leFh0DmHBI+Y2d|razgya)n z)4W68+qC1!ui<|iZPSlPG2M-@%Q)oE5oge)MJuR`a;cVn7MYFbEIG_69OHyEIVj-L zaUn>yu;Y146q(DkY`YX-P#f~)wiqpFi*QZyUCU}qs=hQ8ca8rVF1lzTl&uk+Pn|Uz zLGH_mJuMYNaC~Du6Y&yY9Bi&bu@Po^yQOWT>_L{Eu$LU6t9qbmDLJMU~#nM;^k zzkoTvGzhlg)f_07>)e>%&URpZ>@e5ckehS4ZEt@fk-RLyI?-5YyOLTlff(d78)#sU zG|+38X~p82*;E4P31gDI^VpRaqRJDj2C+&Jy6 z`bRgdmXj`x0=*n z)V^!$8jk(P`qNFSGWg&xUD;cJguQ~0k*sTO`WYSvG*v7y0vkxaC0w3gVbjut69 znK@mPTZkW5kiA6gAP}NOp=GZ<=$CfyYun4oiMaD2eAgb%1~Sqo8A8-YNfu->h8Z3} z!G}OJQi4qq%)4u4Q6mBZ4vWoL%G-9##<;bIIydIk z6e1|e%AtgLRz?gvC~bHpA|z?TEk16jVmi))uFhwiXx5eu(0KHMUIUr}Id~G0EBo;a zqMY(;OD2W(-OJB@jiwD7c!*;mgjd)=CWZ^_)X5a=sVQhR>Z^E^C}Tx#D?HdtZwF4c zG!PMvi+#f~29feapDSq~`Phn7`g5UK8J!lDE^A4#NkwX0T0RYB3xklBd|9f<=I}To z$I^!&%auF2-hNqh=6JOv#yis2E>8<@ z!YtYAYMzB2tAEWOImIv>--eV3)R{ZtaBHJl6GbiYRAr0xJ8N%1Xt?_#{f)E&vUXa}}Fa|vR6Db2ILle<7h_;m_V4YHq|w2R_nu#29m zQcI(AB=MIb!pX_NNeBoDLus`i1++*%1G(YJGd-Zq@O;21@b*I-Azl+!!Qa1bocBoa zp-sEKekaHZ3cKehA)FiuK~Rf(4ZqH!{tDv>c!de1TTh{c+m(L`q;5{(X83VNS6;5I zSk2DBRoQ)9?<};VRIQaql_25YMknwkEiVO_=!jbikITJ;sK`ngiP7{WaZXBxKu?B7 z8>w%2a#Vy#2E_v49!H5b0dpbhp&PRdXP=I;R|1|iTo%Lj*!?KC3EDuNDQwExi_9sN z2FP1vQ;*s7X^pBhfcgXv^>im`mqGRz@WnT7z0t+#H4jE*Qq1YRR0=(_qA=Jr@%Q#& zI_gVLyt-IbfzkG71g_XFBtNU$p_;W6VowjkC=N>%3tnCl9kGLcev8E6rtT@_Vbs!R zgGj+c!mO12k;9a58k(0JWft{7LRc-su=>H|-nO_^uDI32>(2g)r2FS>IqEbCXSQUS z(WdZJn-bBD|A(}9jFL24*F`%kD{V)*(l#n>+qP}9(zb1@(zb2eR;7Kjy4TwKpjY2> z@A(lUe#IEy7xSHSzIXs?t?CsfZ^`H!-%Ioh!~92i5)2t(nmN&l+anK#c3xjQl-IXp z_Z9(7UnZqu6b#cjEm>7T)%a%YPV=iFi&fK3KQx$S|@$IHCJ6ng`<|7ENGp^=o zSi1L?pZ2cV;5epeH{(iD_pnBEsTuxBf+9_Vsa?3^0*p%9;6aVTJNqQN+^tyzHrU`1 z@s+Qpmt=+47$|ya)v)ZLr|~O)!;sP?%7drr(D!ak*0`tdo3Ya^9DPY!NedgylWVI= zI6(7o8xCJOF>?Pn5WOF=c(-gC&J+}n4N`Lhh_S=*%O2v{VELw$07bpJJDMwFT>3f{ zNO$jr^3zf=N#bRWh<`c#__H1-b8FX*hxy|VV!D3l9rt^Ti(-+;W8v2!XVs4jEpZx63kBV#ViJB3UAOr(>7;6Xjl- zjbB450vTFUeE2Br+V)+to4L`H^w7oanW*{F`8UR$-P-8={cwco50uAIze^65^X>0Ge5MblBG|U)R{25cu^>@i*P^hnV~EHpXfupGudmROgUQsO)-nFu_9K#J$YpH zrENn-*Ucx;#0f>aHeDbnRNh6XztM%SUo~)Yn<&wf(z=9Yat)-d^7k;0tgSc~Q% z%0l_NGF1C2^%Fj#+V9wwy927FXp7XUxW*^fkNc?5pXDOwvxv*&^g{3(KmrMTgMktb zHD+;Bcca8dJqfVxOm?nEzr#6+Um3(eHhlGoh851jV`UB0-Z+JKs| zn;1)lFfBnPpqk8>q^q7IL{WM9KT=aW|zqczQ@+GAi4$}(ux#{Cd{;%zF-Mi2!>%_RmV8R0#1h)-R>O0 z(JNs3jgwv(Fx)`ftb*@eV=AI{tU;a>GR{B}z0R)3&IF;ug-W+H#?yI5o_MEZt)K3V zRBNLnRC&eHty^nRNCg98_Z2BlD`k}+t066u;j5uy8WbSEH}T*WyRVr(>4pcUg`;RT zn|+Oi(3$lkeUNp)dWk0c{_ zLx+|@BTUG>Mqw*#>AF(MJz>{GlTmuIYJONn!ub`&a+^qTe!AiPzF*C;ju^Ev|28)g zTUjPoy>eC%4BfEuGRK~dm!GaTl>DksV!R-OfYc~$+K6*NX3U9kc273#;xPu6zW(%r zEklkphbE1QLDbcat*#H$6OPp-T)@+Ktzer@Xop*~p;VLMsJgRns9d4D_nH_Y1Bx(TPTH+%DB}Y7r9+t z?X`MlovYEnqQ+6lK0Y9|ez|CZ&mEA#t@FYp)4pgH66wJGNU&xOS$NP=Y@!O`iB>_R zdsa{iC!Q?i&lRR&Nn-80x{8&zz5!aSh8(%Dpm5}AQGK;zQ!y~Uv*LVF_im%^SfHee za%^3timX*fsD@IwLHKf>irmV)sX_*`D3#Ra&ovtr(5^k}NF#!=mz-}Cxh#n4rcIOM zY%7-ZK6rJC*WNqXW8&3GI4XQ=PTvksajOc?AziwKG^5vm9Qo}6nT>v}&H+nDg*IT= z`0H)j>d_vwnhdF}vW`L1F{2|;6i89N46&rQh|!>!#djxzB{{ctntW+#n~^zWBe5%0 z7FC~7`DFZR;Xm;uJbZd2n(#}zHbnPcB_=_JPU(F(-P9r2H2d!GMu8bV_|L#%zWee; z1+-!#j}IKJ2gD9J=3fxqprW9TI3Of-(|$}ui`np6tja;GLsFDkjvN-*WX1PX1AX?4 zUs*Sj?sdZzmi?Aeu#;1r>%MA#B2KncO?%mTjwV!-PVP zrdA+*3_im`IU96Op`A~W`}E8U#2_q!41k?@qQnD9+G?G()&d=FFTUcc@fbj3=X(E> zkY#VLyr}qz76JQ5w20{c$p!Y`JYOKTbPb=oXk|}^brV{ z%}YRVkW`FqKxV-fR1kZ2CcU2y8woID3pD6BN7t8~sFxZ-ip9O7UUYv#TJ2|sfX0T*@CNyto?7BUoKwoX(cb*Dpk)PxBV$0T#xtW z7#hqQC+YLeH_Ef6TI{f*LUo1~EtUI%9Q`I){3!_)IXen+(E~dJrQ1f5uE4WeF9LY^ zHGUeBlr|yR74=Wy((D0;vgdZ}!aKfG=Q0bv)TkI7<@8Hl_~%bC4655-16_!?1Ve){ zcd`sNA1qV%lUy?U(}WewtxNy%n`&LFc3En`4qKrkS$E7KrN6D0MM0LQ#f{b*iSx~>`TaoPEX^e8UW+VllqG@9m2SFBD zh0f|`UG{D;53&R&$&m{_<0gzI$&n$nQ?}tq7JjE+wkO4u@)X70sQF`IS`)88Y6`Yq zj0wGPr+<~;D@QZ;rE|3we3+0;&|Rtu=U!G~Q4w>rYEN&sqk&LfX80JA$0%)TW`|z> zC9gQmuERbGqanJIA@B&|uLMU@p=UZLDt+7l6)ZjaI(zix&=@U#T?p?Xz5F$+*x5p1 zMgFk60T?D!Ec!W->P?w~d{j}Kc-{pFdUV3+blx)&rnru(sY_39kQSc&cR}qFBAlZ4 zUL-Y!=gDfe_DGp3|E`>06u0kwbGHPy+%yTw)5;hc{S9)4C{Hs47T!~idXqzuQ`7-S z)_HfnRA^~temI#LrxR)D9ve9u*N$E(KjS{pF+ z+h5pgU5>LpKipydq5nwdEBBRxKGA@3)P<0MChyxWJGvoy8U2C*N6fXtzym!a`YHAi zPbsi0G?a*QHk98NE2trIliou}zDEh()*nA*2t>$bz6#ii^L`J^m5D*1#SnxOQdsN$ zPHvYBPfMmF)};ybpfI-W6ACg`s)Bq4uU-6zx=S7kkiQ0KmplS#Gx*0fT-qk(d}m7E zoOZq|FHu)to*96xa+!txKBQ50RG3+@R4B^a+fXCEm*d_berQ>;O#)QOa9c1T3}3`g zbykt`5hio5(Ype9mR52jNQq=S(csjqPEemnc{Cxf?2L6m9M@u~K3rXxZV^-N`0S1* zhSU^j3_W^fgwAJwPh2E5@wg8Ag~8zvCJgQE1k_pi4kj+ST1o%X0#&Y#DwD9itRx9b z78$+L`$~zJW)93xgLJ%VA1xMIZ25j2QkzjAWTuu5Xjfoq!CbK8kWL`nh%L}34m6Q; zGtnxMbA1=e9R(+TbjrJj^tte9iJ9^jm*5dAJFpuQNg-3&0ej@Rn@8YTj$dNI~nBr$9n(A&OM&vkaVh0*sy9|j2#ccy`t5%NfU#0EhM66Z5GH;(6QipFYe~11C*(kJLhF z*Lc{xG4DcEx6+fLDt_ZCilvIg&4~-Yc<;aVUIv@>bJNX!V|_-Xd-jwYgsI@!Y@Px) zZN8TP4n>8Vzb;9Hy3DKcv7$nP>t8)R`l$B_W>y=#>t(P!Lpr!JWVqwU2ota)y+O-` z@vmd6#3^KWIg(sNkgf$@GgunjzlGAsh3?Ea4w9ImfR>qszUdA(-uTml8lYYWDfPK| z3qLR7|Garw|Nm{?zcy`#ocbro2l<^iY26@h!CbGb0%JKyW2z$fn*bycDXA=EA}TJ) zsZ3!xyDkm>J27r~;}0NDcH%-NiBTmf zQ&c7h{Cq(Ia|7Z788_62b~F)?9#sS&W-*LHi*;hVe=1^y{J_(!IH!v*tG~zf8pvRY z&8!th!70g3iT9?rP+ZL!<-BMbx$(yo-DiIhuqOME_=;DMgE4nxdYC z1Z>ktMZ##?_M_1r;J)EZq685KV#G{jE_bbBR}g5awbdP4PZmh9K6GtRzVz+iR)g#{DyQX zAUnWCd`^qs{&8CT*;e@9wf_It&cS~tUVp81g1n{ltjuSEMb>IT@iT{%bR-PrK`f=1 zPzRc2+T{M1E@`*5bA7j-sqrepZCR(U_Rq&yk>HqbAvD^I*PqazMxtao?kVEKV^+@# z`%HJMmzT+;sz2xdD(ZY?oazhh2Qtenc2`llT^U*`IF@6z zdSZ=9%G!`4Hxah)9tB0gSS!+B(%NFX3B7QS-f-EG8xsSxYpAJ$1LH^_G8id%lX%tE zcJ-Qoy>1fekGHTKhlQKfZ8v|Tne$R7oZFC&98krrZL2wG82qjuiB(?{S}PbiQv|^3 zUHcp&kXsBY*+UbMC?S;TbY$Nnk!V%4;KM^1X*M|K4rM;2{y5h`#Nq_6#QC{Zo6jP; z%z{U-cLr}82zNVV6QD;*Dh?A$reP7cIkltJ14ii?oa{ftmAkz2Oe3&J&po}_#dE;$ z>?~UD<8lwi*9sW(5E3)2^Zf}c<60g` zq8ntJc-WTH^VJtxjoKr>G;#>K`mEU?z?)%4leUw|Y3J~Xbb%mV{z$He>yK!ZQDxgr zxf!2)GIGNCxLb$&V`4`L*NCaDNSC)aH;@R12xgJ7ttzKHCpV`wr%+e17wm1yrUsWy zjtw6$o3m3FNvZ`CDT+;(K1KIK-jErWwy?r{DL-ScUrci^v>j^YEt*@(aOsP zSzJcZipw5PX<+EpS+*0XSKLJ2aO^NevST@;1EyDwkf|F zXc@$W*`_-(w;E;WuR-ZMWoVXvt?Rc8EH;IbdZ0r%k_?RdC?sEMTc*4!)Lvr{OoI-b zUH7XYx;LZhY-rzWNUtzLBVT5ag60?A?9yRm>!Wf@{6OdDD-~3q3*dk7o4JAuxiv9W zz3XeJ7e-n$L7+YeX=xRQ!#t6Z+wO2JJ970AZVMy&H(U!}~^quV_K=zYvk1F*RT z?yyE{IIrqJc}rAI(elD_Ia?`C-DVRHw;*vAAx`#dAjH0NzQ{p>wr$MbmtnzuILa6a z&=(a{2eDuI)%F!dkkJkSO&@SeW)-O(GhU|=-~F~I5G8tEoO{o1YQ{IiO<9b{|^{@m#g|Coq; z`@7vPVQOWlYxjTTKjW1();8Cd`}g)ULZE-+jYjrAd81i;&<5B^=bpV$EmgEsLr%xxsC(!7*!#hw5pnHiQrw4!Cl(LEyV9qM z%O|6k9jyzDoqhbdWWmAd|BvH~gR4MQjUuFum`)eSi1`gixGkED#j}2L` zT_6f{5?#50O8?JPT^N4$UbR3+Xl&?Dyi@JwUSW$Z{X|dS7bcHTFPN+x*zqFCiUJ2$`_xv_o3@I|3k*t@Il7cP+`Vb zpK}bU9a$Lf9atFeZ6}ZzlaV!SNw!3NisEoK)kzza_WphI2Y;=2GRAI)8SM>h1u#;_Op08lYF;u&Awt(^gWpetfWlgg2OZo z&&F+|TdmQfhB(k^R1wD7PVQ5>l8)IioA{evRZh@2)JWc68vK_usLQnGk_&ZmpfoyK z0IDqDkuy}>MzZ1%M}HCnt8#tlfv4(eloK5|G!eL0l8Tb9WOpYSj1KrK6Ux z3Y!8RN9Hl6BzM3>NWF``@xP$PGh!)PuNcQq?pQa{8EXCDRxgg_cLu<33XMMcb>I zq(33SaHMw<SJBs=E=-7OWbrWJ?Aw zeW~}I?_uOb9r)C(nF6rC5GhsCTF$DMWVfx0qC-yXcQ?|lZkjixErs~rOaKDJJ>Vb@ zka25dH#8%dWKoJW*Ui5>SBA}WGz<>6i0ZcwUl1RcD0t~Ymc~wWou{SGSC-A*I}5$9 zrdqIBHcby~2&mu8C|KvTDQc&hiISw62$C(L%gE^7!nx#YnL^K4G4TJYGc6kOUYTj9 z&Hy0IMkbsv&fvt%M3$+eayuTaXP)J?a&gv91M-9RJp(5~jf^xm_81xL)#55PD%0xd zs#p{DBUDFQR_H9n##*}bw>xoBB*-X%&*@&?Se~6>t2p#98$kkyIvXy%a?h81cEuKl z45(riM-{V8SVkd@P%C16Rr@JK%D}5|Eoplpri-Pvg)mD{dc2Zd)pL;F&&A?>soH(} z1<6iJW-xChvNoru2|vh557G6}_L=oJcn?0iAj4K8oa=S3P=kuwcj)LXr5iSIQg?wb z<;{38Y6~+a>3HHKYDl9F7owA_cUqCeZb{6?lHLB+Aq)edYM&&ve{etK#SB? zx(lMo{`g8;NSTStT?!UWbdRRmJ$}0d>p*l3&hWw_&Ch&t*OR1j|SS;X_Mr@fAY;D77E)j)oBAgb4AD#lfdV zMywmJ+A(uz*eDo1qgNs1Ro10jDP$b-5PWuDTRVaSOb+plWG;_{;-QdMl^J;}kTq zQ|ObMGYTI#3F@tpI(F+m{8N#7C-4}^>5xXLS!Jm2gD6nbfuJo}BOk9I)GNSfP`^Oa zfL9NpeVo4YtGKoJA>_q*CL{!J+0E(A$D1DsM&p>ay^760!edBB8L&`6YW4U5H8T}Y zoV9Gq+`+_G4BA|TxU3_jBXJ8g=cWx8v3=i>Poo8fh;19z2d;YmQ}cKGefwwj=bi5H zk6nJk|JR%Eulh0|S`xD7(=T@O5$&U*$u_SH-%;O|stMt1&Q=dnA0i_LqD5Yi{WcIs zJU{?hmK>M?`f_(%zq0WLqzhmfh{2zc(>+Ca>#G|-pm9yhEEcjPme(7A0?|mE0{`53 zD`M6im%zfJgJEzKvlcdt>o)<$>GsqAWCSZtWvl?vzs3U;--X?4CuheEiF%RU_z>S{ zlLWd3X!CHz#9QLX%ZDbaaDBw#9ueHpe8W5>8n~If3HK1nA64u}8qSiwlB@&d=TBkx z*?*@5(e{=`gP*Vcn{D%-Q-Z(sF8r^A=f50Z{`z+!VkE^sT`vPSODyy5%DfafA@uxj z^+b4S+Fh#UNN;7Xwk*>m%uk$<;uNN`4!VEqb^?#I3Ua;|mts zlH-((4UfuoswQ9WDYHA$UA{-f*wQ5&2^OlQ@As&~oS6|gqzWPc$-rfQpd$0upqE|+1{uXG#DPX-25sFlqUdrG>VGa6m1q@YW1;|;xIOy+f82Bx{SFx(|O;jp*#RRKQB z=&_n;_G%+O!GaPm;`lsDQ9+^V5b$}~Sh5O;NGz3r2mfbl68{;Cfq*s5<)rnj#I^pD z%Ct$ji8T$$C$*b+$(UB&Q#}WC0ji$s&A^Ed0hYI*T}5{%`3cdbm%UDMfoEW<4TSF`xP)GFB0In3!Y?hp zM^y7H1+g1nq`Oa?t$nsTCbj>_?55I>=sj3ESlV9NH?d;@g-l+|G1CKuBFsSLqK8fT2`d5*Y76&7?(6oxtn0n?2Te#D~ws1@pO;M-kq@M0f0e;_5w+EFy;v zDq(@h|4tk;1s3?r2e(+HfCeB2#!A|-(_x^q$FM0IRs?a)t0^X<-Gr_S6BlJYXy z8@_CBD&G@i6rJN1@S!PlV1-C{B9z{)&5?y%Y!WO{gVjbM%IeHU%3n|hNWL&aB8#ie zlwi%)ag3Buu+~2U!*PEMt`(#)5?8v{%`TU+`|f-3E?3lC(sp(?rOq9b%{v;h#Ji@x zjXwX$Xhc%RDl7e5^1Xjta^nB6DFaH{4v`jYn-N!?7j-f@wdT2oW)REsm>EvJb4A?oDVQ@1O14H{wc7|tW;Um}oc z>2fGRl_j^d^VWDWUU!pL`ihiS?$kQGG2_JqG{f0(IzKiI9p45+_ zd{r5q6%%cXk{Es(JlX`LJ|HPzu;xqLeJX!Pr@a)O8tCuwae@PJ^IGQaOrbAETMFpy ze$QD~Te6bI#ra9K4k+9v$A}C{C`KeCk>zuo1-}QBGFO%WuV~?utMn{T_A^tZKG@#) zC4Q3JZ4F6F?=W>EexhI%4(dUu^K@%*Qsz2q;RSE^O=vT@vr{!>z>2HoXB^VbEG*~LX zu-nl)T=u0I9T4OVTX|@3BUv`t8(HC4b+Hm;ObHfX998BO>&wP{b?}jl$VP#;hmvr{{dnS~&(NIk*ASOqqJ`Njf)>ohi#F0#?pvkSKp4JBq;O8XDq~S}A zGUIY=dgSC3f)?b^rE_g)pp}{{Vv?!QL`a{zCt8LMq6g@v-E!pKPBY!rFh9}kCvu5F z9k<4PDe9!ZrW3*~rJz;zu$mqUa%lALkRfV00I`Y@dkl6^GTVc-HU8PVO)ls4qlX6( zs*>&~N&W~CY<%>vX>{q>CZWzDGZ`wLM0S|U+u!)m=8UNAo{-z0Pg^q`F{ zap)Fb{>km}F}-rlxIV-c@^7!6QqWkp!ccS}e zCh*_a!f&3tu7SZP(cRR__}@hL2zkxVxd^}u7V}hzJq3w48{8xV^OKlJZwUMvtYP^r zGWl#Iz1~>mY^A_l^h#Nj?}tdPmg{yDeMdnY0ANOwX`lJfHtE_a<^B5l3exo@Q;?2q zJH}rSB8rGolvLCOf4jv0lT{3n0Kf;_=~4!oHxcOn;cVz{BQ6BkrPmXAvRMs+K&O|3 zq))!btI&b1g@oi9+x6iCQUsyk((I4-Wm+=^m)=({0t7NFpHOe^g2bt_2eh057Nk<} z;)gp7MU3BBRxK_d$El2I(%#QqysQhV^p$ssX-pbS5x-lMP2Fp3FbYaP^e{+hvJhr{ zgsojuE}De(*bi>MSbyP*jqyS(rBa)g=%rs%3=}D7gy^hu?50&rZm%S7pM)YP0i6famnY!+}SBB0FB zjb=XJ;@{Eh9YUT{+O52QPuN|84S zW{Qj%!{%e{jv!QrwZ4^j;@9Bro5Km*4+Tk za5yY6C8h*Dfu+Bf9c7-m^oL#NpAAH=%g0;J&x8f_A6eFaE5q==MW>Oe#iyH!l(oaZ zqtj7d{dfNQUWlBSNCkongmP75M;2f3Ef`gl$Cu9ZG(azx#;8T&bjQJ|cAaR8`&v7g zOxA;z<0UWLcy$1ZD(#Gw;qW=b{^0fX?hWh*G&7?C$|?h~^f1A|%9Q0rXnBTaV6P#`Va0wAEjIYXtPD#ux_oww z1xo5>Q&zc|b*K0zg<=lpVXvmRw^gC^!daega=YIJ4Nl-!l%!hj z;DA5~evd^64)L=k-TNc0ZPggOU(~%*@{6oy&J+0-KmK$PnQF{r-O2QcFycG#_yi#$$+I8P6(^?xt)4{*%HA<>CI#pag z7{-)t$yw)ub6x2v=J)$GI!H2M#MP`*&2jH_hh-we?F{>IfA-o0zoLxk65ntdAs(E+ z9i#Jr)FS60>m%C+l=*wCi9Wh!#$?^$vhM z^wtMt!FQWH?XhFyVj`_dCvvOLe);#d-wjaFBUY6}Z+; zjhsp{Y`%usIlIIO(y#5VW57WueULb47S@>wjaZ)iVE9<8fFC2;_$HUrw6kH9hYdOg z=S*<7JWOg5G#re}JYGs~g!|Rg$dYVN#^17(R~|(Nk86)BasojJI$mZ!vs=}T_XK6G z#V5f{;GQ}$&p>a$U~56(uxvYgm@ZYzgXiKM?fB=gC*b+dvuT%-Sj*zGpc4GYg6jWf zVgE%%h>+X-Jg4B_?^#sbG%U%osor_6e>4;)Jr^Q|$hubdCnF+djybLOY}+kTx)VRr zJmEtG7Kc3V`FY3NRxUn%W4exgGdfILwsx{QJgl132CAFdQ#oiFMzATjDYq-PonyrS zX(C1IlD*!_-qXcEGwPx}%?oXYM)jcFu8BHLey{~r1`wY_Bte(m-TSyu@KNowU%6iL zh`waoX6mT`l0?4T+}+0@tNE*&Qd-9MRp4>9<2fP=-{bYp4Vdh5Noi38%CU#eJGm{% z8wuYs(k0k}7*8J4BeUyGMQ#+c_rm)vO3D$;E5(CsLN&S`RgtGZ44%Fk-d z)-&=}G(Z?wdZ?hBB64)3=|#j%;ymM{uYLUl}!gYeJN|66H3xEMD5-#d6WnBiI?UKm; zA)ZW7D5Mo*MEJ5K8n}`}bY;KOF>rHNVv0qna2infE6O)$G*0 zy*_E1ZhRb!kENwNEOwW>QIf1THOnYmFgNonTw$uOhSfSN>$T8O3VNdDwArq}QP^h1X+n1hRCTEgBGrYP z=wSe4_J5AhX#M(ae(3t+jo7gAhvQ8|>i`T{`l9tj7!1%Wby#|A;D59!)|YzIKo z=(*7KiUTV3T*!Jy0a_DRkb%l{ZK>P!&{g^_oV}QUjiD>z!0!xg0o#^Z3cCqfbKm;l zule`~!q~8kTq4kL5?!x6Xb!egpkL54&jmSSwZ{7AwZ?jRu@|=n*(&>Ov8@Mj9W@5C zDhHh%+JLjBuBAK#2MHeGdquImyR!iA<-4-48iV>x@`K!0uGmq#d2E?mRnj{Q(A+cE z)~^!EUHmO}+wIVua|xKZi;7HKWn~!Q3?A~z82nTn6=iHSg042Wz_M+1`7_rPn&Ul# z4osL?GxJy;n#y;Nv6ZM>LYh>4%8vEHhGEx+pffYUm-Mgx+eIyAJ!lT|;kYMe=r{|C zV5>WWFfAuL&Abg&n1c@W5S>%=Mjo=thdb^qE1}j$W|$pSWm>O1%B$C=ns?jgEh|2o z$ECQNa#w)uBVB^{a3Ul~`*0V=7)i|cACr(y_sNh$V5 zy|@tSG12JBYfIpZ0KX-4N}rmI1d3RtrTmi{v~sb3A}Wz%=v6%~Cxu6)hNM4~5qdG= zOd6?}zUVNH%c`hmCP2-fn2Myt8xDxO(s!?9Mhx_z4jQ_$s!`v@10^jrX?L&FcO6gh zz>(oD)FF!t?zqLMu<>FLaxK^LM8o}ThAIGZCnODLp|~6)K67L@$SZO=By3IoKm+d+ zBh0Otrvh1cGB{-xZ;KUeVZg}S0f-Tg*)SR?+g+cD8(e$2vqSbfX+fioilM#4S#| z->XHzg~7n!0cnYFXDDVVZF{rH$N+*wVwWCHG+M_I9B>!UJ4I!;;&AL?mN zeY)>go~ofaOu6|o{%CrCsu zg#Sh0?30@^?chC8<2rqtZ+Fbe)1AF6dzhO~+eoU6oR}GVJ~E((MwAD!^MDJX8u|R( z({@R%qOhUtB>_-GX~d4eO<&TG?rGVH7Y;G#sb0>g5fYt=X5Llu9t%|u zkyyEM{EDyHVI!QwF@^n09f;d=#fH#R_zrY3BO>qbt15h%JHX@C?Mrx$n$eX!z= z?+7+rDcf8=%RGRJ}@wy1@<+RxjqU0x;TgnIo^ zQE#~p@I`EfXk^-6IHSVr~0oGy0v2&Wj4I4x`uO1jKKhE zkps}H>@nQ*^5ExhW(7JfHdzMUprV%bW163m)VrF# z^<2aL->hilRv7~;5S91msU1g(3gr?Y&I~Jwl)gf3=fz|< zv;rNz0`FHfFNL(mpy2sz~94w%~Ng%JNK1S9GLP(bJGY(VEb*zf8V zi8IO79rBMqzI_gk{_{8&=O3!uf6Z{?w14+v`X_m8!I2@sL%)Tx%u$gWJ}rW2q3~1w zn)<$J)L1*-McZm>Ecra|aRvOC6Wllpigd)xah7Q?ak;VVE`5Kwe#!j@pU0ld=yfK) z&6%A7E0@hzET%}#z|f|7=F5=0INDe_Y+4J@dLx=dp@rQ!01A7?%JX+a8bc?gvxayA zXY+LTZ3Sf^A<0|K6(Vo8;x~h0&~fu8@r8)GjAum$$u-sUYlPkj6&sgsMP?JJsY8gV zCWys>>s!SvU)gC_D^F3D>hUk4@|(q;QM6o-#r}+(>pc_gd zV-fl)3XLhJVM4icw(KG3iAa_v!rXEsu=CrZyNtP#j>tJ04;`>NC;7ta1}bikPw~GZ zPUQLK7$ACu97bevky`3GqQhQhM*JYiE`1@u@?q=0Bi zLa9 z@0MOh_qRv6p@pg2Z+!A!5nw3E4*BUMo1P?)r?kM#jJndKq7JFV_qDxi^jEaMF{*PM zGR}ApunQ?&HBu1(^x(mbJM#}_KoOu&1R`wz1fG%t^HiaKA)st**`9*SrZQGhslk7_ z5~98pXW2e!W@p3w)H;TiwRu2&8I9cEbeIZr|3t5*?FupTV%AypH;qjT<>Gt=R34~e*SZS|Lvsw6Po#N%|AIw=~W(h&smRh%QZyA8OAzO zF}mozAM^GIU}6M#0r$}j&b#HYf{|2ML8Tudz(A0QB5y!%vWmtLY5ejfTy0NF-x;hq zUv3sZk=52y^jC5^{5tYF;ySAWu+og>$)q~VJp}QTu(;|&=A_vcd4Z)7pqb__y875C zm{qck+d2u+=Jjw8Qf2FDk6K&JUdUf(q!o}`c1?G%!DiO8N_D>NHj+I-buZlkqm&v% z$*hg1zqXougXn8dbSIC;E!@>d_nx`)U(vhl;Bai(e%=Ye_((~!?VBF-n{0o8jXH$x zHdqrK1y|xu9q2eC=NtgI#!dT0&Yg?1ev6lCW0-_#vpKqtQwiX&gxp^|lk-2}u18=- z{l$+yqVn|IcKRfW>`lPcVSRbv z1Wll!CSPtmfPxX|z6cE^(;b9^ z=!=5OHJ@J!sY`;-)|7-3Hly^EPQg`_JE1vFK+-Ycow$>mrkC8z^!txTCC{iGr?3T^ z<3hCZ+Laf~s5-af7`z$0z-9d^d-R6@*LMNbU*sX_uC##tMJ7PjYsXIdeE6+@jNZRh zNc?=g|9QHjZ*AqEYx=2U`7ao*eN@deA-_?0gzQA18jzSvR;GOK*NyV<35$-26Pe$97|U ze0_3#Vts0SRDtb5v4G)V#^CTHbdfMB)28V3@B?H~DpLRg3q4;4C^@<~p=O&5bwR&V z?$KH40yQ!b-UL{}^m`a)VbcrKsyZv(t8}eJgNYw@jh}Ei_ZIX-2BzE$W6I{Oe79%Y z1nz{6xz!3;^|Zmq8NEzDblraOWe^oOHdJ<*fyy!ks9adxqIf5n14>T?K5C2_`kK-!DX_qP>$T(zG=eSC~+87KBAIv_5;(g7e$*4rf7z z|7A!o1&5FB26Mqt)xdXGS1;leO4iF_fWMQ*lzPDkL$3P&u=bWgb*@Rfa0u@1?(PJK z;O_43?h@SH-QAtw?h;&s6WoGJ2>Pugd%v@1W>4n4=X^h&1y!t~>gl`htNUsd_Yr`l zlJ<;`<-#B~ikHDmx4bncr!vtc1aJ31l^=amWgAfxb-R3Jni^5Y7SWdV03m3*Q&#mF z9A~vtSM@4zg`uwb8djLaIk><|hMIcl4+(0r!eCwg0(H_Nm`g5ezn5?v)sB)RVunv4 z^TkB94@F#T+5$~dm%_7uT0G|I(Mdw@_~&xCrKoL9cEFXJ{jKfzKhsMib4Mq08^ixe z>m>b=)-eH8C0kWh5eds7pdU&vTHFX5UV4{F(@E8~v?bVO8pTBXhKePc7+114$BhbwWKiEIdB!u1g= z=JA-5QE@_XQH=eUvteBF$hWIeiv}E01leVVoFeGx4s;W&u(IjT(iH?b2qFX63SOyqQ`URy2U}m~9#z@i~tJ&EDOVX*>F9>6Q z#?YbyBqMkXk@T`M(^jrEIg?ir4D{n^b;p&xCoP1tR&j0U>H|m zoS!`}BdO7}K+9!kE`u_bf0Q_@K6tLc_EQlliou}59_;;qbt)FToXu396C{XfyB7&_ zJ%_h=AMz^hMB&;LW-f7NYa6`6*L7azdbI)3_XQ@O25z_b6PNnxv#BNwvw@GUPp+H= zrot=ZD<V*{%Ik>B9gFltJ*MI^`5H8B=@OrGY!aj?f3~;JsIvYn00awwMrctO$bj z@o@-QoCwo#10qLG$cHukZD(P3^JS2FF6 z{uYnaYXy||#^DkzFF#}R@1oKIh5&8=(QonSf2SmWW^d$2C4l)6ym*jJ>!T#swz1;m z>q7;(-54RL+`_|!I=)G1n4jJZsK0$s21A(%!2co|#v^V+wF0ZGt$ePG`pnC_xdAjU z&s1k-Fgj2lhKgK)c{pOs7;FVcX>>JWNEGw7k{gXdGPbFm2+rC?G&hAh1teo{ANEvU zRL=c0vz2W;;*M%l5v2Y7TfrEpW4A5@s{Jk}PJ1Ek-pu~g2v%qm`+_Gns%4V(3y>7< z6>Vs0QOuW43|OWFyW^`}6|1glp&J%+>FM+_qE+V(YF zy^SXzLHg}z(CnyB`s{`G)2L$Cv^lycW}Kcx?@O*!LbC%i02Fk zJ+KA7A7<=nzysT6HB{^4dl!%xlU_KPQvHjV=BVluw?_$|2V}AfLNDdb1?6u*6soeg zdzU^4YxEoX4@TApUlDK^L#N`%FQN)7@@iJ`^p|$^e0}xJQH=|3{kc%Cn@rQi5pbfh zzjf^XmYDu=?SIjdBt>aiY(^9uL9CL%z{Iz(MK_Ll86cs3NyJDL3WazMClfR_c5O?& zCj^b}qk!N0;s-r!uM;F%&8twKF4+!UKV_eW47Yo{!EIpQaPa(cKXMti!#s>0?Q0mL z&tc_+$=WAkOJO6-h4x|k#f333jw=oJTbA+%zAbGi^0-i8E+%^^1!Jf7lOI#sw@=(X zbvF={A=(o>GeF!SGqyaP@;;kn-?zthLGxwKJdF3F1I6lXiL1!yyHttw9VnRVeLvc&lu$x+lx;-E@(A7jl*Q0 z62qz~pHpfte+D2pt~XCJ0q0!w+nf6z_?gq6{48PBW)4t6{z_=i%33U~C_rfqX$?$5 zH&D(F*|_a5h^X);59F+QGM>)CYL{KO0~tM0FoJIz$chDYwVGP{oz z5hAmRT+FoD;_mURVz+Zj7~*I4{GMv)*ud@cYsHx zRsGi>;xOw;sm5d6FAd)?Noyn?Yr6%$;@+m^^xPz=Bb{rq-v4qp2O^1O)&NQ(@LMHe z`Fob<^v^Vi#Lmju@z=+nL`Z&2qMr|iXHNCOrYc5-e@)2Di)9|1iq;>(S&>QsQA=i< zLa`(vp3J}ciKHD#JJj;ML?+3g^eQI1<2la(Z|dXq<9ESSl=u@03_7ixEYs z0x06u3*#p8ePQ@@@S$uA&+%E8W_lq>LscxMgDy_bb!;N$GfqSb=5&8MRTUb3g8#dm!3K^nqP?na5)N;+^`gogMLs%i>( zu`5cmsr1V)B1Q*Ik3h8kY`PJkL1DQo6@7g^jkn@4_lsOeJD9V=w<2;6KdCvrziy)< z;Lzy*ZxH3GJAJa*wNU{pQL3KB`+0 zYnkNcLX7Z2g*ndR^vuNl;xb;Ta@_5sS~UkZHPib{VUBqMdqv@|_onRwav=TbI-oq9 zMfuslRzO&mOC!@gCc!7AMrs*j=liw$;r=ISgPq}=d7GV=%Xt327?KXSL zfmVBffssQL<}Aip4(~Kuh;9&!v`VTB!kspum*F2!KtpfXcdX2R5z{4 zkDN#>t;Q+AZXw${U|c%0ec|w6o7J{4zTUR8A3dTK6{qbVhAGhO>>p8VCd6=1E*sRD zuZHeMyT-Tz7RNxNQ@oR!8jW$HST(Yb5!&&skE3lr7wIB)G0xPl-@lH&2uRgUvK_QY z5hca_%2UBu*m6*2h~>JjVd}Q~K%3#Q=1y$2%+YjY+fWE+74ztKae9~DNu18DuqD~t zNz9#xExibD*?sjI823(jwALjq!r1=w*rc$kSSjzK z>QiltSAt>cqpOYVG5?_>`BTMq*{f8tKGCf6x6*x^_G%=pxd^eDc*XOEy&$)!qa>8= zEHWvgfYns86CCp$4F0FOIlA4Q)AKYuBo2Od)tA7pSE^DZ7ohv#QnA^DJSjb+qLEnx zR^tgcq#)jLnS)h2v7}IOB;!dbQ?ax)nS_cjk(YD4v$FAM`ftCz{0h#NXPuD&^l;&~ zxX$%=xc*;s@h@efK?>_K$ov>QbBi|W$=wo&!l3qpO&H{6Vp0TTR9oe~!9&nz#bXU{ zGN!C^>q8G@ynfB`W(^zZeoyCcevvtB1Jv#?_vuhld8#E%f^vWg3RvipFLb%Y~t^ zL+I+Dql9MVh)?lyG#dPuQRK!Z@GB-sh#y7zq=<#!| zF{{Ja7lDy5KK5LUx51&;g>HyfKLF#MG!#h14ENm@?yQr0iZ%(CCwy*dCpI*XE(!zl z$O=58fnyszAXS9fBXpq1LelWUN!P{nk6X~lM*dFv5hmlcATW<%u|W~JDM_YqwOL$j zQxk8pb)KZ0`g?>5y*-fx9ecWr^q9BVijU9fZIH|!e+H9Mq1$xPT*jCHf9og1w`_7` zyJV-uhcq5b<|?L}MwghlJVkNbw?z*R8>)$T&wR1{m`}17S2^+&nflw3tYMl;5%azE zZK0pU^pZdv=AR!P*b^L>=fD5`M zkV4DghfH51Uwhc+?)(~})}<%c(>%UqGn2b|zdm0g_t0@8XOJ~Xv6CH>X(7WROCzre z@79%W=|EnqQ3QOk!2~-?k>^|JL&DQk>#YyM!qcd&Guh%$q5b*=h=1@bj0W>s&6L=njkfq z@^jBpa2jV12{h|83osk2kr9e$kLh`P<0{O;7nlh&`+1+k3#Qs+#8Wq+K~uQBq~(^d zy~aBuWH9*~CrG)*+khfuof?_!1n7E)tOh}fb|@kQo2HyoiZmrwpO9BESqMhvbYwl{ z$-wTQgK(o37zO+XwsIVxu zA(NHw$~2T2z@KBOqoB`kOJfwes1`=kSJ|K3?4k^=v5HQ4D{;Jg4+{jQhjBX@x{u)8 zifeO;E=0GW@S?D+DQyG?%U4_YN3Az<54}adTYQo|0nWqzl_*o7swBAu=vK#XQHu3{ zg;G}fj{g(z3{tSB2SgaWn?$D;a>78ugcLyb9`CG-E!)XIbM@8;gcjmI zy#ZAGHp21`>M%JnPmN!XcXf1r)&;86?;U6e!$kf_wl6iFVMl{;*f6E)V z^d_Elh|$UrNszUep-;~!c@hJSIS8tI-+k~__@!0}(WbWRhHH=^j&feB+HC{gnV=#+ z2q)s|!@ykQ7FC&X?I^;g*|XDEw*!Ad^;iv@z=imWh>k*v1PgM8hX9||cEp2{fqAgU zSrf5Lx4iRkZA`|J2;OcqZ`xVP5y`sQ!vc93_`7hJtfym%bp?vVQiW0rzmFo~`@(3mD_#Mt%t*!oPKIyO~{ zH{*JO)K3C z>NBe9sdeOYf%}<_#a-0?_Oc$oDOA^IK1$9pzV=S`?QEp85Vi7X2xJ3)&hT4=xs?oQ z=0%6cu6PX*aG2%i88Sx-0`eD*nL%e4<^-7%1jN=eyDh!V9%&%^2-0z(*K!CcV>>|5ZR z*cN4CzC{I$<{9D-)yDJnsfpH;uYKDffNI6(St^n;reK9B+mbxW%`B2qGLt5bDJhkA zjgui2NDtA@(D-g7@Xq2CX`Tf=V|7@UccW*emH+G(7m9K%FSQR|u%EeH1nOSU8R(k$ z4h*u`oR+QGHqP>Ft(P)3(8QHU07Dt_LLXSyCzA({CQaHT!(Rt{fWZ9m z!+6v01#fgQ&S%LeDCocTcXGL!O!9o&;Mnx>dU%@~iV2Jib_E6nW`mi^q=(sJotr9z zgB~xI`<@9{+-XM(3T7F_07I&mk2Q)ebLtCGwK0Q#9?XzEPe4XXB0roVrj&>YRzHka zF%)p##8LXrnqxSeAUE5kzdpC>#+Qt>__YensdDT-I0@uoK^Wjy&inK&_v%Wg^=;nf z&p~@OHtJn4C)J-DJgYMVG^>IebXa%u%YzigBZ{lE-^+cFZXYO7vRTn`uq3?s+M}#f zhZ1+3-B@(e+KVG-&~!a_tF{2LAn(Fchr(Jepz5Zqu-jX9#e9Qb(8Dkvb~FvW7F>&0 zmRwqegt#JERVue?&>IyGuf(#6(uOHd*={>9-^*jr-geW_XBYg~wU)2LwT~Md*AGS) zq(DdgnHWk|qv@hx0LE3ex{%R&0y-pwNlhineG9sVjN8Mq4v!&GW(Fh}Fegt%Av}OA zRrHwY^O|0kM09rS(HutfO{9)7S3Oa~SkYq>Pe!ZD1;S@e$P|xHjsjCoiD*ip;-v*X zQ(O9tm}WNMKD)S2Q-v5G1Z&DQP;bR3DO*GpBx6m?C`fXw3Nhn#S!W@?v&Xmi6RzME zDZ`IaJuFT7oy4FlEVs#~814>VwTqn+so#fwWc1`a=ekh!?W*T09g+k&+CiPhWgVm^*5~ z&CFUJpLF(A*>XW@jCgezh5%r^Ux%qBNU}sh)=_F(?^kPRWS9 zmuq|o$;N0cxqIN!#8E|b5cc-melSXpUt4T!!cV1vIMvad{n4x;a*j24@3P85sda8K3QjcU zWgUNVW9K;hR(kN>r%}yX4yRFfuAKTqSf2Sqaz%)yQz>#kyG9@6I5Tcm^NpzCA&1{! z^o+c{Ylt)Ts#OSIbWYgPcKu%4B;Q-d0D9d9(l5@Wjo>U2_?Ci2QlwEQJr5uz51B5t z>fybgwiFS4;D{rlpgx-pfruLl@eT_2Ii;#hQG2J;A_QaN5iXLmr_#~;HHts-5RyFO zLRh`5Y9N+I!meRoNF1TT6F0z*(GQ%FK3lk{W^Mt|q8{#spGLH}sF&SG%fD=Sd8G7I zE(D~&WpSW3y~iL5u>msa2aLt!gL@TDHifunfcEu-zK*8Xo4KT1v-gB!!{uo?Xn~ZB zj$!5v^xr9nl>Jh@6A)>s{%jiS3RC}RFcc-FJ8^d2Pdm&+!*0`?xx>0P6n44fmgT?K z{R3O8RK$E406GW(=wSWN)j9u-0Lt1J8vg@O|E-Jl&*`3Y0ZGwBOngJ0m=;+cMUm*; zwW*cceF*V1%>Ernf zmg{ed#mb_F#R>@YS(z^_))o*48a zUwfmocXNrp=^n4`;&43$yro6rY=WJJ38 z`=Wlda9ZyMC3wCZ5FM7I4bP%sKXYR}?j9;<`E?d413Fd<`xykm&w-y$i`DZWw!&Bg z799d?%xq=^0+Qb$wyZh6hilTpUp^2K3|{vMN+nC_vJ#%%jb`@chmFmGp$Z<)QR^^s z>GxgT;wO1DSE70cClS2B$cKcZJ;O*nP>&H8luMMQ^p#+r6qWWNon3$v4EI}Hek#oR zewhBTeu`IP=-b8MYS+|!^d(9y{_egt^1?$F&5j{Y!88M&aj#(eyvQ5UYWs>y>MsIG z1DvBQj98VSAk_L{Npboa6!lO|qeG$d--6dA<|U)*wJ7dNb;?47^40=$?)`(tHW#)F zd!a~jrmdltbAIS#>@9dJH9#kIe~XKp|Ha<_vrd}WS{X?hoBWT{R!Bubw=H$d+BER#9m@$?ZeYxb_zQg4`VhHuo@FxM*AAlHH@Oc}R2U5wDQuSWtbq9q=* zzdne2k;ZB(I>@8Nq<4rG`kVb~A1f;+(%>w0@TtL0CScji?y>V}ix=Hmtxh^EBhEPC zi>YPf5=M*G)rx%K+Vu5;HxC_198_m^Xc@h5b2U#9y!H_#z2`tIXQWTO>~pl<$UAP* zbom-ue>7Xk^2$j?tu#cV5m|HI%gxY9%2QtO$I}o($8AwR>S*q&rVXh3AP6vxN3~W1 z*EP{QnT#n7n23(e6kBu3!bAD1$2(w7Spu*)m{YH-_j?gfvQ*?m7cDm%(s0VDDB8Lb zjAV<&PMCNv5eBPT-^;Nn+fT-11f2vuV)U=40(foU`+VECWCaJ7?#JFu5OmXpfw71i z4%aexn-7?oq2DCR%k&I5S0a80P>~#%hi{*^{|+j>{{aRUp|ICq(I%S*zpoM|u6PQE zNb@`9$@>7pTxVzOr``1`Ie}_rRm{=Do;?zlUwG_v6D8m$2pPzv( zDkFOM9|;TN-};RIjWYhT4mp~C_)F!2wInekg7<9Ix{Y?#fn?)Bd$mGE%Y!;c3v!@< z@;uf3gp+iqSbCBd;;m{Kg+Kli&`1mR&5j0hqj~6OO&HLHGke&`7}ybIAGwk`5E?}p$%s)@WEecCaOKXv>&MDWzqv3=MiGfCKK1f(#J{yc!xv(D5zDhCFLJ4+JH7-SE zI3v!xX~#9+L!QL=&`w5E#DzkwyJI+nc;mPkXp%t?H~Q*^5lCI!HIS=liVJx(XqGYF zT#QLN3l_`ei=)E`VN{H$n_^fM;D_@$klL<(OEc%<(WNX`d02S{nX0|h4&AXkJrSwQ zlG_Lmw0JNi>^{*)v7=v!c82yP66tmW3*NGc;iF|M&8vK;D2{!EjZw=G%IzLJdMxNs z`zQqW)$lEFR)I3qJ2CW6|z7b`iXt91cS1ggI4p zy#3X^Mb5e;J>#69-oO5w37T=nvY7@r{M_Gi$p7Y8{W$zzj#X6R7$Cce68uVOo>{D= zkuG=EIe-!ZY9p2_tS&DkFDHjyrdFG=w6ncE-f%~f&kZy+;15_bjxek!DjbGNB>BR_ zyqWoZY6j=^<@y@k7c5)d%}Rf)k5zE_u0QUyOiVo3Ql_(C&i;5^k$2l>U@vTS&YnVc z1qPcX%Z+=RZYJ)+skCr4uIrxcDVv?VZ<09LkQd3m?=cQAmz6%fFh zzq2YQS1@CSZxc{ZOq??f=9+v@iF+#xY3+KPgRyvxHvUOxQo?0vOD?@Yr{QD{razil zyL&C(dj#kpZ4_3;o^0*iiaD0%)c;$fME!S0$<)}! z*umV8UQpjgz}ei&NW{U``j=uENAdh9*$zAyd@4tuMFrE#wY4rf35sbb$1#i>}D>#Yk7!oKNe_af&0bcE63EiU~_4?eYL+{6_N+r)dF zC`cfP01A?UR2WG4bD+MTk(k><_rWy4`-}sg|Lhw6ZM*HCZ~TjDOWWG$8<`vadS&HE z#viXNbTFK%OwB(e3M0ErqVXqFTjQm)Cs^VSYFF{c{oC}Ydg__58&tPiP?fBxj zpV|Cv*z^sy+6-ffq0VrAG%4;FVjx14KOvfM4;%l$4u&XMGzyYEjh;XM*h)4vPOuf7 zZaYKkZm_u|rPSy8q?t>tjo7}WTbgAQnKXTxl=p^9;|nWK@!0Vd$@WWT$U1nVovDw~ zje$tf##oO6xED4_8P%sZpSmKOtSQIV^&wQg7VC-m2ucKq!Gxt339X;71Uep%F-Fq# zA@J1k45KKIeS^NVQN6=RexFd`p@6rXod15MomKcV{7~#u8V{(#wnZv=|4R1*YBiYB zLLC>(NkL}VJSt&1Pg0afPYsW=1c7^`I>`}P8^o^Xe%B$@f%%_hEi4s&Uo8MOp!l~o z;J4cQPiOjvC6ND>3P*=p&;T9yW|+qcrnc0gUtmuNX(NY@g`6Uo7E5@VIIq&LdnAG+ z4Tu(ha?@PCC$L95;GJxZS#y7Na|gByg+(4CQ<1JrQ)}i+t&0sX?(M0JLfThUtE~&< z=rgI>XuS@k*P_4iIvS*!KTDfGFmMyHdoc(gLDSZ4D+#;uU@}neQt2kNS}1mrINGk8 zrQ#*TsH!E0T%fO}?O=}-0avHYo0r$nA>mgQ`Fw=p?~FRG)7t;{$rK%pHTv{YBu- z9;^iXqz))`Er+JbCO_}asVYV@U}eNG13m{C4HFu3_2@_rYuS=Xh6!{OBb4m>!uCkz zCJrUupOka~BqX03*Sw4DHpa_dl z_o5IhM|+(s39lrn{QVnz(tSFtsI*<_jq!|>pPes5W?-e@W`PvFmOY}vx!f2wpig0} zoEcDV%8&H+>}*<%)bYY$1{;TM==wqY zy4!NZn*ic?1U&y)9)DY9^cNidOKj|)3JYM{NE;{e1rRKw+j*^T2NWQ@%v!_U=HY zkO00q=IVCyS=_wkCX_YYx zfPX3NiN)98=xxJqfG6iK9Os94=!T#NB9II3OTraP4!zIJP-|grs<#1=ybls@rN^&K zFbZ0jhQa=m)i6#Fqjm#?1N~ctp$B~9cZTYRGa_hfZKrPouwcKgQI_Jb2`NpqJXSTn z;i2!#b3>Ynwn%)<7x*r1K((v5>^f{E~J!dd6b|#jW%fjuxp8zm`*)m?d1Ma zf<&86Daz^-gB?5t!KugD-S>^zt5heN)o_jZh(xK-v`OP}o%B=&SLR#g5S9>Q>V-~T zZxl0?V9a)>2Kxn+Ci%Dijd&a6gS37J& z1GZXE;K@*jMsmzDB8r(*P01k&J}I%RF&R%F*mRNu@dsH7&Z0Jvp)W+-}8@;~QGlCfa8i2(Lt4Nu{V3 zD9x3$M+|k`vGkV<&~=NyeuT{MIT|mIwytpK?^cn5;PVE1;evvEyR}Mqhehz=C-Q3D zsTCFhuE{;%`Oir6x77;&B7?u6&A-SZc0xvg5g};mjlw0k_B~jeZxHJMDVMaK;I_cF zcPTNNho-{W-j>$=8{S0w4jtquvSM*rrt68!xvnmr?l%})BVT&HOy|n7N;xQVY1%qXwFqn=R72>L~7AZE5Y+s=gIAxZ!f&YWXgWqX1nFzYvX`bQHEE>H>g}ssvkeF zAXqSLBnE~wQ81ux+60Q0VJ?)&Iv>@GY-L(M9~v{bWe=?D0s~1!7*ts@lUb<56b`ev zxOws(v>Ekt$(_xS6#L8U_&Gvl#>S~Dqv@GXc@qx}&Jd$K1%7nf? zZJ05jO6?hy%B{67w5oO3KkpQqT{Qs^lM$zN@plk+1J?&s*ys)xhTDAMFc1t*MAIoN zwG58S$v6d(IBiv!v#iqP$VWP6FglqPR@jl)m`9Fgn;5QLLNQd=YJVV^R_S~f^`ZtE zJj1yR-9SIqr$ejk6~px+l542|`T)y#%pq9FT6SLI~z0$pD4<*7+YhqlCYb;8q$_*;4XZ+NM$(nHk7P^`#R^J zsyX4q+L+zuQv%r-G}f^%Zk0Oy(h)QLeOUtI*(gwMaoO2qa8-67SAfD#PKuHMr1Z^q zI^uC;fyNQU^`F++4>KvL1Ms>(XvcpBmA~!D`x8{et*!rO<<$SH^CCXAwl*98!*AIf zc>6oo1xf_0`peU7{;pE96oM#trPOQzp>ax#Xg_qSX*;ZVV}I1R z;-E1I-l|owzHL2ehl}ylwc8WUbEFm^35Bl1!t?MLYchD}ug7Ilb2dr*3 z)e+0XTUM-ZHt86(Ut^D6a(z|cofC)0*Pv`peTK>WY&{BX5OH#%bUM{IoUf5s3`?~1 zC@J>SsPzSqU=_H7@e#!(^zII&V0)bb54@W^ZFv?Hct~6@gOGF>P2@8Ak?6q_VfU}| zQ@)`RQkd8c`7Af$e)^Ccvm9;K^2zcIy82q5v*X;svHLs=QTn;fU6klzXB84ePf(m$ z6|spPZ=f5+(M4WSx_+KccoQ2mFW_Y8etU}+BrU z`Gbc9nm%(yLAe;{*ZAWt-uV-#9YSkOl9J${FQtaom}~*x%WYFGQPeYV)#5XMeYWoe zQp>NIRym{)g3|7DV_HXBWG}Ej!&y~K_vqxPOI^AQXL!1IrZXiLO&-K17kcbK89SM0 z)Q?U31c>1ujp~>QBXMJS&Q(z6QY?P$h3i#7N*bwHNZ5BOVjX_p$VS zn*y^( zMGV|6Yob%GYi^@pAund204FEZ!&cWr%h*cNU8+#S&cR3?7w1RK-rYdKu|SJVQUiuI zFFQFkDlSboIzBZgD?1@c`$=liRdzz~(AY?Bs0<-596&K34@G`b3_-ttqZn^~Pz=&P zD8^6Es8D}yq81>AE5P%gLF~8sApHx%{`IZ~#ZCQ;e{~mBXlAr7(^DT3h`+Emh;>60 z2>NMPMj$z9eJ+8iY-gM#dr@^3rwE;VmY<3I5c(;--(%b(hi#ModgFbzE|5~KvBA2~ z;y`eiG&G|c+IH$kJw0Af>QjwO|SRZ3vmMnRB+7Fx!RrS?mpNL8DyYrgDsSUICWPhmXm$pJA0f34M{Pw>7 zR!#pkhVlzAq`e3KRA!%4?DBT^A290n>c_;CI~{ACLCs* z&z9jWA#f_l1DKe@8~1uS`5%^s=6@^=6iKTez|x@q#4@Z51m{73-eUds3ja1n|3fza z0%NlLKUT+NRxOw*BV2v~2f5X|_}WPTf%o9gLRUmuh@m-H<9-3eD-nq&B&WWF8N9r{ zy!z|_S}UZROFNAi3epyM>D0twW}G-*#8_F;(ZICz9L{iqkaAYI7QMc0Oe7?(&NX3g zlgdCbFE3tTMb>P-6#`kHI|gx0&tW)5_t`jF-1e6x@H9a`5)mV ze^vwJI@w8KxNY2l!i$fUAU_&^kl4%>TA@FV8|w3Je#Y)W(#FTd*M9><9tf`5lRYW( zc0Qmd)_5treYAtu;;VD&3^^4znyyhCzk((*Zqq^1j&wByyZuzTy!2LhiKhP?8m!_g~9U&JNK`R6}$rO zH2fHeX`;`Hbu-6(5A>a4^iVvMz}c+4tv&pK2RHf+83siIDkqe-fFp|zQvD2j4J_AU zv4hNbb5+L+Oh=NO;}%J?E)%)a`*LZ9`dS%?7}ZB%RB!f5k`65`i2irB>Rfk4-t>zz zu@_GCz86pDg(&pBSYFf-vGXqYn|Q3e3e6w8#*2gZtW`RS>#!*D5vkdb1n_AICh_7+ z5oSNa-Tsu2<9VomFF-=5zr8B7fN%U8Z!`bXN2MnLBtmR$etrJo!YgUY&hr5#f-sSgdK0a-UhU}&FdHZ z%EKPnq1R*)6GgcLn^A=+*{SbhR82$2bVIWmZ6|D(LD-)8c$vgI}C9U>>T^7X-Olb^se7*p{o=J9@P_r^v(K<=BP;vZ+de zm(376wGe_9;)pdvtfW~nk~$Gy2#lBBS&w}V17oReX|ER!^Q8Urj4(**exx|Zx?e_o zM6Xg{GflyKWS(I)uzC3f<1vDGzDIj~tw!g1i^IUQ1mFhU(y_RQ0F zhj~Q|-bAEIM3WI45@^zR1(Qu3m5=g>*Qy568j$qL#FEgqMdZXj#`<@W9Xe-P058l= zvw_g+bNfY;%aLC526;%fEKlZ-3E8@!X$w$eNaqHDq+=SL`WnNO%zc$9p|nr2#sgyflU=%(&CYLBVmA%keAFP zk4BP?gB-_}mv1_&I*l0$s2yK^0^+x_&6Q|?tt|X)f!p5#;y)B!OyAKA;N1Pu?3|RK zNe?WKF#IaoynaSiG+$*?R49xBuM-NsKtxGI7dG4|!(NZBz8dnNLi4#R=zM?QT5*3k|s(B5MAB#fv zkvKsB{}c?P(N)lKV>PCRd^2poNp)@2Hd(VvLj`JZpKi&)+IIAG1nGoE4dhPc_7Yj$ zoKIaJi)FQo#l?2Aix*<3$W@SG@Uo}l9)XYWoc3y2jhmCFGFu?&+PmN-i~Y=?vCv2lh40c{Q{t1{GTE1lNcU;=kj+nrL4Nybuemx&p7VK)xs z4}z{3`D@UIVxX9Chw15rHvSLEnwCP*$wjv1j?%!*?~h-gOjE74KEOb~TEB(O6H**{ za-Ghg#TutTxgiqf^4h(f{_L@9Di$Mm?pcCOd6qGCL$CC?}&oCm;CRN+lY(f!MyD;WI2r zHHY|eX)teFzEF0Fzv3CE^fyROyGy}Y`8-m(w!>51yGiM)yPMIejpLxIdk z@v=hX3=x$LG^?znl=kXPwuf#@S@=kHW3zi@Om#3UNm@m6sHc=Evx?`P5j5XR2-598#50uH+Gpo?mv#yIb}!tJg$U)_*^ zT;Vr7s;xzU40I4Tt(dUgP!aGU7MIlA1#G0M^mAlQ!$d8eL~Rc%JMxC#-T3ddojT6p z|BxwL&Ajggz!-S{)?EB;%=q7B`iD6Gi0f6UYC0+^qrA8zIF*o>NJ9y+PlpT8np1&5 zsYn6!qLHL3kOU#gT{J9P3p(zQP@*6r%FkT#Zd@=CKsnE2La#u(IN1&PVZQpoPu?F# zzV9J-I^9K)z%s-099c1~>0E5?YP?%H`1<6sGn@ld*=LD^7>zPa2p2#+W}mtPF^n)Q zA8pDJZO=VypSj~6jf_KYm$~Z}x1$95RZSU)d{MkkCmdG6N=a{!7|QBGawb3bZ3YU3 zT3(M|5|mPL zK?cfd5rlZ1rZ8K!JmEB*a0!Zs(v3kyIImT*K}5KBLwRVv;2=MDp1~V_R7mbBq9|Z0 zrfKDlNNp;av7-tIW;JeDAh47Y!ql@F!?qgcjPS5xwBVJD@@yj-vJXr%Mt$fb5Q0-A zuo^Y=O?XkmA2B<*$UHS~oqXD})7Kd=q)o4ag^bzynPmIBW_@|G`6zpk=1(9KL9j-! z!Qqbs+USePA3Up`+w#82tqjM7?GwW6|tm4vu@I>7qX0CPgc9N zCT&Ze|1>56OgwbW4JxdWq*!w|4ugUe5OWXsB4Og6s$kfV4;#Fz($r#2zR0aN|(N)v%p%_ z&Ji~NNx~!wua1>}&HMC?8L)O@=I#h_$G8Fg2Ji* z*-}_uygb5nf%Kg0yOM+TUNb@~DYH=x44#){x*3NHAM3`WjXIaaJj@1@G_UazW*wOM4z?qRCnh z&BcL0SDtj56-CQ=%IfRUp=^c9t+>td@avIB)!YsO6`z8tvhkWY3tL41+XZI; z^Yna0RTOQ8Fp}R{sKoD-wl@+((hz;;y!!&%hTSFpc*VNW$90c!zRkVS584q^a{<)V z8`2UPb3r}%#Z}`OIOmJ3XL$Do=1Z^oS`SC@wj(S2fjUo@9_@X+^#CYZk1*N*!%5F) zE8lG;Lj>7$FLp66nn%0t)@$uoNBk?uwfEb!U%Spj1(B@DLiU_K6)sA9971Sz7^-2l zL9qf+t#1o_mfXb>OgYjJy%{r8!Y=xZ%95COqTn}de3gsca#ZK{f+rU9VcGe!UP^zc zYy4YT>sj_QSa4Cs_st5vw>;u)4;gCGfkp`-8>{RRUGEC)JW^r|{8T?myxOm+jtmkk zCwRDX-~KcddF5lwp8&2F9PIDHM>4KYj`V%B{>D0l8|%f24?KHjqYCEYI`kOM%y3uvQ$y(q_r-_fA-K zpYMtwNAyl{ZPgTJDD*XBMd!i5RI6$rx!D2^l#8jL{0cO~FCY zW5*Dv1CVxmkJqbgcOk6|EDCJ7f-6&-jaoYTDGAeePB{VV!?MQ^gC}x4K2v6v4f(09#wmaQlt-ojc5YeAIQV3d?=JX<`Rs=iRrq+-<{bF~-BTA^!yWsy_C!&4M-Xlez6TfYHw zL(xmPC|3UUJ4q00oCa4fg2ahZ%Z9v$r@$d9k4KQ@A)@v}2rh`SLqIq7!mQdhie9cW z9S_0Q>j0PdvhVv}Qq&3}_Mq+Og~0M3rKtahWBp%-U>V9*f8t!c8|!gs)hJZ@sX}+Z zOF_r03WEuM5h$(i1S0cfS~>#`T53CV7n|naS7Ss$k_eD9e^nS8AU=PKK!^NQspD%EzUu|9qLmC{QP#c zmy7>J3(kiS$>%FN@YRk6P{lEHHY_40l=Ucvu<|$QL96i5X8-u`HefPP>D|_FN)n^> zAzDoM=)`Lb5SWV41Fm&r(-2t0(#KdrEX{T0ruqDwPd%~dNv4t$86<)VaA6GC-*!Uw zBAYALq_HT?Axqkb?U@^@b6c9n*DyM(R-{MM7MsB4X~|AxuQd-*N74V# zE1Ex`SvfXhl8`aP0$xOIwBT}&S^D0&ZLyHQO2#V0?qm{L%Jv=K)uaO01Y$7*UhRRI zfy(G8*ZL-1bKJq>JAxhrJGGt~2qd6p?m!5AKMK^i+)78CuvBJIHr4=w=6hwuwVU?z zUFAO07)+)bbSPRfDvJZuwj=mjC!3{g2Hp|Km;i-y7uLDoll{wi31qhPP~|Z0&bRrLbRUrif?| zkfdN`s>Lc&0*WA}QZw<-H348~u`;Qa)2d~kFSYhtjgD_cq@u6K#dCJ;r6?Ncva_LD7dQ)*kl7=vGN+#-R8j3SA|Li%$MP$SE5 zE^XVf{k{$fLhP7*As1Tq-$QE0i0wv1zA6tO$cz@pP9g|tU_e4Lgpo8AQ115eK?WWK z^mwVcU=S<1i?Asej zX^x+UP*&7)RiF-9y4#9fX!Tx=%`a%hy8D9E1*+&e=)U^G2pTUAkgUoaC~mT_fq~os$O&szj#uzjH{xSVP08m zbv!Uz>Ga@FJ~chpf1kbaz>pUF*?=~#tipVIXuqO77Dzgg+gbXNM}7J%A6I!83b1}C zRv^y69k$)(`+#2+Al;*i@aN{72DHS%cHjs)?)Ybd;@O3)b{*f7U@g;d!Ycki?KXI$m9pml<#UD zufGCUq_Gxm#ZYP=MxJaTT%v3%4LRySuJ=-|b%yG|^Yvb_37X|ZQ21K8&e@5TxXbqt zhjxWp(_8np1x7-_S#BB)p>CnIIw#)eHxU+fY}#~ppyHHn*C^0&tVPeF;*{rZ3DJ43 zK@=HeNPgQvrBJzZ&AvKYc0uOZoGE!k=|Wl-T(cI~a6a8ae)kc2L{uU{w}no+M&|3c z@)T@^kRIn6(8;BHl7QnHxaaaRmy3EXJvLCL{XmMkn9qI`zNpzS4f;j+AwA)z?Iq;69q(Px6EZ^s%Xo%dbN zLX+XanDCEs2sKL}98R-tl$2&(j6X~rq5DulK`o+KQVvrtt5)X`R1rbRh``)ZA<45E z))edELSwN-6B&jpZrCuUr9@Pl-1=kXn8s&{HY8fCwWF$68fa`-V^kMTOh_GeY1L)w zUSR%)^Dru$NN;poJBKi@xN8#TU@2snCuQRale(QHCbS868&c`!Ex}?!8cH+(BM(?k zc^QW+iBU14AkIHa?6c{$j`M(}Br1ltK5lpnB#m1q`6-zGDFZ{py1Y*Bxs2aHNTz)~ z%9UMy`$;0`HrfKF3YoCdD9n*6lYKUBhpX*KC5s?=V+-qh7V-lX9bYU{{17iq+`pQn z)~`Fb+tAI&6n`|W8Cg|W8Ui#UG+D&Be-TWrw&Belf8@(Ch{L4;N?9#@;@{&a+2r4( z^LeG4{l_caeHy|lb3rah3oFy=`3g&02ursvZ#_~**ty^n0yi;KpD*_bQ) z$qkRGr%9KtI*-__ceh83ZU(wh4%)r&@H(c=SVmFk%?H~_5DM0q>F&gR2m+nDF1kH~MFf;2Py zeEcSf+CsMIXx2c#2i{kQ{z};#->+3U_BtKEG)>n znvZg1*eyt@JMJ8FzuQI6<#2u&rRHvt7M_+;v6J!X;-M*bvE-bx}C2- zUULX!l(L;UEJA7qSXo^EQZW8CsNV41i~(OKaRTc$rEJj}30QT6Y6|*CCF|pN?;?$+ z@fK9QtqyBcPBDk?xFt&yY2%!)Sz`!n z9p6$UBG;B4`80ocV+-0qYZJiKFH_(51@x$$(R685B_c$nC76>=Cgi)lXm^@@<4~0} zTMRY!7uhQ*xe5l66U$0gZodiPJa}xsEzgc~ndPB|zfub4t$`=$3T*6#?$v7)FuA1+ zy1~#SX1Q4?o$lg{|C){d%;*rkk`*?%dA4j%O)SysNKA~>*V%TA)ZiEbKMOLGtf=1K z3TBu8)p|o9sx3CO2``%#*-_A++HUKcw%*E+kfyR)@U$qOV+{Tqi5@zbZNQ_8zhLqp zbKxNIa)Btfx)0mr+d>|hRI4X(9)}8QmHDujK3kIT+t3zqw93G!O|iGh(U(I!d$)gE>DCVpLGa{=mq`-w1nH zh`e_Y_TB_k1eXST@tP9nQefmV@Ti=&kU!M;7>IHM_JQcj98hrE!>hTh%4NB8=Y9E3 z`w=So^faR4@(lp3y0fC~a^QTq6OZw&*Q6rX!~<8*X5}~^iEg7A2ddBqneEH-N6!cM z*I%m~p32WzZM^u(JcF!zaZ%Q**Pf)0;mEWpL#Im0zb28uNrIuV9P99?$A?}B^hXCK zrc_7!E-3u2ZhixCq@{U~c9^L@$i##(p-_@-C3@b`MPZ6L&@prGKbZ^-XFlFNP))2! zG77H?UsE6CgHZ&2DaJR5@2AA-#`JEWn~_~+30{luC1(kkqKIg#Pb1y%4~p0O{;+F; z)nmgFx>XW%kkOs}FmZRqPTfgj?M;!@9yL+cDazHq^D5Q{%|D1=pXD|Gli*bA(xDuk zFBMN+_$Y|)d3WW<4|-z9keqLBkEY;ytq>V>cI0$dlpj;q2x1rt`dx%8Uhq5nksi~r zH6FnXNs*}ePO{_p;SVTO@50=fS0l;Hx^RETg8!ya?K}QF5!)dD=|rUXUkM?9@17kR zkgmu}h?z(J4F@I$I5-R#45H5Amc&xEG-=GbmbUsJAu@g=0~#8bVA{t7(S_(8voCzJ z)TF$zWM*m6DAKd}oT-H}?u8FtTMu638Q&|=h$HeFZW|u=wl*H#Q`l`b+f$k>_^fqBW|0%Jpf8i>WbMD0ZLEq3f4mnsUvb_?ze={9=Qz*YD416+@lGxDS5@_mjGnPkCn?!`zGi+Njp*@3-khn8)>C~CoW#q9#U z3PPcJ(+{A$3PACD3WWl_N<+co#SuYtA?enA6%;#vSLZ-Ot)9k%6q%D~8#+r>RH+dA zR=|rVOCBPswwKdx@GID70M|mSN+qgzvKlMH`69Db^|pv@pDn5gDd(7lZOOo4whEwf zbm2;c5~e?AUk4ckpU1++?P5B^$f}d?qLrthYS%$LN=t!2;jbU$=bs!`!hoq&kfrC1)6qK?~+=^7ZB%n|f>WN`& zmN#!Ev>qkT2o0C`mI?O(y7(9)hry~J8oek2w@$;QfLMt`6p%Glc<)@9p|n1V}e6ocYRTkRIa5Q{GZ1*LPCPSD1DS zVIqdlu2Ir6ER9b*jL}1L!#Sf%!pU~2{khZDSC6i8*_S(9FFM(^_x3dOAn~U80)+I& zV-0GGu^^s)2V~KEVQ#hmfZty?_DYL9Y|@lYF8{jdykG7dbMlaQXs^&zkY#+5 z4P&Ix&@y?HnpUCXenM!{c*Ua}o0I)xV}R0Am9rcAAhE%u?i-$Qu*1=igcUulQNi|I zqGDfCc&DO%;H=)rNUdhUzF`GeNR)zssQ3)=^kIv{q1yTHTcv2;ZdAlUH*NL&E&USb zNf|IWE-h(&I?jj0*-(zD;{<}U?I%^Vu+t}>DPCaBP09h#38lb2muh~a(o=%26YIbj)((eqn zN@~(S%Whc3h3fq?LIC$t)Zuz|6H83kp6^OSYHJ4(i{rYX;01Jlp zFh3srHEO?+?y1ro#r2d6kkyOxlo!&d+EpHA-12t!jz~qVFNEi6J*=yT-+okhS4Or} z2`z~JN(lGesol&XW-8&<%Jm%?|oP7<(VvA(~OYA(5n~Iy2 z293^iYF&2)5NN;GZ$HP34IW*-_~juMbPhbKpzLGClPLShS}tX3x57#X+~i}NgQxe;q6BR?%KdHsZ}7~MQw z^3YffMohzSb8RDa?d(kG1#7bL>3PBTOO*{!1oW=+{DvlKcpylg27?Hd0VFF<;U-3B z^WzM*(_ki>AIg%UhLACsKKv-2aFUTWRn@#qfW_&%n&yOz2rW8JC2<f=pAnZ8F_Pg$0e(@*;|~FvA?@EtgYBERu6?=ZeKzmiz+jF;S4A+aKhpHe-FFeeu)NK|e5@HjuZD0~VXSk~8uqyZ z*m}#aP46zTTXzN~b7Lz>#~+7gI7ydT3&_pdvdn=*%@nz%rOI(L6HuvQbIyJTE)l!HCq_&wSPtBwF4D8eYM<&-aGFLE z>5jC(I%oY@0DmrOZ{}56(Q2Ydm@w%Oh?-)~*2gPhUn}&aYdY+qnN1GO;B8*zRnloX zz4(39`0(4+{TZYKZ~gnS^~C^K2}Xtz<4J-3Gg)PC*?SL(N7?0IsUwdM`PWsh#xubZ z|C9%0byz3)NPg&VpU&=DFbh{K>EMJ{U(5+_<<={Hpv_!XhF2QNgF7l6Hitpv!r1w4 z&PKNDG!Jc8bhEXVIF5H4N46sJjmigcmLeNtoECCpn&a~5%B$t(xM$@#fU6sz_X*)N zh0UG$b7LSk+M2&?JNvheW>bAy6N!;Vw$Ilm-6{ScO`Yyj(A<%j)S@5iK$2xVjI^KE zcx)1|3Y2{PQq9TL)|y6TtV=u81+!^$m4sb7hAqse5bPt7Zq-+iJ{f3BW@A2PlFj^+ z6!fh$QK2m}uW}!Kd4kp!#It{!4cnYrwwGygS;E0<3hN3%*>2wX5y#{kzezV$=Gjj3 z#ueY-S8NXC`yQ`!3;NyWF_{MH2<^^pU%le)q@Oe=II_v zZTKI$BWoPB9Y8Hn#7*7E=>{I07hI1(dw~%+zwOz&MLEp*r8=VX;Htqb4~oAod$xt; zf$uPPGVcaX^VaMRQ8ZiP?e1DPaomHx4QZqy6(+0|79hS*Q2)|vlT;*bMpye^0 zUfjMbsT!OKaWJRk2P<><-5PBRoXG^5o_!1~g_H5n`*fGpf7P0=lPgnUIR zv>d1>d|_?xmEIRSvgTlIhb?xVd#oBS9KQbW3sg)?p4pch&~@_^m`mR% z?0q&ztO}2cJO}%7YUIL-^kK+H{dd&?iapEkdaL@Glg8N*25f0IR3a(CQ^{4_CQTn^ z>X}&#fz=8a5Z6EcU^PM{JgigDKtRq^|MW=sU%+Xfto^^>wAeM-&!j}ahXkx<5$87! zRu)~m!)_LSj!PSyL6 zXAZX!k)&4NYqi!?6RS3yPsAG%iYjK8G8=i^O4}oO>Q$sT$s6w$$a~K}m>TvR-+*YW zMkei)o?tqY)A$M6*Xu1e7T-+-KaA@cOD|kR#Kq3-)gz;=jvH4`xPnG4z#1T!8Ph{8 z8?b`PILY5v{7J5@9>%=$R7ptEte5JPwZCS1QEfw{$cH6F9l$TH4#Epo9?k3*#WeC= zIyK)N6I*YwcRM?oJyi~g6lGq-G2reo1tc8lq*P-Nlg={bAZ1lD<#C7GuG){+Cl>h^ z?MSe>N>jex_5|EGVlyxX;ADl{vh-tl?!pbiDkkaa=KAzNPzd|j1@41U>xD19Un#?$ z@6dwdMc+YCwnXf76~-Z&(XDSU^sr^FWje~YL68T0=i zPy0V6fP9VYf5y`gbSbKf27#qy{WMIr>!<*d1lM0YQFd3Lm<1EFErOvAj7sfuCEw66aczr)sN0+YpT3+N?09u~hXg+n#q zV#&|PTtXDm_}Uj7$)OW_$ch_u6&i`xUf$PDX$HX^9>w=T6A_`EUnIPG>2d0^v zEHMcSR-)X9-I`yhw34=AI@~z&Qkt&4M@O&7$Sjx0*O{wp$^T@Y zr|3%Mz=G-*^xJ_dep$u9xD|Q>TFcq3f=m)!JIO}=uBO^l(VGYKftwY#G5Xg{y2ze5KB#qeubNxcg&WhvhVMlrC^&E^(vJdAC|Kv4U}aRR zK}{qoR2UH47`#U;>$%BIB*c;xb?n1ok`u)E$S&CHg&qY}UW8wZa3!AB5Pnq$RjUl4 zwl|C3)fPi7QH|DJ!TZH02D+rKV^OLgH2LYH;6SEGjZSm=eE8ivD=&c+WZJLSj#$*! zF;pA^B$`}SN*5=vT+Vt~sZAz0Yeumfn~s$ag=Lhbay8bQ6$?S_)RfZ;DPn3jBlmJD z8)HP&;`>iOS^OeANZL#;S3VRNeCb@}6gqWZ@`SjJI+GBCg|z`0N6f~3jozjAa$x_o z-z#9?BvkTcnh&O{yAc{89`l~=EyL_z5OUyoi{Gf-&Eul-vZCkY8Kk-Z!_@nEC-~D% zGU!l;BZHo{8O0*WUz`}SMeD!f@}#+jxsq?Y)vJWRR*)Q=zo9KLGd(!6@K=!LZuy#N zHb+l;e^qK?NSXE)F+}!){Dm%zG=mmtxswl?0HjR9Q4Pi`FvESkC9Ld`?M!7W{-q-n z7_4gsTb641wF@P`>1}*V_YpC?y_8Q>YhxgLU1V2?+m$em#6$lD(fwuw4X{587mN9P zL3Ayb_th(hPPNzyjHM)+-WLDiidFCJasM(lRp3)5A*KBoBCSEo+>&*H>rTqwF-R?F z1hBkqMIZazW0>nC0Nc~zSna5rtgE58pk9L85x=7V1FD|H+A(@=9&iyKMs#`u3G~As zsh&0+4+$Gso}w$A?b?}5GtP~E)*t;H^IB+Sl(zf1AuIo*TKB&|S^kd=B7iP_U+2*B*_Jw|E_mdv109d<4z*F#-elX+l|3tIOMp_2l}sK_%)36HyKn zqa+!~+Y$l^i zH8$)PvxOh_nXyOFsF5w}8@{)Dw%W*$b(~a=I#jPl=iUk|-n>^&Qy9|Ab||Uk(EWvy z4RsGu?dc+?5YaH}QiO@_{FVHQv+ID(ysL%KqLKK$xnm7~F^s`aAFut3+h?+#Q$230 z0D+2D@3CEr#sHL+@_06kT;n=Qow-ElO@aoY-xz~(JF1?9nJtY%=+CZOyD}kh0*Ibb z&>i2J)}oo*6dOi08!6A($+hJ^+V}y1-hl4vO}XUCw8%5dm&gnDv%;aXTcnr%OJomq z3lG}4n?=u}v}7L952j1z#2<5o?6-BGh{TewdJu?Ng0aHFqtW})YRFdNL5O`Miu8lqyo5fARaY3Kzobt%A&eDr$=5x>?XDQ4^B>2Mt_A*l`>TuR zAc<3x`?FMM{Ewz|{|lJpKTg}fW#AtUx`MDEg=*l0Z?N^DD6@m#NrKp z!1|?fQk=trZBnZhA)RT-$ruKnS4gm(FY?GaR5Op~pZ+l4(WT9M06}lY59ya_Da(O>fgZbf=W7W!<-eHTWrf;>hF(~LN^ zcFru*UV1Hb5I2(*ha21An5xOfn8|G@S?2aQIYu*iS9aWqeQy^RTqewhTr!zcEI8oX z(yj|NBm(-rl7{|FvyJc^z=@uVu}%DZx$(qGSl^V2JYz*aBa%Vcm=~|A8g1fWV82n} zmM8p>R*nDIn6W@*n`3uHZ#S7!txe(3a?^es=0RWP#%;;sD%>n(k`vr)Idcz&HPhc) zt-t&D-2%CAVCXjGdu9e&560=oL`G|EEfuwrGhf)jf47giR05uuIM2LO06;JJ`BlJ_4aJa03mCw|UtQo8=9@W2OFE@~XAE9Xr zp~o=ehnB4k`xa&igTUTB_WRgvnbtX*ssy^7bK(1oV-$Fe*3ps2`W^J-}aNR!xrNf_Zj5d@gN+zLdPWX3fxHquDN;fRm zg}iAH+sX5PMv)(rwPsmXbpAwDdEQZ>a7Q#RIY!(eZIYUN2mfy2yul0wjC3Ya#cuZAHHq1~MO_((pQZTz> zD5PYyXun*dsPs@)n&Rbe0Qu>%iOeTmtjH6mh%(tI%^kv=Ts50iuQ5mIz8hRIt<_GG zq|up^CXQRX;QbRtU38|u0tZjASIy~CB$oWeFlQN z{aptEWvzplr8KfS9pLm?!`@Ile|>x>j$$FidL;apxo7Y}wy#38Re-o=-n zexeCljrxMILHu6n4601CpRgK8|5>K8|*>% zTaWj8<+jeb5<{>o*jEHKTZAS`48bHn&}WozqTFo?7ulSs9^4CY2GBm-k(BVnIsFuU zx*`+lJoVsDuc!Q!fmt9S57xGrsU#UIrlKK0LkcqG22S-XZrwg;--Yea^=fW_tzTYn z05U5WiEBin?M=S3WY|G{ud=1#4T;+t;N2V`9P!ws3ZJ>XU8-Kng$y{XRp!7M4O#D=GmdgLqdVlUKBd;Q>*`f1rF){*uG_dkY4JLKCMM%3a^VYrYQF-U!CN za*fvLu@Lyj=NpAMU!o6(CJ~xUrNBrht*|s@_l`y=BXjPSS?j1s<{1s2D$$YF!4Id< zu`Z1*JjK6Jf+;<%Z@O7lK3DftiR+gHDen)S0uxHn8iil<3H2)oeT}dLkgBt;$+FeA zAJ@$F3*B@=^@DiP7kbC7HIa<{HZ%k%vrq$q!BmjUgL#g#T$RDtB|I4m(zzjp;o^B; zSb(%%Q|3ko@LLL^G`5hi%*aKV59ywvVvY09h|me>r~UNcC88SKe6`HLmF<3KyhNQb zwanhxYGvh)ki^P@`9-l>W`#lf{ct6j1nH<`Hr;7>1;%vG^UaRe5$xl%zFC@>FbL{E zs&$~2-^CkSsJl4QpZT%ZukbuNJ7~ayMWCmw|vQb_m^ZK?u=ehOmHhif*zjP+OIp;b?zv{w8(Q& zu5%Q@RHZBUFMdM4a;ThHaL_vT*LL8l)9vyn$@(TrRJcm0yGoMZciM5CBOTj2XXyui z*f^PHj6$|5cbf+%4oLzex-$HS((*8@#P>#KX7)beXU z81KWA{EGoK2|G`1y6ghF`_E%`sY{K zZ{#hvJu^6F+5zmdEfAbYTKSJgmRvtb?W3Wy z?%xc9mI~^-P3$3Q+!BlSEbB&*6_-S^G+A5$w$)rBcx+>`q73S_zLqhKYcqPyH9+fNBbKXI_Q+!t#RsjdJ2qyRoDANhr@@*Maxm9!-baw zzU%IV#~0W6Myyu}r8<^{?2?sKjp7ocRE^3K)Kra<5|vbq3d*9wMN{*N!bP*v?@DK^ z2c;soncu^{yC#C5x;B8QGzSo#cEj}Jepcna@zNQ)!U+ZJfWcJRPcw9Zz;bMN2G~ugS3;@8QH1q%(g{V13*+D3X8o0p{oQHG+5Up5A)p9^!i5o=vmL zP~Ph^2H)6~7w$)Ql+!D6*331BwV`WKB)P5XP+ap5weBUS0t~7v8Euu>{Zc3wW_cS{ z??P%=awipkb|z0jwT4HZN}R{|%F`~Pt z$sIYE{G>0}9q46k7<9qk05-Mt9{p>Z8>~q3#g;l-RSZ>bZ{0?N$vbe^GYSuzF=Ns{ z%?&zTDH0$=#3a_g?dESmU|)1xD^sA6&cvn%Tf)zoBDK*owb$6T^f&D85Xyk`+o?yhZu@QVzqwm}rCwTrerkX!>6Clc|d{%`={t4tQ;+vVPHz zX-1qV_pa!N7a=-9p?11tiw$CqRM$@BK#*jzP%%>#T}Y+;jX}Z=^YxuZni;0JceV11fvKjle3!Yr0ZG>D49M(rSle>;oOSItV*=G z`pxZgzR`#gQEASln`DSJTIcmsr{1&)>QS-!*Ayc{JXN%+b3FUd@R@=sb}mb1`U<+) zAwP!~CzQ7#)daWaug{Vn2{Z<4phi!a#k#d`t#{;_x`={cZYrb_6-}tR$$MYZCX5G- zDqTCKAEOo~p9`4HN0x{sD3x*-DH5ZvRh9ry%V?D7RAG)N(KU52tP$t;{mRpkE&A?I z3@@=s!Q-5IklsEP%6U0{3(v`6P%_=nix>qdm-x0kQzrm=ech6VZnJs`o}OIfTODLa zsOx^lNlE?cjM z{>TSw1~t`YS6@`?M)2W8gi2_k7ShgxN;}oNm6OeWN@bW#6f#&4ps+Ik=zm#ZXpKVW zUrE<;8hw;E$Yg7F86pMY&bSMdMFJa=r^&cVb9|r1X%HodzV_*bA;yEpCmt(b#JqlZ z;EuD?2-pg~;E84H6@D=J*_zlat!FN;N!ZXDK!Cza1J)%m8;$wVjZ~m z&flVW^NdR~L~#p^jGZep@t?%U&e0U$(nNi5#$*mJ1hIrhGBCrA0%DQvP1YzlL*@g| zGnA9H5jdb=pZG~qNJjf{C!}IVut#N_H%EgT28Rz;w4H<}sO`f_LI!gizw_4g7q11+ z<;u92o+oqT$wtlIx9o`$nUSr>z&*e*o96eGc-0KUF8WRon$%fMK|}SNA&3t|+NFnrhAt-=cmS610u#x=Z1Uc# zpX45>qI6VO-`$(9W=iRA)PxskRPeK~cv_?0{UWzqUqp4p{<%e25r%An7>a8rfTuzd z5Sd2PuxpR6vNWv#Rbc2&Vpez$9`fyg*pl@;{CR#yzLuUnr`K-;*raKT!NWt;Gv2(! zvLYp8bP?I6s;RAS4LO?Q~7*dWcmd6+6eCiqm3OR1R-H!z z$c?PS4E;cCkW>#7q_4K$W=9(((jKbPSB;$CmD@@OYjk$`0Bx2RP?85!<`!DCGEze| z_jEm>q>s(@&9?;uR-X+W1KI?b*GXC|G=+%^s>%{nWdk_(;_n*iw*n(rerHz_jJ|3T zlGLGwFK$Sf9JEiRD!WuEYh-q3S7<$u8AuJ<;Z+q#PLbljVv8L_Kw3%7>8(+Uxk*n| z!=>nslbV?w2JC7oR3nqX^bt3R3VyLZ1~)fQ7d(C1{A?%&yy5aEp)JzvqZ{qv28N~$ z*2qc@%DyTpwrkhAf^3qy>L&^R>iG@q6eMibU(&lfv&t4J!}O9D+ktIV>8NZK{rZC7 z?R!4M?EkJICTFomRO%`l@}bt=cgVg{S|+Qn!{>mD%VX8zPmXS!@KiG=+WxE+Pv7LW zDlyI*DDou9E1c=e%;ttj9pJ^EDt;TA5OpZIYi>_7cdA9ZUqPBsG#zgf5CyTc+n5WI zcz``C!>knszXqZaJekJ6v|x`Frg`#X4fDLD`}_P6{5sIrk0Vr%ZBplGYiC&aP56o% zG#bke_=F_`gOJyOySO-( z^z`i~s+n7KL>uUf@n2qKK)4Utdd&qVEUNoyok5Rb_OI!ieoA^j`UXbBnhq|D&tSk& zp@2(w^joRD7>=Go%a5kEd!b5ABYJ^aw>q__JJsf8_pwLGn|3RV;;uDXQ;dc!$~Ad5 zmY*gUQ$tb;W}}x50Mg=mq39tmm+7=EJlI3_Ay2Mu(!F0D@X?P(d(Fn_%OjzFHrAds z{T)i9+NtFA@Yx=N`RR!IKex#I$3TLAw^m448ydM98GgEy{a3j^BKEILWWZKV!B{1A zmLCxm8kT8`tx$V_t6mNue-sQqB&qLCo@`}u*Ae~T2P9dnc<*1iYL;lEpwTTH%npYs z51#w4&u_O#-B4&e<8H18Ij4G?Vvtc@@r*d%2w)g|#J;>N$1@`7BkfH}2PF5@GfxSP z>q}=V*wnvB4l*f9;KWHf>F3g6)@q2zm~9$yUDR97YN8fz4Bpo&Uksrf5PTn${5_c2 zo%X!ZdO32g?k3%M+1Z#ciR@8(>J3{_qK=)>iCSEa8f4FW65V}VcrvYaYB_hb)Kc3m zu8IAe{QG|7f`imo)9-XsCG-Q-yfXD=n%AC_bNr>cp>u7s7!q_%!Wgxo9vY?8qGN|$ z=tn3V1Pxgu_6Tta9Odi~xkBP8hdJ5dHy37V?R@R~J2141bA%)H`oIIc&o<8259r+% z%4z9kQw0OS|6cT=wP|AqpQvX^QmRk zn;?-vw|XegNSCAElk!i#+hpziBhv4%I<&f3PCB>n?e?S zN=Fd?Sor@WLH@~d>t9}*{~CUUzx+>m_fq$)(^T(`(B8`OzY^x)Pe+Ab&|$@ZPIRwFaXze%wcuyjTms$KOw{z$jMNM$u<3QzM*Et@ zh{>T9MgUu4la@0nTR6E;@N78}K zj<%4=I9FIsf(fh>Kg};rsbxgbMjZ}wosMfw+sC{=#4hP6Unp;voOpaXxNh)JG^(!b zRfk?tkmx`iE;@!*VFD=&C$(qE`M#<6MNYCnDJ#2xBYu3o0Hr8=cO)Hkiyz3r0nv^mH z*3@{KgoMIbxxkWv{dsj5`-^80I5uFOs0r?f1Yc~#JyG=2L0y3mrSR66kJ4*~Zr_If z#C+BuZd>~t08xKY1t|w>Rue*1fDj}W24f~HoG#7_w?H(uqZUMlMqFe>CmiI_qnd;! zr`4-zSm8t~BAV%Km(9G3G&;S9z&^*z7YrkrXfVzimC3Cwx6LgwnsrhQ4Yxa3JaiCT zf5val1Gk>)D|}F@2)Bpl_;`0!a*OYZ4@ik*x~U#^pT%=SBD|~rl|!RvmCJ|R>ZYgU zmd(>f_3ZVQSp9lr6%@EA=V0`>61A0Id#M6(OT1HAFKPPQI^N@Vm8F6a)%aomQ5rDJm?Y3%U=#yI0`TeTc9n)d2TkT*Bp_E|k zDB)DiCA!i*&3+-Exu!DNc<8aZ99hDkv=Xa%2tsu*@8uO=oJ^&5h+377g^FN0C=D41 z`WjHkW+oBTqzctk2jKd2jTzJyTSizn*kp3r09IyTdXPiY4Pe$29)OmHj00)$L3piHJeKI2o+oiHAx$i%XE#GiOVZk5 zn-+76%Cst5Q%F{ch!K~Pl8(`0d(za)tW=7*t;%`8!P0!v$?BQPTU1T-I@AnQk4R-( zV;wv38d}zZOdr!aCt%rL!INTvFqrS<(2krcqO_4RxYy z;LFFFk-L(BMITg#DV)$0W(8ziDPQ2-rj{n%NoMdYEl?}m>cLmpIu0c9yg27)vjxe# zS4y#9yj;DlAN9HzNx%d$CrA=FzE+0tF|IpmN|b~9Aosquu()3~rPRXu_r!v&aB{Cn z-eTtKjv-AsB{fDmMUnd?9+7)KebTmZ5+Wr-vhNfPE8F%kQvpG6(e95dy_p}cJf1!d zAEl1+xYhK*byecEHYC@$R_9;2R_t!QV0I(}Be={ADCP&b{i^otfn2~c^XL()h)Sfl zwl)hvm;|h(jDHBMRP~-%7koxn{tZS5u>$c`d`|wQf3zzg{pUmL&yM(S{qIi?Q;kto z%%Tr09EJfMG(&B~B}kktCJY&$I0HhQ18s$+FP$S0nOfQ`TgN4_Sz6;SQU$vJG;ypi_FwrrzTgO#z-Rzm(S%`(dCQF!>^CSiSaaco6P*8_lIpEpy^cGaFUlz9{!)#;LK zDFJPPCO=}RNUGzMFAjt(Aas6LFcd+-s5d<(V9=g}P*k85@=#x>$9)k23II(EKGQ%Z z4&A_~?WSJfD~5BEU?v>B=FA;%05+zzesju>3QU!;D{P+vV2PSi~)0P?1n$^hG}E?@?#(b<4W&^en1q1clt&=*bv5* zVRPzRJY0auX%YavFuY-W!LTuH9RUNi8wCrZHwhqoDTOuZ&A{BY0LJ$62Pm?V2M4gG zLSEPQQNN_ZnmD!~v0DYW+m%2)_cKvu4l1Bt31&b&540$|{kT_V4+;;z-xUwO{zuk> z2VoZ-8fuNri`S0}m$Xj~pV+N{k!A9V0C>XuQK^H{k^A-+Ne(65Z~--zw?dhgpZNZ% zPy_ywt3&Iq-m`dV3V*y&Ma4$=1~O*;J*BIW?v%}U5iZp7b-+GFdswM*b;zG3dm3&M zLytG2sBAa;A{eUn$nWAqv{%iTG%%Ecb>W(7zuBhc?6KcPhVbffS_UF0pZW?Yp9c7p zH-G+7y=8pq3V++RrMwCe#lTaXn-P2k@KJ&R2v7ytHHYZY$}oabl(|DeDe;0ED5L92 ziPTE>U~hC$Z3Asf-H-`^Vuf8yd$Dgu5b~VP3_Is6T;1V@@xw3pZJDiB;O#o>Cw%3& z|LS#4*61t+(6@LfU-Mq(s$4I<6FmYCyg8W~Sv%ZMi;eE%=N+vr;<#AoRQdeExCMD% zY-FDk+V?DDV_A8A(SFt6YEtC;9VroqI48JXt2*<-iv%(ECOrS96QjE#Q1|mlGH{iu z_%8@c1!Deh174u|@d)c)8+;KCH_37pMeWTqaSa*$h>G=@Ps{DN`hlseZDGtW5)L)azZ z@VFMbk}-PihFC+}g+zXY%Hi^g8gf>fLy=#*<9Q|z*MV78>$+p(ZyCsqnKH-ZSv4An z3>c`sKbM$XMiQitm>%O3?v}-P$BA}@sHs|=N1aYg&Qce$3ZD{}sVIu)GkFOYMq|cf zNDdnWHN1S{9M>_Q_i=v+)L)V}!M%)I{Ehw^a9dB786MpiG-e(ABKm8$=;&iGc;~SY zG#0)_l0-ov@Pq~GrlONx?sL-Fex%}V%f+^qR&H?Z9>#f1*h!TFV3fgod%A*7Fd#?f zyU41s=Li>FP3x#9*qVxbrL94Odp7rp(S$GI*bxULJvfgh@`E|+&Q=IBR!53Q6)ER< zMNR-7tY`QquPDGPV32(*NvEl}vcixDIHXhnLwG%8^>BUjsCK6i4Xv+dF|Waov%la4 z=vHwdRJto!4~VEIj(EB7C7I%0d3280>;Etr@9aT~45wB;z*=oDbqvXjU%&_n%49X` zE7%1eISgs}$>o9uY%>HJ>{~o@czRp_ehI`W-rI5@EOiRG4);XtANlg-yT9v$WH2G% zV8d9!1TmlKqIrfFxp?l|If_bFShBZuXPZs*0~!GXQg#w#BXqPvmH#y@N2 z{fN??MEA6eNLZalbLf~J5Bz&XZ~u4LdJ&~i_oA4uw`&WpCj!QG0qS&N70$Ky{A_6p z@d^RLT!*d)HZ~`-0D@`C*d!QaPPFO9%*ueKpj{-t(kpDqY;ggZH8Vo)3{qA-0b6vG zY;nVI*#M#;K`;bF!OmXmhJ?0uw&G2cuu$CoV)$j9NvPoX?Yk=ZZ6DeA@jQfd=>hz` zm=M=*R;&w+*zv(*rmyZbmyJ{4UQyPd-=lPUMX5*K)l%NpNVK;_9sW!S=_Sk1o2W1h ztFZMM=PIh=cg(vFJp%Nudf(~_2oGdI_*62k2w#2Z_aGS3{2F2>#$?ABt+D=8iS6wf z?-oAf26l$VmUUEh@xc?C>qt7BtEZrYr)5Zkx~+Bb-yOZcE^o$*%d!%@4jvp>g~TFE z2NeaLw0VKdJ#*dI*)_J1c!WI0eGXiG{#PRG{`|=h%RtmLNS6+gIX&CNMCnJw{&%Mm z%0QAei394yRJP<}B?6+#$=I>tFZu>{$-Bfw35gnrsWtLo{4Mz&9wd5XbHk%N4id?G zwo6fP^G^jv7gb*-3+L)G+&tDxycOv*TWk^rL#BEzcfkZgq9(lK-4#){xj|$BpVXz6 zKnfdj^}r~hM77sOXtyY6(V@xFQnJmY*l2oNkASYNvQtaWU`+~jp*>9zapG>8MrP)Y z>-ltz%&|*r*eO`t>SPm%%*AXm(I7;o zHq=?-b0?Z`S(kU#WUGQtKretFX%zA;mW&aLt5+krA7`{A21tb*%$JwEZvv3w%bt^s z#*gRy0}7V~+3wW7U6A_N^mo6G$2!D5X-n4n-D&&kT*e>qDf3n^04uqg+q)UwJo4H- zJ<2m4*PB9kEYfEmn{jVAWGN4U-NvpdsVgu8jZ>A&;=|(#Ch#^~Qj?P_OTYPQ2DsKJr{9=#x6LiD>_GQXmtv^g#{NhC(XxU z@^pW@OpzADmb#jbTYhJ5XDl`hQO#tC%sz#QNr}V7MMFQrxqd%Re*AiCQ!m5Mb7E_- zgRvc2^)|?E!l@0gunu?~`nHHDUtl>L7HG0uRYKd48R%CV$tCn#g*$;r4Sj`sNqI(U z2>7B=FN{hg7AQK|yn1a=7W2{+V&*Esf>aB~hwPuur6xGZ61YzTh9)7PLdVP8A%pj% zXv+qQ+C%nO|ljf~6aXmw(a~$=F_kFpG=t2M=aSCv_;s zcfe*REN)STU5d3@m9IvSoLQ_v)5N2fACT81v{C^&NL?WhNvCCmQ4W)k<>-v@Jq02_ zO)?MFiOV#1?fYtJ;(OMG&osi;Gg7xE<5od?5K3GMgluF4b%^4ycgg0d8^t$8(l%)A zxd8lqkSSuP19ZV|ImFHZ<8&m);gKpc{QG!k}q)j>sdx&X>;mg=kL^pw4II%^d8xlW{ zHlZyTJ{j%(CTHYYb(1eK#wy7p33vMhA?5=FXH6%_oiY$)M&RyRmVLTC77sz>~zK2 z4ST5SRFv;lD@eN4UeUP=5`OqzTWS7{$HW{C!j@Cwr5DZ* zOEMni%jBo8mS-~j%GU>CBh$QW-U!q{(9^yGjs4V_;{DdO5X$YZffD3&V@lKndyj>> zk*mg>xs<$y$Ql4EGR8U5G z#(Vn^N#vej`at%-!wF-5Cf3IzrT>(cfXoV~Ap=r}YR&{}XTx>kpD2bVFKRS4e7gQ9 z@bIX){2gX%g*<(Fd#k$ao#T4F$0zw@=(37re}EOAo-&DVC*0NvAOHyyO2&3@gF zPy5b+M`Gq&@Pow$g>Y8{(PGO^pJ#%JdZq)?eLd<{ZCYWZ-oGfj$ge1_D9o5-f--rD zTDGoIUwx$B&zMM(NfLU3G}%;rU2R>RQ;k!-xz@_qJ-R5Or~_qaRV&PjlhG)7Nd9`4V*q_ozOk8S+ zZnb(3#j)g;bRvWrzl=TL+lw*eN%`5&MoGIXjEa6694;AJ(K_i@t(OyC;ifv|+*!0o zx~*aleO=lbRT|EpOl9XOFq}(fmeE|4JKIsH+=p@<-<-9cd&I_4F|)JQp$N}e zA(GyaS*rgS=gymE$Un9CelSS9SWEM#0(Cy*K`x~`BeabWa3VidSHOG~H{4OMo{JY{ z-c`J94(i;GCXtF5Hi#SiZY>hoP&p~xWC;wSVg%md^wdlaejwybBcRnb$YVF8oV#u# zc?Ji1-&wUL$+)wT9*}x8rB$F3O|Ok57EO&N!bl^%$mN}RMkiHEph zQVybUGUd;MAAw=y=PIbxw@q1x&!wQ%^}KwGkk9>Z=3Zp+!$BDh_0eh(;w z(@a?zyvXplU-dG&Si#0$k+mq$;bUckx1DoG%|Rppp5e((F=9O555}A?F#HZ6+M+rc=UXeqGwJW?TQsP1nPTkc{BRyb zQ=@j9@_GRn&>)P)fo`Ql{rxkcyzl*B5UFXc7Th4hKL^HDNR_&3qUIsWJzpW(!4c{Y zc^XbF`C3di_l}3VrR~CvwWta6#GnHmq_9Rw4!aKQ*?#FfSK1tcXRly-z7MFydt7_~ z!#Y<9^blRgZbZ#{waZPOhY`N~DWQpu}VO0CCajJ zAfI9L+UN7SR5Qf+K!D|hS$8z*D@Y;5YVc)V?(Ch;FZk#_*mVtthpqeXe^ z*6mNll8G#iGi&+2$Vdjt-x+lgXWDU|VJI&Sf_O0^W$qRzN-^s}G)jM^H7fbmXtRH{ z;Aq=Ru?;ANrFLh?Ms_7C*-tY2J5f6@PtP5~Jogftx@Dawg9}JMvb+j5+vO6>7TU|Q zKLu=5kj-}Ft5j4$M>np}!BgA9EP#f>Il$nI7#dP`S=yGJp`o=+Qu3Ejh{9l8Sw+!0aXCijfd+*tr0qEV>CXa%msb@y|9nE{NszonZ5n0g51g zL8*ipQHGpMSuJ4Xk~EO*O`L5 zeq#ms36~>&PA^@4!5_$kt9UPyv`h6O;Lxciu}7+dMoCbu0mKF36c(rxK$@!q*%hP; z+9)9)J1}WAd{zXhH2NS@1_f=Ta#rsE$>Te-UB_$ zRls0qVR^n)u3S-PfoQ4STGx7`fVdv6v^By$`kR(S8vRvC2MrYTpW+NM>p%h2eVEC~ zG_SABw1J$MnHAR`!j(+ajMNO&Oh(^+Nfk_|V3y!aFjR5a0kNT6J!aB@Ird_D#1ZcK zK6=sAoz+U_ZwVQy*iwD-A?O@MrcIFr>n(6oRyWAH#cO|fFDTsXRol+a@IIYhi;)HZ z@#UMF+#lp{uLRJwgd{Hs!Uv~WVHL%tQc1F-WPs`mK)QtptC zY~b8-nAS<{&joYqOBCrEQ|Ty+k?eM&L%))89L*I^Kk%9eyWq-GwjBX>=HPC`9zvtP zmCUn3YV|?M{pKTYo1N5yFQXPEhbzDfRD>OUej?mI{Rdh`E#_C5xS<$y#eC-C>1l7u z{4x}?l!3sN)C75I!0>i%bDY~5$ae=0yk8K4TsFfUxoLkS&eF6EsWZV?9bUW-!2$-v zXH@KDaAC47QT8AHi!yjJcbr1Ar+@W#ribabMSQc3sQ;8lvi{f7?mu_fGw{$U-Nfr&s_sR*UOPQMb9!a>Iq`+IYESn#Mh#FsdMW0V<`fWp9c^AJp_CLz>$ z|F}Lcm~Q>6CUn)pZ>PMswlkb|3jb zpK2=Sl|;uTFY2RUsxGaXi%PZ3DkFWUe&H)&QcJ_c#`bDP?c^wDr{($q^S0WWo-vYd z3B)%0?ld-*dvQvkT=TLs=Bt@7^Xr*!Tc~#-;Mxy6L)kY~b}-3$8mt>S6dnZIs>)ly zMw-TMutsoi4s2@o#wjFV7FE zA5AY>gH(h%sJH2%E5e;5d#RxtI^5G^9(2X}Rp^IqfeBQ#RX*fJ?vKw7j-knDcoiC>l0Evo%$ZxgR*<+~WSm?DK23yu1V za4*dgSumzeP+nq13@%DFtBWJj%R#t-Hxrat;8JVbVr>}D^8ooP6H(h-D9lUv<5c3G zgMriGLfVXxY>WugmQnW7p=|W~|E3CEMuGSJ4S@VjL}+c$o)n^-FtC#&u-@{j|D;Y@ zg1+8viNi}`Ci(MI5L=g%!}vf(aY-%8+Po;Fh)K(|B3myg(uFarXi2D$!W!p*y}qr< zx45obdYNt}J+S!9!Ugwn6zgrMZSF*GQJ6v5bXnhWzrD=e>T*H~93&ByQ*Ef(lO}i? z=%LFE>B>P%&w)ah9tx4aslHpfx%fM(6vQ@(P7Tvn$gm~7Elz0hqZxrwT z*@p1)g)D#xxI?%IUC}bMtvD6wc;HaJ)+T_R6I0$4U6FG%Gu>ROxW3RLXI>#mQy1TYmIs)?@r&0e1Ukp9Rqtro6(pVk^9u z-3_7bl!c;8TvW(JjVrz^jJ3*3QexbKIR=(&-~qmX>^+rLIFx0Ai3D*IL;eeO?RB01 zSZ;1na9R9jO^QeyKmKtpS<*Z*p35=wjlXx3A$}?mqMD)jicGj+c+@y%W*a{UiRDaH z7@VpAk5F620xk;9jdm%|2ou*?H75TM1D5TOnKSkWoW`lEG_Wk@7ukZwAULoF2^LQ~ z0+|w8MV{W!wK`)UfE{9W0uOJSmH9H&%P4mOe&e8pW!( zHx2skW~{LT|9+-d@XQ}QOo{C{WG2R8AKGDGnA^6)-rxVw=HV0GMherPY+4($mvAA2 zfAP-h)p|xodw6ng=q0}M4F?dHTS0UsSc6|sP^PWVOoiW zZiA=ns02(JGl>GH)Ohmfg+0(3jIR+o?G=&9Q0#N@C0j9}Sk5Nlb2=lTY*trdhU#q* zoQ1#3;R4hMX3skO>PtnR9DJ#KpyzGYoRLY~6IaumCbA>yT@m!7pG1V$ke%;uj9aL6 z`lw}$#(`HDLJx76S)Ijr z?6LX`=)Pp#i7Gnzfu-}boBT<(8Btud6g-xPbVJ-UDH?t%>OJsGoE&Fferb;-lRNjN%ZI5HJG<2FwJpu?TVmb9z<0 zCfk7w12AbM`PNnIQ&*l%>r+KlDw->#gN+udnjP0~-Pavk*59^_qqs?&U~r!*ug_=u z>p#;x&zp|ZT`AmuABbXr%;Zy&cTbU&0V+Bz>$@tKHcJ4NESsf6mE+bW)|T}vmE-m$ zxE9Wts%h7XESsi@s`f357R|@Re25v^AZ%6olOi?W2_EvnP;gqUvpxZ>Njt~^Y##Cv zd3eejYIsWfH1G)fPyqb^xu7q>s2+VFMl^jSLm+0gQ7C;RlMp5;#<#5s6A4B<7-hgd zxB`O|v-;cHF>sGjZS2N7;0&|EwAp(D6pYT;844hQp)+>F9H7F`8M~njR%Pl;1~|ji zn7RQEWQ@y&r=;`+U<^1;$OWUP@DEsy%LQRi)IzM7wLbeEKoxM?X$-oi@L48SutWm?HoxjB*W)|4r~e7jh7zyB8@ zYgrDqGq=d;rL34?2S0$%tQ@kztQre?*ru@b|@%;LLab_V1ntcbohl8)8y%EA)af#H*17VuJ53ycP-+V7|IL7*E)|fT_ElVkMB=F^uUM`8 zbfd1Rul%!)UP*HB!ddM@#$!;Nbr~L;IrOyT!iEX`)$z@gHBU1py=?_MCTaq2HB+7~Gd&ce$~|R=l}iR8JLpn-}d&3ktqik=o zU5#P#n1E%kS9n^WLQu|dM0wk?FZw$aGTZHVQkMVdyF)y6+1V%CK19^;ep6|-u1WK%NJObbU=w^gUHhuD6@GFs&;6`JKZyPT+yq&i?O!2F1tv zzvd2Ju_DI>UulWY*qO2~JzRz+VPuLu7_wZrkVk*CA17r+r=dm4j~(X|+>_Ylm<)90 zOb@5*_IbJKiTR}PylT?d^WOt^@1noPU)aW|2tBGt*r$G4NQ z@9jT(N^B&zHlKTpJbxiY2Pbf$gtutoS6`{k%8_5gi$?h^zLX;3AN@r^Q{$=|$`r|y zm~ZW6H28(a()m6*b$mQoln{0W9MA#(^z4QTWex{ABS(!2kKq(Mw+NcjLf=T4c-E)q zaR?>+Iw*8tk-lKnweQ!>YwIs03FOgutR@kIz!Up70?~COUz`U=nmiHI09h4&#ASG& zb4<15&qfkdP-JV1@#4zbcjdInv+K#Hp6942Yiodrn5=L-6zXzHi}uP#mSzCIgDyQD z5jL9L{$mTM7~!l)v%(dzd9^}A73MJsEpty7F2R5qbTCjOIGASEZjAUH%b&o>fXst= zd|-z3ruomOW~{2uXqWmXi}soV#p|;>SEsl<0twZH43a!G;#J$g_)12!6(U5us5wV+ zdXZo?4qH%U>psMQR1!lE18Q8@Y6iiQA@^2s-CQx=%oB0Ca>`8h0;vmb0U6TMT4#Ae zHe+z8F0H9*B}SY)IZO%AVGqLLRz&G1DKHqxKLGgvCv;@EI+pJjiGylsQITO|R%AZj zUEHvA4o_M{fA$HU4P!A{`X?|*Ppgzq{7H2*%B0Y(;pwfSSnx~6~C+Q;X#@664c*~Y~5DA=9iYYV|$6CR{)f(+H=Bp(7fb zl>ll@5HwiF~AI^!sH#;%{$y(r$0`xw^^_7{;_t|a{{sU5Z=2cX}QRKlVp zD5U`__e}qR8vC5}On&Xd1)QzuZ?<+$(T~K)F)5Y`DHBQ+&CwOW`Ft6t(yDg>E=^T% zjD?^xBdfRe?!4DTK6f(mLRZ7dT)U-^J#q>W~V_xa% zM>5hNr`^AU3Qk(_#OK1fRf7@?Pe&k^^$DQleGQa;uEr?5gFf>EY92O#3cnQLQ>971 z2BKLS{e@7PKp&r(e*Ph%X7%(M+z9lf-nOu4Ybo`C(ne~+yhbhMax9am8u$Qey4Ug3 z2o0e=!(wdo9XZ1;-c6#m+byG$ovaR6(R$l8i=-P~G1-Bb-;C&k#jv%O;u zj3*i-GPe2QCbu4p7uGSW(lJ~PG4zPS z2#eGLQbKJ5tMve!xX4zi!4o7MPqcmmX>PwYzXu1<(YODIM-+CUudHRP>Ln@DB3UkC z6*sQ94CquwY}r$E5hgFhuZDn)QY=kMWCa)!{?!WWv;w4U~a za?Zif>macR5ZfY^#})=|mRuyLV|^B1MD!AK=<^+{YLd-DP1fMh(->SZPx(D0%}!&p z!fF0_$P}wd>_G8JPGi-Os7I1IuH0z><&5P1<4-cmvxErDK5ZZJx79|tRXg`8sT6#O zYse?89W>?$Eq%C{B9pFx?-$J_cN!J030I+`ZH@QuHG9QWT$qi}7A$zf*FnJP>0sIP z@~ZTtx#;6F_mnDt< zjCm`ZDr)9n_((lzNAyd`ac@udi>7^x;U_FAdO5ILNfzXSY7;-8T;#x_!)sX$AkgOw zg^m(>!MRCV%1d5xHPiuecOzwV>cau-4!;dZ>_oO5ese_L15tqrrx@g=wO17vQCb&S zb*-SNj7vRy;v%9;xS@EfW6X*rtKN}W`_*>xo-w%TS zAJDk}DVL)w1yWT}^}9Sch6W+qxlB?RVJMU!ms%gy5;QOzv<%d|>72;juY>?T0Ue~t z#kQ%Vx%sM@R=d&@HGroCKNr8ego(g6o*>LkMi%ai*zu3+)n?k*VfUX;?-w>-iFf0H z8P<1$fg09#lYwhitp-30PGJEdxcW)0Ab(p{_wrg?m`1V+7pztDKrk$I616h5O|nHY zR`R#5lpIAJMV+CRZR+l43@wTLFAf($eM!Bcu48a&wAMF2bjLU$ldP)N?id$bMZMt= z=f{f5<0d=A&{?pD?NzkL&Z}S##Ggz=**y|MDPtfu1b?D@OXO0)9;A<2f~a5*mK@%2 zv5T`YO;ouD#EgorU*0-ckiyV^$2$Q|SzVh#aS3X3CWkNl>*xwmAq zZ;9%9Qy1RBUb%-SqRidPkBW$+z!2>l9)h8y=js-vA2RZg@e}lPI^G2|B8vpqvq`^d zvj+5aqT}`H46nkzxNQRp%g=`qJ)YxW?p=d};}+Q>)$i`fR~3J*YwOjA5K5?jJ|Nu~ z%zTp|>KUX-p9f?9_1#0Yw*a|Wq(8E+uXaezJp^%&JR(7Hi!(qttPj%68fg^jZ2gyg z1+mRKs;QsD2KEvUlnUnta!KLrT6Un?S!9hoS52)HR%rAhBvBp9dcWTZlya28X++j- z5_*Py$B$wI#H=cp>CozWdqy&C!S}3G*ajwtZ8T_P4`U|O^3Al~G>nJjK_&(B;v`6I zIVIRIUt;eD2EZn9LIsj^cGV>-NoHbW|D|I(?MO~-rhzUuAd<(+h?sBkC9z66D1oz* zq9`mZ=3u$)Zzz{_3vX3Um|?TKjSU{3o}vUdYAG+MO>%?tb7pa|%dm|n<4zcy%vYw? z)Za`SapShYq5VxAPfofyz4U?LoeJ>d&Nh^9{3gS*ntNrK-#JthB#@=idvp4*fT*dL zwafB()<32}6U+2{wnn`FDz2FSdoVV-ZUh2j+29|EwopNpx}y*BfD7p3wKR8@TVdaV3Fq;^8h}1A*SQ^LO)>hIDp+wQ~BIRR#~Ien29LpNAk| z^o;qFE(maMsZt`M*F5v!$>o$I96JMJ;lme`?xcPLGh6bLrcn?-k3R#^BI~AU6oO|y zQu-sE-}McY$x$k0uCMf~&pQEo0z!z#7<}^g2dMpU5hj>A*-!rnG9FMA%cApEBb)R)y+CHNK0dX;as+ z0mABn#7fZh9IHlSa>e}V^ldZu=OOM$o+2k{6ISn6$?Du)L7s5lkvg;_)+y4o3F}Jq z#V!;mwvlHIFDQ*>p+{Ykj?jf_<-*;lhpI70GzC@?#ZCk}DRc8zuvbZPD~eFGP&ZtX z0-t)(S6GXT5hqgw*twwJGeQ1@$QE&OQyD*jblygnxuc8U;Nm3Va25W-+jS;DJ|}+Q z)j^&$fc!wVg+9@MjM|m{=`3+#3^T}Pxb_MGW!$bi-Y=1s)L!{pAZEV-LnBbiNCYaCDTrhOu1st#WaH&8(>;MdVU!kvBUyd4M$hIVbj@)pyQG%~}Q);_uCV&4H6 z3eEr-Lo0i-Xvg_a+jNzjD_&10Vku=z(utP$g^`qha{{iBG24B8&(qiZ-=ejDHkW@> zZ;}$$WH#i!Q-6OZ1PDSPC_FV&2UC})2Nx+T4+?Ak7NdY5sTwR|0Cys$i`#Fs$SF^Z z5c_regIj5-&JayrPFpj5$MIpxL1e#T=Q!m@IXUDcLiK%IaHleN5qunr+=f&}@&*+PHEb7(o-> zA2a+;B5{mz3e~ITEWuN@ezkr*tpnr4?Ap{^f&fVxG5~9q$<2pM>`UYGt)unAuflVk zjh2ICD9@7w0?|mtroqk5%cvk(UuSN7&xEfcI@gdV-3G! z$DqG|*re6O?3e^Nej2A%A7d%^SA5~3<)!~3Ud}*7MH;-gBfQzR>|8|YN&rya_soLh z%Elfi*guJ=4~T00KL6#2#le|PfDX*WkozIcR#%+8(VC-M~$Vh*lZ}6MR-TdD& zxqlvw|Ch=Ach5=eKjFaN&&j`-92zGBz3+dt*BAc~^!pc+gS5!X`yZLym+SLs%T;&x z4&TpAL-JaxTJ-;s$?YfnFPYrVHMB=Qc zHtYKH)=w|s`(!*?RR#+8*IA5vJD1Bx(Dc$T(eMNt90z>6<@sM)>E9AorvIABiT~G} z^PkK3k7oM)E8uKyWn}E|-)w#k#*WTbPXA%@OZs_x8VJb? zu?U7=%s%JPC_B};ja(09PN4ru8$Rk(wIm(XGEAoD@wVr5*W|~m&)W~Vq3CE>+zife z&Ss}XC##ZNf+PW)F&GE2w^)+UWU6Nn)5B$7UU@~izh>JHV3Q7XM?^_~5&GQQSei?7 zi;bc41U8dN@!@z%*1!_e`d~O@I@Z($X;scwXAinZoh!@8AsN z&L_iXGvly61zIXW!hPn>!|@v1s+_vb9}1f2JPub2(zKk8`MQ>);VVLuo&m`d*ZDWz z-Jc~XV}8Yk#r)N*BtkBp^aVkQ;YH-QYPQs%d4j9b3{PsLcu>s=QVzQP-BM2fT+xB; zvH({iCWV$*O@Yl1_*eENs+f4T-uF^T|4*%XZ2y05>%YBIt5mF27e!EfOb~%O_+!JD z_MpvtNg`+^mBZDVes#=N=Ytk?KagBZ3(#li)z20174_s_mernd@c2B5x_j6{G=&r* zaDT*_GBVA0k9D~+K7WL;{rO4Wt5jRLo98bIg-7)w-zN*@qjD1gB-j`3AMRfcwDuFU z4X$yYd%z6~h{(JVAdp^sjNueq$IR(Q6hJMQ4Ui0uFfX0OFl#{$fEj=x!Yrgu=ZA*E z#OaR>mwZVFa1OwtrZ5cHCFrXRwWJ!6m$gQ0P@>&Ng_CT`+r#UYwZ`O!n<590m|B_`I?L|5gk7tW+Jl1e6sM-H`Z3cQUnrgyikUj7*)lY z$Y)uoyEo~&Vv9tTJenw8QA?Dv8jIdUXHU#J)+gt}PttIRGM#@^tG42eQ-0gk67I4`Cr^N>Q;b? zcy#P28^I4SyEZowUxtra0olx|imj zZ1}yrC|#UlMl+!BaAzj9VH{nBOS%=umrAHnS*uEKSZNhF^6V=W{Z?ITJhblVCfp)1 znYSf=4~6y<@Vg}zwQKI4bE{*@qtB9dFatwYiFdFB9Ch7QlE&&dd9@HmkaDeCSXrsj znI$GydUDqzKS)+`s5DVq!unlpVHW;qoxi8?kk=Xm5D0rCvPsO7qH|aPjSzK2+?_rx z_jVVF$F1^t039O*s_9+ih9QvSygXCTJL+LdGg*yP|7~u@)vEe6+oL-K75Pb4WQGY_ zosdi16mIOct%EfXA0A=Ym|Ob5Lt$NxJmW=iER}K4?xItIyHh<*>5O=}2q)J!C%rw8 z!r+X?~A0{ie5@mSe4fi@l>|R;G zK(_PlQQOVbiw+Y|Y8ogs6m|373?<;e)8<=e1o4A`FB9^feCDTEHy%knn(dri3Be(I z;LnbicmDZ?=ZZ|n>-^qFyT3yh{`cST{|B=e|GfAA+bl+u@|5F(3c?qUW;2d(px4|m zWMDFsoe}0wND}-&A7a1aRf;Sp&IFxx^uwjl@%VH2V7pVN;^(JDOqokaU56jV1Td40 zhl4O*QeOe{-toS>rGm5_vgbQjpDxc`j}3qDKOlEuC@FbW>T(U_hvLJ($77Yf%J(GT z9Y!VXl?0&-u~0e-2m)M-*o*mxLcdMJk&?IBD8V9lL<*4-_G;looYDK0(Fv(b^06h3 z}$sPyxe3>ff&00#G~NOnG!fkRpT z4iYRF%z`U%G2K#Eg885@<6uC)&|l7;KI@n_<+p~%^M3OwhRk1%3jn;9tls<4Be3qv zWs6HR-RD4Umy9Z90zZ_)2|1r3Jv5gfjn-l8O#&!K&wa_8v8~-C7weBdaYX|D-gF%k z{W0Xy*YG(?Q+dRy`SgmbF@OB&9!Wj>9=Dc-!Hfx35tu9L&m&5A=?b|9{MdWT=)ni{ z=7l0Zlyj@VE`y0^+ZgSpA>Ywly|H=-x#fu%_ug2aPdfNGM=69ULe-22E$FDrH?+!^ zG&5iXAp?vtc6j(_Iav8Cr~Eln?lZ)t2jt19@dwlU~D4F|gzBAaF7vaSb* zS8f+DB01YBv1p$+HRtG~yV|Cl+;?CYRr<}Tq@R;>qv8u#eqWXpF_7c4Sf?d=y%u^h z{MzCCc;)rRC72R3o#e~iDgqvGk6)X8Vd6gpkp1>-&+9>GgkcKKF19kCCo=Kq*7EZ0 z+Ze^!&L5bSrBJ!z{wkSxQ@tx@LH3Lp4pTU-qTRF&EHFmh_sOMJpI2^ZRjA9SKBeJ= zNi<=yY>?n}1TUZO^U`sLV%h3b(sB2HuzomB>WP~+#mlb=D3~8LAW9ep#x--l^ZAiQ zNXfV-S9JgWOv~sR3PUrW8Yx`CA?&eQ$hbqwcYkGguspd3w|!=_jOLE_=kza;yNTF+ zerDgeMiI(fc0J*8GHU!)L#QJ~A-6?PWKmF(KsQGPO8rz&L!7@FPNp((P$%Jjw@3k9 z<}?rWK5sVxkFVq#_Pxv3F+2YXm4CG2Q2kM5rMuhACBcOdy#vdipSiu$HVOk2RGC*;qgeofvF{D0OyYpU*B-uK#9`A;);{}1^1_uBt& z&SmT&)X$ORuINs294?vBD&ma6xJ&kCYJ6g1dz_HeS6|M;Ce7%k#_$xej__!lhuosZ zpm_m`E=jr*e)GKFntQQmCtLI1jQq{U56xq)TfHifh-WN3@8iMyt8)AO{%I=Xd3$oy z=jPV)8hDoW)$%3->%AK6PkCPsYMmUSbGTYzC`BAn8y7EFf3J+|mLqNjmeba6XnXm66HEj+a$XPN+1P(vNTfvR&ms=mpn?38+GN(sG14`uI|!s z#~W$oi@_14I_Ml#VSJym@U546UsdP~(Z*QZCK^8r8WWAv!ruPL05phZ?5(V>_TNcp zG2_{wGF$}%8Hh$6*x4`npoIEFNb>!p`S<8z%?G zBIHdpy4=YUsmuIGD{<`KQ}7-eX57Z>XkjV`LPkD`A_=XRS_4>2h1o1x*SV;y(U<^+ zeR>9!y2!-1S+W9=vCqV;T%V}(ot)VijIcr_1|vMi@!Tj40&&YjGjP}mu~Y&Msg%wlb-DTj5;+=lB8zXH1X4*sD=i{D-G53R08X794>+3QMsDfP`PE!gvDA2>|JD* z#*#0`vAY}STVgIZEKS6swQFTXnOHxllB~uZ?=imbsAGK)Jg<~Yf5*?UP^1;IurT$| zDsmyM84Zd1?j`FOnIUHt&Eg>B@#r4ei39l-4=e4i(sHk2yYD89Y*d3;G>Fqo!($bk zNvXj8`hzapjx?LjMZmgLPG>XFXUjpr)3!hQR&iYVo6L8xGGqnJLi8Gm&BAv@Nq1vqeMx6|9kG(X0K z*^}isPR`5g4ELGaWKPn)uWCE_!^mJ~{Fm;1XK6rvZ;WP5iu*B%rS<{QBYBs+n8>wK zdR265;Hh_$85PH^ji>qdOrT?FMw045Br9)8-eK0#Q&!rQ0(jL)rqa}3trgwra|spO z(~XVnOtg{I3l+wdv7|B`y-F`bWDz08_16LqMh+?0*3dX=2OfaHuWLLpu5}+Gw3xH9 z`4I+Mg?pu`+Lpxeerpfpf{q&~;+4i0YEnniqB_g8y1xvL&J@#MY=r9>(_URE%+iyI z*Uguuc@te79v$XsWw=hamn{Qe+H>Frw~O{U=kz;?*$?W?GcpULClMRP!d1Qf1PhEJ z^L64xb-L-~xvjt;M;)@7IM`+1#l5sxnE5aqpF+)XePUB!@vQ1ln2b*sDat z@WwwE35LG=zNI6SmV+-|Za-Xf3G)_Sh9_ILPQ?br%q|oBvZwXj7BVj?x}sRd!VhOlfllf@_y z5_v-+(;Q32-{aeq&%b^Yo3qDb8t--)-fqI0F8xN6dUXhx2Xe7a{NqPKX zTw3QWOaGM}ZeA}W*^1sbj0CP5{cOI$TTmADj|y`ZN~4?}cPyW1F8CVbNjQ3wUBu3k z-5&^q68v<2*BLo;rBn%z~HRvejs@M8RefsqCV#wPkG{xa{Fh9!wubzF5*a zBr8%askpHxu~=ES*nMgwf2S1A`(oez$VfS7FUlqq?+hYR-4)EN(O|bBnXx`>ZE^$N z5BwzxTYK9SQLx;wNE^N0i#0nX-?BdiBy4db z;1S9#8D%eQ$Fbc{PybyVpy;K$IIMYp0Z!!F@q`C2+@^S;B1_Fl0 z=U35wx%fuP>^^7H*Iu0}iu%Hs=UhQ>U#Teiu0tIv36w$^6WNx&{wfa|;p-4nnfG6?U__x;K7^a&HG>4~L@! z>P;1Vo#)*3f~!^HrS`xk(4Eg9z65=d_W9^fM+S?U_Luw{xU*ruizT}t zZPc9xn1g%%`soespChqN9iC<%6om=@OaI`1WAgc5%RGr`hAP97&>Y)3ufDo%Wwj?; zfz=f&9;qJ~+}O_C8t30T{vlN|2|FEWAoEU66Y}n%WJE2I(!mwc_jvusH1}|CXm2m< zx^yhnc#1tECoWOezq>rEgt*Z@M9|gE&y++lkv4EViVTfm0n+Uzjz64hN>+KdI6IF{ z+x0|_^c5~HaP;2-SJalgSBxXp`im92ykrO)oTJ@NE@7$kF%ukyFTJE|>WrVKdEC3a z509~GXlj_%T8_mZqO0bFj}_J~Dy388Y`jyHKAS^jQP)|QYw4YHILZT)yPokP-T1A` z+I(xi@SOx0x{({2butTvew+24{YaxrZiQsgAUUi!Mm#F}BarnP72ma@l z&j0!j|7nyr9>scH%VIO$rBxpkB0bg*w-#xlGt4Rw;Zo=H!LE8oSAttu@f9c==tV0; zds|5$=5QicLcrEqfh2~UAy{5<>vr-_pFD*ms+fa)VWAmz0PWO~^8v?OzrfSg`T6?srgSy|$0|s?}YWU223Juxo)C;ei(c;vBNz9wEZurX{Sm6C6pgQRD)mIYx6u52F!9lMZ7=I};{xlUjFX z?VX^_S@_crw-6T2Kun^~2n*-_=jd->GyzX<C}zmzWdD zFp&genn-fZmuL?5jlOq!h8?hcVEcE%_nP3@=%4`&bI+R?^MHpq8ElrQaBY}44uKi= z<9&^#aCj#1a}Z5Rc0kx3g4os7cqrF81*@`P_J7l3OL9IOjWAl-t2J6lCz>4ZLw~5! zCze=r;=f#S!Vv$aN7G-mViIgrOTRq{v93Ehps&N<$1!3XI&O2ao3zqj32)#DPTqSm zVgoU>8O6Cfx4}`Y)x^7=9AR0uHy{Z(uf$V0wZRrGI??o(Kn3hdwQLWS`o`C6b?tj0 zb?pbJMi+wyxWTW${m&9#)GOJdgkE)ri02-m&Fh~nb4*eAUYmaUXE|;QbF9=kBjL@u znu>gA^Bno5O1ui1?V~ZsTi716y7pjCZ^TIJOdZ*%O63*4PddbQjF*i!#ZoIOU!c9* zwl(TWY~t=4rk`t8Jtb?kf;!jvUC8Ojuv1fcc*YbLS+|N*?5loo++9r(Dom?p>7mhK z1u98 zC&i0sNioyN?NghzXlg?4dGFz&!{3(@%I@W{cPNFcQb|?xPUR^^OTvcC*amSUrH#f0 z(X@iedRh zMl#Ks74e=vS1agg&vZk@>T6|%)zO)gZ%hO%V`pgcWL1N5nNj+(ng?Tg8}B|$-O8(r z%kodFO6@&x@Z_wv)v1O?eN{WYg^ultOeN;1oA$ao$)(w!JDGYwRBJ<{P7isR`X%-9 z%?}Z~zozK^q{n3K#&jtdqZC^7(3#woUQSpRK;sQSw_ zde=bO$t1bc`PSZKo6WQ!kML8vJbzD~IO~C3`-l?#1#n^fged)-N5@);#$b<$y&%B; zW2wG}c?*qSRB_e1*jo0%bEw9Ekuvb8@Ot@k z?#B2H({wuTv={wOdb_GbUq^;`XJ?R_PzS+fxT;H@70%1M>ZWb35Xxr1``sH3B4WtW zv6vF>DPV2u055p>qRg}kh7y-os;jP%xCNzHV* z5fLMqM5$B(b%4dvb90giz*0N^UM-nyD|@$af{?eL>8;&blw5%4bx^UYLxhxsDN7;2 z;~N7xHAycQk6QwRH%j7c`oK8nP@icNp{opQIz5S<=K-dlf}?C}o(o^V4ob_#Dn)Fb zogXWC#S72IBC7;x1>7vA=MyLW!3}26BJYyP{RRbZ#GWGNhQL)T>!S1RT@=kn8`JU*cq)Sm+Wo;X+ zTUay$Abanr(^xEQ+rZ{i^&UZds&~v(wji^e^qm-ycLJ@NA&o5t?M#N_Ve_jbC+QVn zBz*L@hdZxLv`niOy8TatI_@`q)k5yHSmzr+#oRNy0Z{k%O>95=WQY+HZ}zNkR8&R% zCCaPvqL!uOUmIN@zZ~?eq5owLrM9wo(Omf;Q^z16Dk<&In<45R0k_Q8^G2adUt7~= zX1JH*Xpe_`DqYRQ`6`Qykw7azjg*clGpu|EBHH3z7V9z~5CcFL%EL%NV!EM7`<_=OBUY{J_5`(1f)^Wu7+d8OyOOvNQ3n_~y0<5UFxcgTUP)$IVTKMpeytg=@Fi9_jg7 z{C#{-WTVXm-1fOTZ)8R@9KTk`Ho&??XJ<%F0cK?Di%0TR)_31Gd zgf{48u<|bWrhM?P$SyD+b$o%gu6BIq#}cAw7}UlS7DW4Q^#`5D$A{t(QwFwgMn~<+ z+d}@C;D#_8H^QrwcFVk0+c(b@b{}SGSCXh{;@Bs`%MYDX)v5tibmch$ppnNkONWdB@m!|uRp!0LUUQMF?g=9*a*FYA;n>xz&5S#f#B%2nLF z^m1?~4Hc2@QuKb=(RX1^%46g2Ghvvry@nv8&rV&$wT2p zoH2c)^5mG=v+BQ2cGO*3e=EsSx1lq5rvcd==3ORVO;&_BD!C%_LGp}@akFmm{;l<`1hY3R>O&lRXQDad0MajsA&r?1FOlcCYyD~$2A zL1Fe6UXs$CD??-Ph1+yd{wI`BJ08f_|3HhlT%HVBC|YnsMY#X_qzyVs``?{Up+{4$ z|8?DtGmaNxJ!NEU*68Umh#0{Im%yr!Wndlf-Znno%t0-oyS=%F zPt{*3LeOBgse$aube5TsI@on-owC?FY3uZ{7ipcU*dnw|I@n@qzcOWuXdmgCRIq#K zxF}<%=scrn5$U*SWA{)PV7Px8e$BgjeEw16a>e9F#G zU_uN|sQdvS6@ZyM#4Ft3eQz3A559rkpAB?|yekQj z4;R4d%?0a+cf8-0hd^Hp!r^r?8*$Cd0V|wpkW%HUPl6z4<~=L_PiP&(SGJuvAymTy zV5)FTtl&s+Su-P+B&7`%DAYzvT@&_4b`&}})dL0y(?$zWlr^gTP|sQeu7w{+ALV{X zXJx0}^?Hb7WsiQcZNi?#3YNn6fi$w7?tNp`2d&vO5r;6x^Fhj4SBLIx7LX;fM}>LS zc(v-tUa8{~ApwRi;T#jCL?Nv4GZ0t9&hN$if^rZ8!_GZ7L;pRJW``Jx$svVa4DG`h zbL{6&;N<3Ck?xbFs39!zDtl+mgj7rN)uV)l-2|7ZqahD1tYdq|&4MrkX`^ir97EDb z^QnE|heTEkP`0fnVF)d16l+dc(F8Tj8?HqQ((w2>tXQctT+R7HW0vxIpZI-bX_YJ7 zNx%r;@nO%blQ%eaSG|zjr_6uPPhzk%Ceq` zBWaeVG7B}|kb9CXoN;@6t2a}ySu{mzv^ku+HeuPS3N<6jPPJ@!8xzWj4d|(zK#%3* zB=niVIX9Tv>ion0%3sju3+|;Dg)Q$KHo9EWR!)YbEsA%wvHPg@pvVZHl$B zUL=0|<-`grtQ8%P2g3$4V6GBK{*`u#B!HY*Nyh`Xa0oX9M0!MWcywgHkHN7t8g+1B zbT|8>CP#N$yogvl*LQl2)to=quhGR~R%CrYd}OX_LS7N{F?egpXm)3+o8T*@Hwx1{ z;kb|Cr$=f(hP5GGi?1(HMy#~MJ6r?Qvpn*JaWP2Ta(@=S#+jK% z5!?Auu}{pn=6Ij3_lLZu7v5ACpVUVE9tO6ok75~QBuF$w z;+%jF*ee}^WZvw|+w6^$V!6i3SWcWEAAHc1_ZM<=-KBl?jvj)_p7~V1W=6euajJXV zB~wscO0bAgZ&oKUb#yI5DJ`~)KMO{6tmqSkfVGgonPEXf^#V{<3BT*!TjUVo{)~Fo zS^j6eHSS^eah7!c+g0I-o7sb?i)(*n6^TdX+>0k2zD|V5zNam5o^eNXn}=5(j7l=O zE`lz4G#}r`PzsvhUjM-Es(WU7kJ z(4}Q6-VAQ`V~lHPVI!1{K|B!NY>Y(dX%GuzgS&8GsgI=n)3(fb@tRW8jwAz{56EPd z%d1(!yd9+zbIZ?cMLwpZX{ur19H-eL=KkYMYWZjh9N)CvuNb7Cwb=1D5Jn3#HpN~~U%TDSI~j)603^^3|@f?j@~5?=11jsAQubCd5-Lr^;u&8`Ot=4KW_WW zq_fjS&uW?OSufQCY58KY4&#`vBZ>Xw#OilH$eu5h^*O1IUYMe)VydO}dGbY1I^ihm zUc^^v>8a44yCrih?_mUb9rn+L8QcYMdDM8#0ttItRw)sU9$hs|Mu8*uUv8fm)$_0c zKT^!5`E{xkuH_0}w4@|WcS&f>s!(?65tfg|&PwT6Zg%wH*W;B);8t;WG2p+ksY>FV z9b1AMdKmEoYaCy9l)`x2Q7*^%I!FkTQY|iEC$9+u0C$XPv4J1-)f;Jt1PF0#Wogu1 z8H3>}a(b!drCwOo)T403Hq79^b!uANBjL8ydJn~%apg{Ym`claStbAd*k_7D|M~Qr z%h1)IDl=8gHl>$m(DG#gBmh?22A!Y3Q~N;W1_PpBRL^Ur6tr{8PC6a?11vf2cX(M5_>G@hil)v|-4UL>-tT z_~kz9Rh}*qgRn5|V_RGa52`YuZBh=kcEy=_v563N`3wD|?zAGXZ&DAucuEhxI*568 zy>H{MnERj*a7Zf;tdkGBlMh?#U!Q#77VgI*-kHH;i8kv|PxiC3&9KdRTF=eLg3w>c z4KH~8Cx19XD9j`oTP=y*ppV4xE=th>qMwjJrwl~O&Z8_J%gZAX4WxTOZR> z*nsH?FKcgEE+AbQC!QbHYOuYzAWss8)$n9rpYkNzC@A06okCJ2=s7^+)a@h_A5&km ztAFze_-;ztMf^I3K0U+M$m}0nzrVsWKbCqwEFm~5laGbux%K5NmPP244`LQAJfziN zJqIxG)RhqB5?By4wznXddleLD4Jn**)%-|$H+f!}F5yP~eG4Y}OE0oI9FfTCzCVmU zW-?~mE<)NNZL3HTO!Y0TU_o-kp+4|b6bYYnx;tGnIc>4E9G5PUp=5jG@%^_xyqa0* zL%|wHLG-gL5d;=*w&lj-tn=^__=yyLBXWd*-dk3<+!Qx=3Na))o_XGPBwlU5s%*uA z7nk?BXOm!9jJ$C3KYgTYlo0rqWMIzp2lUj;<^Kw9DH3|1?4ru+B%X;Kauj=KklpAa zusu`NpJgBhcV+*5?I(@**EZ3gVvxf92^QdQ*etZG$Z=IK(gMrT-(h98{#7fnNmuM7 zSMAH06$i(Q!YaaHn`f-@@Oo>8oRO!K34}%}L)tn(Wob016jPt%6Yauv$fY6S)4)4% z=Rct&%2Q3f8w=dxw=l~k2zNWSJ6U_eDl9}aiGIds#EnhX>`&_2;}U-B+y3#O@DUl88moM_nvh^KWyx(QeWTTBWsKt)ib7PJ-VP zlYX_F*^Xp@FK|+cJRJBZiufogTeSh*t`q(*x9k4@@dEe%#S4Z2|9^Qw#mLYM0vr8# z7%J{bwY8b(=;GqfS&tsUkfn|MR*}gzA4U64ebJ;3+AT15>$VxquVySwl`~&n?wHPh zlF8qPqdzT0H9?nyp5o&M<+Zp zr7oPFhX`m2c~=D@PLc!C@YqJg?<@o(l4K0-@H0k*p+D8|Ge+c~;}3F?u$DLIy>A6k z6E=b9ir=H*L;_0TJ)!IX2&a{bvq%6lf&@W{@Enw#xnTToV}KjYKe&KwrV7Ch&&KWz z2WP@JQ2XJQ!9?uGrBu;LSCf&zrM;WCKArV0NAM?LT~+}^ecqn`;?AXouE5kg|< z4`yg~po!T82!lzX&;q@tvTcG5VT1w(i=hkkZ@a}UxKL{UXmC;UI*LTvDE~tqYake@ z8Il@`R5jlA^HNHX4i({*4OQY9hz|?Jh;WFpVZxxf4HrsirW|rVOS}xk*^rbTa|j;m zp^vq+6c{#xg-qu2V7&*_mCU8Vj!U8=Q)4T%l+z+X$RWFi0Mi_niOu8AR{x*?fgSBv zDPFpsF`2(U2i-%cR*H;@T*+1Gq!+Mwv&)1|42I%@{6z8MpgwwHDtB*p3x5m#uw182Flour8{Y6FVCS^em;~a+OW!%UiNFIl{QjKs9^p?qZKQxwWF-S zuEB@kUs6 z+5Yra;|W`l2$YgP6*g7j2XG9u>!b}<>AS#gjlM^1YP707ZSK8+>~;y7UFI8^zZWz0 z4V1RrmVarg!bbkvHYwJ3Z?@ZOKDFf1(jNO%x#it6Um@F8x{&@^?g9Cm+7x=?=x~j| z)h&-!!6T>XZCjjL{n~F)+yy2GwLv<9D0l-VM@mp*O1B z!GGNfs*UR(`3`J0M)EzIqv(`KXwoNAdIe(CTF!)CZqM`cBp-Lvzw*pFs}vq16vkX` zD1nQ`{svB04(;%@yeA;4yR3XCx+ZK^ELzmjD)@IPdW(MJekT0hpg$>B=QE4fl8D`o zH~(BIj>G~P0wb=kD0xnvHi*IM1N-A#Meh&&hp@LE9=qt^#oLYGO(V_m=LXju@1>kS zP9p_yEsIJ61bCDY2=#q1W=T&y?&_wfJv*2A1-|_(&i?A!-u+a5c-$ttWhqu7z)odp zbnM|BJpHtZkYG&AnhDZliViEotKaE}MA|ghjcLq~^JDc%Cih!}?X}kFz6`8MjJb=y zf^R=~mUIN3eAYGne59_nPq^;sGW<=@UDRkZJkQ-W-p562=fUT6Tfyo{8k#fKoDLA8 zVVO(P37m>rhkXL-`plEq^tUORv&1sqUG0Y0ns>-Za@CiGVUPG2nZve&)`3F}`>>J2 z>=u=D#lMPU2II*xB_)%KV0hG-M3c-M>uC0`E_`UUwoeU8cNE4lI2T_|D6+nj##d%& z@-h`KQ^Y4@jT*iJxE)c~d@@VBS=Dw(>qT#S9RA94$y0YC3YRD`TswnfX*qVIWPXvGCRF<=x9z z*&K<1oWNdpIUD|+;HtlkarkCHBG0k}+6cr^?x#%;CKBE}fDvY@Jl^~34`g`9h0o$F8L`rXfQs%KScq$@b%yfw^%&Ln(~ zVt}WZw4Fy67MTg96^k=ud5WNzgs1oQ{5MtFsjC^cqZ?HzW^JFXn#!J zViN8Zk1z3W*YI#!ar5=%mZ)`(r?{2TU-)~t~#(fP1lNL5qh$C zCZwKXkdgU~4_njZ053b!6l(x}BWJty3}b~rn;v3=>x|^b@h64v7m>sYO*Q4%T0O%~ zvWb;x7|9)U>!$$!mJ7CvSNgUJz|B|IvzjO&QAZZ5{*xbnXmwSUp3SO5DMnUwhDy%v z#9Vn2nvg?M8Q$6{2McdTv3}J4?$<>Drrk`&7n-DvrJPa~3%^xU7Sl_>(&^6* z4oYZ9vARorqlaa&6Q6|d^jbY5-!2yM%ywK^-~0Lw_J*gLyd+8|S(TZx(&ks7t5941 zC_Hj>QW+kLDbU#=|8MnzW-|*V#-R6+4S*Cgqo`w8H#@T z?*!M*nFGDN#w^_mD+2eXq{=<~hJDvR_FKjzdBj`qW(WyMHhFsH)8ZB?@XoB6D_zZ5 z+Be3QMn(+Kp@Ho1QZ<4SU zLF*1%iTvN%wo1ZZ4^SgH4?1iqArxVbaI1wQ`pBrIaxFVys^-yM@Cua9sp$tkgF+D< z+nT&03>G2^J1ECQP814vgEDAsXJ*kxfMmjDrw^^43x&=Yic`}~;gda0IKKA4ijQ^9 z&rH7Jj|G8j(J?>Q5?+7Tk!b6+DG|7-_9$D}cf5z&-=ZB%I?X8BK9` zh>X{TKegNsF#*Sff;e0H3CKEOZUk#^3inCZM}!*Fj(t5$1h%I}ELe_*+B`m?LQnKP zqBO>?=U(wROT~!??9p??+eY8`>0Rn(A8n@?iciF&chG-;=>-PW*x@lb)3?TjnmAhq zRT6e|Z1!nQUFtTbmE7h=DBUX}g1}q3re+Cm4%urx^cG|c-9-IkX;84}CjE&{x&dpx z54SbpNwP#^ADtE@@s{{v-C3)_|ZH%1@ay3?uHEQZ&ZfX!TUp5>h+>ikiB9ruz|o(>NcKw!-epR=}V%(5>^^$LL(! z=ttUW2jL_BVm8{x_!4)jj}!CbyE?boJ*{Y7g&#b3bff5CtgE6bWi(o01%q*M6e3N5l0kQRnpSsSWeJU| zVq643O}RU*@`U_kH|T1o^|<$**5$7vKKsnE##`p>%azb^VbEpQy_o;$h64FJm}EgK zJa0NK4Q#SBAV(~ETBkJjyR>zx*n+f9Wo!c#D^2VkTF*pUUlprl+WO3q6xu(^Rz5#JvcgzBmZ zd(|OD7wJ-EJA&v<=@>BurYD(>&?3P=-LeoTZ3IapzlR6x!^a?N!#jqff&K|Y6aYzB zqD5dicnv72c16=XbNB^<-r1C0hq}_0>ifeyOJLHV2p4Pz$JBO9u#C4kauMv z_~9Mcy#-)0_$%7Z7_ce)J$7#%SUmh*FC0ErIS1XnHj)`iNA1d0(Rs+bCID2h5BE0a zTk0Mx6bxJ-THzWv!Aam7cyr2!Qr4OgmvEuxDy$Y8t|ANAXMKFsJ*9AStY>w6--k-p ztic9bEmR45e|`MBQdNAF*bX#k&VVG!9G?l{1>M^;p@=iZ%lk{?*X}VidkhuX?uFai zET9dfjuJoAvTpw>2dea$S*FRM1{>gqJbYp`+DmSBh%ler_wsU}0nvxgKnTM*r>syD zCQ5Na=;PU;_|fesi^;Tu7^?6dn$a|og|Nq)J#?@>?tN`u*HA}vj8qnn0VlA^%XP70 z4070bzJH~PXM9$|=X&_cx(_8TT5wFWAbe2>B~E99LS8XW>gror}9r#TSwxP=ATX+2HB(1NN|SKZ&sk>pp4kofBoB zxVdPKdbBkSW4*K|bFnP$eUhv&X>y5!r+VZOjUe&prK{67b0yppGTjMfv9&D{)Eb2q zaB(0qQ5!2w9ew8BXQNgLG7YPNh_fG=x)`++xYfT^oKlpsO6W6B@@u*ADHX&3S^d?&huA+bbJb}t!_~H&`r2Y+tr&lJ?5a|~oFUiMq?&J1*oF5|{7HPey2nHv_PTU;G`>09OWYARG3iCaeEDG;5O6&P- zPBg~~UD&9or18tEvCG8m+Vh)jf3LO=9G}LuakrL3Jm}>oIN`X?@HNjz;oa1685z~> z45~C3k(00O*?^gAP0Fq+oHI6p_CyW4rav2dOgz=D4{kG!;B{5MyKytmW4C%QvEWMQ zzekwlHmTmpdyDQ5Lep;No6ffS7yY+Pd-Dp z#ymDk#;IoT?|2<>mn0e>?_0efE31_4|Hqr*u>2m4?jN!-on)NEoU0oZoV&<9aqNBY zS)cE0C>MY=%nM3WrY=G3piQ6TB88GoQq&TZ+Jz6K^ak_$JychISf5k(hdIrjvJMYQ zidG8$0yM81*)(LzW<2_RFkM=^c(0js##jvJnaHmn=E_;9gs3YTHh*qA-3p(v^NR|y zFG)wZe84c6pc6+b)6lw#g?{@-Si5SQSkr1JhFcl_BOR#jIn|LW{akE@-IG1u78H%dV|o87_mYT^7nMqE*fuiS>(5ms+ZK%utuVQURBf^gn~ zJ4t-HbZpNZ0RAkA`WPodmDq0g57?B~uz@sT{sU~7Ur_r37LOBaOXEFEo3Y~qah*k* z*{Cjh$B0hN5% z`Z|ZJ?~lLNNF6PwZjDIrLyid6M1n;~7mhM_1^tdzLyyW_a>2Dj_7~0T#$<$~yX2I6 zCB9FSPTJ6@B&kY^FH2oW)GV}W1?To?>g58q==9!_cxIx<(~?1UGBl+J4tzJER#tv1*kbiOn2|u@&g^D z$+#+p1RSTT4_WCY8G~^vfDmdU8DG1v3d7a2`FOCX^r)n+uyUqr;^*UH`Lp`tT_-LG zE`YzA!DqOv%z8o_9(m)nkB#Nabx;~V zxs0>rCR=N7z%#yBQTbEbj>#62sVIMEGvME1uDFxx5!gCj75f`Y^rBv0W)p4j9iK%s zccT;^u=vaCC>qRG`GOnoX2yaW_h##=FUaIEM&x}db|@R{r%;$mPY2!OpnsncMlnfQT1736 zq`mo@bP(Qg*S%ZcA;E(oe)S7|L2J!3^qQZdr~)Y9@909IN*sSJLJdnms(T;!L56Lt}dh=%(|3Jmak1-_mVGEA%w%|D)bSmsA zJb2ubqDL&qM`aEUN4;1f^$SpUEohozQ&)}fA??5yclIC|j0}a*w_i%z^n^%!Z+rra zK=&=Y(p3f(awgC{)mbOmAk^0P8M%U)X6>skd%9-1@lp%QNSK57-aWmfP{0xB8na~k z+KSXan$`gR?u?Qm;AO)QoQZmln6!mLVO2Umsukf9ZaS*f13dLj+ME%dpVBIG>D!sn z;(_q7!#@eS1zpk_+Eg9lRA!5L`a<&)ivAJy?7WsMcFEn5)U(K_JRnI2$Z3y92akg& zMn6fE`CI0`j3?5^s&Ym>{c z?zA$DV~hfmUEuHZJ>Q>y!8*d2$<aW7_@BlrdbM7 ztONb`wnXKo1HmNf`9fvYAa zVC88KEBYHQwRguloDG6{wz?T8?Y_n40Y=o;rD}(<&9Y}zLB|_*^G~lK14uVX~r1BUgOz5=M|{QynJng~DiYNi_p3B8L-ol22WX>)Q@7jI88PP}p0uhGXGs2WL z3py}HpuA75diR}BhQLH}k@;rBSf zVPF|}3i=0RkODwl9-2;Jg^M6x(mn`+K2}gl)!-h1Tq`s&8mK!faPf!k;3-(6=pepC zmjw7k|r)neKM!~Y-QLPNgICy)sI5Sh7STTTpsp@m&lTP7pxD>dV~zWMVU{Q0$!<5 zF-190Ii9uf#UHX+zi=roNNa4##|B2JYU>RLBs--H&SDk(c-(h`9K{zXS#h;pmZ4aMKx8{+@{C~#?@Fl`Kxo~X{)EzKR-3C zYpNdw_#_nEm=9yO*m+x%V+#xrcs`p7ofBSIK@%dpEDvZxRGZ7tyCf?$-C|3XEHF?` zj+$7L7_Uq*T-emfA8rlg)nfLtDkbFWxwu z$#wzE);;W#H@$;wJtxz&^Nt$*#P(^T?s=x4&xqJ6^2kKj*h(b>LUxZ7t#vBVJ5-yK zM-Unsg7J7rNV9+1=}!yCnffDUK192*OO`Yi+)s8f;q`8e9hbgYEj}eii9EkuaJ5V` zs1+4xK*2iaZKgTMjhJV5x6>4y@L}4Z1I`Be%On__x{kxj(ODV7m^GeN75U*PD9lhT zMA0vT>Y44wQr=I-YC=~X9lNkhq^DgSLPrj$ttN#|pjjCfkyHw9RFBD*DR&R{6>Twn z{rG_q+*>)botdENd%vBLe6x6&TiDG*5rgiNO>k>Cc-tjXosd*hk~mS$owVXgLVd0H z2nU-rE31gdIlzH!aFeq&@ONT@iXN6?plM1pz)RFbU@HD0UuxWf)RDwqVP z{C>^J2SE@K$9kd|I)m12`khxra$$uWjzeEpnyT*&9@wO`;F+MzV%lWk-oO&I|EC!H zYFDCCHkF)r;A5%~mIh0zzp|twCzo(PPstr(KhvpS-q?^O-^p*Zzngi+(lQ%=_$%m^ ziT_xlX)B_$STD4Oc^zce>S-Bmu0=_&mz3DRSu>Umtt^s`5QFRz@C>AhKh7*E8yExi z@?`cXlOmN)M}F;2MJ=K-&I>KVlbpB^D1lLJgnl7f*cYnT&ML&`z-Nt|6m1QEi)w9 z%IY<)U@+f$W3MKcci*k*%WCU1gT^e+KYkiCVzFU(zCaCo6xeFVC`tGqFc+iyNfs8b?a#PMny8e!%NYte3{bw}E@Z?{l(b6GXjQkyO)pJRWiI zGQ9k`TsY1uURzo9BB_RpuTZ^9bT`YRmf`12dq*qMN2hqqEdhx$a1>AAToRz5C_&jrDg8RAE)T$-}tPyuJH|kgIZE zWMW_pr~nF*wy%k@l6oO<%FDAugHs0YYlT9WxG&tl=k^A9v}t2HRBzyw@4$YwSKiS$ zsJtvnQ)8|EZt2R~#uR@*kwhxZQrTCfpHdWXlB!t4b09aEc7kYNtf?ik8Dr!@_jQJ} zi_-N7bkx`|vsPl^#5A`ay(wz85)*5yCG7dAc=j8;%^%O*n{#Wod$yxqY&l`)STe{C zYjY7G46BhgI(&OwrvDWo}^_Jj-l; zJj-~HZ&Gz5lJUQCdqS{{1UE7vq~op!AAL3ct|=U~ag@cSrJhVxaLXFPe(Gh$#G8YsOd zy|#!J^tlGG7eU4ZCDcZ+`JEX*9SeF!BdDG#H;HLS#EJ^d4Dcn?>5Ah){kD&QprMX+&yq%r*O#@%yL! zbYx!LK#4U>MAqmE?JKfYAi7HCE`94dno1kF(hHc0@k=CM`h1F5d{XI^{zKAU(133m zwEoDxOAOY40Cf4ty0%nT`E;Vs1Hbd7@0^0;2bC3s(3`U_S}2D)z|_vVnjasVy$7cl zx9NNa!bwy2d(qNApDM;HwQK1E#&6Di#{6@;38^eJ%t3;Ls28~Y;0d7kVLin!cbCBu zk)ZgdKeVhU#Wr+ieExcSdhrH65&Z3ENofO>YB#GB3vf+vxdOfGSUsI-E`q$zRyn_yKxn6YHC~@ z8{Ra-qQ(ya9SZ*3r@Qtjo|@$pxp@^ehr$00zX?b7sa{f}SR+T( z{#uXaYdL#9&*m0&Hvg2H8QtvnI$<^dTl8NI=$cskmRw$@PxSjQ=RYYT8TI_=3MMhV zvdhn<317SF)n>&b7bse;kF@(~5^7sKcAcbLlFzk!zIaDyA~hiUOS@4lcOetQ92Cpi zjB&{WFmGZ=Zpxqq8lolB=uF2TLm#TUlE>OQ>^J#WvtS zC1r_v#s)Y_?_Y((*(SpFKlYUf)=~k+zFA0`xWi`h5#UX;|5CC4AEdogbY*Y1?Ol~r zl8P%UcEz@B+qP|1P|=EQ+qP}nwp~$0-}=Ar-n)Gl=j?r2+ZVI7c|Yef+Hdq{jNVU9 zTH;evq}j*iZuM$-E5ftt$J385qSuTbjQNRi`=l0A%c{GowB$aYK8t<77~h2LGYoSN z)8WXm-{C++5sHQGQw%dj9pk{_gyYab>hH{b0n%eZ2yusf7tPyg!5~S;6vC$+o{27J z!0F>4nWAza0DgVL=>J)Tfzco67YLBpV5SeqP8h*z3FeaAV2SB3@Y9gSbPq}JV`PN% z>2L8%ZW**+saUlwx7bRV7`Yf|(8pjrpK=2l*$Ay`>cpx@Ni$|dMjxgp*$c1T;fQH= zUACTz&FRE^!1a4%x@{%HD>x9p$YlD~5~!NwgJS4kpWb^Us|xk^IajPs$AC7a83|J! zjcxX$0?!J#TUCpiU4$~tQtUtQshG4wkGI2AVqF7YId#f9NNy(7izNX4BFU6GPdh3#+(a=lV*NCZpC&|EC6P!b1AkN zFo@mYyhnVE*qqh{m}joxO_uwjFLn29!@zKShLFuh6N>Fa9|9WV|y#hRVG$S;Wt22))_xgB68+0zU}29n(mK*}Z$a*96?SK16&-@P7FdIqZZ%FbgQAxI)spLfX1Q;)z0%z{_Os9Smd$ zzaPA1WGo!SO}ufxPi$<=#SIi>D8KJ40IWnIb+-q=vL`N-d1@p@A**)Y=JF1dnTQWQhi~{+y;cSr52%u z93)%%?W-7=DVP|3G0-PgqG+O!vXH7j!u@c?85wH6f+`o(cOz;|qH<83`tjdZvj4$E zM%KdmKWyuRl-FgE6;O17(Td>uD;vmvD&|O1h!1l4qtl9{qlA^FEHvlx1ofUur%wI& zLw3{a{sM(ZOsY(Bdv_*=`D8&RzxM65X8OWqslq#blh3#N^9y}sAmTjl#-o8opAPgB z@q~B_oq=i}5p*=kgp_(r4Y@%nDlSIMu1I%;Dcq=euL)+Hy?0+jz8tCKAY4-}d@_pk zPwI&l|ML)vmH;Z4Jq(qbZ$vOzdz1lWj9L2Mg@@wT&#JKIRg9RD)>e)*el_9At|ysV z1GpvaDay|*sjbd96Y0BXjJGUXn6FI|fA zG`u3~g2aV6Pe!~V$L|Dw4X5fFT@}!cXkGgyo}qg-4lKhv=wc%i!S1={-3X^B;V2D? z+ARwmYpe&)x?{CUFA(z#j)l*TTo^Iccl4$0q z%q^KLB=d!2vnc2Wv^(d%#&61A2nAT``_IuirJYqd0eO`NgOVTq?-1}m933SShyOe} zF*0%>5&~}9#Lj^c?j1{n;`|oIo(3S6WR&5_BS)KPOpva#Ca@0qlHAS+1AKq;jj`K= zK{4Fc+}}=3Fr~k|d47Je590w}hjfLzNSo2u)F;IdnB&6F|Z{b=ypmi z&~0;IweV z^q~U-e9=A!HI{HW!vEeH6AE(Xi@Je#zW-50Ii*(+NKJ4V!_q*dN1Fkl9BU{Jqg8b5 z(nBSn&EO+rRAW|%Hl>^nD_4K)q#Y3>_i{s$BVeS!aT<(B<-@&13P+=8IWE&H<&N_B9g2Q@=aGa=(FDK&m<eT~05+y#GJs2}{linYB!Z*3N+Snq?L|HU8opBW>O@iubE0wBjPaGPc|NiD6! zV$E-q+ERy#U*Y|pAxjBL0rO?m$wm}rSAN>n#6DEB!T^cqFFxe`8xBA8LM}<#nYdow z%}nX)yFcIBz-9(CvF^d9za>GaeslIq=77fS6?FQo;JE}Cp0ljPd6oEs%KgKNhtRkXWrUcsiK=s*iH^^I-@*Z zNb*}FrT&nz;F$XPT_8W-|FUF+Owe<?3Hh$$H3WZ?1N&CTP2Zyp7`|LZc zP@O7=jW1ws*9y5hjb3xl7ERA4FmQli;7@?Z@p=&z^vUl!a1Noy)qq<;NMGS+n8oY@s!0Z;`JFu7glN6|(QvmaF7Q0eCgzzRNVC!pI7ncfE&lDT0T$(K_HA|A@8}_PP zwx62T+7_-{oh(Y9Iy{(&nat4qCHX#Hk2*fHeoeI=XZ*s~yXJO!#r?98R|(ebvZIIa zwoWjH+qe4z6+z+jhiq_(PVp$W$jtl$!tkkv$cLoE_h!ver!%RnfgV=Pn= zj^i+VQmX+w~)g2}(94G+Iz`DOp)&q4kM z{rDpS3FF%@=trc$@1Q*(uz39peLjJNpRzGO*~RTcK-WJ={>jMy-mj_>$%jFB zE|U)hS1IYMq9g~w1lYEsJ!)6pjthI~ju~jZ>m9}?0Ay5fH)!X2=V$}Pn=t?$o2<&B_6^32-nhVs=*010ecUKo?N zI)tM|776x%S%D570MA|)Lh2=l1nZDan05=A@9j-Snf0%x5*(xnQx8)LQwL^~8*+3} z?~0YH%j!GCVMg&#nMn963huo^5Bd4=*&3Wip8$2UtN=cWH2X({Yy;+jn)`It#)7o zT5zLtAAdNozmqF*L!PW|zjjS;BFHR4$B6l0~uUhEnOaq3*6?KI+m`kl5 z=Sz+q8Kdp1-$7;;xz--G$bTC{60_6uO;g+`F8?kl zOEiL)Cyi#1W{FXVRHGkzWiI@Lzn+lWjlU-JQ8dLKO}FW&R=KQTy3!4j}tD(b5b zJC)T5TDW;HY@cb+9@Lr9V9kW()Y8VPGFvE!uSg;7H1&j0Bs0qzoomz2diq=}+thK- zJw=C0#!MalHA|6bXV~`XSV{29aERrk&naa<&$tN%WP1>>TX$+f`S&D5TI}87NI)_tzml+OJfV{>e4?^y-BEP zNo5v#XMe*rXzluoSGJseG^vdgi>IA4sT4bV%x{N8AAT}${uR=sW~Jt<^7o`3hJs86 z!^Ahg+gZsC9v0H%{+#VIu>_V$JRT{fde_kzIGbG^E&j8))alrq`6@eMwa%l_HOnm2 zCTeCd$mu&0f^qu*c6fdfT<61-13Q7mNcgu&c%@j(Er#HZGW%6tnIZ~;HzRbPascyt z^f2PO@$k(^kyJ3G^2;t230NY4uI31qU2IZ#p?dsY7E84h0&J8XdSYg1SST36R5Ksz z5fP!R*5R~NFC6Yd&YVEvsrWTXS6p{j5rVFTRfn<1LN&}E@ejL#sSp|01f;DP*5Xr}<8aX1R207Z# z#b_2VTVvvFC9mO_3G=xx^@U*8+K`8z?JFvx&-eZ17e_pgb(eN=^9f zw7w`oSVrs3zj}J>YM|XJzb_9dv0Rz);+@lrRTR!?^gc?Noc;&HxfsnyNb_1CDF>kNg57kR zP|QzHoVwXqrv$aKWy+Lwq)=RQZ_nK+>|nBo%tp7Kxhp~1sj)hPpG*vL%OD>0=Lc!{= zMQ-OIV)YW|0k4Bg%4Yu?njpB>)Y1V?PK4`iPaj;54SI*myD5!DT5Khr9V^cy4DTTR+F z8h32uP*{#L!;a8g-e{*w4lm+%j|y#;uXNKN^k#PG(}%Jwh+>_ve1c~usRW<(QCW%Q zXC}i|dvE1Z_++OC6q6I*s~3LRxhjqAxkzpC&2wy7pF81*UD~81x4LH&f8yddk3JN{ zV5Dlb7n!!352Q1>w8!xp54mXZ1`T*WZQhKy)`?lwvexqr%D_L$6 z+G8{L>IOU&vJsz>&Ciisk1$(a3wzbx}{r;~ct+nf*qF(4PUxY!e&;Pwo_#f8s z|6xMV^I!H|Q0`ibs2|ldud8?AMqIG#s|q1R>*RGE05dLVhDPKPp^|w)kEASXKUHQy zK{g%dxw+Ii5iM~k3+RMh{@fW6DbW1YtWHi4ZO$<_5409%9O^vl$DQ8k#=ns;>(_QY zzWd(PuP^S4RakElN|9-na|>WHGz-e*MXHvTiAAbbmXSr;X`1DwOe~GEN;YYl)fCM| zYb=YorLr=aD&eHuxVBld|HeQ|~Y%qV*8A^U_~jQ&Ho zX(M;uepn3MksHGQGLkiNhv=umtTJ{-?JvjF?Z43kE-wr8=W;IWuEL=&3V^(>9^5@} zv3`^Y5mn7Np!($OTY59O;#J z)7#2OwGx0za0^=D+2>=S(I;gHPbZSYWG*a-dW$MNT{-0<&vjL3_sb;Dgy0nH;M=Cp z1pU#@ixJc3Wf0r*i;f%BREU@bZ64}~8fBgikyU?hm2_+dz}v8T+?*RB^yU{;z0}m! zZz*&@wzAHR$lXheKf7{woFN3ov_I6@(Nt2d!vgb83&xlCSzg=NuwWjl+}(jasYvwX z)b3~Dodz-c=9FIT6_gHH8$@zBtuC3$gFwDS?~MFDIW{`UGB_@ z-$ls8H~9Aow!dR%z27R6qmd|B$qKUuTQG<6n5EAu#6MKTcn0Tvy)VJUU_nGSN*xhT zg*M{0PP&jA1T0*~{Y4~>ATlK!r34BkrW&Jq)#Zo5j5S!duLe( zPHRH2DDXFI<_iMiMf+~%0dOda8p~~`2i~V6?g~e#HvZaa-Acr0)9rqD&#Ug*G4y*5 zOf^KY^ilBkD1oz3;UlAwgZ^G2P}H%U6JPfxBZb3K zu=;f=F+f-+#j$KwWZaadQ5Z6*L3z*a()!C}_9_CBv7?+>M0Wo2F6WI>-BM3SJ1eXh zSvZ@1aN@gUzUw4!S;pQPPdZjB_u~|6g*s-+q6R~znM+d#a{}pceC{8=b`_jNO0F4)WMoL2(*7`;S>!SBh@+H~++GP_nB}Ig`Nh4nG2miw7A=fBl_UY-L=eib%4v z?t^3OXTo5O8%6TWgG(DFjOJ5>Ip9g0>viaf80Z%*`!h>uVwqKD@tCBj;a*n_a6Ya+ zN-PBNXX9N{=7%J;Ef=#+7ZVz6I%IYbOZxk+JafWYnSAH!#|}mWgmlnLe{M;t2Lf*o zkepefOl|Tmsz(IJ#`v@9KG?1Oq&AZnrZLb=qaP(Bzt*|WZmw5^X#vZRe@HKg(z68~ zUn#|`u`0yZd?YacWLjQInjJOprcai5QAfu5f+&gk@A4z9quRmZ z8sx29`wB|r95#rTDLMLcCo$fa3g*<}KZ79pUQz1^KSjpi6j;9@jA>bBY#3LTMySh* z(N3EBF0yc>>iOeiog(sB#yBTr(;m1vH1jbp{6Z7PzT9ypEh~xGx#Ahky-*G- z<TweZy&zo!t=w;jG_d?L1{yHN( zyNi5s#?;yT@bb8Z+y;K|H%B+2jXI^bbE5A5B3o=$p3n@(vaEc9U5#FHU5LWiZ5WN> zpo1+WWs1hWG5lI4@2AP^56JMH|Ef>s{Rv-1Gzto%;Wqv#O|L9M^6me1QB}A=X^Gpc ze=AY6;}*D>Wc@wT61%D~m4+EwifRKetwd@EaaHF*B`?AE`^fe>rSG?}?a-^u9pm~0 z4gLwX-bo*)FC4 zRtkyZC>)2jDJNnj{9y;bHCj;ah77AqBQ98RySMcL%A*QD<8pAj$%jwubhl=F-V(uS zRlnA^`gW4W=pA9dwIzxSiqgwL76{AhL2im6ELn*qR|!ihW830^#aq?hdc4s zYkAEM-@^sk=4Q>tbp{BF_FaYzYbUin0Y3i7cJI?Vii&oP8beH8T8?HRM_aBfGIlm% z%-dyc)&}`;ODdeWF-?c|*8+E%$-x}s*51Vyxs7ux;zAV>n10Am?a=i?^-K~6T*hT0 zhvVH#!5^L^mdz=8TqI|Nv1@b+=;4so2_;%LvVNXT!ngytir=#c9FC9A%)={Hr;^cX zw``LshgF_aS{lj=U&_N{Iu(KMxZN>0T67n5YmsEIsC1cv=?$2#w^S0hw4-~K?$tj+ zI{_|CTLkh?;tNdnS_Zc#CmDUJsov$r*J!C!nU?8=&%A+pgunhM&qllaadu`^8q}gH zeLRYa-y3y8IQ3>$OQ0n+T5avTywa*Q-d+(uewe`_o7r^WYZyZYO%otS?U!7GViFvE zPN5rO3K)q>i6zdV7o459qLTJY95>Wc`PS(*7Gu^aY}3LEWun>@SX4)qqGbIj@9w